You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hugegraph.apache.org by je...@apache.org on 2022/06/02 08:42:53 UTC

[incubator-hugegraph] branch master updated: Add system schema store (#1891)

This is an automated email from the ASF dual-hosted git repository.

jermy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hugegraph.git


The following commit(s) were added to refs/heads/master by this push:
     new 2af01bc92 Add system schema store (#1891)
2af01bc92 is described below

commit 2af01bc92ddfd69e654889b9f6e69a729d4f3043
Author: zyxxoo <13...@qq.com>
AuthorDate: Thu Jun 2 16:42:46 2022 +0800

    Add system schema store (#1891)
    
    * Add meta table into system store
    * Add system schema store
    * Add test case for read version
    * Let AbstractBackendStore impl default storedVersion()
    * fix small bug og HugeProject
    * add test for SystemSchemaStore
    * fix initStore schema missed
    
    Change-Id: I5c9d65c4ae022cb84592e8f2aa57bd7485cd12e4
    Co-authored-by: liningrui <li...@baidu.com>
---
 .../baidu/hugegraph/auth/HugeGraphAuthProxy.java   |  30 ++++--
 .../com/baidu/hugegraph/core/GraphManager.java     |   4 +-
 .../backend/store/cassandra/CassandraMetrics.java  |   4 +-
 .../backend/store/cassandra/CassandraStore.java    |  49 +++++++--
 .../store/cassandra/CassandraStoreProvider.java    |   9 +-
 .../backend/store/cassandra/CassandraTables.java   |  42 ++++++++
 .../main/java/com/baidu/hugegraph/HugeGraph.java   |  10 +-
 .../java/com/baidu/hugegraph/HugeGraphParams.java  |   3 +
 .../com/baidu/hugegraph/StandardHugeGraph.java     |  56 ++++++++---
 .../java/com/baidu/hugegraph/auth/AuthManager.java |   2 +
 .../java/com/baidu/hugegraph/auth/HugeProject.java |   2 +-
 .../baidu/hugegraph/auth/StandardAuthManager.java  |  40 ++------
 .../backend/cache/CachedBackendStore.java          |  11 +++
 .../backend/store/AbstractBackendStore.java        |  16 ++-
 .../store/AbstractBackendStoreProvider.java        |  44 ++++++---
 .../hugegraph/backend/store/BackendStore.java      |   6 ++
 .../hugegraph/backend/store/BackendStoreInfo.java  |  58 +++++++++++
 .../backend/store/BackendStoreProvider.java        |  20 ++--
 .../backend/store/BackendStoreSystemInfo.java      | 110 ---------------------
 .../hugegraph/backend/store/SystemSchemaStore.java |  77 +++++++++++++++
 .../backend/store/memory/InMemoryDBStore.java      |  13 +++
 .../store/memory/InMemoryDBStoreProvider.java      |  11 ++-
 .../backend/store/raft/RaftBackendStore.java       |  11 +++
 .../store/raft/RaftBackendStoreProvider.java       |  40 ++++----
 .../hugegraph/backend/store/raft/RaftContext.java  |  15 +--
 .../hugegraph/backend/tx/GraphTransaction.java     |   2 +-
 .../hugegraph/backend/tx/SchemaTransaction.java    |  31 +++++-
 .../com/baidu/hugegraph/config/CoreOptions.java    |  16 ---
 .../com/baidu/hugegraph/schema/VertexLabel.java    |   2 +-
 .../baidu/hugegraph/task/ServerInfoManager.java    |  36 +------
 .../hugegraph/task/StandardTaskScheduler.java      |  36 ++-----
 .../com/baidu/hugegraph/task/TaskScheduler.java    |   2 +
 .../java/com/baidu/hugegraph/type/HugeType.java    |   1 +
 .../com/baidu/hugegraph/type/define/HugeKeys.java  |   2 +
 .../java/com/baidu/hugegraph/cmd/InitStore.java    |  41 ++++----
 .../hugegraph/backend/store/hbase/HbaseStore.java  |  34 +++++++
 .../backend/store/hbase/HbaseStoreProvider.java    |  11 ++-
 .../hugegraph/backend/store/hbase/HbaseTables.java |  33 +++++++
 .../hugegraph/backend/store/mysql/MysqlStore.java  |  35 +++++++
 .../backend/store/mysql/MysqlStoreProvider.java    |  11 ++-
 .../hugegraph/backend/store/mysql/MysqlTables.java |  48 +++++++++
 .../backend/store/palo/PaloStoreProvider.java      |   5 +-
 .../store/postgresql/PostgresqlStoreProvider.java  |  52 +++++++++-
 .../backend/store/postgresql/PostgresqlTables.java |  43 ++++++--
 .../backend/store/rocksdb/RocksDBStore.java        |  52 +++++++++-
 .../store/rocksdb/RocksDBStoreProvider.java        |   9 +-
 .../backend/store/rocksdb/RocksDBTables.java       |  31 ++++++
 .../store/scylladb/ScyllaDBStoreProvider.java      |  44 ++++++++-
 .../java/com/baidu/hugegraph/api/EdgeApiTest.java  |   3 +-
 .../com/baidu/hugegraph/core/MultiGraphsTest.java  |  24 ++++-
 .../com/baidu/hugegraph/tinkerpop/TestGraph.java   |   4 +-
 .../com/baidu/hugegraph/unit/UnitTestSuite.java    |   7 +-
 .../hugegraph/unit/core/BackendStoreInfoTest.java  |  50 ++++++++++
 ...temInfoTest.java => SystemSchemaStoreTest.java} |  48 ++++-----
 54 files changed, 1002 insertions(+), 394 deletions(-)

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 4a6ab9512..d3efb5aa3 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
@@ -74,7 +74,7 @@ import com.baidu.hugegraph.backend.id.Id;
 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;
@@ -604,15 +604,9 @@ public final class HugeGraphAuthProxy implements HugeGraph {
     }
 
     @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
@@ -734,6 +728,12 @@ public final class HugeGraphAuthProxy implements HugeGraph {
         }
     }
 
+    @Override
+    public void initSystemInfo() {
+        this.verifyAdminPermission();
+        this.hugegraph.initSystemInfo();
+    }
+
     @Override
     public void createSnapshot() {
         this.verifyPermission(HugePermission.WRITE, ResourceType.STATUS);
@@ -998,6 +998,12 @@ public final class HugeGraphAuthProxy implements HugeGraph {
             return this.taskScheduler.graph();
         }
 
+        @Override
+        public void init() {
+            verifyAdminPermission();
+            this.taskScheduler.init();
+        }
+
         @Override
         public int pendingTasks() {
             verifyTaskPermission(HugePermission.READ);
@@ -1161,6 +1167,12 @@ public final class HugeGraphAuthProxy implements HugeGraph {
             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 6a4217f4d..749e350aa 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
@@ -49,7 +49,7 @@ import com.baidu.hugegraph.backend.cache.Cache;
 import com.baidu.hugegraph.backend.cache.CacheManager;
 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.config.CoreOptions;
 import com.baidu.hugegraph.config.HugeConfig;
 import com.baidu.hugegraph.config.ServerOptions;
@@ -393,7 +393,7 @@ public final class GraphManager {
                     }
                 }
             }
-            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 bc17e207d..f385c0a73 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
@@ -38,10 +38,10 @@ import org.apache.tinkerpop.gremlin.util.NumberHelper;
 import org.slf4j.Logger;
 
 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;
@@ -77,7 +77,7 @@ public class CassandraMetrics implements BackendMetrics {
         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 90dcd5af4..75bf12f0c 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
@@ -299,7 +299,7 @@ public abstract class CassandraStore
         String tableName = query.olap() ? this.olapTableName(type) :
                                           type.string();
         CassandraTable table = this.table(tableName);
-        Iterator<BackendEntry> entries = table.query(this.session(), query);
+        Iterator<BackendEntry> entries = table.query(this.session(null), query);
         // Merge olap results as needed
         Set<Id> olapPks = query.olapPks();
         if (this.isGraphStore && !olapPks.isEmpty()) {
@@ -307,7 +307,7 @@ public abstract class CassandraStore
             for (Id pk : olapPks) {
                 Query q = query.copy();
                 table = this.table(this.olapTableName(pk));
-                iterators.add(table.query(this.session(), q));
+                iterators.add(table.query(this.session(null), q));
             }
             entries = new MergeIterator<>(entries, iterators,
                                           BackendEntry::mergeable);
@@ -608,11 +608,6 @@ public abstract class CassandraStore
         return this.sessions.session();
     }
 
-    protected CassandraSessionPool.Session session() {
-        this.checkOpened();
-        return this.sessions.session();
-    }
-
     protected final void checkClusterConnected() {
         E.checkState(this.sessions != null && this.sessions.clusterConnected(),
                      "Cassandra cluster has not been connected");
@@ -758,7 +753,7 @@ public abstract class CassandraStore
         @Override
         public void createOlapTable(Id id) {
             CassandraTable table = new CassandraTables.Olap(this.store(), id);
-            table.init(this.session());
+            table.init(this.session(null));
             registerTableManager(this.olapTableName(id), table);
         }
 
@@ -769,7 +764,7 @@ public abstract class CassandraStore
             if (table == null || !this.existsTable(table.table())) {
                 throw new HugeException("Not exist table '%s'", name);
             }
-            table.truncate(this.session());
+            table.truncate(this.session(null));
         }
 
         @Override
@@ -779,8 +774,42 @@ public abstract class CassandraStore
             if (table == null || !this.existsTable(table.table())) {
                 throw new HugeException("Not exist table '%s'", name);
             }
-            table.dropTable(this.session());
+            table.dropTable(this.session(null));
             this.unregisterTableManager(name);
         }
     }
+
+    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();
+            String driverVersion = this.provider().driverVersion();
+            this.meta.writeVersion(this.session(null), driverVersion);
+            LOG.info("Write down the backend version: {}", driverVersion);
+        }
+
+        @Override
+        public String storedVersion() {
+            CassandraSessionPool.Session session = this.session(null);
+            return this.meta.readVersion(session);
+        }
+
+        @Override
+        protected Collection<CassandraTable> tables() {
+            List<CassandraTable> 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 da98fa602..b5898e94f 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.AbstractBackendStoreProvider;
 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;
 import com.baidu.hugegraph.config.HugeConfig;
 
 public class CassandraStoreProvider extends AbstractBackendStoreProvider {
@@ -41,13 +42,18 @@ public class CassandraStoreProvider extends AbstractBackendStoreProvider {
         return new CassandraGraphStore(this, this.keyspace(), store);
     }
 
+    @Override
+    protected BackendStore newSystemStore(HugeConfig config, 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
@@ -64,6 +70,7 @@ public class CassandraStoreProvider extends AbstractBackendStoreProvider {
          * [1.9] #295: support ttl for vertex and edge
          * [1.10] #1333: support read frequency for property key
          * [1.11] #1506: support olap properties
+         * [1.11] #1533: add meta table in system store
          */
         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 3ead2a5ac..c94dd69ee 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<HugeKeys, DataType> pkeys = ImmutableMap.of(
+                    HugeKeys.NAME, DataType.text()
+            );
+            ImmutableMap<HugeKeys, DataType> ckeys = ImmutableMap.of();
+            ImmutableMap<HugeKeys, DataType> 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 caf706ee1..ca388899c 100644
--- a/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java
+++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/HugeGraph.java
@@ -36,7 +36,7 @@ import com.baidu.hugegraph.auth.AuthManager;
 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.HugeConfig;
 import com.baidu.hugegraph.config.TypedOption;
@@ -179,12 +179,10 @@ public interface HugeGraph extends Graph {
 
     String backend();
 
-    String backendVersion();
-
-    BackendStoreSystemInfo backendStoreSystemInfo();
-
     BackendFeatures backendStoreFeatures();
 
+    BackendStoreInfo backendStoreInfo();
+
     GraphMode mode();
 
     void mode(GraphMode mode);
@@ -209,6 +207,8 @@ public interface HugeGraph extends Graph {
 
     void truncateBackend();
 
+    void initSystemInfo();
+
     void createSnapshot();
 
     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 c9853c90a..3fbdc00bd 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;
@@ -80,6 +81,8 @@ public interface HugeGraphParams {
 
     ServerInfoManager serverManager();
 
+    LocalCounter counter();
+
     AbstractSerializer serializer();
 
     Analyzer analyzer();
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 569d3d9e2..217fc5cbb 100644
--- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java
+++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java
@@ -45,6 +45,7 @@ import com.baidu.hugegraph.analyzer.AnalyzerFactory;
 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;
@@ -60,8 +61,8 @@ import com.baidu.hugegraph.backend.serializer.SerializerFactory;
 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;
@@ -153,6 +154,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;
@@ -173,6 +175,8 @@ public class StandardHugeGraph implements HugeGraph {
         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;
@@ -241,13 +245,11 @@ public class StandardHugeGraph implements HugeGraph {
     }
 
     @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
+        // TODO: pass storeProvider.metaStore()
+        this.systemTransaction();
+        return new BackendStoreInfo(this.configuration, this.storeProvider);
     }
 
     @Override
@@ -257,6 +259,9 @@ public class StandardHugeGraph implements HugeGraph {
 
     @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);
@@ -326,7 +331,12 @@ public class StandardHugeGraph implements HugeGraph {
         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();
@@ -365,7 +375,7 @@ public class StandardHugeGraph implements HugeGraph {
         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 {
@@ -375,6 +385,18 @@ public class StandardHugeGraph implements HugeGraph {
         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);
@@ -447,18 +469,15 @@ public class StandardHugeGraph implements HugeGraph {
     }
 
     private BackendStore loadSchemaStore() {
-        String name = this.configuration.get(CoreOptions.STORE_SCHEMA);
-        return this.storeProvider.loadSchemaStore(this.configuration, name);
+        return this.storeProvider.loadSchemaStore(this.configuration);
     }
 
     private BackendStore loadGraphStore() {
-        String name = this.configuration.get(CoreOptions.STORE_GRAPH);
-        return this.storeProvider.loadGraphStore(this.configuration, name);
+        return this.storeProvider.loadGraphStore(this.configuration);
     }
 
     private BackendStore loadSystemStore() {
-        String name = this.configuration.get(CoreOptions.STORE_SYSTEM);
-        return this.storeProvider.loadSystemStore(this.configuration, name);
+        return this.storeProvider.loadSystemStore(this.configuration);
     }
 
     @Watched
@@ -1206,6 +1225,11 @@ public class StandardHugeGraph implements HugeGraph {
             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 91947ce50..393913874 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,8 @@ import com.baidu.hugegraph.backend.id.Id;
 
 public interface AuthManager {
 
+    void init();
+
     boolean close();
 
     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 9f7b1fdbb..b10af0f7e 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
@@ -274,7 +274,7 @@ public class HugeProject extends Entity {
 
         @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 c6640bc9b..cb71e6bc7 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
@@ -45,16 +45,13 @@ import com.baidu.hugegraph.backend.id.Id;
 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;
 
@@ -63,7 +60,6 @@ public class StandardAuthManager implements AuthManager {
     protected static final Logger LOG = Log.logger(StandardAuthManager.class);
 
     private final HugeGraphParams graph;
-    private final EventListener eventListener;
 
     // Cache <username, HugeUser>
     private final Cache<Id, HugeUser> usersCache;
@@ -91,7 +87,6 @@ public class StandardAuthManager implements AuthManager {
         this.tokenExpire = config.get(AuthOptions.AUTH_TOKEN_EXPIRE) * 1000;
 
         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);
@@ -125,36 +120,8 @@ public class StandardAuthManager implements AuthManager {
         return cache;
     }
 
-    private EventListener listenChanges() {
-        // Listen store event: "store.inited"
-        Set<String> 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();
@@ -164,6 +131,11 @@ public class StandardAuthManager implements AuthManager {
         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 b2daec42f..9060f9396 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.BackendFeatures;
 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 class CachedBackendStore implements BackendStore {
         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 class CachedBackendStore implements BackendStore {
         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 41727ace5..5e3ea6b07 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,12 @@ import com.baidu.hugegraph.type.HugeType;
 public abstract class AbstractBackendStore<Session extends BackendSession>
                 implements BackendStore {
 
+    // TODO: move SystemSchemaStore into backend like MetaStore
+    private final SystemSchemaStore systemSchemaStore;
     private final MetaDispatcher<Session> dispatcher;
 
     public AbstractBackendStore() {
+        this.systemSchemaStore = new SystemSchemaStore();
         this.dispatcher = new MetaDispatcher<>();
     }
 
@@ -39,11 +42,22 @@ public abstract class AbstractBackendStore<Session extends BackendSession>
         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> R metadata(HugeType type, String meta, Object[] args) {
         Session session = this.session(type);
-        MetaDispatcher<Session> dispatcher = null;
+        MetaDispatcher<Session> 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 b7e6951d6..ed01b378c 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
@@ -44,7 +44,7 @@ public abstract class AbstractBackendStoreProvider
 
     private String graph = null;
 
-    private EventHub storeEventHub = new EventHub("store");
+    private final EventHub storeEventHub = new EventHub("store");
 
     protected Map<String, BackendStore> stores = null;
 
@@ -66,6 +66,8 @@ public abstract class AbstractBackendStoreProvider
 
     protected abstract BackendStore newGraphStore(HugeConfig config, String store);
 
+    protected abstract BackendStore newSystemStore(HugeConfig config, String store);
+
     @Override
     public void listen(EventListener listener) {
         this.storeEventHub.listen(EventHub.ANY_EVENT, listener);
@@ -76,6 +78,11 @@ public abstract class AbstractBackendStoreProvider
         this.storeEventHub.unlisten(EventHub.ANY_EVENT, listener);
     }
 
+    @Override
+    public String storedVersion() {
+        return this.loadSystemStore(null).storedVersion();
+    }
+
     @Override
     public String graph() {
         this.checkOpened();
@@ -145,13 +152,14 @@ public abstract class AbstractBackendStoreProvider
     }
 
     @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
@@ -171,7 +179,8 @@ public abstract class AbstractBackendStoreProvider
     }
 
     @Override
-    public BackendStore loadSchemaStore(HugeConfig config, String name) {
+    public BackendStore loadSchemaStore(HugeConfig config) {
+        String name = SCHEMA_STORE;
         LOG.debug("The '{}' StoreProvider load SchemaStore '{}'",
                   this.type(), name);
 
@@ -187,7 +196,8 @@ public abstract class AbstractBackendStoreProvider
     }
 
     @Override
-    public BackendStore loadGraphStore(HugeConfig config, String name) {
+    public BackendStore loadGraphStore(HugeConfig config) {
+        String name = GRAPH_STORE;
         LOG.debug("The '{}' StoreProvider load GraphStore '{}'",
                   this.type(), name);
 
@@ -203,8 +213,20 @@ public abstract class AbstractBackendStoreProvider
     }
 
     @Override
-    public BackendStore loadSystemStore(HugeConfig config, String name) {
-        return this.loadGraphStore(config, name);
+    public BackendStore loadSystemStore(HugeConfig config) {
+        String name = SYSTEM_STORE;
+        LOG.debug("The '{}' StoreProvider load SystemStore '{}'",
+                  this.type(), name);
+
+        this.checkOpened();
+        if (!this.stores.containsKey(name)) {
+            BackendStore s = this.newSystemStore(config, 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 9f8e1e496..828f35cc6 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
     String store();
 
+    // Stored version
+    String storedVersion();
+
     // Database name
     String database();
 
     // Get the parent provider
     BackendStoreProvider provider();
 
+    // Get the system schema store
+    SystemSchemaStore systemSchemaStore();
+
     // Whether it is the storage of schema
     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 000000000..a5daaa697
--- /dev/null
+++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreInfo.java
@@ -0,0 +1,58 @@
+/*
+ * 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 com.baidu.hugegraph.config.HugeConfig;
+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;
+
+    private final HugeConfig config;
+
+    public BackendStoreInfo(HugeConfig config,
+                            BackendStoreProvider storeProvider) {
+        this.config = config;
+        this.storeProvider = storeProvider;
+    }
+
+    public boolean exists() {
+        return this.storeProvider.initialized();
+    }
+
+    public boolean checkVersion() {
+        String driverVersion = this.storeProvider.driverVersion();
+        String storedVersion = this.storeProvider.loadSystemStore(this.config)
+                                                 .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 2b8485326..490fbb12e 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
@@ -20,27 +20,33 @@
 package com.baidu.hugegraph.backend.store;
 
 import com.alipay.remoting.rpc.RpcServer;
-import com.baidu.hugegraph.HugeGraph;
 import com.baidu.hugegraph.config.HugeConfig;
 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
     String type();
 
     // Backend store version
-    String version();
+    String storedVersion();
+
+    // Current backend store driver version
+    String driverVersion();
 
     // Graph name (that's database name)
     String graph();
 
-    BackendStore loadSystemStore(HugeConfig config, String name);
+    BackendStore loadSystemStore(HugeConfig config);
 
-    BackendStore loadSchemaStore(HugeConfig config, String name);
+    BackendStore loadSchemaStore(HugeConfig config);
 
-    BackendStore loadGraphStore(HugeConfig config, String name);
+    BackendStore loadGraphStore(HugeConfig config);
 
     void open(String name);
 
@@ -52,9 +58,9 @@ public interface BackendStoreProvider {
 
     void clear();
 
-    void truncate();
+    boolean initialized();
 
-    void initSystemInfo(HugeGraph graph);
+    void truncate();
 
     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 a47cb068b..000000000
--- 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<String, Object> 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<String, Object> 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 000000000..cd1424ef9
--- /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<String, SchemaElement> 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 extends SchemaElement> 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 extends SchemaElement> 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 8a106b1a3..5d90fc95b 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 abstract class InMemoryDBStore
         }
     }
 
+    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 d0107423f..d9eef07dc 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.AbstractBackendStoreProvider;
 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.config.HugeConfig;
 import com.baidu.hugegraph.util.Events;
 
@@ -76,13 +77,18 @@ public class InMemoryDBStoreProvider extends AbstractBackendStoreProvider {
         return new InMemoryGraphStore(this, this.graph(), store);
     }
 
+    @Override
+    protected BackendStore newSystemStore(HugeConfig config, 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
@@ -95,7 +101,8 @@ public class InMemoryDBStoreProvider extends AbstractBackendStoreProvider {
          * [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 803269181..8dacb8fef 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.BackendFeatures;
 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;
@@ -74,6 +75,11 @@ public class RaftBackendStore implements BackendStore {
         return this.store.store();
     }
 
+    @Override
+    public String storedVersion() {
+        return this.store.storedVersion();
+    }
+
     @Override
     public String database() {
         return this.store.database();
@@ -84,6 +90,11 @@ public class RaftBackendStore implements BackendStore {
         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 145951604..695da68b2 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
@@ -25,12 +25,10 @@ import java.util.concurrent.Future;
 import org.slf4j.Logger;
 
 import com.alipay.remoting.rpc.RpcServer;
-import com.baidu.hugegraph.HugeGraph;
 import com.baidu.hugegraph.HugeGraphParams;
 import com.baidu.hugegraph.backend.BackendException;
 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.config.HugeConfig;
@@ -96,8 +94,13 @@ public class RaftBackendStoreProvider implements BackendStoreProvider {
     }
 
     @Override
-    public String version() {
-        return this.provider.version();
+    public String driverVersion() {
+        return this.provider.driverVersion();
+    }
+
+    @Override
+    public String storedVersion() {
+        return this.provider.storedVersion();
     }
 
     @Override
@@ -106,10 +109,10 @@ public class RaftBackendStoreProvider implements BackendStoreProvider {
     }
 
     @Override
-    public synchronized BackendStore loadSchemaStore(HugeConfig config, String name) {
+    public synchronized BackendStore loadSchemaStore(HugeConfig config) {
         if (this.schemaStore == null) {
             LOG.info("Init raft backend schema store");
-            BackendStore store = this.provider.loadSchemaStore(config, name);
+            BackendStore store = this.provider.loadSchemaStore(config);
             this.checkNonSharedStore(store);
             this.schemaStore = new RaftBackendStore(store, this.context());
             this.context().addStore(StoreType.SCHEMA, this.schemaStore);
@@ -118,10 +121,10 @@ public class RaftBackendStoreProvider implements BackendStoreProvider {
     }
 
     @Override
-    public synchronized BackendStore loadGraphStore(HugeConfig config, String name) {
+    public synchronized BackendStore loadGraphStore(HugeConfig config) {
         if (this.graphStore == null) {
             LOG.info("Init raft backend graph store");
-            BackendStore store = this.provider.loadGraphStore(config, name);
+            BackendStore store = this.provider.loadGraphStore(config);
             this.checkNonSharedStore(store);
             this.graphStore = new RaftBackendStore(store, this.context());
             this.context().addStore(StoreType.GRAPH, this.graphStore);
@@ -130,10 +133,10 @@ public class RaftBackendStoreProvider implements BackendStoreProvider {
     }
 
     @Override
-    public synchronized BackendStore loadSystemStore(HugeConfig config, String name) {
+    public synchronized BackendStore loadSystemStore(HugeConfig config) {
         if (this.systemStore == null) {
             LOG.info("Init raft backend system store");
-            BackendStore store = this.provider.loadSystemStore(config, name);
+            BackendStore store = this.provider.loadSystemStore(config);
             this.checkNonSharedStore(store);
             this.systemStore = new RaftBackendStore(store, this.context());
             this.context().addStore(StoreType.SYSTEM, this.systemStore);
@@ -197,18 +200,7 @@ public class RaftBackendStoreProvider implements BackendStoreProvider {
             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
@@ -220,7 +212,11 @@ public class RaftBackendStoreProvider implements BackendStoreProvider {
          * 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/RaftContext.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftContext.java
index 7197dd461..bb1297b2f 100644
--- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftContext.java
+++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftContext.java
@@ -53,6 +53,7 @@ import com.baidu.hugegraph.backend.query.Query;
 import com.baidu.hugegraph.backend.store.BackendAction;
 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.raft.rpc.ListPeersProcessor;
 import com.baidu.hugegraph.backend.store.raft.rpc.RaftRequests.StoreType;
 import com.baidu.hugegraph.backend.store.raft.rpc.RpcForwarder;
@@ -93,10 +94,6 @@ public final class RaftContext {
 
     private final Configuration groupPeers;
 
-    private final String schemaStoreName;
-    private final String graphStoreName;
-    private final String systemStoreName;
-
     private final RaftBackendStore[] stores;
 
     private final ExecutorService readIndexExecutor;
@@ -128,10 +125,6 @@ public final class RaftContext {
                                     groupPeersString);
         }
 
-        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()];
 
         if (config.get(CoreOptions.RAFT_SAFE_READ)) {
@@ -210,12 +203,12 @@ public final class RaftContext {
     }
 
     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 ade53c1dc..2dbd76776 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<Id, HugeEdge> updatedEdges;
     private Set<HugeProperty<?>> 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 7d6a4b100..d0496333e 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 org.apache.tinkerpop.gremlin.structure.util.CloseableIterator;
 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.Query;
 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;
@@ -70,12 +72,17 @@ import com.google.common.collect.ImmutableSet;
 public class SchemaTransaction extends IndexableTransaction {
 
     private final SchemaIndexTransaction indexTx;
+    private final SystemSchemaStore systemSchemaStore;
+    // TODO: move LocalCounter counter define into SystemSchemaStore class
+    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
@@ -379,6 +386,12 @@ public class SchemaTransaction extends IndexableTransaction {
                   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()),
@@ -395,6 +408,11 @@ public class SchemaTransaction extends IndexableTransaction {
     protected <T extends SchemaElement> 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) {
@@ -416,6 +434,11 @@ public class SchemaTransaction extends IndexableTransaction {
                                                     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);
@@ -454,6 +477,12 @@ public class SchemaTransaction extends IndexableTransaction {
     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()),
@@ -568,7 +597,7 @@ public class SchemaTransaction extends IndexableTransaction {
     @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 e158a4e3f..a20d4798b 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,22 +72,6 @@ public class CoreOptions extends OptionHolder {
                     "hugegraph"
             );
 
-    public static final ConfigOption<String> STORE_SYSTEM =
-            new ConfigOption<>(
-                    "store.system",
-                    "The system table name, which store system data.",
-                    disallowEmpty(),
-                    "s"
-            );
-
-    public static final ConfigOption<String> STORE_SCHEMA =
-            new ConfigOption<>(
-                    "store.schema",
-                    "The schema table name, which store meta data.",
-                    disallowEmpty(),
-                    "m"
-            );
-
     public static final ConfigOption<String> STORE_GRAPH =
             new ConfigOption<>(
                     "store.graph",
diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/VertexLabel.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/VertexLabel.java
index aa2e9b598..49f4d84cb 100644
--- a/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/VertexLabel.java
+++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/VertexLabel.java
@@ -40,7 +40,7 @@ public class VertexLabel extends SchemaLabel {
     // OLAP_VL_ID means all of vertex label ids
     private static final Id OLAP_VL_ID = IdGenerator.of(SchemaLabel.OLAP_VL_ID);
     // OLAP_VL_NAME means all of vertex label names
-    private static final String OLAP_VL_NAME = "~olap";
+    private static final String OLAP_VL_NAME = "*olap";
     // OLAP_VL means all of vertex labels
     public static final VertexLabel OLAP_VL = new VertexLabel(null, OLAP_VL_ID,
                                                               OLAP_VL_NAME);
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 477a148b3..cdae8d3fc 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 static com.baidu.hugegraph.backend.query.Query.NO_LIMIT;
 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.Condition;
 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.HugeKeys;
 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 class ServerInfoManager {
         this.graph = graph;
         this.dbExecutor = dbExecutor;
 
-        this.eventListener = this.listenChanges();
-
         this.selfServerId = null;
         this.selfServerRole = NodeRole.MASTER;
 
@@ -91,32 +84,12 @@ public class ServerInfoManager {
         this.closed = false;
     }
 
-    private EventListener listenChanges() {
-        // Listen store event: "store.inited"
-        Set<String> 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(() -> {
@@ -160,6 +133,7 @@ public class ServerInfoManager {
             } while (page != null);
         }
 
+        // TODO: save ServerInfo at AuthServer
         this.saveServerInfo(this.selfServerId, this.selfServerRole);
     }
 
@@ -253,10 +227,6 @@ public class ServerInfoManager {
         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 449877bea..3a27c13b5 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.Collection;
 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.query.QueryResults;
 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.Cardinality;
 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<Id, HugeTask<?>> tasks;
 
     private volatile TaskTransaction taskTx;
@@ -107,8 +102,6 @@ public class StandardTaskScheduler implements TaskScheduler {
         this.tasks = new ConcurrentHashMap<>();
 
         this.taskTx = null;
-
-        this.eventListener = this.listenChanges();
     }
 
     @Override
@@ -133,9 +126,9 @@ public class StandardTaskScheduler implements TaskScheduler {
              * 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 @@ public class StandardTaskScheduler implements TaskScheduler {
         return this.taskTx;
     }
 
-    private EventListener listenChanges() {
-        // Listen store event: "store.inited"
-        Set<String> 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 <V> void restoreTasks() {
         Id selfServer = this.serverManager().selfServerId();
@@ -482,9 +456,13 @@ public class StandardTaskScheduler implements TaskScheduler {
         });
     }
 
+    @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 aa12854bf..22e4864ff 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
@@ -50,6 +50,8 @@ public interface TaskScheduler {
     <V> Iterator<HugeTask<V>> tasks(TaskStatus status,
                                     long limit, String page);
 
+    void init();
+
     boolean close();
 
     <V> HugeTask<V> 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 bfb146ecf..a5f69ba45 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 fe006f3a3..e7aba0fb1 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"),
     TIMESTAMP(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-dist/src/main/java/com/baidu/hugegraph/cmd/InitStore.java b/hugegraph-dist/src/main/java/com/baidu/hugegraph/cmd/InitStore.java
index 99b4e060a..cd45bfa24 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,7 +19,9 @@
 
 package com.baidu.hugegraph.cmd;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.collections.map.MultiValueMap;
@@ -29,7 +31,7 @@ import org.slf4j.Logger;
 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;
@@ -73,36 +75,39 @@ public class InitStore {
 
         HugeConfig restServerConfig = new HugeConfig(restConf);
         String graphsDir = restServerConfig.get(ServerOptions.GRAPHS);
-        Map<String, String> graphs = ConfigUtil.scanGraphsDir(graphsDir);
+        Map<String, String> graph2ConfigPaths = ConfigUtil.scanGraphsDir(graphsDir);
 
-        for (Map.Entry<String, String> entry : graphs.entrySet()) {
-            initGraph(entry.getValue());
+        List<HugeGraph> graphs = new ArrayList<>(graph2ConfigPaths.size());
+        try {
+            for (Map.Entry<String, String> entry : graph2ConfigPaths.entrySet()) {
+                graphs.add(initGraph(entry.getValue()));
+            }
+            StandardAuthenticator.initAdminUserIfNeeded(restConf);
+        } finally {
+            for (HugeGraph graph : graphs) {
+                graph.close();
+            }
         }
 
-        StandardAuthenticator.initAdminUserIfNeeded(restConf);
-
         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 061b65ecb..628048a5d 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
@@ -551,4 +551,38 @@ public abstract class HbaseStore extends AbstractBackendStore<Session> {
                       "HbaseGraphStore.getCounter()");
         }
     }
+
+    public static class HbaseSystemStore extends HbaseGraphStore {
+
+        private final HbaseTables.Meta meta;
+
+        public HbaseSystemStore(HugeConfig config, BackendStoreProvider provider,
+                                String namespace, String store) {
+            super(config, provider, namespace, store);
+
+            this.meta = new HbaseTables.Meta();
+        }
+
+        @Override
+        protected List<String> tableNames() {
+            List<String> tableNames = super.tableNames();
+            tableNames.add(this.meta.table());
+            return tableNames;
+        }
+
+        @Override
+        public void init() {
+            super.init();
+            Session session = super.session(null);
+            String driverVersion = this.provider().driverVersion();
+            this.meta.writeVersion(session, driverVersion);
+            LOG.info("Write down the backend version: {}", driverVersion);
+        }
+
+        @Override
+        public String storedVersion() {
+            Session session = super.session(null);
+            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 f8dd33a66..dd4da243d 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.AbstractBackendStoreProvider;
 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;
 import com.baidu.hugegraph.config.HugeConfig;
 
 public class HbaseStoreProvider extends AbstractBackendStoreProvider {
@@ -41,13 +42,18 @@ public class HbaseStoreProvider extends AbstractBackendStoreProvider {
         return new HbaseGraphStore(config, this, this.namespace(), store);
     }
 
+    @Override
+    protected BackendStore newSystemStore(HugeConfig config, String store) {
+        return new HbaseSystemStore(config, 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
@@ -64,7 +70,8 @@ public class HbaseStoreProvider extends AbstractBackendStoreProvider {
          * [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 8aa9d8cc6..885351817 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.BackendEntryIterator;
 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 5cec83de1..858b22edb 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
@@ -472,4 +472,39 @@ public abstract class MysqlStore extends AbstractBackendStore<Session> {
                       "MysqlGraphStore.getCounter()");
         }
     }
+
+    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();
+            Session session = super.session(null);
+            String driverVersion = this.provider().driverVersion();
+            this.meta.writeVersion(session, driverVersion);
+            LOG.info("Write down the backend version: {}", driverVersion);
+        }
+
+        @Override
+        public String storedVersion() {
+            super.init();
+            Session session = super.session(null);
+            return this.meta.readVersion(session);
+        }
+
+        @Override
+        protected Collection<MysqlTable> tables() {
+            List<MysqlTable> 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 13357f739..511c3d10d 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.AbstractBackendStoreProvider;
 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;
 import com.baidu.hugegraph.config.HugeConfig;
 
 public class MysqlStoreProvider extends AbstractBackendStoreProvider {
@@ -41,13 +42,18 @@ public class MysqlStoreProvider extends AbstractBackendStoreProvider {
         return new MysqlGraphStore(this, this.database(), store);
     }
 
+    @Override
+    protected BackendStore newSystemStore(HugeConfig config, 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
@@ -65,7 +71,8 @@ public class MysqlStoreProvider extends AbstractBackendStoreProvider {
          * [1.9] #295: support ttl for vertex and edge
          * [1.10] #1333: support read frequency for property key
          * [1.11] #1506: rename read frequency to write type
+         * [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 25e9e66df..03b14986b 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
@@ -84,6 +84,54 @@ public class MysqlTables {
         }
     }
 
+    public static class Meta extends MysqlTableTemplate {
+
+        public static final String TABLE = HugeType.META.string();
+
+        public Meta() {
+            this(TYPES_MAPPING);
+        }
+
+        public Meta(Map<String, String> 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 versionColumn = formatKey(HugeKeys.VERSION);
+            String insert = String.format("INSERT IGNORE INTO %s VALUES " +
+                                          "('%s', '%s')", this.table(),
+                                          versionColumn, driverVersion);
+            try {
+                session.execute(insert);
+            } catch (SQLException e) {
+                throw new BackendException("Failed to insert driver version " +
+                                           "with '%s'", e, 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 (ResultSetWrapper results = session.select(select)) {
+                ResultSet rs = results.resultSet();
+                if (!rs.next()) {
+                    return null;
+                }
+                return rs.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 434cd2d40..fe08261c4 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
@@ -49,7 +49,7 @@ public class PaloStoreProvider extends MysqlStoreProvider {
     }
 
     @Override
-    public String version() {
+    public String driverVersion() {
         /*
          * Versions history:
          * [1.0] HugeGraph-1328: supports backend table version checking
@@ -66,8 +66,9 @@ public class PaloStoreProvider extends MysqlStoreProvider {
          * [1.8] #295: support ttl for vertex and edge
          * [1.9] #1333: support read frequency for property key
          * [1.10] #1506: rename read frequency to write type
+         * [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 955572a65..902866581 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.ArrayList;
 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;
@@ -32,9 +34,12 @@ import com.baidu.hugegraph.backend.store.mysql.MysqlTable;
 import com.baidu.hugegraph.config.HugeConfig;
 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(HugeConfig config, String store) {
         return new PostgresqlSchemaStore(this, this.database(), store);
@@ -45,13 +50,18 @@ public class PostgresqlStoreProvider extends MysqlStoreProvider {
         return new PostgresqlGraphStore(this, this.database(), store);
     }
 
+    @Override
+    protected BackendStore newSystemStore(HugeConfig config, 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
@@ -67,8 +77,9 @@ public class PostgresqlStoreProvider extends MysqlStoreProvider {
          * [1.7] #295: support ttl for vertex and edge
          * [1.8] #1333: support read frequency for property key
          * [1.9] #1506: rename read frequency to write type
+         * [1.9] #1533: add meta table in system store
          */
-        return "1.8";
+        return "1.9";
     }
 
     public static class PostgresqlSchemaStore extends PostgresqlStore {
@@ -173,4 +184,41 @@ public class PostgresqlStoreProvider extends MysqlStoreProvider {
                       "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<MysqlTable> tables() {
+            List<MysqlTable> 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 f61022490..6d9e83a04 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 com.baidu.hugegraph.type.define.HugeKeys;
 
 import 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<String, String> TYPES_MAPPING =
@@ -75,6 +75,31 @@ public class PostgresqlTables {
         }
     }
 
+    public static class Meta extends PostgresqlTableTemplate {
+
+        public Meta() {
+            super(new MysqlTables.Meta(TYPES_MAPPING));
+        }
+
+        public void writeVersion(Session session, String version) {
+            String versionColumn = formatKey(HugeKeys.VERSION);
+            String insert = String.format("INSERT INTO %s VALUES ('%s', '%s') " +
+                                          "ON CONFLICT(name) DO NOTHING;",
+                                          this.table(), versionColumn, version);
+            try {
+                session.execute(insert);
+            } catch (SQLException e) {
+                throw new BackendException("Failed to insert driver version " +
+                                           "with '%s'", e, insert);
+            }
+        }
+
+        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 2db4547c8..cd14d3e6d 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
@@ -301,7 +301,6 @@ public abstract class RocksDBStore extends AbstractBackendStore<Session> {
     protected RocksDBSessions open(HugeConfig config, String dataPath,
                                    String walPath, List<String> tableNames) {
         LOG.info("Opening RocksDB with data path: {}", dataPath);
-
         RocksDBSessions sessions = null;
         try {
             sessions = this.openSessionPool(config, dataPath,
@@ -403,6 +402,10 @@ public abstract class RocksDBStore extends AbstractBackendStore<Session> {
         return tableDBMap;
     }
 
+    protected ReadWriteLock storeLock() {
+        return this.storeLock;
+    }
+
     @Override
     public void close() {
         LOG.debug("Store close: {}", this.store);
@@ -1093,4 +1096,51 @@ public abstract class RocksDBStore extends AbstractBackendStore<Session> {
             this.unregisterTableManager(this.olapTableName(id));
         }
     }
+
+    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 {
+                Session session = super.session(HugeType.META);
+                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.session(null);
+                return this.meta.readVersion(session);
+            } finally {
+                readLock.unlock();
+            }
+        }
+
+        @Override
+        protected List<String> tableNames() {
+            List<String> 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 7f83f9cf8..35c0b5db2 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
@@ -25,6 +25,7 @@ import com.baidu.hugegraph.backend.store.AbstractBackendStoreProvider;
 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;
 import com.baidu.hugegraph.config.HugeConfig;
 import com.baidu.hugegraph.util.ConfigUtil;
 
@@ -67,13 +68,18 @@ public class RocksDBStoreProvider extends AbstractBackendStoreProvider {
         ConfigUtil.deleteFile(new File(walPath));
     }
 
+    @Override
+    protected BackendStore newSystemStore(HugeConfig config, 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
@@ -90,6 +96,7 @@ public class RocksDBStoreProvider extends AbstractBackendStoreProvider {
          * [1.9] #295: support ttl for vertex and edge
          * [1.10] #1333: support read frequency for property key
          * [1.11] #1506: support olap properties
+         * [1.11] #1533: add meta table in system store
          */
         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 38e981e37..c4b69d385 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
@@ -35,9 +35,40 @@ import com.baidu.hugegraph.backend.store.rocksdb.RocksDBSessions.Session;
 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 0fe6d670e..8d2bc5b7a 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 class ScyllaDBStoreProvider extends CassandraStoreProvider {
     }
 
     @Override
-    public BackendStore loadSchemaStore(HugeConfig config, String name) {
+    public BackendStore loadSchemaStore(HugeConfig config) {
+        String name = SCHEMA_STORE;
         LOG.debug("ScyllaDBStoreProvider load SchemaStore '{}'", name);
 
         if (!this.stores.containsKey(name)) {
@@ -61,7 +62,8 @@ public class ScyllaDBStoreProvider extends CassandraStoreProvider {
     }
 
     @Override
-    public BackendStore loadGraphStore(HugeConfig config, String name) {
+    public BackendStore loadGraphStore(HugeConfig config) {
+        String name = GRAPH_STORE;
         LOG.debug("ScyllaDBStoreProvider load GraphStore '{}'", name);
 
         if (!this.stores.containsKey(name)) {
@@ -76,6 +78,23 @@ public class ScyllaDBStoreProvider extends CassandraStoreProvider {
         return store;
     }
 
+    @Override
+    public BackendStore loadSystemStore(HugeConfig config) {
+        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 @@ public class ScyllaDBStoreProvider extends CassandraStoreProvider {
             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/api/EdgeApiTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/api/EdgeApiTest.java
index 9e0b76ed0..00aa867be 100644
--- a/hugegraph-test/src/main/java/com/baidu/hugegraph/api/EdgeApiTest.java
+++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/api/EdgeApiTest.java
@@ -21,13 +21,14 @@ package com.baidu.hugegraph.api;
 
 import java.io.IOException;
 
-import jakarta.ws.rs.core.Response;
 import org.junit.Before;
 import org.junit.Test;
 
 import com.baidu.hugegraph.testutil.Assert;
 import com.google.common.collect.ImmutableMap;
 
+import jakarta.ws.rs.core.Response;
+
 public class EdgeApiTest extends BaseApiTest {
 
     private static String path = "/graphs/hugegraph/graph/edges/";
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 3545185c1..0480a7f92 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
@@ -38,6 +38,7 @@ import com.baidu.hugegraph.HugeException;
 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;
@@ -55,6 +56,23 @@ public class MultiGraphsTest {
     private static final String NAME48 =
             "g12345678901234567890123456789012345678901234567";
 
+    @Test
+    public void testWriteAndReadVersion() {
+        List<HugeGraph> 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();
+        }
+        destroyGraphs(graphs);
+    }
+
     @Test
     public void testCreateMultiGraphs() {
         List<HugeGraph> graphs = openGraphs("g_1", NAME48);
@@ -238,9 +256,9 @@ public class MultiGraphsTest {
         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 be8c7e809..e0f873c30 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 org.apache.tinkerpop.gremlin.structure.io.Io;
 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 class TestGraph implements Graph {
 
     @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 896147127..fe8f0d14d 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,8 @@ package com.baidu.hugegraph.unit;
 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;
 import com.baidu.hugegraph.unit.cache.CachedGraphTransactionTest;
@@ -30,7 +32,7 @@ import com.baidu.hugegraph.unit.cache.RamTableTest;
 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;
@@ -108,9 +110,10 @@ import com.baidu.hugegraph.unit.util.collection.ObjectIntMappingTest;
     SecurityManagerTest.class,
     RolePermissionTest.class,
     ExceptionTest.class,
-    BackendStoreSystemInfoTest.class,
+    BackendStoreInfoTest.class,
     TraversalUtilTest.class,
     PageStateTest.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 000000000..7f5f9e9ad
--- /dev/null
+++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/BackendStoreInfoTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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 com.baidu.hugegraph.config.HugeConfig;
+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);
+        HugeConfig config = Mockito.mock(HugeConfig.class);
+
+        Mockito.when(provider.initialized()).thenReturn(true);
+        Mockito.when(provider.loadSystemStore(config)).thenReturn(store);
+        Mockito.when(store.storedVersion()).thenReturn("1.11");
+
+        BackendStoreInfo backendStoreInfo = new BackendStoreInfo(config,
+                                                                 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 8b388db38..92778c536 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 @@ package com.baidu.hugegraph.unit.core;
 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);
+        }
     }
 }