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

jackietien pushed a commit to branch force_ci/object_type
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 602d4650aaf0f9a26f41f2898fb51f420fbb344e
Author: Caideyipi <[email protected]>
AuthorDate: Mon Nov 10 17:27:18 2025 +0800

    Fixed the potential non-consistency caused by initialize of 
DataNodeTableCache & Cleared the storageGroups in compatible codes (#16693)
    
    * fix
    
    * refactor
    
    * may-complete
    
    * fix
    
    * refactor
    
    * fix
    
    * Update IoTDBFlushQueryIT.java
    
    * bug-fix
    
    * fix
    
    (cherry picked from commit 639de5255ce51c4c104160faafad3fa26ad60e82)
---
 .../it/env/cluster/config/MppCommonConfig.java     |  4 +-
 .../env/cluster/config/MppSharedCommonConfig.java  |  6 +-
 .../apache/iotdb/itbase/constant/TestConstant.java | 14 ++--
 .../org/apache/iotdb/itbase/env/CommonConfig.java  |  2 +-
 .../confignode/it/IoTDBConfigNodeSnapshotIT.java   | 31 ++++-----
 .../it/database/IoTDBDatabaseSetAndDeleteIT.java   | 66 +++++++++---------
 .../it/load/IoTDBConfigNodeSwitchLeaderIT.java     |  2 +-
 .../partition/IoTDBAutoRegionGroupExtensionIT.java |  8 +--
 .../IoTDBCustomRegionGroupExtensionIT.java         |  2 +-
 .../it/partition/IoTDBPartitionCreationIT.java     |  4 +-
 .../it/partition/IoTDBPartitionDurableIT.java      |  4 +-
 .../it/partition/IoTDBPartitionGetterIT.java       | 41 ++++++------
 .../confignode/it/utils/ConfigNodeTestUtils.java   | 16 ++---
 .../org/apache/iotdb/db/it/IoTDBExampleIT.java     |  4 +-
 .../org/apache/iotdb/db/it/IoTDBFlushQueryIT.java  |  5 +-
 .../org/apache/iotdb/db/it/IoTDBRestServiceIT.java |  8 +--
 .../iotdb/db/it/IoTDBSetConfigurationIT.java       |  2 +-
 .../org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java |  2 +-
 .../db/it/IoTDBSyntaxConventionIdentifierIT.java   |  2 +-
 .../apache/iotdb/db/it/path/IoTDBQuotedPathIT.java |  4 +-
 .../db/it/schema/IoTDBAutoCreateSchemaIT.java      | 14 ++--
 .../db/it/schema/IoTDBDeactivateTemplateIT.java    |  2 +-
 .../db/it/schema/IoTDBDeleteTimeSeriesIT.java      |  2 +-
 .../iotdb/db/it/schema/IoTDBMetadataFetchIT.java   |  2 +-
 .../org/apache/iotdb/db/it/schema/IoTDBTagIT.java  |  2 +-
 .../it/query/old/IoTDBSimpleQueryTableIT.java      |  2 +-
 .../iotdb/session/it/IoTDBSessionSimpleIT.java     | 22 +++---
 .../session/it/IoTDBSessionSyntaxConventionIT.java |  6 +-
 .../write/database/PreDeleteDatabasePlan.java      | 22 +++---
 .../iotdb/confignode/manager/ConfigManager.java    | 28 +++-----
 .../apache/iotdb/confignode/manager/IManager.java  | 16 ++---
 .../iotdb/confignode/manager/ProcedureManager.java | 27 ++++++++
 .../iotdb/confignode/manager/load/LoadManager.java |  2 +-
 .../manager/load/balancer/RegionBalancer.java      |  2 +-
 .../manager/schema/ClusterSchemaManager.java       | 26 +++++++-
 .../persistence/partition/PartitionInfo.java       |  2 +-
 .../impl/schema/DeleteDatabaseProcedure.java       | 28 ++++----
 ...ageGroupState.java => DeleteDatabaseState.java} |  2 +-
 .../impl/DataNodeInternalRPCServiceImpl.java       | 48 +++++++------
 .../queryengine/plan/analyze/AnalyzeVisitor.java   |  2 +-
 .../execution/config/TreeConfigTaskVisitor.java    |  3 +-
 .../db/queryengine/plan/parser/ASTVisitor.java     | 18 ++---
 .../metadata/fetcher/TableDeviceSchemaFetcher.java |  6 +-
 .../security/TreeAccessCheckVisitor.java           |  2 +-
 .../plan/statement/StatementVisitor.java           |  2 +-
 .../statement/metadata/CountDatabaseStatement.java |  2 +-
 .../mtree/impl/mem/MTreeBelowSGMemoryImpl.java     | 78 ++++++++++------------
 .../db/schemaengine/table/DataNodeTableCache.java  | 17 +++++
 .../schema/table/NonCommittableTsTable.java}       | 31 ++++-----
 .../apache/iotdb/commons/schema/table/TsTable.java | 24 ++++---
 50 files changed, 357 insertions(+), 310 deletions(-)

diff --git 
a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java
 
b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java
index 1df43c16206..6cb7e008242 100644
--- 
a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java
+++ 
b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java
@@ -580,8 +580,8 @@ public class MppCommonConfig extends MppBaseConfig 
implements CommonConfig {
   }
 
   @Override
