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 94c067cac3..93233931cb 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 @@ -67,7 +67,7 @@ import com.baidu.hugegraph.backend.id.IdGenerator; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.store.BackendFeatures; -import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; +import com.baidu.hugegraph.backend.store.BackendStoreInfo; import com.baidu.hugegraph.backend.store.raft.RaftGroupManager; import com.baidu.hugegraph.config.AuthOptions; import com.baidu.hugegraph.config.HugeConfig; @@ -591,15 +591,9 @@ public String backend() { } @Override - public String backendVersion() { - this.verifyAnyPermission(); - return this.hugegraph.backendVersion(); - } - - @Override - public BackendStoreSystemInfo backendStoreSystemInfo() { + public BackendStoreInfo backendStoreInfo() { this.verifyAdminPermission(); - return this.hugegraph.backendStoreSystemInfo(); + return this.hugegraph.backendStoreInfo(); } @Override @@ -721,6 +715,12 @@ public void truncateBackend() { } } + @Override + public void initSystemInfo() { + this.verifyAdminPermission(); + this.hugegraph.initSystemInfo(); + } + @Override public void createSnapshot() { this.verifyPermission(HugePermission.WRITE, ResourceType.STATUS); @@ -967,6 +967,12 @@ public HugeGraph graph() { return this.taskScheduler.graph(); } + @Override + public void init() { + verifyAdminPermission(); + this.taskScheduler.init(); + } + @Override public int pendingTasks() { verifyTaskPermission(HugePermission.READ); @@ -1130,6 +1136,12 @@ private String currentUsername() { return null; } + @Override + public void init() { + verifyAdminPermission(); + this.authManager.init(); + } + @Override public boolean close() { verifyAdminPermission(); diff --git a/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java b/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java index 1a9f778ab9..e8b8b867f0 100644 --- a/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java +++ b/hugegraph-api/src/main/java/com/baidu/hugegraph/core/GraphManager.java @@ -44,7 +44,7 @@ import com.baidu.hugegraph.backend.cache.Cache; import com.baidu.hugegraph.backend.cache.CacheManager; import com.baidu.hugegraph.backend.id.IdGenerator; -import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; +import com.baidu.hugegraph.backend.store.BackendStoreInfo; import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.config.ServerOptions; import com.baidu.hugegraph.exception.NotSupportException; @@ -273,7 +273,7 @@ private void checkBackendVersionOrExit(HugeConfig config) { } } } - BackendStoreSystemInfo info = hugegraph.backendStoreSystemInfo(); + BackendStoreInfo info = hugegraph.backendStoreInfo(); if (!info.exists()) { throw new BackendException( "The backend store of '%s' has not been initialized", diff --git a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraMetrics.java b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraMetrics.java index 0443a5f910..fa22e067dd 100644 --- a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraMetrics.java +++ b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraMetrics.java @@ -34,10 +34,10 @@ import org.apache.tinkerpop.gremlin.util.NumberHelper; import com.baidu.hugegraph.backend.store.BackendMetrics; +import com.baidu.hugegraph.backend.store.BackendStoreProvider; import com.baidu.hugegraph.backend.store.BackendTable; import com.baidu.hugegraph.backend.store.cassandra.CassandraTables.Edge; import com.baidu.hugegraph.backend.store.cassandra.CassandraTables.Vertex; -import com.baidu.hugegraph.config.CoreOptions; import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.testutil.Whitebox; import com.baidu.hugegraph.util.E; @@ -70,7 +70,7 @@ public CassandraMetrics(HugeConfig conf, assert this.username != null && this.password != null; this.keyspace = keyspace; - String g = conf.get(CoreOptions.STORE_GRAPH); + String g = BackendStoreProvider.GRAPH_STORE; String v = BackendTable.joinTableName(g, Vertex.TABLE); String oe = BackendTable.joinTableName(g, "o" + Edge.TABLE_SUFFIX); String ie = BackendTable.joinTableName(g, "i" + Edge.TABLE_SUFFIX); diff --git a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStore.java b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStore.java index 9442a8454f..8ef969dad4 100644 --- a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStore.java +++ b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStore.java @@ -670,5 +670,45 @@ public long getCounter(HugeType type) { public boolean isSchemaStore() { return false; } + + public CassandraSessionPool.Session getSession() { + return super.sessions.session(); + } + } + + public static class CassandraSystemStore extends CassandraGraphStore { + + private final CassandraTables.Meta meta; + + public CassandraSystemStore(BackendStoreProvider provider, + String keyspace, String store) { + super(provider, keyspace, store); + + this.meta = new CassandraTables.Meta(); + } + + @Override + public void init() { + super.init(); + this.checkOpened(); + CassandraSessionPool.Session session = this.getSession(); + String driverVersion = this.provider().driverVersion(); + this.meta.writeVersion(session, driverVersion); + LOG.info("Write down the backend version: {}", driverVersion); + } + + @Override + public String storedVersion() { + this.checkOpened(); + CassandraSessionPool.Session session = this.getSession(); + return this.meta.readVersion(session); + } + + @Override + protected Collection tables() { + List tables = new ArrayList<>(super.tables()); + tables.add(this.meta); + return tables; + } } } diff --git a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStoreProvider.java b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStoreProvider.java index 6104bf2bdf..7024de895e 100644 --- a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStoreProvider.java +++ b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStoreProvider.java @@ -23,6 +23,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.cassandra.CassandraStore.CassandraGraphStore; import com.baidu.hugegraph.backend.store.cassandra.CassandraStore.CassandraSchemaStore; +import com.baidu.hugegraph.backend.store.cassandra.CassandraStore.CassandraSystemStore; public class CassandraStoreProvider extends AbstractBackendStoreProvider { @@ -40,13 +41,18 @@ protected BackendStore newGraphStore(String store) { return new CassandraGraphStore(this, this.keyspace(), store); } + @Override + protected BackendStore newSystemStore(String store) { + return new CassandraSystemStore(this, this.keyspace(), store); + } + @Override public String type() { return "cassandra"; } @Override - public String version() { + public String driverVersion() { /* * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking @@ -62,7 +68,8 @@ public String version() { * [1.8] #746: support userdata for indexlabel * [1.9] #295: support ttl for vertex and edge * [1.10] #1333: support read frequency for property key + * [1.11] #1533: add meta table in system store */ - return "1.10"; + return "1.11"; } } diff --git a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraTables.java b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraTables.java index fea2b988d2..42ab336a39 100644 --- a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraTables.java +++ b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraTables.java @@ -72,6 +72,48 @@ public class CassandraTables { private static final long COMMIT_DELETE_BATCH = Query.COMMIT_BATCH; + public static class Meta extends CassandraTable { + + public static final String TABLE = HugeType.META.string(); + + public Meta() { + super(TABLE); + } + + @Override + public void init(CassandraSessionPool.Session session) { + ImmutableMap pkeys = ImmutableMap.of( + HugeKeys.NAME, DataType.text() + ); + ImmutableMap ckeys = ImmutableMap.of(); + ImmutableMap columns = ImmutableMap.of( + HugeKeys.VALUE, DataType.text() + ); + + this.createTable(session, pkeys, ckeys, columns); + } + + public void writeVersion(CassandraSessionPool.Session session, + String version) { + Insert insert = QueryBuilder.insertInto(TABLE); + insert.value(formatKey(HugeKeys.NAME), formatKey(HugeKeys.VERSION)); + insert.value(formatKey(HugeKeys.VALUE), version); + session.execute(insert); + } + + public String readVersion(CassandraSessionPool.Session session) { + Clause where = formatEQ(HugeKeys.NAME, formatKey(HugeKeys.VERSION)); + Select select = QueryBuilder.select(formatKey(HugeKeys.VALUE)) + .from(TABLE); + select.where(where); + Row row = session.execute(select).one(); + if (row == null) { + return null; + } + return row.getString(formatKey(HugeKeys.VALUE)); + } + } + public static class Counters extends CassandraTable { public static final String TABLE = HugeType.COUNTER.string(); 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 e9b93f4d86..c85e3f96bb 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java @@ -35,9 +35,8 @@ import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.store.BackendFeatures; -import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; +import com.baidu.hugegraph.backend.store.BackendStoreInfo; import com.baidu.hugegraph.backend.store.raft.RaftGroupManager; -import com.baidu.hugegraph.config.ConfigOption; import com.baidu.hugegraph.config.TypedOption; import com.baidu.hugegraph.rpc.RpcServiceConfig4Client; import com.baidu.hugegraph.rpc.RpcServiceConfig4Server; @@ -133,8 +132,7 @@ public interface HugeGraph extends Graph { public String name(); public String backend(); - public String backendVersion(); - public BackendStoreSystemInfo backendStoreSystemInfo(); + public BackendStoreInfo backendStoreInfo(); public BackendFeatures backendStoreFeatures(); public GraphMode mode(); @@ -154,6 +152,8 @@ public interface HugeGraph extends Graph { public void clearBackend(); public void truncateBackend(); + public void initSystemInfo(); + public void createSnapshot(); public void resumeSnapshot(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraphParams.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraphParams.java index e307415635..7cafd45e5f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraphParams.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraphParams.java @@ -20,6 +20,7 @@ package com.baidu.hugegraph; import com.baidu.hugegraph.analyzer.Analyzer; +import com.baidu.hugegraph.backend.LocalCounter; import com.baidu.hugegraph.backend.serializer.AbstractSerializer; import com.baidu.hugegraph.backend.store.BackendFeatures; import com.baidu.hugegraph.backend.store.BackendStore; @@ -67,6 +68,7 @@ public interface HugeGraphParams { public ServerInfoManager serverManager(); + public LocalCounter counter(); public AbstractSerializer serializer(); public Analyzer analyzer(); public RateLimiter writeRateLimiter(); 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 1d4b1faee8..c82d49d26f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java @@ -43,6 +43,7 @@ import com.baidu.hugegraph.auth.AuthManager; import com.baidu.hugegraph.auth.StandardAuthManager; import com.baidu.hugegraph.backend.BackendException; +import com.baidu.hugegraph.backend.LocalCounter; import com.baidu.hugegraph.backend.cache.Cache; import com.baidu.hugegraph.backend.cache.CacheNotifier; import com.baidu.hugegraph.backend.cache.CacheNotifier.GraphCacheNotifier; @@ -57,8 +58,8 @@ import com.baidu.hugegraph.backend.store.BackendFeatures; import com.baidu.hugegraph.backend.store.BackendProviderFactory; import com.baidu.hugegraph.backend.store.BackendStore; +import com.baidu.hugegraph.backend.store.BackendStoreInfo; import com.baidu.hugegraph.backend.store.BackendStoreProvider; -import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; import com.baidu.hugegraph.backend.store.raft.RaftBackendStoreProvider; import com.baidu.hugegraph.backend.store.raft.RaftGroupManager; import com.baidu.hugegraph.backend.store.ram.RamTable; @@ -147,6 +148,7 @@ public class StandardHugeGraph implements HugeGraph { private final EventHub graphEventHub; private final EventHub indexEventHub; + private final LocalCounter localCounter; private final RateLimiter writeRateLimiter; private final RateLimiter readRateLimiter; private final TaskManager taskManager; @@ -167,6 +169,8 @@ public StandardHugeGraph(HugeConfig config) { this.graphEventHub = new EventHub("graph"); this.indexEventHub = new EventHub("index"); + this.localCounter = new LocalCounter(); + final int writeLimit = config.get(CoreOptions.RATE_LIMIT_WRITE); this.writeRateLimiter = writeLimit > 0 ? RateLimiter.create(writeLimit) : null; @@ -235,13 +239,10 @@ public String backend() { } @Override - public String backendVersion() { - return this.storeProvider.version(); - } - - @Override - public BackendStoreSystemInfo backendStoreSystemInfo() { - return new BackendStoreSystemInfo(this.schemaTransaction()); + public BackendStoreInfo backendStoreInfo() { + // Just for trigger Tx.getOrNewTransaction, then load 3 stores + this.systemTransaction(); + return new BackendStoreInfo(this.storeProvider); } @Override @@ -251,6 +252,9 @@ public BackendFeatures backendStoreFeatures() { @Override public void serverStarted(Id serverId, NodeRole serverRole) { + LOG.info("Init system info for graph '{}'", this.name); + this.initSystemInfo(); + LOG.info("Init server info [{}-{}] for graph '{}'...", serverId, serverRole, this.name); this.serverInfoManager().initServerInfo(serverId, serverRole); @@ -283,15 +287,6 @@ public GraphMode mode() { public void mode(GraphMode mode) { LOG.info("Graph {} will work in {} mode", this, mode); this.mode = mode; - if (mode.loading()) { - /* - * NOTE: This may block tasks submit and lead the queue to be full, - * so don't submit gremlin job when loading data - */ - this.taskManager.pauseScheduledThreadPool(); - } else { - this.taskManager.resumeScheduledThreadPool(); - } } @Override @@ -320,7 +315,12 @@ public void initBackend() { LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); try { this.storeProvider.init(); - this.storeProvider.initSystemInfo(this); + /* + * NOTE: The main goal is to write the serverInfo to the central + * node, such as etcd, and also create the system schema in memory, + * which has no side effects + */ + this.initSystemInfo(); } finally { LockUtil.unlock(this.name, LockUtil.GRAPH_LOCK); this.loadGraphStore().close(); @@ -359,7 +359,7 @@ public void truncateBackend() { LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); try { this.storeProvider.truncate(); - this.storeProvider.initSystemInfo(this); + // TOOD: remove this after serverinfo saved in etcd this.serverStarted(this.serverInfoManager().selfServerId(), this.serverInfoManager().selfServerRole()); } finally { @@ -369,6 +369,18 @@ public void truncateBackend() { LOG.info("Graph '{}' has been truncated", this.name); } + @Override + public void initSystemInfo() { + try { + this.taskScheduler().init(); + this.serverInfoManager().init(); + this.authManager().init(); + } finally { + this.closeTx(); + } + LOG.debug("Graph '{}' system info has been initialized", this); + } + @Override public void createSnapshot() { LockUtil.lock(this.name, LockUtil.GRAPH_LOCK); @@ -430,18 +442,15 @@ private void checkGraphNotClosed() { } private BackendStore loadSchemaStore() { - String name = this.configuration.get(CoreOptions.STORE_SCHEMA); - return this.storeProvider.loadSchemaStore(name); + return this.storeProvider.loadSchemaStore(); } private BackendStore loadGraphStore() { - String graph = this.configuration.get(CoreOptions.STORE_GRAPH); - return this.storeProvider.loadGraphStore(graph); + return this.storeProvider.loadGraphStore(); } private BackendStore loadSystemStore() { - String name = this.configuration.get(CoreOptions.STORE_SYSTEM); - return this.storeProvider.loadSystemStore(name); + return this.storeProvider.loadSystemStore(); } @Watched @@ -1133,6 +1142,11 @@ public ServerInfoManager serverManager() { return StandardHugeGraph.this.serverInfoManager(); } + @Override + public LocalCounter counter() { + return StandardHugeGraph.this.localCounter; + } + @Override public AbstractSerializer serializer() { return StandardHugeGraph.this.serializer(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/AuthManager.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/AuthManager.java index c10e11a099..7750fb10d9 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/AuthManager.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/AuthManager.java @@ -29,6 +29,7 @@ public interface AuthManager { + public void init(); public boolean close(); public Id createUser(HugeUser user); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/HugeProject.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/HugeProject.java index e92f3804e0..46fbaf3a69 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/HugeProject.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/HugeProject.java @@ -270,7 +270,7 @@ public Schema(HugeGraphParams graph) { @Override public void initSchemaIfNeeded() { - if (this.existEdgeLabel(this.label)) { + if (this.existVertexLabel(this.label)) { return; } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/StandardAuthManager.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/StandardAuthManager.java index 2a8f67c30d..af7a32616d 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/StandardAuthManager.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/auth/StandardAuthManager.java @@ -44,16 +44,13 @@ import com.baidu.hugegraph.backend.id.IdGenerator; import com.baidu.hugegraph.config.AuthOptions; import com.baidu.hugegraph.config.HugeConfig; -import com.baidu.hugegraph.event.EventListener; import com.baidu.hugegraph.type.define.Directions; import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.Events; import com.baidu.hugegraph.util.LockUtil; import com.baidu.hugegraph.util.Log; import com.baidu.hugegraph.util.StringEncoding; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import io.jsonwebtoken.Claims; @@ -62,7 +59,6 @@ public class StandardAuthManager implements AuthManager { protected static final Logger LOG = Log.logger(StandardAuthManager.class); private final HugeGraphParams graph; - private final EventListener eventListener; // Cache private final Cache usersCache; @@ -90,7 +86,6 @@ public StandardAuthManager(HugeGraphParams graph) { this.tokenExpire = config.get(AuthOptions.AUTH_TOKEN_EXPIRE); this.graph = graph; - this.eventListener = this.listenChanges(); this.usersCache = this.cache("users", capacity, expired); this.pwdCache = this.cache("users_pwd", capacity, expired); this.tokenCache = this.cache("token", capacity, expired); @@ -124,36 +119,8 @@ private Cache cache(String prefix, long capacity, return cache; } - private EventListener listenChanges() { - // Listen store event: "store.inited" - Set storeEvents = ImmutableSet.of(Events.STORE_INITED); - EventListener eventListener = event -> { - // Ensure user schema create after system info initialized - if (storeEvents.contains(event.name())) { - try { - this.initSchemaIfNeeded(); - } finally { - this.graph.closeTx(); - } - return true; - } - return false; - }; - this.graph.loadSystemStore().provider().listen(eventListener); - return eventListener; - } - - private void unlistenChanges() { - this.graph.loadSystemStore().provider().unlisten(this.eventListener); - } - @Override - public boolean close() { - this.unlistenChanges(); - return true; - } - - private void initSchemaIfNeeded() { + public void init() { this.invalidateUserCache(); HugeUser.schema(this.graph).initSchemaIfNeeded(); HugeGroup.schema(this.graph).initSchemaIfNeeded(); @@ -163,6 +130,11 @@ private void initSchemaIfNeeded() { HugeProject.schema(this.graph).initSchemaIfNeeded(); } + @Override + public boolean close() { + return true; + } + private void invalidateUserCache() { this.usersCache.clear(); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedBackendStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedBackendStore.java index b2daec42fb..9060f9396e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedBackendStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/cache/CachedBackendStore.java @@ -28,6 +28,7 @@ import com.baidu.hugegraph.backend.store.BackendMutation; import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.BackendStoreProvider; +import com.baidu.hugegraph.backend.store.SystemSchemaStore; import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.util.StringEncoding; @@ -52,6 +53,11 @@ public String store() { return this.store.store(); } + @Override + public String storedVersion() { + return this.store.storedVersion(); + } + @Override public String database() { return this.store.database(); @@ -62,6 +68,11 @@ public BackendStoreProvider provider() { return this.store.provider(); } + @Override + public SystemSchemaStore systemSchemaStore() { + return this.store.systemSchemaStore(); + } + @Override public void open(HugeConfig config) { this.store.open(config); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStore.java index 41727ace55..9b0cd90b25 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStore.java @@ -25,9 +25,11 @@ public abstract class AbstractBackendStore implements BackendStore { + private final SystemSchemaStore systemSchemaStore; private final MetaDispatcher dispatcher; public AbstractBackendStore() { + this.systemSchemaStore = new SystemSchemaStore(); this.dispatcher = new MetaDispatcher<>(); } @@ -39,11 +41,22 @@ public void registerMetaHandler(String name, MetaHandler handler) { this.dispatcher.registerMetaHandler(name, handler); } + @Override + public String storedVersion() { + throw new UnsupportedOperationException( + "AbstractBackendStore.storedVersion()"); + } + + @Override + public SystemSchemaStore systemSchemaStore() { + return this.systemSchemaStore; + } + // Get metadata by key @Override public R metadata(HugeType type, String meta, Object[] args) { Session session = this.session(type); - MetaDispatcher dispatcher = null; + MetaDispatcher dispatcher; if (type == null) { dispatcher = this.metaDispatcher(); } else { 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 fc35b7fb84..0a5a9d084e 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 @@ -25,7 +25,6 @@ import org.slf4j.Logger; -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; @@ -41,7 +40,7 @@ public abstract class AbstractBackendStoreProvider private String graph = null; - private EventHub storeEventHub = new EventHub("store"); + private final EventHub storeEventHub = new EventHub("store"); protected Map stores = null; @@ -63,6 +62,8 @@ protected final void checkOpened() { protected abstract BackendStore newGraphStore(String store); + protected abstract BackendStore newSystemStore(String store); + @Override public void listen(EventListener listener) { this.storeEventHub.listen(EventHub.ANY_EVENT, listener); @@ -73,6 +74,11 @@ public void unlisten(EventListener listener) { this.storeEventHub.unlisten(EventHub.ANY_EVENT, listener); } + @Override + public String storedVersion() { + return this.loadSystemStore().storedVersion(); + } + @Override public String graph() { this.checkOpened(); @@ -142,13 +148,14 @@ public void truncate() { } @Override - public void initSystemInfo(HugeGraph graph) { + public boolean initialized() { this.checkOpened(); - BackendStoreSystemInfo info = graph.backendStoreSystemInfo(); - info.init(); - this.notifyAndWaitEvent(Events.STORE_INITED); - - LOG.debug("Graph '{}' system info has been initialized", this.graph); + for (BackendStore store : this.stores.values()) { + if (!store.initialized()) { + return false; + } + } + return true; } @Override @@ -168,7 +175,8 @@ public void resumeSnapshot() { } @Override - public BackendStore loadSchemaStore(final String name) { + public BackendStore loadSchemaStore() { + String name = SCHEMA_STORE; LOG.debug("The '{}' StoreProvider load SchemaStore '{}'", this.type(), name); @@ -184,9 +192,10 @@ public BackendStore loadSchemaStore(final String name) { } @Override - public BackendStore loadGraphStore(String name) { + public BackendStore loadGraphStore() { + String name = GRAPH_STORE; LOG.debug("The '{}' StoreProvider load GraphStore '{}'", - this.type(), name); + this.type(), name); this.checkOpened(); if (!this.stores.containsKey(name)) { @@ -200,8 +209,20 @@ public BackendStore loadGraphStore(String name) { } @Override - public BackendStore loadSystemStore(String name) { - return this.loadGraphStore(name); + public BackendStore loadSystemStore() { + String name = SYSTEM_STORE; + LOG.debug("The '{}' StoreProvider load SystemStore '{}'", + this.type(), name); + + this.checkOpened(); + if (!this.stores.containsKey(name)) { + BackendStore s = this.newSystemStore(name); + this.stores.putIfAbsent(name, s); + } + + BackendStore store = this.stores.get(name); + E.checkNotNull(store, "store"); + return store; } @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 bd1727790b..2cca181f44 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 @@ -34,12 +34,18 @@ public interface BackendStore { // Store name public String store(); + // Stored version + public String storedVersion(); + // Database name public String database(); // Get the parent provider public BackendStoreProvider provider(); + // Get the system schema store + public SystemSchemaStore systemSchemaStore(); + // Whether it is the storage of schema public boolean isSchemaStore(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreInfo.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreInfo.java new file mode 100644 index 0000000000..0dca87ecc7 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreInfo.java @@ -0,0 +1,53 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.backend.store; + +import org.slf4j.Logger; + +import com.baidu.hugegraph.HugeGraph; +import com.baidu.hugegraph.util.Log; + +public class BackendStoreInfo { + + private static final Logger LOG = Log.logger(HugeGraph.class); + + private final BackendStoreProvider storeProvider; + + public BackendStoreInfo(BackendStoreProvider storeProvider) { + this.storeProvider = storeProvider; + } + + public boolean exists() { + return this.storeProvider.initialized(); + } + + public boolean checkVersion() { + String driverVersion = this.storeProvider.driverVersion(); + String storedVersion = this.storeProvider.loadSystemStore() + .storedVersion(); + if (!driverVersion.equals(storedVersion)) { + LOG.error("The backend driver version '{}' is inconsistent with " + + "the data version '{}' of backend store for graph '{}'", + driverVersion, storedVersion, this.storeProvider.graph()); + return false; + } + return true; + } +} 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 12058d7b73..fb773fcbae 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 @@ -19,26 +19,32 @@ package com.baidu.hugegraph.backend.store; -import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.event.EventHub; import com.baidu.hugegraph.event.EventListener; public interface BackendStoreProvider { + String SCHEMA_STORE = "m"; + String GRAPH_STORE = "g"; + String SYSTEM_STORE = "s"; + // Backend store type public String type(); - // Backend store version - public String version(); + // Current backend store driver version + public String driverVersion(); + + // Stored backend store version + public String storedVersion(); // Graph name (that's database name) public String graph(); - public BackendStore loadSystemStore(String name); + public BackendStore loadSystemStore(); - public BackendStore loadSchemaStore(String name); + public BackendStore loadSchemaStore(); - public BackendStore loadGraphStore(String name); + public BackendStore loadGraphStore(); public void open(String name); @@ -52,7 +58,7 @@ public interface BackendStoreProvider { public void truncate(); - public void initSystemInfo(HugeGraph graph); + public boolean initialized(); public void createSnapshot(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreSystemInfo.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreSystemInfo.java deleted file mode 100644 index a47cb068b7..0000000000 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreSystemInfo.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Copyright 2017 HugeGraph Authors - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to You under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package com.baidu.hugegraph.backend.store; - -import java.util.Map; - -import org.apache.tinkerpop.gremlin.structure.Graph.Hidden; -import org.slf4j.Logger; - -import com.baidu.hugegraph.HugeException; -import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.backend.tx.SchemaTransaction; -import com.baidu.hugegraph.schema.PropertyKey; -import com.baidu.hugegraph.schema.SchemaElement; -import com.baidu.hugegraph.type.HugeType; -import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.Log; - -public class BackendStoreSystemInfo { - - private static final Logger LOG = Log.logger(HugeGraph.class); - - private static final String PK_BACKEND_INFO = Hidden.hide("backend_info"); - - private final SchemaTransaction schemaTx; - - public BackendStoreSystemInfo(SchemaTransaction schemaTx) { - this.schemaTx = schemaTx; - } - - public synchronized void init() { - if (this.exists()) { - return; - } - // Set schema counter to reserve primitive system id - this.schemaTx.setNextIdLowest(HugeType.SYS_SCHEMA, - SchemaElement.MAX_PRIMITIVE_SYS_ID); - - HugeGraph graph = this.schemaTx.graph(); - E.checkState(this.info() == null, - "Already exists backend info of graph '%s' in backend " + - "'%s'", graph.name(), graph.backend()); - // Use property key to store backend version - String backendVersion = graph.backendVersion(); - PropertyKey backendInfo = graph.schema() - .propertyKey(PK_BACKEND_INFO) - .userdata("version", backendVersion) - .build(); - this.schemaTx.addPropertyKey(backendInfo); - } - - private Map info() { - PropertyKey pkey; - try { - pkey = this.schemaTx.getPropertyKey(PK_BACKEND_INFO); - } catch (IllegalStateException e) { - String message = String.format( - "Should not exist schema with same name '%s'", - PK_BACKEND_INFO); - if (message.equals(e.getMessage())) { - HugeGraph graph = this.schemaTx.graph(); - throw new HugeException("There exists multiple backend info " + - "of graph '%s' in backend '%s'", - graph.name(), graph.backend()); - } - throw e; - } - return pkey != null ? pkey.userdata() : null; - } - - public boolean exists() { - if (!this.schemaTx.storeInitialized()) { - return false; - } - return this.info() != null; - } - - public boolean checkVersion() { - Map info = this.info(); - E.checkState(info != null, "The backend version info doesn't exist"); - // Backend has been initialized - HugeGraph graph = this.schemaTx.graph(); - String driverVersion = graph.backendVersion(); - String backendVersion = (String) info.get("version"); - if (!driverVersion.equals(backendVersion)) { - LOG.error("The backend driver version '{}' is inconsistent with " + - "the data version '{}' of backend store for graph '{}'", - driverVersion, backendVersion, graph.name()); - return false; - } - return true; - } -} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/SystemSchemaStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/SystemSchemaStore.java new file mode 100644 index 0000000000..cd1424ef97 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/SystemSchemaStore.java @@ -0,0 +1,77 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.backend.store; + +import java.util.Map; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.schema.SchemaElement; +import com.baidu.hugegraph.type.define.CollectionType; +import com.baidu.hugegraph.util.collection.CollectionFactory; + +/** + * The system schema will be initialized when server started, and the + * initialization process is thread-safe, so it's unnecessary to lock it. + */ +public class SystemSchemaStore { + + private static final int SYSTEM_SCHEMA_MAX_NUMS = 128; + + private SchemaElement[] storeByIds; + private final Map storeByNames; + + public SystemSchemaStore() { + this.storeByIds = new SchemaElement[SYSTEM_SCHEMA_MAX_NUMS]; + this.storeByNames = CollectionFactory.newMap(CollectionType.EC, + SYSTEM_SCHEMA_MAX_NUMS); + } + + public void add(SchemaElement schema) { + long idValue = schema.id().asLong(); + assert idValue < 0L; + int index = (int) Math.abs(idValue); + if (index >= this.storeByIds.length) { + this.expandCapacity(); + } + this.storeByIds[index] = schema; + this.storeByNames.put(schema.name(), schema); + } + + @SuppressWarnings("unchecked") + public T get(Id id) { + long idValue = id.asLong(); + assert idValue < 0L; + int index = (int) Math.abs(idValue); + return (T) this.storeByIds[index]; + } + + @SuppressWarnings("unchecked") + public T get(String name) { + return (T) this.storeByNames.get(name); + } + + private void expandCapacity() { + int newLength = this.storeByIds.length << 1; + SchemaElement[] newStoreByIds = new SchemaElement[newLength]; + System.arraycopy(this.storeByIds, 0, newStoreByIds, 0, + this.storeByIds.length); + this.storeByIds = newStoreByIds; + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStore.java index 376b658840..99c049d0bb 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStore.java @@ -353,6 +353,19 @@ public long getCounter(HugeType type) { } } + public static class InMemorySystemStore extends InMemoryGraphStore { + + public InMemorySystemStore(BackendStoreProvider provider, + String database, String store) { + super(provider, database, store); + } + + @Override + public String storedVersion() { + return this.provider().driverVersion(); + } + } + /** * InMemoryDBStore features */ diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStoreProvider.java index 86de96f74a..9aae55554c 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStoreProvider.java @@ -26,6 +26,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.memory.InMemoryDBStore.InMemoryGraphStore; import com.baidu.hugegraph.backend.store.memory.InMemoryDBStore.InMemorySchemaStore; +import com.baidu.hugegraph.backend.store.memory.InMemoryDBStore.InMemorySystemStore; import com.baidu.hugegraph.util.Events; public class InMemoryDBStoreProvider extends AbstractBackendStoreProvider { @@ -75,13 +76,18 @@ protected BackendStore newGraphStore(String store) { return new InMemoryGraphStore(this, this.graph(), store); } + @Override + protected BackendStore newSystemStore(String store) { + return new InMemorySystemStore(this, this.graph(), store); + } + @Override public String type() { return TYPE; } @Override - public String version() { + public String driverVersion() { /* * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking @@ -94,7 +100,8 @@ public String version() { * [1.5] #820: store vertex properties in one column * [1.6] #894: encode label id in string index * [1.7] #1333: support read frequency for property key + * [1.8] #1533: add meta table in system store */ - return "1.7"; + return "1.8"; } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStore.java index 259b6ef576..19cd8f495b 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStore.java @@ -36,6 +36,7 @@ import com.baidu.hugegraph.backend.store.BackendMutation; import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.BackendStoreProvider; +import com.baidu.hugegraph.backend.store.SystemSchemaStore; import com.baidu.hugegraph.backend.store.raft.rpc.RaftRequests.StoreAction; import com.baidu.hugegraph.backend.store.raft.rpc.RaftRequests.StoreType; import com.baidu.hugegraph.config.HugeConfig; @@ -72,6 +73,11 @@ public String store() { return this.store.store(); } + @Override + public String storedVersion() { + return this.store.storedVersion(); + } + @Override public String database() { return this.store.database(); @@ -82,6 +88,11 @@ public BackendStoreProvider provider() { return this.store.provider(); } + @Override + public SystemSchemaStore systemSchemaStore() { + return this.store.systemSchemaStore(); + } + @Override public boolean isSchemaStore() { return this.store.isSchemaStore(); 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 b8d8bae370..76b5fdafcc 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 @@ -24,11 +24,9 @@ import org.slf4j.Logger; -import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.HugeGraphParams; import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.BackendStoreProvider; -import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; import com.baidu.hugegraph.backend.store.raft.rpc.RaftRequests.StoreAction; import com.baidu.hugegraph.backend.store.raft.rpc.RaftRequests.StoreType; import com.baidu.hugegraph.event.EventHub; @@ -86,8 +84,13 @@ public String type() { } @Override - public String version() { - return this.provider.version(); + public String driverVersion() { + return this.provider.driverVersion(); + } + + @Override + public String storedVersion() { + return this.provider.storedVersion(); } @Override @@ -96,10 +99,10 @@ public String graph() { } @Override - public synchronized BackendStore loadSchemaStore(final String name) { + public synchronized BackendStore loadSchemaStore() { if (this.schemaStore == null) { LOG.info("Init raft backend schema store"); - BackendStore store = this.provider.loadSchemaStore(name); + BackendStore store = this.provider.loadSchemaStore(); this.checkNonSharedStore(store); this.schemaStore = new RaftBackendStore(store, this.context); this.context.addStore(StoreType.SCHEMA, this.schemaStore); @@ -108,10 +111,10 @@ public synchronized BackendStore loadSchemaStore(final String name) { } @Override - public synchronized BackendStore loadGraphStore(String name) { + public synchronized BackendStore loadGraphStore() { if (this.graphStore == null) { LOG.info("Init raft backend graph store"); - BackendStore store = this.provider.loadGraphStore(name); + BackendStore store = this.provider.loadGraphStore(); this.checkNonSharedStore(store); this.graphStore = new RaftBackendStore(store, this.context); this.context.addStore(StoreType.GRAPH, this.graphStore); @@ -120,10 +123,10 @@ public synchronized BackendStore loadGraphStore(String name) { } @Override - public synchronized BackendStore loadSystemStore(String name) { + public synchronized BackendStore loadSystemStore() { if (this.systemStore == null) { LOG.info("Init raft backend system store"); - BackendStore store = this.provider.loadSystemStore(name); + BackendStore store = this.provider.loadSystemStore(); this.checkNonSharedStore(store); this.systemStore = new RaftBackendStore(store, this.context); this.context.addStore(StoreType.SYSTEM, this.systemStore); @@ -157,7 +160,7 @@ public void init() { for (RaftBackendStore store : this.stores()) { store.init(); } - this.notifyAndWaitEvent(Events.STORE_INITED); + this.notifyAndWaitEvent(Events.STORE_INIT); LOG.debug("Graph '{}' store has been initialized", this.graph()); } @@ -185,18 +188,7 @@ public void truncate() { store.truncate(); } this.notifyAndWaitEvent(Events.STORE_TRUNCATE); - LOG.debug("Graph '{}' store has been truncated", this.graph()); - } - - @Override - public void initSystemInfo(HugeGraph graph) { - this.checkOpened(); - BackendStoreSystemInfo info = graph.backendStoreSystemInfo(); - info.init(); - - this.notifyAndWaitEvent(Events.STORE_INITED); - LOG.debug("Graph '{}' system info has been initialized", this.graph()); /* * Take the initiative to generate a snapshot, it can avoid this * situation: when the server restart need to read the database @@ -208,7 +200,11 @@ public void initSystemInfo(HugeGraph graph) { * will not encounter such an intermediate state. */ this.createSnapshot(); - LOG.debug("Graph '{}' snapshot has been created", this.graph()); + } + + @Override + public boolean initialized() { + return this.provider.initialized() && this.context != null; } @Override 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 7a609a6e00..ba4e21cad1 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 @@ -46,6 +46,7 @@ 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.BackendStoreProvider; import com.baidu.hugegraph.backend.store.raft.rpc.ListPeersProcessor; import com.baidu.hugegraph.backend.store.raft.rpc.RaftRequests.StoreType; import com.baidu.hugegraph.backend.store.raft.rpc.RpcForwarder; @@ -84,9 +85,6 @@ public final class RaftSharedContext { public static final String DEFAULT_GROUP = "default"; private final HugeGraphParams params; - private final String schemaStoreName; - private final String graphStoreName; - private final String systemStoreName; private final RaftBackendStore[] stores; private final RpcServer rpcServer; @SuppressWarnings("unused") @@ -102,9 +100,6 @@ public RaftSharedContext(HugeGraphParams params) { this.params = params; HugeConfig config = params.configuration(); - this.schemaStoreName = config.get(CoreOptions.STORE_SCHEMA); - this.graphStoreName = config.get(CoreOptions.STORE_GRAPH); - this.systemStoreName = config.get(CoreOptions.STORE_SYSTEM); this.stores = new RaftBackendStore[StoreType.ALL.getNumber()]; this.rpcServer = this.initAndStartRpcServer(); if (config.get(CoreOptions.RAFT_SAFE_READ)) { @@ -180,12 +175,12 @@ public void addStore(StoreType type, RaftBackendStore store) { } public StoreType storeType(String store) { - if (this.schemaStoreName.equals(store)) { + if (BackendStoreProvider.SCHEMA_STORE.equals(store)) { return StoreType.SCHEMA; - } else if (this.graphStoreName.equals(store)) { + } else if (BackendStoreProvider.GRAPH_STORE.equals(store)) { return StoreType.GRAPH; } else { - assert this.systemStoreName.equals(store); + assert BackendStoreProvider.SYSTEM_STORE.equals(store); return StoreType.SYSTEM; } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java index 4143ba9cd0..5205db2a10 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java @@ -120,7 +120,7 @@ public class GraphTransaction extends IndexableTransaction { private Map updatedEdges; private Set> updatedOldestProps; // Oldest props - private LockUtil.LocksTable locksTable; + private final LockUtil.LocksTable locksTable; private final boolean checkCustomVertexExist; private final boolean checkAdjacentVertexExist; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaTransaction.java index b6ce55d5b8..97afe047d7 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaTransaction.java @@ -30,6 +30,7 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.HugeGraphParams; import com.baidu.hugegraph.backend.BackendException; +import com.baidu.hugegraph.backend.LocalCounter; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.id.IdGenerator; import com.baidu.hugegraph.backend.query.ConditionQuery; @@ -37,6 +38,7 @@ import com.baidu.hugegraph.backend.query.QueryResults; import com.baidu.hugegraph.backend.store.BackendEntry; import com.baidu.hugegraph.backend.store.BackendStore; +import com.baidu.hugegraph.backend.store.SystemSchemaStore; import com.baidu.hugegraph.config.CoreOptions; import com.baidu.hugegraph.exception.NotAllowException; import com.baidu.hugegraph.job.JobBuilder; @@ -65,13 +67,17 @@ public class SchemaTransaction extends IndexableTransaction { - private SchemaIndexTransaction indexTx; + private final SchemaIndexTransaction indexTx; + private final SystemSchemaStore systemSchemaStore; + private final LocalCounter counter; public SchemaTransaction(HugeGraphParams graph, BackendStore store) { super(graph, store); this.autoCommit(true); this.indexTx = new SchemaIndexTransaction(graph, store); + this.systemSchemaStore = store.systemSchemaStore(); + this.counter = graph.counter(); } @Override @@ -278,6 +284,12 @@ protected void addSchema(SchemaElement schema) { schema.type(), schema.id()); setCreateTimeIfNeeded(schema); + // System schema just put into SystemSchemaStore in memory + if (schema.longId() < 0L) { + this.systemSchemaStore.add(schema); + return; + } + LockUtil.Locks locks = new LockUtil.Locks(this.params().name()); try { locks.lockWrites(LockUtil.hugeType2Group(schema.type()), @@ -294,6 +306,11 @@ protected void addSchema(SchemaElement schema) { protected T getSchema(HugeType type, Id id) { LOG.debug("SchemaTransaction get {} by id '{}'", type.readableName(), id); + // System schema just get from SystemSchemaStore in memory + if (id.asLong() < 0L) { + return this.systemSchemaStore.get(id); + } + this.beforeRead(); BackendEntry entry = this.query(type, id); if (entry == null) { @@ -315,6 +332,11 @@ protected T getSchema(HugeType type, String name) { LOG.debug("SchemaTransaction get {} by name '{}'", type.readableName(), name); + // System schema just get from SystemSchemaStore in memory + if (Graph.Hidden.isHidden(name)) { + return this.systemSchemaStore.get(name); + } + this.beforeRead(); ConditionQuery query = new ConditionQuery(type); @@ -350,6 +372,12 @@ protected List getAllSchema(HugeType type) { protected void removeSchema(SchemaElement schema) { LOG.debug("SchemaTransaction remove {} by id '{}'", schema.type(), schema.id()); + // System schema just remove from SystemSchemaStore in memory + if (schema.longId() < 0L) { + throw new IllegalStateException("Deletion of system metadata " + + "should not occur"); + } + LockUtil.Locks locks = new LockUtil.Locks(this.graphName()); try { locks.lockWrites(LockUtil.hugeType2Group(schema.type()), @@ -464,7 +492,7 @@ public void setNextIdLowest(HugeType type, long lowest) { @Watched(prefix = "schema") public Id getNextSystemId() { LOG.debug("SchemaTransaction get next system id"); - Id id = this.store().nextId(HugeType.SYS_SCHEMA); + Id id = this.counter.nextId(HugeType.SYS_SCHEMA); return IdGenerator.of(-id.asLong()); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java index e4a65c608f..c46d79bf3d 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/config/CoreOptions.java @@ -72,30 +72,6 @@ public static synchronized CoreOptions instance() { "hugegraph" ); - public static final ConfigOption STORE_SYSTEM = - new ConfigOption<>( - "store.system", - "The system table name, which store system data.", - disallowEmpty(), - "s" - ); - - public static final ConfigOption STORE_SCHEMA = - new ConfigOption<>( - "store.schema", - "The schema table name, which store meta data.", - disallowEmpty(), - "m" - ); - - public static final ConfigOption STORE_GRAPH = - new ConfigOption<>( - "store.graph", - "The graph table name, which store vertex, edge and property.", - disallowEmpty(), - "g" - ); - public static final ConfigOption SERIALIZER = new ConfigOption<>( "serializer", diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/task/ServerInfoManager.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/task/ServerInfoManager.java index 807673eae2..d8aef5fe8f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/task/ServerInfoManager.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/task/ServerInfoManager.java @@ -24,7 +24,6 @@ import java.util.Collection; import java.util.Iterator; import java.util.Map; -import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; @@ -40,7 +39,6 @@ import com.baidu.hugegraph.backend.query.ConditionQuery; import com.baidu.hugegraph.backend.query.QueryResults; import com.baidu.hugegraph.backend.tx.GraphTransaction; -import com.baidu.hugegraph.event.EventListener; import com.baidu.hugegraph.exception.ConnectionException; import com.baidu.hugegraph.iterator.ListIterator; import com.baidu.hugegraph.iterator.MapperIterator; @@ -52,10 +50,8 @@ import com.baidu.hugegraph.type.define.NodeRole; import com.baidu.hugegraph.util.DateUtil; import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.Events; import com.baidu.hugegraph.util.Log; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; public class ServerInfoManager { @@ -66,7 +62,6 @@ public class ServerInfoManager { private final HugeGraphParams graph; private final ExecutorService dbExecutor; - private final EventListener eventListener; private Id selfServerId; private NodeRole selfServerRole; @@ -82,8 +77,6 @@ public ServerInfoManager(HugeGraphParams graph, this.graph = graph; this.dbExecutor = dbExecutor; - this.eventListener = this.listenChanges(); - this.selfServerId = null; this.selfServerRole = NodeRole.MASTER; @@ -91,32 +84,12 @@ public ServerInfoManager(HugeGraphParams graph, this.closed = false; } - private EventListener listenChanges() { - // Listen store event: "store.inited" - Set storeEvents = ImmutableSet.of(Events.STORE_INITED); - EventListener eventListener = event -> { - // Ensure server info schema create after system info initialized - if (storeEvents.contains(event.name())) { - try { - this.initSchemaIfNeeded(); - } finally { - this.graph.closeTx(); - } - return true; - } - return false; - }; - this.graph.loadSystemStore().provider().listen(eventListener); - return eventListener; - } - - private void unlistenChanges() { - this.graph.loadSystemStore().provider().unlisten(this.eventListener); + public void init() { + HugeServerInfo.schema(this.graph).initSchemaIfNeeded(); } public boolean close() { this.closed = true; - this.unlistenChanges(); if (!this.dbExecutor.isShutdown()) { this.removeSelfServerInfo(); this.call(() -> { @@ -162,6 +135,7 @@ public synchronized void initServerInfo(Id server, NodeRole role) { HugeServerInfo serverInfo = new HugeServerInfo(server, role); serverInfo.maxLoad(this.calcMaxLoad()); + // TODO: save ServerInfo at AuthServer this.save(serverInfo); LOG.info("Init server info: {}", serverInfo); @@ -251,10 +225,6 @@ protected synchronized HugeServerInfo pickWorkerNode( return serverWithMinLoad; } - private void initSchemaIfNeeded() { - HugeServerInfo.schema(this.graph).initSchemaIfNeeded(); - } - private GraphTransaction tx() { assert Thread.currentThread().getName().contains("server-info-db-worker"); return this.graph.systemTransaction(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/task/StandardTaskScheduler.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/task/StandardTaskScheduler.java index 0db34f6fdc..f7fa400ddd 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/task/StandardTaskScheduler.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/task/StandardTaskScheduler.java @@ -24,7 +24,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; @@ -47,7 +46,6 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.tx.GraphTransaction; import com.baidu.hugegraph.config.CoreOptions; -import com.baidu.hugegraph.event.EventListener; import com.baidu.hugegraph.exception.ConnectionException; import com.baidu.hugegraph.exception.NotFoundException; import com.baidu.hugegraph.iterator.ExtendableIterator; @@ -66,10 +64,8 @@ import com.baidu.hugegraph.type.define.DataType; import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.Events; import com.baidu.hugegraph.util.Log; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; public class StandardTaskScheduler implements TaskScheduler { @@ -81,7 +77,6 @@ public class StandardTaskScheduler implements TaskScheduler { private final ExecutorService taskExecutor; private final ExecutorService taskDbExecutor; - private final EventListener eventListener; private final Map> tasks; private volatile TaskTransaction taskTx; @@ -107,8 +102,6 @@ public StandardTaskScheduler(HugeGraphParams graph, this.tasks = new ConcurrentHashMap<>(); this.taskTx = null; - - this.eventListener = this.listenChanges(); } @Override @@ -133,9 +126,9 @@ private TaskTransaction tx() { * this lock through scheduleTasks(), then query tasks and wait * for db-worker thread after call(), the tx may not be initialized * but can't catch this lock, then cause dead lock. - * We just use this.eventListener as a monitor here + * We just use this.serverManager as a monitor here */ - synchronized (this.eventListener) { + synchronized (this.serverManager) { if (this.taskTx == null) { BackendStore store = this.graph.loadSystemStore(); TaskTransaction tx = new TaskTransaction(this.graph, store); @@ -148,25 +141,6 @@ private TaskTransaction tx() { return this.taskTx; } - private EventListener listenChanges() { - // Listen store event: "store.inited" - Set storeEvents = ImmutableSet.of(Events.STORE_INITED); - EventListener eventListener = event -> { - // Ensure task schema create after system info initialized - if (storeEvents.contains(event.name())) { - this.call(() -> this.tx().initSchema()); - return true; - } - return false; - }; - this.graph.loadSystemStore().provider().listen(eventListener); - return eventListener; - } - - private void unlistenChanges() { - this.graph.loadSystemStore().provider().unlisten(this.eventListener); - } - @Override public void restoreTasks() { Id selfServer = this.serverManager().selfServerId(); @@ -479,9 +453,13 @@ public void save(HugeTask task) { }); } + @Override + public void init() { + this.call(() -> this.tx().initSchema()); + } + @Override public boolean close() { - this.unlistenChanges(); if (!this.taskDbExecutor.isShutdown()) { this.call(() -> { try { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/task/TaskScheduler.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/task/TaskScheduler.java index da67f13b6b..ce8a8e5b31 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/task/TaskScheduler.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/task/TaskScheduler.java @@ -48,6 +48,7 @@ public interface TaskScheduler { public Iterator> tasks(TaskStatus status, long limit, String page); + public void init(); public boolean close(); public HugeTask waitUntilTaskCompleted(Id id, long seconds) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/type/HugeType.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/type/HugeType.java index 9d03a7ab31..9630416559 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/type/HugeType.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/type/HugeType.java @@ -34,6 +34,7 @@ public enum HugeType implements SerialEnum { PROPERTY_KEY(3, "PK"), INDEX_LABEL(4, "IL"), + META(40, "M"), COUNTER(50, "C"), /* Data types */ diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/type/define/HugeKeys.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/type/define/HugeKeys.java index 488312a69b..035bf85cff 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/type/define/HugeKeys.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/type/define/HugeKeys.java @@ -28,6 +28,8 @@ public enum HugeKeys { NAME(2, "name"), TIMESTANMP(3, "timestamp"), SCHEMA_TYPE(4, "schema_type"), + VALUE(5, "value"), + VERSION(6, "version"), USER_DATA(10, "user_data"), STATUS(11, "status"), diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/Events.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/Events.java index 70bbd0469c..0e594e5fef 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/Events.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/Events.java @@ -28,5 +28,4 @@ public final class Events { public static final String STORE_INIT = "store.init"; public static final String STORE_CLEAR = "store.clear"; public static final String STORE_TRUNCATE = "store.truncate"; - public static final String STORE_INITED = "store.inited"; } diff --git a/hugegraph-dist/src/main/java/com/baidu/hugegraph/cmd/InitStore.java b/hugegraph-dist/src/main/java/com/baidu/hugegraph/cmd/InitStore.java index a51e66ad26..86653e1fa9 100644 --- a/hugegraph-dist/src/main/java/com/baidu/hugegraph/cmd/InitStore.java +++ b/hugegraph-dist/src/main/java/com/baidu/hugegraph/cmd/InitStore.java @@ -19,6 +19,7 @@ package com.baidu.hugegraph.cmd; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; @@ -32,7 +33,7 @@ import com.baidu.hugegraph.HugeFactory; import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.auth.StandardAuthenticator; -import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; +import com.baidu.hugegraph.backend.store.BackendStoreInfo; import com.baidu.hugegraph.config.CoreOptions; import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.config.ServerOptions; @@ -90,39 +91,41 @@ public static void main(String[] args) throws Exception { E.checkArgument(!graphNames.isEmpty(), "Must contain at least one graph"); - for (ConfigurationNode graphName : graphNames) { - @SuppressWarnings("unchecked") - String name = ((Map.Entry) - graphName.getReference()).getKey(); - HugeFactory.checkGraphName(name, "gremlin-server.yaml"); - String configPath = graphName.getValue().toString(); - initGraph(configPath); + List graphs = new ArrayList<>(graphNames.size()); + try { + for (ConfigurationNode graphName : graphNames) { + @SuppressWarnings("unchecked") + String name = ((Map.Entry) + graphName.getReference()).getKey(); + HugeFactory.checkGraphName(name, "gremlin-server.yaml"); + String configPath = graphName.getValue().toString(); + graphs.add(initGraph(configPath)); + } + StandardAuthenticator.initAdminUserIfNeeded(restConfFile); + } finally { + for (HugeGraph graph : graphs) { + graph.close(); + } } - - StandardAuthenticator.initAdminUserIfNeeded(restConfFile); - HugeFactory.shutdown(30L); } - private static void initGraph(String configPath) throws Exception { + private static HugeGraph initGraph(String configPath) throws Exception { LOG.info("Init graph with config file: {}", configPath); HugeConfig config = new HugeConfig(configPath); // Forced set RAFT_MODE to false when initializing backend config.setProperty(CoreOptions.RAFT_MODE.name(), "false"); HugeGraph graph = (HugeGraph) GraphFactory.open(config); - BackendStoreSystemInfo sysInfo = graph.backendStoreSystemInfo(); - try { - if (sysInfo.exists()) { - LOG.info("Skip init-store due to the backend store of '{}' " + - "had been initialized", graph.name()); - sysInfo.checkVersion(); - } else { - initBackend(graph); - } - } finally { - graph.close(); + BackendStoreInfo backendStoreInfo = graph.backendStoreInfo(); + if (backendStoreInfo.exists()) { + LOG.info("Skip init-store due to the backend store of '{}' " + + "had been initialized", graph.name()); + backendStoreInfo.checkVersion(); + } else { + initBackend(graph); } + return graph; } private static void initBackend(final HugeGraph graph) diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java index 4dedf66050..a9bbe29689 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java @@ -540,5 +540,45 @@ public long getCounter(HugeType type) { throw new UnsupportedOperationException( "HbaseGraphStore.getCounter()"); } + + public Session getSession() { + return super.sessions.session(); + } + } + + public static class HbaseSystemStore extends HbaseGraphStore { + + private final HbaseTables.Meta meta; + + public HbaseSystemStore(BackendStoreProvider provider, + String namespace, String store) { + super(provider, namespace, store); + + this.meta = new HbaseTables.Meta(); + } + + @Override + protected List tableNames() { + List tableNames = super.tableNames(); + tableNames.add(this.meta.table()); + return tableNames; + } + + @Override + public void init() { + super.init(); + super.checkOpened(); + Session session = super.getSession(); + String driverVersion = this.provider().driverVersion(); + this.meta.writeVersion(session, driverVersion); + LOG.info("Write down the backend version: {}", driverVersion); + } + + @Override + public String storedVersion() { + super.checkOpened(); + Session session = super.getSession(); + return this.meta.readVersion(session); + } } } diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java index f1c6634d60..02035ba7c0 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java @@ -23,6 +23,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.hbase.HbaseStore.HbaseGraphStore; import com.baidu.hugegraph.backend.store.hbase.HbaseStore.HbaseSchemaStore; +import com.baidu.hugegraph.backend.store.hbase.HbaseStore.HbaseSystemStore; public class HbaseStoreProvider extends AbstractBackendStoreProvider { @@ -40,13 +41,18 @@ protected BackendStore newGraphStore(String store) { return new HbaseGraphStore(this, this.namespace(), store); } + @Override + protected BackendStore newSystemStore(String store) { + return new HbaseSystemStore(this, this.namespace(), store); + } + @Override public String type() { return "hbase"; } @Override - public String version() { + public String driverVersion() { /* * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking @@ -63,7 +69,8 @@ public String version() { * [1.9] #894: encode label id in string index * [1.10] #295: support ttl for vertex and edge * [1.11] #1333: support read frequency for property key + * [1.12] #1533: add meta table in system store */ - return "1.11"; + return "1.12"; } } diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTables.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTables.java index cc029b2b74..089bde2b50 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTables.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTables.java @@ -38,10 +38,43 @@ import com.baidu.hugegraph.backend.store.hbase.HbaseSessions.RowIterator; import com.baidu.hugegraph.backend.store.hbase.HbaseSessions.Session; import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.NumericUtil; +import com.baidu.hugegraph.util.StringEncoding; public class HbaseTables { + public static class Meta extends HbaseTable { + + private static final String TABLE = HugeType.META.string(); + private static final byte[] COL = Bytes.toBytes(TABLE); + + public Meta() { + super(TABLE); + } + + public void writeVersion(Session session, String version) { + byte[] key = new byte[]{HugeKeys.VERSION.code()}; + byte[] value = StringEncoding.encode(version); + session.put(this.table(), CF, key, COL, value); + try { + session.commit(); + } catch (Exception e) { + session.rollback(); + } + } + + public String readVersion(Session session) { + byte[] key = new byte[]{HugeKeys.VERSION.code()}; + RowIterator results = session.get(this.table(), CF, key); + if (!results.hasNext()) { + return null; + } + Result row = results.next(); + return StringEncoding.decode(row.getValue(CF, COL)); + } + } + public static class Counters extends HbaseTable { private static final String TABLE = HugeType.COUNTER.string(); diff --git a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStore.java b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStore.java index b44e228b59..4bf1480ca4 100644 --- a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStore.java +++ b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStore.java @@ -470,5 +470,46 @@ public long getCounter(HugeType type) { throw new UnsupportedOperationException( "MysqlGraphStore.getCounter()"); } + + public Session getSession() { + return super.sessions.session(); + } + } + + public static class MysqlSystemStore extends MysqlGraphStore { + + private final MysqlTables.Meta meta; + + public MysqlSystemStore(BackendStoreProvider provider, + String database, String store) { + super(provider, database, store); + + this.meta = new MysqlTables.Meta(); + } + + @Override + public void init() { + super.init(); + this.checkOpened(); + Session session = this.getSession(); + String driverVersion = this.provider().driverVersion(); + this.meta.writeVersion(session, driverVersion); + LOG.info("Write down the backend version: {}", driverVersion); + } + + @Override + public String storedVersion() { + super.init(); + this.checkOpened(); + Session session = this.getSession(); + return this.meta.readVersion(session); + } + + @Override + protected Collection tables() { + List tables = new ArrayList<>(super.tables()); + tables.add(this.meta); + return tables; + } } } diff --git a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStoreProvider.java b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStoreProvider.java index b07b6b101a..45153e2522 100644 --- a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStoreProvider.java +++ b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStoreProvider.java @@ -23,6 +23,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.mysql.MysqlStore.MysqlGraphStore; import com.baidu.hugegraph.backend.store.mysql.MysqlStore.MysqlSchemaStore; +import com.baidu.hugegraph.backend.store.mysql.MysqlStore.MysqlSystemStore; public class MysqlStoreProvider extends AbstractBackendStoreProvider { @@ -40,13 +41,18 @@ protected BackendStore newGraphStore(String store) { return new MysqlGraphStore(this, this.database(), store); } + @Override + protected BackendStore newSystemStore(String store) { + return new MysqlSystemStore(this, this.database(), store); + } + @Override public String type() { return "mysql"; } @Override - public String version() { + public String driverVersion() { /* * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking @@ -63,7 +69,8 @@ public String version() { * instead of sortable B64 * [1.9] #295: support ttl for vertex and edge * [1.10] #1333: support read frequency for property key + * [1.11] #1533: add meta table in system store */ - return "1.10"; + return "1.11"; } } diff --git a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlTables.java b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlTables.java index e9e5fa969f..3c78549aeb 100644 --- a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlTables.java +++ b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlTables.java @@ -85,6 +85,54 @@ public TableDefine tableDefine() { } } + public static class Meta extends MysqlTableTemplate { + + public static final String TABLE = HugeType.META.string(); + + public Meta() { + this(TYPES_MAPPING); + } + + public Meta(Map typesMapping) { + super(TABLE); + + this.define = new TableDefine(typesMapping); + this.define.column(HugeKeys.NAME, SMALL_TEXT); + this.define.column(HugeKeys.VALUE, MID_TEXT); + this.define.keys(HugeKeys.NAME); + } + + public void writeVersion(Session session, String driverVersion) { + String insert = String.format("INSERT INTO %s VALUES ('%s', '%s')", + this.table(), + formatKey(HugeKeys.VERSION), + driverVersion); + try { + session.execute(insert); + } catch (SQLException throwables) { + throw new BackendException("Failed to insert driver version " + + "with '%s'", insert); + } + } + + public String readVersion(Session session) { + String select = String.format("SELECT %s FROM %s WHERE %s = '%s'", + formatKey(HugeKeys.VALUE), + this.table(), formatKey(HugeKeys.NAME), + formatKey(HugeKeys.VERSION)); + try { + ResultSet resultSet = session.select(select); + if (!resultSet.next()) { + return null; + } + return resultSet.getString(formatKey(HugeKeys.VALUE)); + } catch (SQLException e) { + throw new BackendException( + "Failed to get stored version with '%s'", e, select); + } + } + } + public static class Counters extends MysqlTableTemplate { public static final String TABLE = HugeType.COUNTER.string(); diff --git a/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloStoreProvider.java b/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloStoreProvider.java index 27bdde28b7..da0fda928e 100644 --- a/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloStoreProvider.java +++ b/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloStoreProvider.java @@ -48,7 +48,7 @@ public String type() { } @Override - public String version() { + public String driverVersion() { /* * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking @@ -64,8 +64,9 @@ public String version() { * instead of sortable B64 * [1.8] #295: support ttl for vertex and edge * [1.9] #1333: support read frequency for property key + * [1.10] #1533: add meta table in system store */ - return "1.9"; + return "1.10"; } public static class PaloSchemaStore extends PaloStore { diff --git a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlStoreProvider.java b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlStoreProvider.java index 04a5c30851..7924659c96 100644 --- a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlStoreProvider.java +++ b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlStoreProvider.java @@ -23,6 +23,8 @@ import java.util.Collection; import java.util.List; +import org.slf4j.Logger; + import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.BackendStoreProvider; @@ -31,9 +33,12 @@ import com.baidu.hugegraph.backend.store.mysql.MysqlTable; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.Directions; +import com.baidu.hugegraph.util.Log; public class PostgresqlStoreProvider extends MysqlStoreProvider { + private static final Logger LOG = Log.logger(PostgresqlStore.class); + @Override protected BackendStore newSchemaStore(String store) { return new PostgresqlSchemaStore(this, this.database(), store); @@ -44,13 +49,18 @@ protected BackendStore newGraphStore(String store) { return new PostgresqlGraphStore(this, this.database(), store); } + @Override + protected BackendStore newSystemStore(String store) { + return new PostgresqlSystemStore(this, this.database(), store); + } + @Override public String type() { return "postgresql"; } @Override - public String version() { + public String driverVersion() { /* * Versions history: * [1.0] #441: supports PostgreSQL and Cockroach backend @@ -65,8 +75,9 @@ public String version() { * instead of sortable B64 * [1.7] #295: support ttl for vertex and edge * [1.8] #1333: support read frequency for property key + * [1.9] #1533: add meta table in system store */ - return "1.8"; + return "1.9"; } public static class PostgresqlSchemaStore extends PostgresqlStore { @@ -171,4 +182,41 @@ public long getCounter(HugeType type) { "PostgresqlGraphStore.getCounter()"); } } + + public static class PostgresqlSystemStore extends PostgresqlGraphStore { + + private final PostgresqlTables.Meta meta; + + public PostgresqlSystemStore(BackendStoreProvider provider, + String database, String store) { + super(provider, database, store); + + this.meta = new PostgresqlTables.Meta(); + } + + @Override + public void init() { + super.init(); + this.checkOpened(); + MysqlSessions.Session session = this.session(HugeType.META); + String driverVersion = this.provider().driverVersion(); + this.meta.writeVersion(session, driverVersion); + LOG.info("Write down the backend version: {}", driverVersion); + } + + @Override + public String storedVersion() { + super.init(); + this.checkOpened(); + MysqlSessions.Session session = this.session(HugeType.META); + return this.meta.readVersion(session); + } + + @Override + protected Collection tables() { + List tables = new ArrayList<>(super.tables()); + tables.add(this.meta); + return tables; + } + } } diff --git a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlTables.java b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlTables.java index 6442014adb..249b450ca2 100644 --- a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlTables.java +++ b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlTables.java @@ -19,6 +19,15 @@ package com.baidu.hugegraph.backend.store.postgresql; +import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.BOOLEAN; +import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.HUGE_TEXT; +import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.INT; +import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.LARGE_TEXT; +import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.MID_TEXT; +import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.NUMERIC; +import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.SMALL_TEXT; +import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.TINYINT; + import java.sql.SQLException; import java.util.List; import java.util.Map; @@ -37,15 +46,6 @@ import jersey.repackaged.com.google.common.collect.ImmutableMap; -import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.BOOLEAN; -import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.HUGE_TEXT; -import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.INT; -import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.LARGE_TEXT; -import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.MID_TEXT; -import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.NUMERIC; -import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.SMALL_TEXT; -import static com.baidu.hugegraph.backend.store.mysql.MysqlTables.TINYINT; - public class PostgresqlTables { private static final Map TYPES_MAPPING = @@ -75,6 +75,23 @@ public TableDefine tableDefine() { } } + public static class Meta extends PostgresqlTableTemplate { + + public Meta() { + super(new MysqlTables.Meta(TYPES_MAPPING)); + } + + public void writeVersion(Session session, String version) { + MysqlTables.Meta table = (MysqlTables.Meta) this.template; + table.writeVersion(session, version); + } + + public String readVersion(Session session) { + MysqlTables.Meta table = (MysqlTables.Meta) this.template; + return table.readVersion(session); + } + } + public static class Counters extends PostgresqlTableTemplate { public Counters() { 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 1fc97c37e4..1db48f449e 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 @@ -265,7 +265,6 @@ protected RocksDBSessions open(HugeConfig config, List tableNames) { protected RocksDBSessions open(HugeConfig config, String dataPath, String walPath, List tableNames) { LOG.info("Opening RocksDB with data path: {}", dataPath); - RocksDBSessions sessions = null; try { sessions = this.openSessionPool(config, dataPath, @@ -365,6 +364,10 @@ protected Map tableDBMapping() { return tableDBMap; } + protected ReadWriteLock storeLock() { + return this.storeLock; + } + @Override public void close() { LOG.debug("Store close: {}", this.store); @@ -941,5 +944,57 @@ public long getCounter(HugeType type) { throw new UnsupportedOperationException( "RocksDBGraphStore.getCounter()"); } + + public Session getSession() { + return super.sessions.session(); + } + } + + public static class RocksDBSystemStore extends RocksDBGraphStore { + + private final RocksDBTables.Meta meta; + + public RocksDBSystemStore(BackendStoreProvider provider, + String database, String store) { + super(provider, database, store); + + this.meta = new RocksDBTables.Meta(database); + } + + @Override + public synchronized void init() { + super.init(); + Lock writeLock = this.storeLock().writeLock(); + writeLock.lock(); + try { + super.checkOpened(); + Session session = super.getSession(); + String driverVersion = this.provider().driverVersion(); + this.meta.writeVersion(session, driverVersion); + LOG.info("Write down the backend version: {}", driverVersion); + } finally { + writeLock.unlock(); + } + } + + @Override + public String storedVersion() { + Lock readLock = this.storeLock().readLock(); + readLock.lock(); + try { + super.checkOpened(); + Session session = super.getSession(); + return this.meta.readVersion(session); + } finally { + readLock.unlock(); + } + } + + @Override + protected List tableNames() { + List tableNames = super.tableNames(); + tableNames.add(this.meta.table()); + return tableNames; + } } } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStoreProvider.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStoreProvider.java index c4206eb2f3..3c6db48617 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStoreProvider.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStoreProvider.java @@ -23,6 +23,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.rocksdb.RocksDBStore.RocksDBGraphStore; import com.baidu.hugegraph.backend.store.rocksdb.RocksDBStore.RocksDBSchemaStore; +import com.baidu.hugegraph.backend.store.rocksdb.RocksDBStore.RocksDBSystemStore; public class RocksDBStoreProvider extends AbstractBackendStoreProvider { @@ -40,13 +41,18 @@ protected BackendStore newGraphStore(String store) { return new RocksDBGraphStore(this, this.database(), store); } + @Override + protected BackendStore newSystemStore(String store) { + return new RocksDBSystemStore(this, this.database(), store); + } + @Override public String type() { return "rocksdb"; } @Override - public String version() { + public String driverVersion() { /* * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking @@ -62,7 +68,8 @@ public String version() { * [1.8] #894: encode label id in string index * [1.9] #295: support ttl for vertex and edge * [1.10] #1333: support read frequency for property key + * [1.11] #1533: add meta table in system store */ - return "1.10"; + return "1.11"; } } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTables.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTables.java index 5971bcb638..29959af9b9 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTables.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTables.java @@ -34,9 +34,40 @@ import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.StringEncoding; public class RocksDBTables { + public static class Meta extends RocksDBTable { + + private static final String TABLE = HugeType.META.string(); + + public Meta(String database) { + super(database, TABLE); + } + + public void writeVersion(Session session, String version) { + byte[] key = new byte[]{HugeKeys.VERSION.code()}; + byte[] value = StringEncoding.encode(version); + session.put(this.table(), key, value); + try { + session.commit(); + } catch (Exception e) { + session.rollback(); + throw e; + } + } + + public String readVersion(Session session) { + byte[] key = new byte[]{HugeKeys.VERSION.code()}; + byte[] value = session.get(this.table(), key); + if (value == null) { + return null; + } + return StringEncoding.decode(value); + } + } + public static class Counters extends RocksDBTable { private static final String TABLE = HugeType.COUNTER.string(); diff --git a/hugegraph-scylladb/src/main/java/com/baidu/hugegraph/backend/store/scylladb/ScyllaDBStoreProvider.java b/hugegraph-scylladb/src/main/java/com/baidu/hugegraph/backend/store/scylladb/ScyllaDBStoreProvider.java index 9e18b15633..b5a54fa421 100644 --- a/hugegraph-scylladb/src/main/java/com/baidu/hugegraph/backend/store/scylladb/ScyllaDBStoreProvider.java +++ b/hugegraph-scylladb/src/main/java/com/baidu/hugegraph/backend/store/scylladb/ScyllaDBStoreProvider.java @@ -45,7 +45,8 @@ public String type() { } @Override - public BackendStore loadSchemaStore(String name) { + public BackendStore loadSchemaStore() { + String name = SCHEMA_STORE; LOG.debug("ScyllaDBStoreProvider load SchemaStore '{}'", name); if (!this.stores.containsKey(name)) { @@ -61,7 +62,8 @@ public BackendStore loadSchemaStore(String name) { } @Override - public BackendStore loadGraphStore(String name) { + public BackendStore loadGraphStore() { + String name = GRAPH_STORE; LOG.debug("ScyllaDBStoreProvider load GraphStore '{}'", name); if (!this.stores.containsKey(name)) { @@ -76,6 +78,23 @@ public BackendStore loadGraphStore(String name) { return store; } + @Override + public BackendStore loadSystemStore() { + String name = SYSTEM_STORE; + LOG.debug("ScyllaDBStoreProvider load SystemStore '{}'", name); + + if (!this.stores.containsKey(name)) { + BackendStore s = new ScyllaDBSystemStore(this, keyspace(), name); + this.stores.putIfAbsent(name, s); + } + + BackendStore store = this.stores.get(name); + E.checkNotNull(store, "store"); + E.checkState(store instanceof ScyllaDBSystemStore, + "SystemStore must be an instance of ScyllaDBSystemStore"); + return store; + } + public static class ScyllaDBSchemaStore extends CassandraStore.CassandraSchemaStore { @@ -133,4 +152,25 @@ protected CassandraMetrics createMetrics(HugeConfig conf, return new ScyllaDBMetrics(conf, sessions, keyspace); } } + + public static class ScyllaDBSystemStore + extends CassandraStore.CassandraSystemStore { + + public ScyllaDBSystemStore(BackendStoreProvider provider, + String keyspace, String store) { + super(provider, keyspace, store); + } + + @Override + public BackendFeatures features() { + return FEATURES; + } + + @Override + protected CassandraMetrics createMetrics(HugeConfig conf, + CassandraSessionPool sessions, + String keyspace) { + return new ScyllaDBMetrics(conf, sessions, keyspace); + } + } } diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/MultiGraphsTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/MultiGraphsTest.java index 7000d31c8d..3e12be9970 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/MultiGraphsTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/MultiGraphsTest.java @@ -37,6 +37,7 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.BackendException; import com.baidu.hugegraph.backend.id.IdGenerator; +import com.baidu.hugegraph.backend.store.BackendStoreInfo; import com.baidu.hugegraph.backend.store.rocksdb.RocksDBOptions; import com.baidu.hugegraph.config.CoreOptions; import com.baidu.hugegraph.exception.ExistedException; @@ -56,6 +57,23 @@ public class MultiGraphsTest { private static final String NAME48 = "g12345678901234567890123456789012345678901234567"; + @Test + public void testWriteAndReadVersion() { + List graphs = openGraphs("g_1", NAME48); + for (HugeGraph graph : graphs) { + graph.initBackend(); + // Init more than once no side effect + graph.initBackend(); + + BackendStoreInfo backendStoreInfo = graph.backendStoreInfo(); + Assert.assertTrue(backendStoreInfo.exists()); + Assert.assertTrue(backendStoreInfo.checkVersion()); + + graph.clearBackend(); + } + destoryGraphs(graphs); + } + @Test public void testCreateMultiGraphs() { List graphs = openGraphs("g_1", NAME48); @@ -239,9 +257,9 @@ public void testCreateGraphsWithSameName() { HugeGraph g3 = graphs.get(2); g1.initBackend(); - Assert.assertTrue(g1.backendStoreSystemInfo().exists()); - Assert.assertTrue(g2.backendStoreSystemInfo().exists()); - Assert.assertTrue(g3.backendStoreSystemInfo().exists()); + Assert.assertTrue(g1.backendStoreInfo().exists()); + Assert.assertTrue(g2.backendStoreInfo().exists()); + Assert.assertTrue(g3.backendStoreInfo().exists()); g2.initBackend(); // no error g3.initBackend(); diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/tinkerpop/TestGraph.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/tinkerpop/TestGraph.java index 22cf61454b..a6701313f2 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/tinkerpop/TestGraph.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/tinkerpop/TestGraph.java @@ -37,7 +37,7 @@ import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.id.IdGenerator; -import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; +import com.baidu.hugegraph.backend.store.BackendStoreInfo; import com.baidu.hugegraph.io.HugeGraphIoRegistry; import com.baidu.hugegraph.io.HugeGraphSONModule; import com.baidu.hugegraph.perf.PerfUtil.Watched; @@ -79,7 +79,7 @@ public HugeGraph hugegraph() { @Watched protected void initBackend() { - BackendStoreSystemInfo sysInfo = this.graph.backendStoreSystemInfo(); + BackendStoreInfo sysInfo = this.graph.backendStoreInfo(); if (!sysInfo.exists()) { this.graph.initBackend(); } else { diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java index 3317faae51..eb6a167798 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java @@ -22,6 +22,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Suite; +import com.baidu.hugegraph.unit.core.SystemSchemaStoreTest; import com.baidu.hugegraph.unit.util.RateLimiterTest; import com.baidu.hugegraph.unit.cache.CacheManagerTest; import com.baidu.hugegraph.unit.cache.CacheTest; @@ -31,7 +32,7 @@ import com.baidu.hugegraph.unit.cassandra.CassandraTest; import com.baidu.hugegraph.unit.core.AnalyzerTest; import com.baidu.hugegraph.unit.core.BackendMutationTest; -import com.baidu.hugegraph.unit.core.BackendStoreSystemInfoTest; +import com.baidu.hugegraph.unit.core.BackendStoreInfoTest; import com.baidu.hugegraph.unit.core.ConditionQueryFlattenTest; import com.baidu.hugegraph.unit.core.ConditionTest; import com.baidu.hugegraph.unit.core.DataTypeTest; @@ -106,12 +107,13 @@ SecurityManagerTest.class, RolePermissionTest.class, ExceptionTest.class, - BackendStoreSystemInfoTest.class, + BackendStoreInfoTest.class, TraversalUtilTest.class, PageStateTest.class, Int2IntsMapTest.class, ObjectIntMappingTest.class, IdSetTest.class, + SystemSchemaStoreTest.class, /* serializer */ BytesBufferTest.class, diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/BackendStoreInfoTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/BackendStoreInfoTest.java new file mode 100644 index 0000000000..9c0b82912d --- /dev/null +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/BackendStoreInfoTest.java @@ -0,0 +1,47 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.unit.core; + +import org.junit.Test; +import org.mockito.Mockito; + +import com.baidu.hugegraph.backend.store.BackendStore; +import com.baidu.hugegraph.backend.store.BackendStoreInfo; +import com.baidu.hugegraph.backend.store.BackendStoreProvider; +import com.baidu.hugegraph.testutil.Assert; + +public class BackendStoreInfoTest { + + @Test + public void testBackendStoreInfo() { + BackendStoreProvider provider = Mockito.mock(BackendStoreProvider.class); + BackendStore store = Mockito.mock(BackendStore.class); + + Mockito.when(provider.initialized()).thenReturn(true); + Mockito.when(provider.loadSystemStore()).thenReturn(store); + Mockito.when(store.storedVersion()).thenReturn("1.11"); + + BackendStoreInfo backendStoreInfo = new BackendStoreInfo(provider); + Assert.assertTrue(backendStoreInfo.exists()); + + Mockito.when(provider.driverVersion()).thenReturn("1.10"); + Assert.assertFalse(backendStoreInfo.checkVersion()); + } +} diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/BackendStoreSystemInfoTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/SystemSchemaStoreTest.java similarity index 51% rename from hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/BackendStoreSystemInfoTest.java rename to hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/SystemSchemaStoreTest.java index 8b388db389..92778c5361 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/BackendStoreSystemInfoTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/SystemSchemaStoreTest.java @@ -22,34 +22,38 @@ import org.junit.Test; import org.mockito.Mockito; -import com.baidu.hugegraph.HugeException; import com.baidu.hugegraph.HugeGraph; -import com.baidu.hugegraph.backend.store.BackendStoreSystemInfo; -import com.baidu.hugegraph.backend.tx.SchemaTransaction; +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.backend.id.IdGenerator; +import com.baidu.hugegraph.backend.store.SystemSchemaStore; +import com.baidu.hugegraph.schema.SchemaElement; +import com.baidu.hugegraph.schema.VertexLabel; import com.baidu.hugegraph.testutil.Assert; import com.baidu.hugegraph.testutil.Whitebox; -public class BackendStoreSystemInfoTest { - - private static final String PK_BACKEND_INFO = "~backend_info"; +public class SystemSchemaStoreTest { @Test - public void testBackendStoreSystemInfoIllegalStateException() { - HugeGraph graph = Mockito.mock(HugeGraph.class); - SchemaTransaction stx = Mockito.mock(SchemaTransaction.class); - Mockito.when(stx.getPropertyKey(PK_BACKEND_INFO)) - .thenThrow(new IllegalStateException("Should not exist schema " + - "with same name '~backend_info'")); - Mockito.when(stx.graph()).thenReturn(graph); - Mockito.when(stx.storeInitialized()).thenReturn(true); + public void testExpandCapacity() { + SystemSchemaStore store = new SystemSchemaStore(); + SchemaElement[] storeByIds = Whitebox.getInternalState(store, + "storeByIds"); + int initCapacity = storeByIds.length; - BackendStoreSystemInfo info = new BackendStoreSystemInfo(stx); - - Assert.assertThrows(HugeException.class, () -> { - Whitebox.invoke(BackendStoreSystemInfo.class, "info", info); - }, e -> { - Assert.assertContains("There exists multiple backend info", - e.getMessage()); - }); + int num = initCapacity + 1; + HugeGraph graph = Mockito.mock(HugeGraph.class); + for (int i = 1; i <= num; i++) { + Id id = IdGenerator.of(-i); + String name = "name-" + i; + store.add(new VertexLabel(graph, id, name)); + } + + for (int i = 1; i <= num; i++) { + Id id = IdGenerator.of(-i); + String name = "name-" + i; + VertexLabel vlById = store.get(id); + VertexLabel vlByName = store.get(name); + Assert.assertEquals(vlById, vlByName); + } } }