-  public CommonConfig setDefaultStorageGroupLevel(int 
defaultStorageGroupLevel) {
-    setProperty("default_database_level", 
String.valueOf(defaultStorageGroupLevel));
+  public CommonConfig setDefaultDatabaseLevel(int defaultDatabaseLevel) {
+    setProperty("default_database_level", 
String.valueOf(defaultDatabaseLevel));
     return this;
   }
 
diff --git 
a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java
 
b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java
index 504b1ae60e2..1b4801f5692 100644
--- 
a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java
+++ 
b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java
@@ -602,9 +602,9 @@ public class MppSharedCommonConfig implements CommonConfig {
   }
 
   @Override
-  public CommonConfig setDefaultStorageGroupLevel(int 
defaultStorageGroupLevel) {
-    dnConfig.setDefaultStorageGroupLevel(defaultStorageGroupLevel);
-    cnConfig.setDefaultStorageGroupLevel(defaultStorageGroupLevel);
+  public CommonConfig setDefaultDatabaseLevel(int defaultDatabaseLevel) {
+    dnConfig.setDefaultDatabaseLevel(defaultDatabaseLevel);
+    cnConfig.setDefaultDatabaseLevel(defaultDatabaseLevel);
     return this;
   }
 
diff --git 
a/integration-test/src/main/java/org/apache/iotdb/itbase/constant/TestConstant.java
 
b/integration-test/src/main/java/org/apache/iotdb/itbase/constant/TestConstant.java
index 8a9d11516c5..03b09bc819b 100644
--- 
a/integration-test/src/main/java/org/apache/iotdb/itbase/constant/TestConstant.java
+++ 
b/integration-test/src/main/java/org/apache/iotdb/itbase/constant/TestConstant.java
@@ -168,13 +168,12 @@ public class TestConstant {
   }
 
   public static String getTestTsFilePath(
-      String logicalStorageGroupName,
-      long VirtualStorageGroupId,
+      String logicalDatabaseName,
+      long VirtualDatabaseId,
       long TimePartitionId,
       long tsFileVersion) {
     String filePath =
-        String.format(
-            TEST_TSFILE_PATH, logicalStorageGroupName, VirtualStorageGroupId, 
TimePartitionId);
+        String.format(TEST_TSFILE_PATH, logicalDatabaseName, 
VirtualDatabaseId, TimePartitionId);
     String fileName =
         System.currentTimeMillis()
             + FilePathUtils.FILE_NAME_SEPARATOR
@@ -184,11 +183,8 @@ public class TestConstant {
   }
 
   public static String getTestTsFileDir(
-      String logicalStorageGroupName, long VirtualStorageGroupId, long 
TimePartitionId) {
+      String logicalDatabaseName, long VirtualDatabaseId, long 
TimePartitionId) {
     return String.format(
-        TestConstant.TEST_TSFILE_PATH,
-        logicalStorageGroupName,
-        VirtualStorageGroupId,
-        TimePartitionId);
+        TestConstant.TEST_TSFILE_PATH, logicalDatabaseName, VirtualDatabaseId, 
TimePartitionId);
   }
 }
diff --git 
a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java 
b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
index 9767e1c089e..f27bdc8d66d 100644
--- 
a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
+++ 
b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
@@ -186,7 +186,7 @@ public interface CommonConfig {
 
   CommonConfig setSubscriptionEnabled(boolean subscriptionEnabled);
 
-  default CommonConfig setDefaultStorageGroupLevel(int 
defaultStorageGroupLevel) {
+  default CommonConfig setDefaultDatabaseLevel(int defaultDatabaseLevel) {
     return this;
   }
 
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBConfigNodeSnapshotIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBConfigNodeSnapshotIT.java
index d906cd43bda..a5c757d8dd3 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBConfigNodeSnapshotIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBConfigNodeSnapshotIT.java
@@ -106,7 +106,7 @@ public class IoTDBConfigNodeSnapshotIT {
   @Test
   public void testPartitionInfoSnapshot() throws Exception {
     final String sg = "root.sg";
-    final int storageGroupNum = 10;
+    final int databaseNum = 10;
     final int seriesPartitionSlotsNum = 10;
     final int timePartitionSlotsNum = 10;
 
@@ -118,10 +118,10 @@ public class IoTDBConfigNodeSnapshotIT {
 
       Set<TCQEntry> expectedCQEntries = createCQs(client);
 
-      for (int i = 0; i < storageGroupNum; i++) {
-        String storageGroup = sg + i;
-        TDatabaseSchema storageGroupSchema = new TDatabaseSchema(storageGroup);
-        TSStatus status = client.setDatabase(storageGroupSchema);
+      for (int i = 0; i < databaseNum; i++) {
+        String database = sg + i;
+        TDatabaseSchema databaseSchema = new TDatabaseSchema(database);
+        TSStatus status = client.setDatabase(databaseSchema);
         assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
status.getCode());
 
         for (int j = 0; j < seriesPartitionSlotsNum; j++) {
@@ -129,7 +129,7 @@ public class IoTDBConfigNodeSnapshotIT {
 
           // Create SchemaPartition
           ByteBuffer patternTree =
-              generatePatternTreeBuffer(new String[] {storageGroup + ".d" + j 
+ ".s"});
+              generatePatternTreeBuffer(new String[] {database + ".d" + j + 
".s"});
           TSchemaPartitionReq schemaPartitionReq = new 
TSchemaPartitionReq(patternTree);
           TSchemaPartitionTableResp schemaPartitionTableResp =
               client.getOrCreateSchemaPartitionTable(schemaPartitionReq);
@@ -139,10 +139,8 @@ public class IoTDBConfigNodeSnapshotIT {
               schemaPartitionTableResp.getStatus().getCode());
           
Assert.assertNotNull(schemaPartitionTableResp.getSchemaPartitionTable());
           assertEquals(1, 
schemaPartitionTableResp.getSchemaPartitionTableSize());
-          Assert.assertNotNull(
-              
schemaPartitionTableResp.getSchemaPartitionTable().get(storageGroup));
-          assertEquals(
-              1, 
schemaPartitionTableResp.getSchemaPartitionTable().get(storageGroup).size());
+          
Assert.assertNotNull(schemaPartitionTableResp.getSchemaPartitionTable().get(database));
+          assertEquals(1, 
schemaPartitionTableResp.getSchemaPartitionTable().get(database).size());
 
           for (int k = 0; k < timePartitionSlotsNum; k++) {
             TTimePartitionSlot timePartitionSlot =
@@ -151,9 +149,9 @@ public class IoTDBConfigNodeSnapshotIT {
             // Create DataPartition
             Map<String, Map<TSeriesPartitionSlot, TTimeSlotList>> 
partitionSlotsMap =
                 new HashMap<>();
-            partitionSlotsMap.put(storageGroup, new HashMap<>());
+            partitionSlotsMap.put(database, new HashMap<>());
             partitionSlotsMap
-                .get(storageGroup)
+                .get(database)
                 .put(
                     seriesPartitionSlot,
                     new TTimeSlotList()
@@ -167,19 +165,18 @@ public class IoTDBConfigNodeSnapshotIT {
                 dataPartitionTableResp.getStatus().getCode());
             
Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable());
             assertEquals(1, 
dataPartitionTableResp.getDataPartitionTableSize());
-            
Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable().get(storageGroup));
-            assertEquals(
-                1, 
dataPartitionTableResp.getDataPartitionTable().get(storageGroup).size());
+            
Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable().get(database));
+            assertEquals(1, 
dataPartitionTableResp.getDataPartitionTable().get(database).size());
             Assert.assertNotNull(
                 dataPartitionTableResp
                     .getDataPartitionTable()
-                    .get(storageGroup)
+                    .get(database)
                     .get(seriesPartitionSlot));
             assertEquals(
                 1,
                 dataPartitionTableResp
                     .getDataPartitionTable()
-                    .get(storageGroup)
+                    .get(database)
                     .get(seriesPartitionSlot)
                     .size());
           }
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java
index e08339823d6..959f37d594d 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java
@@ -74,17 +74,17 @@ public class IoTDBDatabaseSetAndDeleteIT {
     try (SyncConfigNodeIServiceClient client =
         (SyncConfigNodeIServiceClient) 
EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
       // set Database0 by default values
-      TDatabaseSchema storageGroupSchema0 = new TDatabaseSchema(sg0);
-      status = client.setDatabase(storageGroupSchema0);
+      TDatabaseSchema databaseSchema0 = new TDatabaseSchema(sg0);
+      status = client.setDatabase(databaseSchema0);
       Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
status.getCode());
 
       // set Database1 by specific values
-      TDatabaseSchema storageGroupSchema1 =
+      TDatabaseSchema databaseSchema1 =
           new TDatabaseSchema(sg1)
               .setSchemaReplicationFactor(5)
               .setDataReplicationFactor(5)
               .setTimePartitionInterval(2048L);
-      status = client.setDatabase(storageGroupSchema1);
+      status = client.setDatabase(databaseSchema1);
       Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
status.getCode());
 
       // test count all Databases
@@ -111,21 +111,21 @@ public class IoTDBDatabaseSetAndDeleteIT {
           TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
getResp.getStatus().getCode());
       Map<String, TDatabaseSchema> schemaMap = getResp.getDatabaseSchemaMap();
       Assert.assertEquals(2, schemaMap.size());
-      TDatabaseSchema storageGroupSchema = schemaMap.get(sg0);
-      Assert.assertNotNull(storageGroupSchema);
-      Assert.assertEquals(sg0, storageGroupSchema.getName());
-      Assert.assertEquals(1, storageGroupSchema.getSchemaReplicationFactor());
-      Assert.assertEquals(1, storageGroupSchema.getDataReplicationFactor());
-      Assert.assertEquals(604800000, 
storageGroupSchema.getTimePartitionInterval());
-      storageGroupSchema = schemaMap.get(sg1);
-      Assert.assertNotNull(storageGroupSchema);
-      Assert.assertEquals(sg1, storageGroupSchema.getName());
-      Assert.assertEquals(5, storageGroupSchema.getSchemaReplicationFactor());
-      Assert.assertEquals(5, storageGroupSchema.getDataReplicationFactor());
-      Assert.assertEquals(2048L, 
storageGroupSchema.getTimePartitionInterval());
+      TDatabaseSchema databaseSchema = schemaMap.get(sg0);
+      Assert.assertNotNull(databaseSchema);
+      Assert.assertEquals(sg0, databaseSchema.getName());
+      Assert.assertEquals(1, databaseSchema.getSchemaReplicationFactor());
+      Assert.assertEquals(1, databaseSchema.getDataReplicationFactor());
+      Assert.assertEquals(604800000, 
databaseSchema.getTimePartitionInterval());
+      databaseSchema = schemaMap.get(sg1);
+      Assert.assertNotNull(databaseSchema);
+      Assert.assertEquals(sg1, databaseSchema.getName());
+      Assert.assertEquals(5, databaseSchema.getSchemaReplicationFactor());
+      Assert.assertEquals(5, databaseSchema.getDataReplicationFactor());
+      Assert.assertEquals(2048L, databaseSchema.getTimePartitionInterval());
 
       // test fail by re-register
-      status = client.setDatabase(storageGroupSchema0);
+      status = client.setDatabase(databaseSchema0);
       
Assert.assertEquals(TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode(), 
status.getCode());
 
       // test Database setter interfaces
@@ -149,35 +149,35 @@ public class IoTDBDatabaseSetAndDeleteIT {
           TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
getResp.getStatus().getCode());
       schemaMap = getResp.getDatabaseSchemaMap();
       Assert.assertEquals(1, schemaMap.size());
-      storageGroupSchema = schemaMap.get(sg1);
-      Assert.assertNotNull(storageGroupSchema);
-      Assert.assertEquals(sg1, storageGroupSchema.getName());
-      Assert.assertEquals(1, storageGroupSchema.getSchemaReplicationFactor());
-      Assert.assertEquals(1, storageGroupSchema.getDataReplicationFactor());
-      Assert.assertEquals(604800, 
storageGroupSchema.getTimePartitionInterval());
+      databaseSchema = schemaMap.get(sg1);
+      Assert.assertNotNull(databaseSchema);
+      Assert.assertEquals(sg1, databaseSchema.getName());
+      Assert.assertEquals(1, databaseSchema.getSchemaReplicationFactor());
+      Assert.assertEquals(1, databaseSchema.getDataReplicationFactor());
+      Assert.assertEquals(604800, databaseSchema.getTimePartitionInterval());
     }
   }
 
   @Test
-  public void testDeleteStorageGroup() throws Exception {
+  public void testDeleteDatabase() throws Exception {
     TSStatus status;
     final String sg0 = "root.sg0";
     final String sg1 = "root.sg1";
 
     try (SyncConfigNodeIServiceClient client =
         (SyncConfigNodeIServiceClient) 
EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
-      TDatabaseSchema storageGroupSchema0 = new TDatabaseSchema(sg0);
-      // set StorageGroup0 by default values
-      status = client.setDatabase(storageGroupSchema0);
+      TDatabaseSchema databaseSchema0 = new TDatabaseSchema(sg0);
+      // set Database0 by default values
+      status = client.setDatabase(databaseSchema0);
       Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
status.getCode());
-      // set StorageGroup1 by specific values
-      TDatabaseSchema storageGroupSchema1 = new TDatabaseSchema(sg1);
-      status = client.setDatabase(storageGroupSchema1);
+      // set Database1 by specific values
+      TDatabaseSchema databaseSchema1 = new TDatabaseSchema(sg1);
+      status = client.setDatabase(databaseSchema1);
       Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
status.getCode());
-      TDeleteDatabasesReq deleteStorageGroupsReq = new TDeleteDatabasesReq();
+      TDeleteDatabasesReq deleteDatabasesReq = new TDeleteDatabasesReq();
       List<String> sgs = Arrays.asList(sg0, sg1);
-      deleteStorageGroupsReq.setPrefixPathList(sgs);
-      TSStatus deleteSgStatus = client.deleteDatabases(deleteStorageGroupsReq);
+      deleteDatabasesReq.setPrefixPathList(sgs);
+      TSStatus deleteSgStatus = client.deleteDatabases(deleteDatabasesReq);
       TDatabaseSchemaResp root =
           client.getMatchedDatabaseSchemas(
               new TGetDatabaseReq(Arrays.asList("root", "sg*"), 
ALL_MATCH_SCOPE_BINARY));
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java
index a49d1501af5..2246ce4314a 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java
@@ -101,7 +101,7 @@ public class IoTDBConfigNodeSwitchLeaderIT {
 
     try (SyncConfigNodeIServiceClient client =
         (SyncConfigNodeIServiceClient) 
EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
-      // Set StorageGroups
+      // Set Databases
       status = client.setDatabase((new TDatabaseSchema(sg0)));
       Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
status.getCode());
       status = client.setDatabase((new TDatabaseSchema(sg1)));
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBAutoRegionGroupExtensionIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBAutoRegionGroupExtensionIT.java
index a82b992a4df..33481d9faae 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBAutoRegionGroupExtensionIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBAutoRegionGroupExtensionIT.java
@@ -100,9 +100,9 @@ public class IoTDBAutoRegionGroupExtensionIT {
     try (SyncConfigNodeIServiceClient client =
         (SyncConfigNodeIServiceClient) 
EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
 
-      setStorageGroupAndCheckRegionGroupDistribution(client);
+      setDatabaseAndCheckRegionGroupDistribution(client);
 
-      // Delete all StorageGroups
+      // Delete all Databases
       for (int i = 0; i < TEST_DATABASE_NUM; i++) {
         String curSg = DATABASE + i;
         client.deleteDatabase(new TDeleteDatabaseReq(curSg));
@@ -126,11 +126,11 @@ public class IoTDBAutoRegionGroupExtensionIT {
       Assert.assertTrue(isAllRegionGroupDeleted);
 
       // Re-test for safety
-      setStorageGroupAndCheckRegionGroupDistribution(client);
+      setDatabaseAndCheckRegionGroupDistribution(client);
     }
   }
 
-  private void 
setStorageGroupAndCheckRegionGroupDistribution(SyncConfigNodeIServiceClient 
client)
+  private void 
setDatabaseAndCheckRegionGroupDistribution(SyncConfigNodeIServiceClient client)
       throws TException, IllegalPathException, IOException {
 
     for (int i = 0; i < TEST_DATABASE_NUM; i++) {
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBCustomRegionGroupExtensionIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBCustomRegionGroupExtensionIT.java
index 820e6835c59..64b35875a20 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBCustomRegionGroupExtensionIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBCustomRegionGroupExtensionIT.java
@@ -98,7 +98,7 @@ public class IoTDBCustomRegionGroupExtensionIT {
       for (int i = 0; i < testSgNum; i++) {
         String curSg = sg + i;
 
-        /* Set StorageGroup */
+        /* Set Database */
         TSStatus status = client.setDatabase(new TDatabaseSchema(curSg));
         Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
status.getCode());
 
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionCreationIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionCreationIT.java
index 5043b4a8426..954f54e01bd 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionCreationIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionCreationIT.java
@@ -96,10 +96,10 @@ public class IoTDBPartitionCreationIT {
     // Init 1C3D environment
     EnvFactory.getEnv().initClusterEnvironment(1, 3);
 
-    setStorageGroup();
+    setDatabase();
   }
 
-  private void setStorageGroup() throws Exception {
+  private void setDatabase() throws Exception {
     try (SyncConfigNodeIServiceClient client =
         (SyncConfigNodeIServiceClient) 
EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
       TSStatus status = client.setDatabase(new TDatabaseSchema(sg));
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionDurableIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionDurableIT.java
index ceedc355295..8d6e4a5f831 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionDurableIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionDurableIT.java
@@ -104,10 +104,10 @@ public class IoTDBPartitionDurableIT {
     // Init 1C3D environment
     EnvFactory.getEnv().initClusterEnvironment(1, 3);
 
-    setStorageGroup();
+    setDatabase();
   }
 
-  private void setStorageGroup() throws Exception {
+  private void setDatabase() throws Exception {
     try (SyncConfigNodeIServiceClient client =
         (SyncConfigNodeIServiceClient) 
EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
       TSStatus status = client.setDatabase(new TDatabaseSchema(sg));
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
index 6ca5d884b07..f110d4a8d28 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
@@ -84,7 +84,7 @@ public class IoTDBPartitionGetterIT {
   private static final int testDataRegionGroupPerDatabase = 5;
 
   private static final String sg = "root.sg";
-  private static final int storageGroupNum = 2;
+  private static final int databaseNum = 2;
   private static final int testSeriesPartitionSlotNum = 1000;
   private static final int seriesPartitionBatchSize = 10;
   private static final int testTimePartitionSlotsNum = 10;
@@ -116,8 +116,8 @@ public class IoTDBPartitionGetterIT {
   private static void prepareData() throws Exception {
     try (SyncConfigNodeIServiceClient client =
         (SyncConfigNodeIServiceClient) 
EnvFactory.getEnv().getLeaderConfigNodeConnection()) {
-      /* Set StorageGroups */
-      for (int i = 0; i < storageGroupNum; i++) {
+      /* Set Databases */
+      for (int i = 0; i < databaseNum; i++) {
         TSStatus status = client.setDatabase(new TDatabaseSchema(sg + i));
         Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
status.getCode());
       }
@@ -149,13 +149,13 @@ public class IoTDBPartitionGetterIT {
       }
 
       /* Create DataPartitions */
-      for (int i = 0; i < storageGroupNum; i++) {
-        String storageGroup = sg + i;
+      for (int i = 0; i < databaseNum; i++) {
+        String database = sg + i;
         for (int j = 0; j < testSeriesPartitionSlotNum; j += 
seriesPartitionBatchSize) {
           for (long k = 0; k < testTimePartitionSlotsNum; k += 
timePartitionBatchSize) {
             Map<String, Map<TSeriesPartitionSlot, TTimeSlotList>> 
partitionSlotsMap =
                 ConfigNodeTestUtils.constructPartitionSlotsMap(
-                    storageGroup,
+                    database,
                     j,
                     j + seriesPartitionBatchSize,
                     k,
@@ -187,7 +187,7 @@ public class IoTDBPartitionGetterIT {
                 dataPartitionTableResp.getStatus().getCode());
             
Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable());
             ConfigNodeTestUtils.checkDataPartitionTable(
-                storageGroup,
+                database,
                 j,
                 j + seriesPartitionBatchSize,
                 k,
@@ -229,7 +229,7 @@ public class IoTDBPartitionGetterIT {
           schemaPartitionTableResp.getStatus().getCode());
       Assert.assertEquals(0, 
schemaPartitionTableResp.getSchemaPartitionTableSize());
 
-      // Test getSchemaPartition, when a device path doesn't match any 
StorageGroup and including
+      // Test getSchemaPartition, when a device path doesn't match any 
Database and including
       // "**", ConfigNode will return all the SchemaPartitions
       buffer = generatePatternTreeBuffer(new String[] {allPaths});
       schemaPartitionReq.setPathPatternTree(buffer);
@@ -244,8 +244,8 @@ public class IoTDBPartitionGetterIT {
         Assert.assertEquals(2, schemaPartitionTable.get(sg + i).size());
       }
 
-      // Test getSchemaPartition, when a device path matches with a 
StorageGroup and end with "*",
-      // ConfigNode will return all the SchemaPartitions in this StorageGroup
+      // Test getSchemaPartition, when a device path matches with a Database 
and end with "*",
+      // ConfigNode will return all the SchemaPartitions in this Database
       buffer = generatePatternTreeBuffer(new String[] {allSg0, d11});
       schemaPartitionReq.setPathPatternTree(buffer);
       schemaPartitionTableResp = 
client.getSchemaPartitionTable(schemaPartitionReq);
@@ -295,17 +295,16 @@ public class IoTDBPartitionGetterIT {
       }
       int leastDataRegionGroupNum =
           (int)
-              Math.ceil(
-                  (double) totalCpuCoreNum / (double) (storageGroupNum * 
testReplicationFactor));
+              Math.ceil((double) totalCpuCoreNum / (double) (databaseNum * 
testReplicationFactor));
       leastDataRegionGroupNum = Math.min(leastDataRegionGroupNum, 
testDataRegionGroupPerDatabase);
 
-      for (int i = 0; i < storageGroupNum; i++) {
-        String storageGroup = sg + i;
+      for (int i = 0; i < databaseNum; i++) {
+        String database = sg + i;
         for (int j = 0; j < testSeriesPartitionSlotNum; j += 
seriesPartitionBatchSize) {
           for (long k = 0; k < testTimePartitionSlotsNum; k += 
timePartitionBatchSize) {
             partitionSlotsMap =
                 ConfigNodeTestUtils.constructPartitionSlotsMap(
-                    storageGroup,
+                    database,
                     j,
                     j + seriesPartitionBatchSize,
                     k,
@@ -320,7 +319,7 @@ public class IoTDBPartitionGetterIT {
                 dataPartitionTableResp.getStatus().getCode());
             
Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable());
             ConfigNodeTestUtils.checkDataPartitionTable(
-                storageGroup,
+                database,
                 j,
                 j + seriesPartitionBatchSize,
                 k,
@@ -334,10 +333,10 @@ public class IoTDBPartitionGetterIT {
         // And this number should be greater than or equal to 
leastDataRegionGroupNum
         TGetDatabaseReq req =
             new TGetDatabaseReq(
-                Arrays.asList(storageGroup.split("\\.")), 
SchemaConstant.ALL_MATCH_SCOPE_BINARY);
-        TShowDatabaseResp showStorageGroupResp = client.showDatabase(req);
+                Arrays.asList(database.split("\\.")), 
SchemaConstant.ALL_MATCH_SCOPE_BINARY);
+        TShowDatabaseResp showDatabaseResp = client.showDatabase(req);
         Assert.assertTrue(
-            
showStorageGroupResp.getDatabaseInfoMap().get(storageGroup).getDataRegionNum()
+            
showDatabaseResp.getDatabaseInfoMap().get(database).getDataRegionNum()
                 >= leastDataRegionGroupNum);
       }
     }
@@ -394,7 +393,7 @@ public class IoTDBPartitionGetterIT {
           TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
getRegionIdResp.status.getCode());
 
       // Get all RegionIds within database
-      for (int i = 0; i < storageGroupNum; i++) {
+      for (int i = 0; i < databaseNum; i++) {
         String curSg = sg + i;
         getRegionIdReq = new TGetRegionIdReq(TConsensusGroupType.DataRegion);
         getRegionIdReq.setDatabase(curSg);
@@ -536,7 +535,7 @@ public class IoTDBPartitionGetterIT {
       Assert.assertEquals(
           TSStatusCode.SUCCESS_STATUS.getStatusCode(), 
nodeManagementResp.getStatus().getCode());
       // +1 for AUDIT database
-      Assert.assertEquals(storageGroupNum + 1, 
nodeManagementResp.getMatchedNodeSize());
+      Assert.assertEquals(databaseNum + 1, 
nodeManagementResp.getMatchedNodeSize());
       Assert.assertNotNull(nodeManagementResp.getSchemaRegionMap());
       Assert.assertEquals(0, nodeManagementResp.getSchemaRegionMapSize());
     }
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
index dfc37e52526..0697a6c674d 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
@@ -120,24 +120,24 @@ public class ConfigNodeTestUtils {
   }
 
   public static Map<String, Map<TSeriesPartitionSlot, TTimeSlotList>> 
constructPartitionSlotsMap(
-      String storageGroup,
+      String database,
       int seriesSlotStart,
       int seriesSlotEnd,
       long timeSlotStart,
       long timeSlotEnd,
       long timePartitionInterval) {
     Map<String, Map<TSeriesPartitionSlot, TTimeSlotList>> result = new 
HashMap<>();
-    result.put(storageGroup, new HashMap<>());
+    result.put(database, new HashMap<>());
 
     for (int i = seriesSlotStart; i < seriesSlotEnd; i++) {
       TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(i);
       result
-          .get(storageGroup)
+          .get(database)
           .put(seriesPartitionSlot, new 
TTimeSlotList().setTimePartitionSlots(new ArrayList<>()));
       for (long j = timeSlotStart; j < timeSlotEnd; j++) {
         TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot(j * 
timePartitionInterval);
         result
-            .get(storageGroup)
+            .get(database)
             .get(seriesPartitionSlot)
             .getTimePartitionSlots()
             .add(timePartitionSlot);
@@ -148,7 +148,7 @@ public class ConfigNodeTestUtils {
   }
 
   public static void checkDataPartitionTable(
-      String storageGroup,
+      String database,
       int seriesSlotStart,
       int seriesSlotEnd,
       long timeSlotStart,
@@ -157,12 +157,12 @@ public class ConfigNodeTestUtils {
       Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, 
List<TConsensusGroupId>>>>
           dataPartitionTable) {
 
-    // Check the existence of StorageGroup
-    Assert.assertTrue(dataPartitionTable.containsKey(storageGroup));
+    // Check the existence of Database
+    Assert.assertTrue(dataPartitionTable.containsKey(database));
 
     // Check the number of SeriesPartitionSlot
     Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TConsensusGroupId>>>
-        seriesPartitionTable = dataPartitionTable.get(storageGroup);
+        seriesPartitionTable = dataPartitionTable.get(database);
     Assert.assertEquals(seriesSlotEnd - seriesSlotStart, 
seriesPartitionTable.size());
 
     for (int i = seriesSlotStart; i < seriesSlotEnd; i++) {
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBExampleIT.java 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBExampleIT.java
index 91bf6462033..d51a058ec51 100644
--- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBExampleIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBExampleIT.java
@@ -58,8 +58,8 @@ public class IoTDBExampleIT {
       statement.execute("CREATE DATABASE root.sg");
       try (ResultSet resultSet = statement.executeQuery("show databases 
root.sg")) {
         if (resultSet.next()) {
-          String storageGroupPath = 
resultSet.getString(ColumnHeaderConstant.DATABASE);
-          Assert.assertEquals("root.sg", storageGroupPath);
+          String databasePath = 
resultSet.getString(ColumnHeaderConstant.DATABASE);
+          Assert.assertEquals("root.sg", databasePath);
         } else {
           Assert.fail("This ResultSet is empty.");
         }
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFlushQueryIT.java 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFlushQueryIT.java
index ae039b792b1..6253e618eda 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFlushQueryIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFlushQueryIT.java
@@ -26,7 +26,6 @@ import org.apache.iotdb.itbase.category.LocalStandaloneIT;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -166,7 +165,6 @@ public class IoTDBFlushQueryIT {
   }
 
   @Test
-  @Ignore
   public void testFlushNotExistGroupNoData() {
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
@@ -175,8 +173,7 @@ public class IoTDBFlushQueryIT {
         statement.execute(
             "FLUSH 
root.noexist.nodatagroup1,root.notExistGroup1,root.notExistGroup2");
       } catch (SQLException sqe) {
-        String expectedMsg =
-            "322: 322: storageGroup root.notExistGroup1,root.notExistGroup2 
does not exist";
+        String expectedMsg = "500: Database 
root.notExistGroup1,root.notExistGroup2 does not exist";
         sqe.printStackTrace();
         assertTrue(sqe.getMessage().contains(expectedMsg));
       }
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBRestServiceIT.java 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBRestServiceIT.java
index 7d6d8774385..d84a2493cc2 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBRestServiceIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBRestServiceIT.java
@@ -393,7 +393,7 @@ public class IoTDBRestServiceIT {
     queryShowChildPaths(httpClient);
     queryShowNodes(httpClient);
     showAllTTL(httpClient);
-    showStorageGroup(httpClient);
+    showDatabase(httpClient);
     showFunctions(httpClient);
     showTimeseries(httpClient);
 
@@ -413,7 +413,7 @@ public class IoTDBRestServiceIT {
     queryShowChildPathsV2(httpClient);
     queryShowNodesV2(httpClient);
     showAllTTLV2(httpClient);
-    showStorageGroupV2(httpClient);
+    showDatabaseV2(httpClient);
     showFunctionsV2(httpClient);
     showTimeseriesV2(httpClient);
 
@@ -1094,7 +1094,7 @@ public class IoTDBRestServiceIT {
     Assert.assertEquals(values2, valuesResult.get(1));
   }
 
-  public void showStorageGroup(CloseableHttpClient httpClient) {
+  public void showDatabase(CloseableHttpClient httpClient) {
     String sql = "{\"sql\":\"SHOW DATABASES root.sg25\"}";
     Map map = queryMetaData(httpClient, sql);
     List<String> columnNamesResult = (List<String>) map.get("columnNames");
@@ -1757,7 +1757,7 @@ public class IoTDBRestServiceIT {
     Assert.assertEquals(values2, valuesResult.get(1));
   }
 
-  public void showStorageGroupV2(CloseableHttpClient httpClient) {
+  public void showDatabaseV2(CloseableHttpClient httpClient) {
     String sql = "{\"sql\":\"SHOW DATABASES root.sg25\"}";
     Map map = queryMetaDataV2(httpClient, sql);
     List<String> columnNamesResult = (List<String>) map.get("column_names");
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java
index aceec6d2075..96ee0bc30c1 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java
@@ -188,7 +188,7 @@ public class IoTDBSetConfigurationIT {
 
     // can start with an illegal value
     EnvFactory.getEnv().cleanClusterEnvironment();
-    
EnvFactory.getEnv().getConfig().getCommonConfig().setDefaultStorageGroupLevel(-1);
+    
EnvFactory.getEnv().getConfig().getCommonConfig().setDefaultDatabaseLevel(-1);
     EnvFactory.getEnv().initClusterEnvironment();
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java
index 19ca146e2e6..c56c0077f29 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java
@@ -1040,7 +1040,7 @@ public class IoTDBSimpleQueryIT {
   }
 
   @Test
-  public void testStorageGroupWithHyphenInName() {
+  public void testDatabaseWithHyphenInName() {
     try (final Connection connection = EnvFactory.getEnv().getConnection();
         final Statement statement = connection.createStatement()) {
       statement.setFetchSize(5);
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSyntaxConventionIdentifierIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSyntaxConventionIdentifierIT.java
index 8be33862554..c21e4e347c4 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSyntaxConventionIdentifierIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSyntaxConventionIdentifierIT.java
@@ -388,7 +388,7 @@ public class IoTDBSyntaxConventionIdentifierIT {
   }
 
   @Test
-  public void testCreateIllegalStorageGroup() {
+  public void testCreateIllegalDatabase() {
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
 
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/path/IoTDBQuotedPathIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/path/IoTDBQuotedPathIT.java
index 4cba7008a22..ed98ff2eac8 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/path/IoTDBQuotedPathIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/path/IoTDBQuotedPathIT.java
@@ -101,13 +101,13 @@ public class IoTDBQuotedPathIT {
   }
 
   @Test
-  public void testIllegalStorageGroup() {
+  public void testIllegalDatabase() {
     try (final Connection connection = EnvFactory.getEnv().getConnection();
         final Statement statement = connection.createStatement()) {
       statement.execute("CREATE DATABASE root.`\"ln`");
     } catch (final SQLException e) {
       Assert.assertTrue(
-          e.getMessage().contains("Error StorageGroup name")
+          e.getMessage().contains("Error Database name")
               || e.getMessage()
                   .contains(
                       "The database name can only contain english or chinese 
characters, numbers, backticks and underscores."));
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAutoCreateSchemaIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAutoCreateSchemaIT.java
index 5453fdc9ab1..620411b8c78 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAutoCreateSchemaIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAutoCreateSchemaIT.java
@@ -164,12 +164,12 @@ public class IoTDBAutoCreateSchemaIT extends 
AbstractSchemaIT {
    */
   @Test
   public void testInsertAutoCreate2() throws Exception {
-    String storageGroup = "root.sg2.a.b.c";
+    String database = "root.sg2.a.b.c";
     String timeSeriesPrefix = "root.sg2.a.b";
 
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
-      statement.execute(String.format("CREATE DATABASE %s", storageGroup));
+      statement.execute(String.format("CREATE DATABASE %s", database));
       try {
         statement.execute(
             String.format("INSERT INTO %s(timestamp, c) values(123, 
\"aabb\")", timeSeriesPrefix));
@@ -177,18 +177,18 @@ public class IoTDBAutoCreateSchemaIT extends 
AbstractSchemaIT {
       }
 
       // ensure that current database in cache is right.
-      InsertAutoCreate2Tool(statement, storageGroup, timeSeriesPrefix);
+      InsertAutoCreate2Tool(statement, database, timeSeriesPrefix);
     }
     // todo restart test
     //    EnvironmentUtils.stopDaemon();
     //    setUp();
     //
     //    // ensure that database in cache is right after recovering.
-    //    InsertAutoCreate2Tool(storageGroup, timeSeriesPrefix);
+    //    InsertAutoCreate2Tool(database, timeSeriesPrefix);
   }
 
-  private void InsertAutoCreate2Tool(
-      Statement statement, String storageGroup, String timeSeriesPrefix) 
throws SQLException {
+  private void InsertAutoCreate2Tool(Statement statement, String database, 
String timeSeriesPrefix)
+      throws SQLException {
     Set<String> resultList = new HashSet<>();
     try (ResultSet resultSet = statement.executeQuery("show timeseries")) {
       while (resultSet.next()) {
@@ -204,7 +204,7 @@ public class IoTDBAutoCreateSchemaIT extends 
AbstractSchemaIT {
         resultList.add(resultSet.getString(ColumnHeaderConstant.DATABASE));
       }
     }
-    Assert.assertTrue(resultList.contains(storageGroup));
+    Assert.assertTrue(resultList.contains(database));
   }
 
   /**
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java
index eb02ca1d7ee..22572aac6ee 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java
@@ -170,7 +170,7 @@ public class IoTDBDeactivateTemplateIT extends 
AbstractSchemaIT {
   }
 
   @Test
-  public void deactivateTemplateCrossStorageGroupTest() throws Exception {
+  public void deactivateTemplateCrossDatabaseTest() throws Exception {
     String insertSql = "insert into root.sg%d.d2(time, s1, s2) values(%d, %d, 
%d)";
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeleteTimeSeriesIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeleteTimeSeriesIT.java
index c278fd89f0a..0866b37836e 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeleteTimeSeriesIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeleteTimeSeriesIT.java
@@ -304,7 +304,7 @@ public class IoTDBDeleteTimeSeriesIT extends 
AbstractSchemaIT {
   }
 
   @Test
-  public void deleteTimeSeriesCrossStorageGroupTest() throws Exception {
+  public void deleteTimeSeriesCrossDatabaseTest() throws Exception {
     String[] retArray1 = new String[] {"4,4,4,4"};
 
     String insertSql = "insert into root.sg%d.d1(time, s1, s2) values(%d, %d, 
%d)";
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java
index cb2d1ff3d47..1b6818d39f0 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java
@@ -592,7 +592,7 @@ public class IoTDBMetadataFetchIT extends AbstractSchemaIT {
   }
 
   @Test
-  public void showCountStorageGroup() throws SQLException {
+  public void showCountDatabase() throws SQLException {
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
       String[] sqls =
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java
index 8ed8d1ee29a..a403758cc36 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java
@@ -1073,7 +1073,7 @@ public class IoTDBTagIT extends AbstractSchemaIT {
   }
 
   @Test
-  public void deleteStorageGroupTest() {
+  public void deleteDatabaseTest() {
     List<String> ret =
         Collections.singletonList(
             "root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,"
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java
index d89c7a6b3e4..b97dc4915a1 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java
@@ -606,7 +606,7 @@ public class IoTDBSimpleQueryTableIT {
   }
 
   @Test
-  public void testStorageGroupWithHyphenInName() {
+  public void testDatabaseWithHyphenInName() {
     try (final Connection connection =
             EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT);
         final Statement statement = connection.createStatement()) {
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java
index e60eb5471c9..38f6345bae5 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java
@@ -406,11 +406,11 @@ public class IoTDBSessionSimpleIT {
         session.close();
         return;
       }
-      String storageGroup = "root.存储组1";
+      String database = "root.存储组1";
       String[] devices = new String[] {"设备1.指标1", "设备1.s2", "d2.s1", "d2.指标2"};
-      session.setStorageGroup(storageGroup);
+      session.setStorageGroup(database);
       for (String path : devices) {
-        String fullPath = storageGroup + TsFileConstant.PATH_SEPARATOR + path;
+        String fullPath = database + TsFileConstant.PATH_SEPARATOR + path;
         session.createTimeseries(
             fullPath, TSDataType.INT64, TSEncoding.RLE, 
CompressionType.SNAPPY);
       }
@@ -418,7 +418,7 @@ public class IoTDBSessionSimpleIT {
       for (String path : devices) {
         for (int i = 0; i < 10; i++) {
           String[] ss = path.split("\\.");
-          StringBuilder deviceId = new StringBuilder(storageGroup);
+          StringBuilder deviceId = new StringBuilder(database);
           for (int j = 0; j < ss.length - 1; j++) {
             deviceId.append(TsFileConstant.PATH_SEPARATOR).append(ss[j]);
           }
@@ -441,7 +441,7 @@ public class IoTDBSessionSimpleIT {
         count++;
       }
       assertEquals(10, count);
-      session.deleteStorageGroup(storageGroup);
+      session.deleteDatabase(database);
     } catch (Exception e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -757,8 +757,8 @@ public class IoTDBSessionSimpleIT {
         session.close();
         return;
       }
-      String storageGroup = "root.sg";
-      session.setStorageGroup(storageGroup);
+      String database = "root.sg";
+      session.setStorageGroup(database);
 
       session.createTimeseries(
           "root.sg.`my.device.with.colon:`.s",
@@ -769,7 +769,7 @@ public class IoTDBSessionSimpleIT {
       final SessionDataSet dataSet = session.executeQueryStatement("SHOW 
TIMESERIES");
       assertTrue(dataSet.hasNext());
 
-      session.deleteStorageGroup(storageGroup);
+      session.deleteDatabase(database);
     } catch (Exception e) {
       e.printStackTrace();
       fail(e.getMessage());
@@ -785,8 +785,8 @@ public class IoTDBSessionSimpleIT {
         session.close();
         return;
       }
-      String storageGroup = "root.sg";
-      session.setStorageGroup(storageGroup);
+      String database = "root.sg";
+      session.setStorageGroup(database);
 
       try {
         session.createTimeseries(
@@ -804,7 +804,7 @@ public class IoTDBSessionSimpleIT {
       final SessionDataSet dataSet = session.executeQueryStatement("SHOW 
TIMESERIES root.sg.**");
       assertFalse(dataSet.hasNext());
 
-      session.deleteStorageGroup(storageGroup);
+      session.deleteDatabase(database);
     } catch (Exception e) {
       e.printStackTrace();
       fail(e.getMessage());
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java
index ac0c7501d26..ac65c6be784 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java
@@ -69,8 +69,8 @@ public class IoTDBSessionSyntaxConventionIT {
   @Test
   public void createTimeSeriesTest() {
     try (ISession session = EnvFactory.getEnv().getSessionConnection()) {
-      String storageGroup = "root.sg";
-      session.setStorageGroup(storageGroup);
+      String database = "root.sg";
+      session.setStorageGroup(database);
 
       try {
         session.createTimeseries(
@@ -91,7 +91,7 @@ public class IoTDBSessionSyntaxConventionIT {
       final SessionDataSet dataSet = session.executeQueryStatement("SHOW 
TIMESERIES root.sg.**");
       assertFalse(dataSet.hasNext());
 
-      session.deleteStorageGroup(storageGroup);
+      session.deleteDatabase(database);
     } catch (Exception e) {
       e.printStackTrace();
       fail(e.getMessage());
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/PreDeleteDatabasePlan.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/PreDeleteDatabasePlan.java
index bc9313f92f1..743f305398c 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/PreDeleteDatabasePlan.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/PreDeleteDatabasePlan.java
@@ -29,25 +29,25 @@ import java.nio.ByteBuffer;
 import java.util.Objects;
 
 public class PreDeleteDatabasePlan extends ConfigPhysicalPlan {
-  private String storageGroup;
+  private String database;
   private PreDeleteType preDeleteType;
 
   public PreDeleteDatabasePlan() {
     super(ConfigPhysicalPlanType.PreDeleteDatabase);
   }
 
-  public PreDeleteDatabasePlan(String storageGroup, PreDeleteType 
preDeleteType) {
+  public PreDeleteDatabasePlan(String database, PreDeleteType preDeleteType) {
     this();
-    this.storageGroup = storageGroup;
+    this.database = database;
     this.preDeleteType = preDeleteType;
   }
 
-  public String getStorageGroup() {
-    return storageGroup;
+  public String getDatabase() {
+    return database;
   }
 
-  public void setStorageGroup(String storageGroup) {
-    this.storageGroup = storageGroup;
+  public void setDatabase(String database) {
+    this.database = database;
   }
 
   public PreDeleteType getPreDeleteType() {
@@ -57,13 +57,13 @@ public class PreDeleteDatabasePlan extends 
ConfigPhysicalPlan {
   @Override
   protected void serializeImpl(DataOutputStream stream) throws IOException {
     stream.writeShort(getType().getPlanType());
-    BasicStructureSerDeUtil.write(storageGroup, stream);
+    BasicStructureSerDeUtil.write(database, stream);
     stream.write(preDeleteType.getType());
   }
 
   @Override
   protected void deserializeImpl(ByteBuffer buffer) throws IOException {
-    this.storageGroup = BasicStructureSerDeUtil.readString(buffer);
+    this.database = BasicStructureSerDeUtil.readString(buffer);
     this.preDeleteType = buffer.get() == (byte) 1 ? PreDeleteType.ROLLBACK : 
PreDeleteType.EXECUTE;
   }
 
@@ -79,12 +79,12 @@ public class PreDeleteDatabasePlan extends 
ConfigPhysicalPlan {
       return false;
     }
     PreDeleteDatabasePlan that = (PreDeleteDatabasePlan) o;
-    return storageGroup.equals(that.storageGroup) && preDeleteType == 
that.preDeleteType;
+    return database.equals(that.database) && preDeleteType == 
that.preDeleteType;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(super.hashCode(), storageGroup, preDeleteType);
+    return Objects.hash(super.hashCode(), database, preDeleteType);
   }
 
   public enum PreDeleteType {
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 65883f3065d..502937713c2 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -3062,23 +3062,17 @@ public class ConfigManager implements IManager {
   @Override
   public TFetchTableResp fetchTables(final Map<String, Set<String>> 
fetchTableMap) {
     final TSStatus status = confirmLeader();
-    return status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()
-        ? clusterSchemaManager.fetchTables(
-            fetchTableMap.entrySet().stream()
-                .filter(
-                    entry -> {
-                      entry
-                          .getValue()
-                          .removeIf(
-                              table ->
-                                  procedureManager
-                                      .checkDuplicateTableTask(
-                                          entry.getKey(), null, table, null, 
null, null)
-                                      .getRight());
-                      return true;
-                    })
-                .collect(Collectors.toMap(Map.Entry::getKey, 
Map.Entry::getValue)))
-        : new TFetchTableResp(status);
+    if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      return new TFetchTableResp(status);
+    }
+    fetchTableMap.forEach(
+        (key, value) ->
+            value.removeIf(
+                table ->
+                    procedureManager
+                        .checkDuplicateTableTask(key, null, table, null, null, 
null)
+                        .getRight()));
+    return clusterSchemaManager.fetchTables(fetchTableMap);
   }
 
   @Override
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
index 3415c2d2d6e..e15c33e04b7 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
@@ -401,18 +401,18 @@ public interface IManager {
   TSStatus setTimePartitionInterval(SetTimePartitionIntervalPlan 
configPhysicalPlan);
 
   /**
-   * Count StorageGroups.
+   * Count Databases.
    *
-   * @return The number of matched StorageGroups
+   * @return The number of matched Databases
    */
   DataSet countMatchedDatabases(CountDatabasePlan countDatabasePlan);
 
   /**
-   * Get StorageGroupSchemas.
+   * Get DatabaseSchemas.
    *
-   * @return StorageGroupSchemaDataSet
+   * @return DatabaseSchemaDataSet
    */
-  DataSet getMatchedDatabaseSchemas(GetDatabasePlan 
getOrCountStorageGroupPlan);
+  DataSet getMatchedDatabaseSchemas(GetDatabasePlan getOrCountDatabasePlan);
 
   /**
    * Set Database.
@@ -429,7 +429,7 @@ public interface IManager {
   TSStatus alterDatabase(DatabaseSchemaPlan databaseSchemaPlan);
 
   /**
-   * Delete StorageGroups.
+   * Delete Databases.
    *
    * @param tDeleteReq TDeleteDatabaseReq
    * @return status
@@ -660,10 +660,10 @@ public interface IManager {
   TShowConfigNodes4InformationSchemaResp showConfigNodes4InformationSchema();
 
   /**
-   * Show StorageGroup.
+   * Show Database.
    *
    * @param req TShowDatabaseReq
-   * @return TShowStorageGroupResp
+   * @return TShowDatabaseResp
    */
   TShowDatabaseResp showDatabase(TGetDatabaseReq req);
 
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
index cb470308134..2e4227af3fc 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
@@ -2112,6 +2112,33 @@ public class ProcedureManager {
     }
   }
 
+  public Map<String, List<String>> getAllExecutingTables() {
+    final Map<String, List<String>> result = new HashMap<>();
+    for (final Procedure<?> procedure : executor.getProcedures().values()) {
+      if (procedure.isFinished()) {
+        continue;
+      }
+      // CreateTableOrViewProcedure is covered by the default process, thus we 
can ignore it here
+      // Note that if a table is creating there will not be a working table, 
and the DN will either
+      // be updated by commit or fetch the CN tables
+      // And it won't be committed by other procedures because:
+      // if the preUpdate of other procedure has failed there will not be any 
commit here
+      // if it succeeded then it will go to the normal process and will not 
leave any problems
+      if (procedure instanceof AbstractAlterOrDropTableProcedure) {
+        result
+            .computeIfAbsent(
+                ((AbstractAlterOrDropTableProcedure<?>) 
procedure).getDatabase(),
+                k -> new ArrayList<>())
+            .add(((AbstractAlterOrDropTableProcedure<?>) 
procedure).getTableName());
+      }
+      if (procedure instanceof DeleteDatabaseProcedure
+          && ((DeleteDatabaseProcedure) 
procedure).getDeleteDatabaseSchema().isIsTableModel()) {
+        result.put(((DeleteDatabaseProcedure) procedure).getDatabase(), null);
+      }
+    }
+    return result;
+  }
+
   public TSStatus executeWithoutDuplicate(
       final String database,
       final TsTable table,
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java
index 54dd582551d..993bfc0e400 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java
@@ -103,7 +103,7 @@ public class LoadManager {
    * @param consensusGroupType TConsensusGroupType of RegionGroup to be 
allocated
    * @return CreateRegionGroupsPlan
    * @throws NotEnoughDataNodeException If there are not enough DataNodes
-   * @throws DatabaseNotExistsException If some specific StorageGroups don't 
exist
+   * @throws DatabaseNotExistsException If some specific Databases don't exist
    */
   public CreateRegionGroupsPlan allocateRegionGroups(
       final Map<String, Integer> allotmentMap, final TConsensusGroupType 
consensusGroupType)
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RegionBalancer.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RegionBalancer.java
index 3528cbe1360..10864e9fba9 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RegionBalancer.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RegionBalancer.java
@@ -74,7 +74,7 @@ public class RegionBalancer {
    * @param consensusGroupType {@link TConsensusGroupType} of the new 
RegionGroups
    * @return CreateRegionGroupsPlan
    * @throws NotEnoughDataNodeException When the number of DataNodes is not 
enough for allocation
-   * @throws DatabaseNotExistsException When some StorageGroups don't exist
+   * @throws DatabaseNotExistsException When some Databases don't exist
    */
   public CreateRegionGroupsPlan genRegionGroupsAllocationPlan(
       final Map<String, Integer> allotmentMap, final TConsensusGroupType 
consensusGroupType)
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java
index bfe6e830cba..9a04d9e9e59 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
 import org.apache.iotdb.commons.schema.SchemaConstant;
+import org.apache.iotdb.commons.schema.table.NonCommittableTsTable;
 import org.apache.iotdb.commons.schema.table.TableNodeStatus;
 import org.apache.iotdb.commons.schema.table.TreeViewSchema;
 import org.apache.iotdb.commons.schema.table.TsTable;
@@ -1253,8 +1254,31 @@ public class ClusterSchemaManager {
   }
 
   public byte[] getAllTableInfoForDataNodeActivation() {
+    // To guarantee the safety of fetched tables
+    // If DataNode discovered that the table is being altered, it will fetch 
it from configNode, and
+    // if it's still in execution, it can use the table temporarily
+    // However, if the database is deleting then it must fetch it from 
configNode, or else the table
+    // is considered to be non exist
+    final Map<String, List<String>> alteringTables =
+        configManager.getProcedureManager().getAllExecutingTables();
+    final Map<String, List<TsTable>> usingTableMap = 
clusterSchemaInfo.getAllUsingTables();
+    final Map<String, List<TsTable>> preCreateTableMap = 
clusterSchemaInfo.getAllPreCreateTables();
+    alteringTables.forEach(
+        (k, v) -> {
+          final List<TsTable> preCreateList =
+              preCreateTableMap.computeIfAbsent(k, database -> new 
ArrayList<>());
+          if (Objects.isNull(v)) {
+            usingTableMap
+                .remove(k)
+                .forEach(
+                    table -> preCreateList.add(new 
NonCommittableTsTable(table.getTableName())));
+          } else {
+            preCreateList.addAll(
+                
v.stream().map(NonCommittableTsTable::new).collect(Collectors.toList()));
+          }
+        });
     return TsTableInternalRPCUtil.serializeTableInitializationInfo(
-        clusterSchemaInfo.getAllUsingTables(), 
clusterSchemaInfo.getAllPreCreateTables());
+        usingTableMap, preCreateTableMap);
   }
 
   // endregion
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java
index 2067783307c..b907527416b 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java
@@ -292,7 +292,7 @@ public class PartitionInfo implements SnapshotProcessor {
   public TSStatus preDeleteDatabase(final PreDeleteDatabasePlan 
preDeleteDatabasePlan) {
     final PreDeleteDatabasePlan.PreDeleteType preDeleteType =
         preDeleteDatabasePlan.getPreDeleteType();
-    final String database = preDeleteDatabasePlan.getStorageGroup();
+    final String database = preDeleteDatabasePlan.getDatabase();
     final DatabasePartitionTable databasePartitionTable = 
databasePartitionTables.get(database);
     if (databasePartitionTable == null) {
       return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
index 6d14b8be050..3ea3f2d0654 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
@@ -37,7 +37,7 @@ import 
org.apache.iotdb.confignode.persistence.partition.maintainer.RegionDelete
 import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
 import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
 import org.apache.iotdb.confignode.procedure.impl.StateMachineProcedure;
-import 
org.apache.iotdb.confignode.procedure.state.schema.DeleteStorageGroupState;
+import org.apache.iotdb.confignode.procedure.state.schema.DeleteDatabaseState;
 import org.apache.iotdb.confignode.procedure.store.ProcedureType;
 import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
 import org.apache.iotdb.consensus.exception.ConsensusException;
@@ -57,7 +57,7 @@ import java.util.Map;
 import java.util.Objects;
 
 public class DeleteDatabaseProcedure
-    extends StateMachineProcedure<ConfigNodeProcedureEnv, 
DeleteStorageGroupState> {
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, DeleteDatabaseState> 
{
   private static final Logger LOG = 
LoggerFactory.getLogger(DeleteDatabaseProcedure.class);
   private static final int RETRY_THRESHOLD = 5;
 
@@ -82,8 +82,7 @@ public class DeleteDatabaseProcedure
   }
 
   @Override
-  protected Flow executeFromState(
-      final ConfigNodeProcedureEnv env, final DeleteStorageGroupState state)
+  protected Flow executeFromState(final ConfigNodeProcedureEnv env, final 
DeleteDatabaseState state)
       throws InterruptedException {
     if (deleteDatabaseSchema == null) {
       return Flow.NO_MORE_STATE;
@@ -95,14 +94,14 @@ public class DeleteDatabaseProcedure
               "[DeleteDatabaseProcedure] Pre delete database: {}", 
deleteDatabaseSchema.getName());
           env.preDeleteDatabase(
               PreDeleteDatabasePlan.PreDeleteType.EXECUTE, 
deleteDatabaseSchema.getName());
-          setNextState(DeleteStorageGroupState.INVALIDATE_CACHE);
+          setNextState(DeleteDatabaseState.INVALIDATE_CACHE);
           break;
         case INVALIDATE_CACHE:
           LOG.info(
               "[DeleteDatabaseProcedure] Invalidate cache of database: {}",
               deleteDatabaseSchema.getName());
           if (env.invalidateCache(deleteDatabaseSchema.getName())) {
-            setNextState(DeleteStorageGroupState.DELETE_DATABASE_SCHEMA);
+            setNextState(DeleteDatabaseState.DELETE_DATABASE_SCHEMA);
           } else {
             setFailure(new ProcedureException("[DeleteDatabaseProcedure] 
Invalidate cache failed"));
           }
@@ -240,8 +239,7 @@ public class DeleteDatabaseProcedure
   }
 
   @Override
-  protected void rollbackState(
-      final ConfigNodeProcedureEnv env, final DeleteStorageGroupState state)
+  protected void rollbackState(final ConfigNodeProcedureEnv env, final 
DeleteDatabaseState state)
       throws IOException, InterruptedException {
     switch (state) {
       case PRE_DELETE_DATABASE:
@@ -257,7 +255,7 @@ public class DeleteDatabaseProcedure
   }
 
   @Override
-  protected boolean isRollbackSupported(final DeleteStorageGroupState state) {
+  protected boolean isRollbackSupported(final DeleteDatabaseState state) {
     switch (state) {
       case PRE_DELETE_DATABASE:
       case INVALIDATE_CACHE:
@@ -268,18 +266,18 @@ public class DeleteDatabaseProcedure
   }
 
   @Override
-  protected DeleteStorageGroupState getState(final int stateId) {
-    return DeleteStorageGroupState.values()[stateId];
+  protected DeleteDatabaseState getState(final int stateId) {
+    return DeleteDatabaseState.values()[stateId];
   }
 
   @Override
-  protected int getStateId(final DeleteStorageGroupState 
deleteStorageGroupState) {
-    return deleteStorageGroupState.ordinal();
+  protected int getStateId(final DeleteDatabaseState deleteDatabaseState) {
+    return deleteDatabaseState.ordinal();
   }
 
   @Override
-  protected DeleteStorageGroupState getInitialState() {
-    return DeleteStorageGroupState.PRE_DELETE_DATABASE;
+  protected DeleteDatabaseState getInitialState() {
+    return DeleteDatabaseState.PRE_DELETE_DATABASE;
   }
 
   public String getDatabase() {
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteDatabaseState.java
similarity index 96%
rename from 
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java
rename to 
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteDatabaseState.java
index bc19a262e64..cde6b2bdd81 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteDatabaseState.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.confignode.procedure.state.schema;
 
-public enum DeleteStorageGroupState {
+public enum DeleteDatabaseState {
   PRE_DELETE_DATABASE,
   INVALIDATE_CACHE,
   DELETE_DATABASE_SCHEMA
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java
index 3f050b1184f..90d7f20fb25 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java
@@ -655,12 +655,12 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
         executeSchemaBlackListTask(
             req.getSchemaRegionIdList(),
             consensusGroupId -> {
-              final String storageGroup =
+              final String database =
                   schemaEngine
                       .getSchemaRegion(new 
SchemaRegionId(consensusGroupId.getId()))
                       .getDatabaseFullPath();
               final PathPatternTree filteredPatternTree =
-                  filterPathPatternTree(patternTree, storageGroup);
+                  filterPathPatternTree(patternTree, database);
               if (filteredPatternTree.isEmpty()) {
                 return new 
TSStatus(TSStatusCode.ONLY_LOGICAL_VIEW.getStatusCode());
               }
@@ -688,11 +688,11 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
     return executeInternalSchemaTask(
         req.getSchemaRegionIdList(),
         consensusGroupId -> {
-          String storageGroup =
+          String database =
               schemaEngine
                   .getSchemaRegion(new 
SchemaRegionId(consensusGroupId.getId()))
                   .getDatabaseFullPath();
-          PathPatternTree filteredPatternTree = 
filterPathPatternTree(patternTree, storageGroup);
+          PathPatternTree filteredPatternTree = 
filterPathPatternTree(patternTree, database);
           if (filteredPatternTree.isEmpty()) {
             return RpcUtils.SUCCESS_STATUS;
           }
@@ -889,12 +889,12 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
       Map<PartialPath, List<Integer>> templateSetInfo, TConsensusGroupId 
consensusGroupId) {
 
     Map<PartialPath, List<Integer>> result = new HashMap<>();
-    PartialPath storageGroupPath = getStorageGroupPath(consensusGroupId);
-    if (null != storageGroupPath) {
-      PartialPath storageGroupPattern = 
storageGroupPath.concatNode(MULTI_LEVEL_PATH_WILDCARD);
+    PartialPath databasePath = getDatabasePath(consensusGroupId);
+    if (null != databasePath) {
+      PartialPath databasePattern = 
databasePath.concatNode(MULTI_LEVEL_PATH_WILDCARD);
       templateSetInfo.forEach(
           (k, v) -> {
-            if (storageGroupPattern.overlapWith(k) || 
storageGroupPath.overlapWith(k)) {
+            if (databasePattern.overlapWith(k) || databasePath.overlapWith(k)) 
{
               result.put(k, v);
             }
           });
@@ -902,10 +902,10 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
     return result;
   }
 
-  private PartialPath getStorageGroupPath(TConsensusGroupId consensusGroupId) {
-    PartialPath storageGroupPath = null;
+  private PartialPath getDatabasePath(TConsensusGroupId consensusGroupId) {
+    PartialPath databasePath = null;
     try {
-      storageGroupPath =
+      databasePath =
           new PartialPath(
               schemaEngine
                   .getSchemaRegion(new 
SchemaRegionId(consensusGroupId.getId()))
@@ -913,7 +913,7 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
     } catch (IllegalPathException ignored) {
       // Won't reach here
     }
-    return storageGroupPath;
+    return databasePath;
   }
 
   @Override
@@ -1099,12 +1099,12 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
         executeInternalSchemaTask(
             req.getSchemaRegionIdList(),
             consensusGroupId -> {
-              final String storageGroup =
+              final String database =
                   schemaEngine
                       .getSchemaRegion(new 
SchemaRegionId(consensusGroupId.getId()))
                       .getDatabaseFullPath();
               final PathPatternTree filteredPatternTree =
-                  filterPathPatternTree(patternTree, storageGroup);
+                  filterPathPatternTree(patternTree, database);
               if (filteredPatternTree.isEmpty()) {
                 return RpcUtils.SUCCESS_STATUS;
               }
@@ -1132,12 +1132,11 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
     return executeInternalSchemaTask(
         req.getSchemaRegionIdList(),
         consensusGroupId -> {
-          final String storageGroup =
+          final String database =
               schemaEngine
                   .getSchemaRegion(new 
SchemaRegionId(consensusGroupId.getId()))
                   .getDatabaseFullPath();
-          final PathPatternTree filteredPatternTree =
-              filterPathPatternTree(patternTree, storageGroup);
+          final PathPatternTree filteredPatternTree = 
filterPathPatternTree(patternTree, database);
           if (filteredPatternTree.isEmpty()) {
             return RpcUtils.SUCCESS_STATUS;
           }
@@ -1157,11 +1156,11 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
     return executeInternalSchemaTask(
         req.getSchemaRegionIdList(),
         consensusGroupId -> {
-          String storageGroup =
+          String database =
               schemaEngine
                   .getSchemaRegion(new 
SchemaRegionId(consensusGroupId.getId()))
                   .getDatabaseFullPath();
-          PathPatternTree filteredPatternTree = 
filterPathPatternTree(patternTree, storageGroup);
+          PathPatternTree filteredPatternTree = 
filterPathPatternTree(patternTree, database);
           if (filteredPatternTree.isEmpty()) {
             return RpcUtils.SUCCESS_STATUS;
           }
@@ -2070,12 +2069,11 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
     return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
   }
 
-  private PathPatternTree filterPathPatternTree(PathPatternTree patternTree, 
String storageGroup) {
+  private PathPatternTree filterPathPatternTree(PathPatternTree patternTree, 
String database) {
     PathPatternTree filteredPatternTree = new PathPatternTree();
     try {
-      PartialPath storageGroupPattern =
-          new PartialPath(storageGroup).concatNode(MULTI_LEVEL_PATH_WILDCARD);
-      for (PartialPath pathPattern : 
patternTree.getOverlappedPathPatterns(storageGroupPattern)) {
+      PartialPath databasePattern = new 
PartialPath(database).concatNode(MULTI_LEVEL_PATH_WILDCARD);
+      for (PartialPath pathPattern : 
patternTree.getOverlappedPathPatterns(databasePattern)) {
         filteredPatternTree.appendPathPattern(pathPattern);
       }
       filteredPatternTree.constructTree();
@@ -2733,8 +2731,8 @@ public class DataNodeInternalRPCServiceImpl implements 
IDataNodeRPCService.Iface
     return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
   }
 
-  private TSStatus createNewRegion(ConsensusGroupId regionId, String 
storageGroup) {
-    return regionManager.createNewRegion(regionId, storageGroup);
+  private TSStatus createNewRegion(ConsensusGroupId regionId, String database) 
{
+    return regionManager.createNewRegion(regionId, database);
   }
 
   @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java
index 30e8426c707..34a289b76c9 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java
@@ -3295,7 +3295,7 @@ public class AnalyzeVisitor extends 
StatementVisitor<Analysis, MPPQueryContext>
   }
 
   @Override
-  public Analysis visitCountStorageGroup(
+  public Analysis visitCountDatabase(
       CountDatabaseStatement countDatabaseStatement, MPPQueryContext context) {
     Analysis analysis = new Analysis();
     analysis.setRealStatement(countDatabaseStatement);
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java
index 1fdf636c33d..40724d4df15 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java
@@ -263,8 +263,7 @@ public class TreeConfigTaskVisitor extends 
StatementVisitor<IConfigTask, MPPQuer
   }
 
   @Override
-  public IConfigTask visitCountStorageGroup(
-      CountDatabaseStatement statement, MPPQueryContext context) {
+  public IConfigTask visitCountDatabase(CountDatabaseStatement statement, 
MPPQueryContext context) {
     return new CountDatabaseTask(statement);
   }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java
index c4e8664cf6d..64d606ef72e 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java
@@ -943,7 +943,7 @@ public class ASTVisitor extends 
IoTDBSqlParserBaseVisitor<Statement> {
     return new CountNodesStatement(path, level);
   }
 
-  // Count StorageGroup 
========================================================================
+  // Count Database 
========================================================================
   @Override
   public Statement visitCountDatabases(CountDatabasesContext ctx) {
     PartialPath path;
@@ -3574,18 +3574,18 @@ public class ASTVisitor extends 
IoTDBSqlParserBaseVisitor<Statement> {
   @Override
   public Statement visitFlush(final IoTDBSqlParser.FlushContext ctx) {
     final FlushStatement flushStatement = new 
FlushStatement(StatementType.FLUSH);
-    List<String> storageGroups = null;
+    List<String> databases = null;
     if (ctx.boolean_literal() != null) {
       
flushStatement.setSeq(Boolean.parseBoolean(ctx.boolean_literal().getText()));
     }
     flushStatement.setOnCluster(ctx.LOCAL() == null);
     if (ctx.prefixPath(0) != null) {
-      storageGroups = new ArrayList<>();
+      databases = new ArrayList<>();
       for (final IoTDBSqlParser.PrefixPathContext prefixPathContext : 
ctx.prefixPath()) {
-        storageGroups.add(parsePrefixPath(prefixPathContext).getFullPath());
+        databases.add(parsePrefixPath(prefixPathContext).getFullPath());
       }
     }
-    flushStatement.setDatabases(storageGroups);
+    flushStatement.setDatabases(databases);
     return flushStatement;
   }
 
@@ -3741,14 +3741,14 @@ public class ASTVisitor extends 
IoTDBSqlParserBaseVisitor<Statement> {
     }
 
     if (ctx.OF() != null) {
-      List<PartialPath> storageGroups = null;
+      List<PartialPath> databases = null;
       if (ctx.prefixPath(0) != null) {
-        storageGroups = new ArrayList<>();
+        databases = new ArrayList<>();
         for (IoTDBSqlParser.PrefixPathContext prefixPathContext : 
ctx.prefixPath()) {
-          storageGroups.add(parsePrefixPath(prefixPathContext));
+          databases.add(parsePrefixPath(prefixPathContext));
         }
       }
-      showRegionStatement.setDatabases(storageGroups);
+      showRegionStatement.setDatabases(databases);
     } else {
       showRegionStatement.setDatabases(null);
     }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableDeviceSchemaFetcher.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableDeviceSchemaFetcher.java
index 9b0bea05bce..84c8c9c002e 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableDeviceSchemaFetcher.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableDeviceSchemaFetcher.java
@@ -459,11 +459,11 @@ public class TableDeviceSchemaFetcher {
   }
 
   public static IDeviceID convertTagValuesToDeviceID(
-      final String tableName, final String[] idValues) {
+      final String tableName, final String[] tagValues) {
     // Convert to IDeviceID
-    final String[] deviceIdNodes = new String[idValues.length + 1];
+    final String[] deviceIdNodes = new String[tagValues.length + 1];
     deviceIdNodes[0] = tableName;
-    System.arraycopy(idValues, 0, deviceIdNodes, 1, idValues.length);
+    System.arraycopy(tagValues, 0, deviceIdNodes, 1, tagValues.length);
     return IDeviceID.Factory.DEFAULT_FACTORY.create(deviceIdNodes);
   }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java
index 9f4c2cf9fb6..d24ec3ef0d3 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java
@@ -959,7 +959,7 @@ public class TreeAccessCheckVisitor extends 
StatementVisitor<TSStatus, TreeAcces
   }
 
   @Override
-  public TSStatus visitCountStorageGroup(
+  public TSStatus visitCountDatabase(
       CountDatabaseStatement countDatabaseStatement, TreeAccessCheckContext 
context) {
     context
         .setAuditLogOperation(AuditLogOperation.QUERY)
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java
index 7ce212c3297..3f8bb158c09 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java
@@ -402,7 +402,7 @@ public abstract class StatementVisitor<R, C> {
     return visitStatement(showDevicesStatement, context);
   }
 
-  public R visitCountStorageGroup(CountDatabaseStatement 
countDatabaseStatement, C context) {
+  public R visitCountDatabase(CountDatabaseStatement countDatabaseStatement, C 
context) {
     return visitStatement(countDatabaseStatement, context);
   }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
index 4ff683f4436..2d186ce4173 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
@@ -32,7 +32,7 @@ public class CountDatabaseStatement extends CountStatement 
implements IConfigSta
 
   @Override
   public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitCountStorageGroup(this, context);
+    return visitor.visitCountDatabase(this, context);
   }
 
   @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java
index 53e8c2617bc..15d51928d64 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java
@@ -141,7 +141,7 @@ public class MTreeBelowSGMemoryImpl {
   private final MemMTreeStore store;
 
   @SuppressWarnings("java:S3077")
-  private volatile IMemMNode storageGroupMNode;
+  private volatile IMemMNode databaseMNode;
 
   private final IMemMNode rootNode;
   private final Function<IMeasurementMNode<IMemMNode>, Map<String, String>> 
tagGetter;
@@ -153,38 +153,38 @@ public class MTreeBelowSGMemoryImpl {
 
   // region MTree initialization, clear and serialization
   public MTreeBelowSGMemoryImpl(
-      final PartialPath storageGroupPath,
+      final PartialPath databasePath,
       final Function<IMeasurementMNode<IMemMNode>, Map<String, String>> 
tagGetter,
       final Function<IMeasurementMNode<IMemMNode>, Map<String, String>> 
attributeGetter,
       final MemSchemaRegionStatistics regionStatistics,
       final SchemaRegionMemMetric metric) {
-    store = new MemMTreeStore(storageGroupPath, regionStatistics, metric);
+    store = new MemMTreeStore(databasePath, regionStatistics, metric);
     this.regionStatistics = regionStatistics;
-    this.storageGroupMNode = store.getRoot();
-    this.rootNode = store.generatePrefix(storageGroupPath);
-    levelOfSG = storageGroupPath.getNodeLength() - 1;
+    this.databaseMNode = store.getRoot();
+    this.rootNode = store.generatePrefix(databasePath);
+    levelOfSG = databasePath.getNodeLength() - 1;
     this.tagGetter = tagGetter;
     this.attributeGetter = attributeGetter;
   }
 
   private MTreeBelowSGMemoryImpl(
-      final PartialPath storageGroupPath,
+      final PartialPath databasePath,
       final MemMTreeStore store,
       final Function<IMeasurementMNode<IMemMNode>, Map<String, String>> 
tagGetter,
       final Function<IMeasurementMNode<IMemMNode>, Map<String, String>> 
attributeGetter,
       final MemSchemaRegionStatistics regionStatistics) {
     this.store = store;
     this.regionStatistics = regionStatistics;
-    this.storageGroupMNode = store.getRoot();
-    this.rootNode = store.generatePrefix(storageGroupPath);
-    levelOfSG = storageGroupPath.getNodeLength() - 1;
+    this.databaseMNode = store.getRoot();
+    this.rootNode = store.generatePrefix(databasePath);
+    levelOfSG = databasePath.getNodeLength() - 1;
     this.tagGetter = tagGetter;
     this.attributeGetter = attributeGetter;
   }
 
   public void clear() {
     store.clear();
-    storageGroupMNode = null;
+    databaseMNode = null;
   }
 
   public synchronized boolean createSnapshot(final File snapshotDir) {
@@ -193,7 +193,7 @@ public class MTreeBelowSGMemoryImpl {
 
   public static MTreeBelowSGMemoryImpl loadFromSnapshot(
       final File snapshotDir,
-      final String storageGroupFullPath,
+      final String databaseFullPath,
       final MemSchemaRegionStatistics regionStatistics,
       final SchemaRegionMemMetric metric,
       final Consumer<IMeasurementMNode<IMemMNode>> measurementProcess,
@@ -203,7 +203,7 @@ public class MTreeBelowSGMemoryImpl {
       final Function<IMeasurementMNode<IMemMNode>, Map<String, String>> 
attributeGetter)
       throws IOException, IllegalPathException {
     return new MTreeBelowSGMemoryImpl(
-        PartialPath.getQualifiedDatabasePartialPath(storageGroupFullPath),
+        PartialPath.getQualifiedDatabasePartialPath(databaseFullPath),
         MemMTreeStore.loadFromSnapshot(
             snapshotDir,
             measurementProcess,
@@ -409,7 +409,7 @@ public class MTreeBelowSGMemoryImpl {
     if (nodeNames.length == levelOfSG + 1) {
       return null;
     }
-    IMemMNode cur = storageGroupMNode;
+    IMemMNode cur = databaseMNode;
     IMemMNode child;
     String childName;
     // e.g, path = root.sg.d1.s1,  create internal nodes and set cur to sg 
node, parent of d1
@@ -433,13 +433,12 @@ public class MTreeBelowSGMemoryImpl {
       throws PathAlreadyExistException, ExceedQuotaException {
     if (deviceParent == null) {
       // device is sg
-      return storageGroupMNode;
+      return databaseMNode;
     }
     IMemMNode device = store.getChild(deviceParent, deviceName);
     if (device == null) {
       if (IoTDBDescriptor.getInstance().getConfig().isQuotaEnable()) {
-        if (!DataNodeSpaceQuotaManager.getInstance()
-            .checkDeviceLimit(storageGroupMNode.getName())) {
+        if 
(!DataNodeSpaceQuotaManager.getInstance().checkDeviceLimit(databaseMNode.getName()))
 {
           throw new ExceedQuotaException(
               "The number of devices has reached the upper limit",
               TSStatusCode.SPACE_QUOTA_EXCEEDED.getStatusCode());
@@ -501,8 +500,7 @@ public class MTreeBelowSGMemoryImpl {
                 devicePath.getFullPath() + "." + measurementList.get(i), 
aliasList.get(i)));
       }
       if (IoTDBDescriptor.getInstance().getConfig().isQuotaEnable()) {
-        if (!DataNodeSpaceQuotaManager.getInstance()
-            .checkTimeSeriesNum(storageGroupMNode.getName())) {
+        if 
(!DataNodeSpaceQuotaManager.getInstance().checkTimeSeriesNum(databaseMNode.getName()))
 {
           failingMeasurementMap.put(
               i,
               new ExceedQuotaException(
@@ -747,7 +745,7 @@ public class MTreeBelowSGMemoryImpl {
       throws MetadataException {
     MetaFormatUtils.checkTimeseries(deviceId);
     final String[] nodeNames = deviceId.getNodes();
-    IMemMNode cur = storageGroupMNode;
+    IMemMNode cur = databaseMNode;
     IMemMNode child;
     for (int i = levelOfSG + 1; i < nodeNames.length; i++) {
       child = cur.getChild(nodeNames[i]);
@@ -894,7 +892,7 @@ public class MTreeBelowSGMemoryImpl {
    */
   public IMemMNode getNodeByPath(final PartialPath path) throws 
PathNotExistException {
     final String[] nodes = path.getNodes();
-    IMemMNode cur = storageGroupMNode;
+    IMemMNode cur = databaseMNode;
     IMemMNode next;
     for (int i = levelOfSG + 1; i < nodes.length; i++) {
       next = cur.getChild(nodes[i]);
@@ -930,7 +928,7 @@ public class MTreeBelowSGMemoryImpl {
   public void activateTemplate(final PartialPath activatePath, final Template 
template)
       throws MetadataException {
     final String[] nodes = activatePath.getNodes();
-    IMemMNode cur = storageGroupMNode;
+    IMemMNode cur = databaseMNode;
     for (int i = levelOfSG + 1; i < nodes.length; i++) {
       cur = cur.getChild(nodes[i]);
     }
@@ -1037,7 +1035,7 @@ public class MTreeBelowSGMemoryImpl {
   public void activateTemplateWithoutCheck(
       final PartialPath activatePath, final int templateId, final boolean 
isAligned) {
     final String[] nodes = activatePath.getNodes();
-    IMemMNode cur = storageGroupMNode;
+    IMemMNode cur = databaseMNode;
     for (int i = levelOfSG + 1; i < nodes.length; i++) {
       cur = cur.getChild(nodes[i]);
     }
@@ -1334,18 +1332,16 @@ public class MTreeBelowSGMemoryImpl {
 
   private IMemMNode getTableDeviceNode(final String table, final Object[] 
deviceId)
       throws PathNotExistException {
-    IMemMNode cur = storageGroupMNode;
+    IMemMNode cur = databaseMNode;
     IMemMNode next;
 
     next = cur.getChild(table);
     if (next == null) {
       throw new PathNotExistException(
-          storageGroupMNode.getFullPath() + PATH_SEPARATOR + table + 
Arrays.toString(deviceId),
-          true);
+          databaseMNode.getFullPath() + PATH_SEPARATOR + table + 
Arrays.toString(deviceId), true);
     } else if (next.isMeasurement()) {
       throw new PathNotExistException(
-          storageGroupMNode.getFullPath() + PATH_SEPARATOR + table + 
Arrays.toString(deviceId),
-          true);
+          databaseMNode.getFullPath() + PATH_SEPARATOR + table + 
Arrays.toString(deviceId), true);
     }
     cur = next;
 
@@ -1353,14 +1349,13 @@ public class MTreeBelowSGMemoryImpl {
       next = cur.getChild(deviceId[i] == null ? null : 
String.valueOf(deviceId[i]));
       if (next == null) {
         throw new PathNotExistException(
-            storageGroupMNode.getFullPath() + PATH_SEPARATOR + table + 
Arrays.toString(deviceId),
-            true);
+            databaseMNode.getFullPath() + PATH_SEPARATOR + table + 
Arrays.toString(deviceId), true);
       } else if (next.isMeasurement()) {
         if (i == deviceId.length - 1) {
           return next;
         } else {
           throw new PathNotExistException(
-              storageGroupMNode.getFullPath() + PATH_SEPARATOR + table + 
Arrays.toString(deviceId),
+              databaseMNode.getFullPath() + PATH_SEPARATOR + table + 
Arrays.toString(deviceId),
               true);
         }
       }
@@ -1628,7 +1623,7 @@ public class MTreeBelowSGMemoryImpl {
   // region table device management
 
   public int getTableDeviceNotExistNum(final String tableName, final 
List<Object[]> deviceIdList) {
-    final IMemMNode tableNode = storageGroupMNode.getChild(tableName);
+    final IMemMNode tableNode = databaseMNode.getChild(tableName);
     int notExistNum = deviceIdList.size();
     if (tableNode == null) {
       return notExistNum;
@@ -1659,11 +1654,10 @@ public class MTreeBelowSGMemoryImpl {
     if (LOGGER.isDebugEnabled()) {
       LOGGER.debug("Start to create table device {}.{}", tableName, 
Arrays.toString(devicePath));
     }
-    IMemMNode cur = storageGroupMNode.getChild(tableName);
+    IMemMNode cur = databaseMNode.getChild(tableName);
     if (cur == null) {
       cur =
-          store.addChild(
-              storageGroupMNode, tableName, 
nodeFactory.createInternalMNode(cur, tableName));
+          store.addChild(databaseMNode, tableName, 
nodeFactory.createInternalMNode(cur, tableName));
     }
 
     for (final String childName : devicePath) {
@@ -1776,14 +1770,14 @@ public class MTreeBelowSGMemoryImpl {
 
   public boolean deleteTableDevice(final String tableName, final IntConsumer 
attributeDeleter)
       throws MetadataException {
-    if (!store.hasChild(storageGroupMNode, tableName)) {
+    if (!store.hasChild(databaseMNode, tableName)) {
       return false;
     }
     final AtomicInteger memoryReleased = new AtomicInteger(0);
     try (final MNodeCollector<Void, IMemMNode> collector =
         new MNodeCollector<Void, IMemMNode>(
-            storageGroupMNode,
-            new PartialPath(new String[] {storageGroupMNode.getName(), 
tableName}),
+            databaseMNode,
+            new PartialPath(new String[] {databaseMNode.getName(), tableName}),
             this.store,
             true,
             SchemaConstant.ALL_MATCH_SCOPE) {
@@ -1805,7 +1799,7 @@ public class MTreeBelowSGMemoryImpl {
         }) {
       collector.traverse();
     }
-    storageGroupMNode.deleteChild(tableName);
+    databaseMNode.deleteChild(tableName);
     regionStatistics.resetTableDevice(tableName);
     store.releaseMemory(memoryReleased.get());
     return true;
@@ -1813,14 +1807,14 @@ public class MTreeBelowSGMemoryImpl {
 
   public boolean dropTableAttribute(final String tableName, final IntConsumer 
attributeDropper)
       throws MetadataException {
-    if (!store.hasChild(storageGroupMNode, tableName)) {
+    if (!store.hasChild(databaseMNode, tableName)) {
       return false;
     }
     final AtomicInteger memoryReleased = new AtomicInteger(0);
     try (final EntityUpdater<IMemMNode> updater =
         new EntityUpdater<IMemMNode>(
-            storageGroupMNode,
-            new PartialPath(new String[] {storageGroupMNode.getName(), 
tableName}),
+            databaseMNode,
+            new PartialPath(new String[] {databaseMNode.getName(), tableName}),
             this.store,
             true,
             SchemaConstant.ALL_MATCH_SCOPE) {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/DataNodeTableCache.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/DataNodeTableCache.java
index fdbca98cc00..2001550ab67 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/DataNodeTableCache.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/DataNodeTableCache.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.schemaengine.table;
 
+import org.apache.iotdb.commons.schema.table.NonCommittableTsTable;
 import org.apache.iotdb.commons.schema.table.TsTable;
 import org.apache.iotdb.commons.schema.table.TsTableInternalRPCUtil;
 import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema;
@@ -177,6 +178,14 @@ public class DataNodeTableCache implements ITableCache {
       if (Objects.nonNull(oldName)) {
         // Equals to commit update
         final TsTable oldTable = 
preUpdateTableMap.get(database).get(oldName).getLeft();
+        // Cannot be rolled back, consider:
+        // 1. Fetched a written CN table
+        // 2. CN rollback because of timeout
+        // 3. If we roll back here, the flag will be cleared, and it will 
always be the written
+        // one
+        if (oldTable instanceof NonCommittableTsTable) {
+          return;
+        }
         databaseTableMap
             .computeIfAbsent(database, k -> new ConcurrentHashMap<>())
             .put(tableName, oldTable);
@@ -207,6 +216,14 @@ public class DataNodeTableCache implements ITableCache {
     readWriteLock.writeLock().lock();
     try {
       final TsTable newTable = 
preUpdateTableMap.get(database).get(tableName).getLeft();
+      // Cannot be committed, consider:
+      // 1. Fetched a non-changed CN table
+      // 2. CN is changed
+      // 3. If we commit here, it will always be the non-changed one
+      // (And it is not committable because it's not real table)
+      if (newTable instanceof NonCommittableTsTable) {
+        return;
+      }
       final TsTable oldTable =
           databaseTableMap
               .computeIfAbsent(database, k -> new ConcurrentHashMap<>())
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/NonCommittableTsTable.java
similarity index 52%
copy from 
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
copy to 
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/NonCommittableTsTable.java
index 4ff683f4436..5f22c86474f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
+++ 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/NonCommittableTsTable.java
@@ -17,26 +17,27 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.queryengine.plan.statement.metadata;
+package org.apache.iotdb.commons.schema.table;
 
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.queryengine.plan.analyze.QueryType;
-import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement;
-import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
+import org.apache.tsfile.utils.ReadWriteIOUtils;
 
-public class CountDatabaseStatement extends CountStatement implements 
IConfigStatement {
+import java.io.IOException;
+import java.io.OutputStream;
 
-  public CountDatabaseStatement(PartialPath partialPath) {
-    super(partialPath);
-  }
-
-  @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
-    return visitor.visitCountStorageGroup(this, context);
+/**
+ * This table is just for occupation, and notice the dataNode to fetch the 
newest version from
+ * configNode. Note that the table cannot be committed or rolled-back, yet it 
can still be
+ * pre-updated or invalidated, because the two can update the table to the 
newest and trustable
+ * version.
+ */
+public class NonCommittableTsTable extends TsTable {
+  public NonCommittableTsTable(final String tableName) {
+    super(tableName);
   }
 
   @Override
-  public QueryType getQueryType() {
-    return QueryType.READ;
+  public void serialize(final OutputStream stream) throws IOException {
+    ReadWriteIOUtils.write(tableName, stream);
+    ReadWriteIOUtils.write(-1, stream);
   }
 }
diff --git 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java
index 00ced1990ea..3bc5ab64248 100644
--- 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java
+++ 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java
@@ -65,7 +65,7 @@ public class TsTable {
 
   public static final String TTL_PROPERTY = "ttl";
   public static final Set<String> TABLE_ALLOWED_PROPERTIES = 
Collections.singleton(TTL_PROPERTY);
-  private String tableName;
+  protected String tableName;
 
   private final Map<String, TsTableColumnSchema> columnSchemaMap = new 
LinkedHashMap<>();
   private final Map<String, Integer> tagColumnIndexMap = new HashMap<>();
@@ -331,10 +331,13 @@ public class TsTable {
     ReadWriteIOUtils.write(props, stream);
   }
 
-  public static TsTable deserialize(InputStream inputStream) throws 
IOException {
-    String name = ReadWriteIOUtils.readString(inputStream);
-    TsTable table = new TsTable(name);
-    int columnNum = ReadWriteIOUtils.readInt(inputStream);
+  public static TsTable deserialize(final InputStream inputStream) throws 
IOException {
+    final String name = ReadWriteIOUtils.readString(inputStream);
+    final int columnNum = ReadWriteIOUtils.readInt(inputStream);
+    if (columnNum < 0) {
+      return new NonCommittableTsTable(name);
+    }
+    final TsTable table = new TsTable(name);
     for (int i = 0; i < columnNum; i++) {
       table.addColumnSchema(TsTableColumnSchemaUtil.deserialize(inputStream));
     }
@@ -342,10 +345,13 @@ public class TsTable {
     return table;
   }
 
-  public static TsTable deserialize(ByteBuffer buffer) {
-    String name = ReadWriteIOUtils.readString(buffer);
-    TsTable table = new TsTable(name);
-    int columnNum = ReadWriteIOUtils.readInt(buffer);
+  public static TsTable deserialize(final ByteBuffer buffer) {
+    final String name = ReadWriteIOUtils.readString(buffer);
+    final int columnNum = ReadWriteIOUtils.readInt(buffer);
+    if (columnNum < 0) {
+      return new NonCommittableTsTable(name);
+    }
+    final TsTable table = new TsTable(name);
     for (int i = 0; i < columnNum; i++) {
       table.addColumnSchema(TsTableColumnSchemaUtil.deserialize(buffer));
     }

Reply via email to