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

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


The following commit(s) were added to refs/heads/master by this push:
     new 61fa8d6f004 Implement schema of 'information_schema' & Make 
AddLocation plan immune readOnly state
61fa8d6f004 is described below

commit 61fa8d6f0047270c40ac89720d1bb7dbc3718282
Author: Caideyipi <[email protected]>
AuthorDate: Thu Dec 19 20:01:19 2024 +0800

    Implement schema of 'information_schema' & Make AddLocation plan immune 
readOnly state
---
 .../it/tablemodel/IoTDBTablePatternFormatIT.java   |   2 +-
 .../it/query/old/IoTDBSimpleQueryTableIT.java      |  20 ++-
 .../relational/it/schema/IoTDBDatabaseIT.java      | 100 +++++++++++
 .../read/ainode/GetAINodeConfigurationPlan.java    |  15 --
 .../request/read/model/GetModelInfoPlan.java       |  16 --
 .../request/read/model/ShowModelPlan.java          |  20 ---
 .../manager/partition/PartitionMetrics.java        |   3 +-
 .../execution/executor/RegionWriteExecutor.java    |   7 +-
 .../InformationSchemaContentSupplierFactory.java   |   9 +-
 .../db/queryengine/plan/analyze/AnalyzeUtils.java  |  10 +-
 .../plan/analyze/schema/SchemaValidator.java       |   6 +-
 .../execution/config/TableConfigTaskVisitor.java   |  37 ++---
 .../config/executor/ClusterConfigTaskExecutor.java |  15 +-
 .../config/executor/IConfigTaskExecutor.java       |   4 +-
 .../config/metadata/relational/ShowDBTask.java     |  22 ++-
 .../plan/node/write/RelationalDeleteDataNode.java  |  15 +-
 .../plan/relational/analyzer/Analysis.java         |   4 +-
 .../relational/analyzer/StatementAnalyzer.java     |   3 +
 .../relational/metadata/TableMetadataImpl.java     |  45 +++--
 .../plan/relational/metadata/TableSchema.java      |  27 +--
 .../fetcher/TableHeaderSchemaValidator.java        |   3 +
 .../plan/relational/planner/RelationPlanner.java   |   4 +-
 .../DataNodeLocationSupplierFactory.java           |  12 +-
 .../plan/relational/security/AccessControl.java    |   2 +-
 .../relational/sql/ast/AbstractTraverseDevice.java |   1 +
 .../plan/relational/sql/parser/AstBuilder.java     |   4 +-
 .../plan/relational/sql/rewrite/ShowRewrite.java   |   4 +-
 .../statement/metadata/ShowDatabaseStatement.java  |   6 +-
 .../db/schemaengine/table/DataNodeTableCache.java  |  10 +-
 .../schemaengine/table/InformationSchemaUtils.java | 185 +++++++++++++++++++++
 .../plan/relational/analyzer/TestMatadata.java     |  10 +-
 .../schema/column/ColumnHeaderConstant.java        |   8 +
 .../commons/schema/table/InformationSchema.java    | 123 ++++++++++++++
 .../schema/table/InformationSchemaTable.java       |  91 ----------
 .../schema/table/column/IdColumnSchema.java        |   9 +-
 .../table/column/MeasurementColumnSchema.java      |  48 +++---
 36 files changed, 612 insertions(+), 288 deletions(-)

diff --git 
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBTablePatternFormatIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBTablePatternFormatIT.java
index a563968f5bb..ac1e96ab4aa 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBTablePatternFormatIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBTablePatternFormatIT.java
@@ -595,7 +595,7 @@ public class IoTDBTablePatternFormatIT extends 
AbstractPipeTableModelTestIT {
           receiverEnv,
           "show databases",
           
"Database,TTL(ms),SchemaReplicationFactor,DataReplicationFactor,TimePartitionInterval,",
-          Collections.emptySet(),
+          Collections.singleton("information_schema,INF,null,null,null,"),
           (String) null);
     }
   }
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 e615c779d1f..3a93991880e 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
@@ -611,24 +611,28 @@ public class IoTDBSimpleQueryTableIT {
 
   @Test
   public void testStorageGroupWithHyphenInName() {
-    try (Connection connection = 
EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT);
-        Statement statement = connection.createStatement()) {
+    try (final Connection connection =
+            EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT);
+        final Statement statement = connection.createStatement()) {
       statement.setFetchSize(5);
       statement.execute("CREATE DATABASE group_with_hyphen");
     } catch (SQLException e) {
       fail();
     }
 
-    try (Connection connection = 
EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT);
-        Statement statement = connection.createStatement()) {
-      try (ResultSet resultSet = statement.executeQuery("SHOW DATABASES")) {
+    try (final Connection connection =
+            EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT);
+        final Statement statement = connection.createStatement()) {
+      try (final ResultSet resultSet = statement.executeQuery("SHOW 
DATABASES")) {
         while (resultSet.next()) {
-          StringBuilder builder = new StringBuilder();
+          final StringBuilder builder = new StringBuilder();
           builder.append(resultSet.getString(1));
-          Assert.assertEquals(builder.toString(), "group_with_hyphen");
+          Assert.assertTrue(
+              builder.toString().equals("group_with_hyphen")
+                  || builder.toString().equals("information_schema"));
         }
       }
-    } catch (SQLException e) {
+    } catch (final SQLException e) {
       fail();
     }
   }
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java
index c48f10a7f2f..4f5ed0653ec 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java
@@ -37,7 +37,10 @@ import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
 
 import static 
org.apache.iotdb.commons.schema.column.ColumnHeaderConstant.showDBColumnHeaders;
 import static 
org.apache.iotdb.commons.schema.column.ColumnHeaderConstant.showDBDetailsColumnHeaders;
@@ -96,6 +99,9 @@ public class IoTDBDatabaseIT {
           assertEquals(showDBColumnHeaders.get(i).getColumnName(), 
metaData.getColumnName(i + 1));
         }
         while (resultSet.next()) {
+          if (resultSet.getString(1).equals("information_schema")) {
+            continue;
+          }
           assertEquals(databaseNames[cnt], resultSet.getString(1));
           assertEquals(TTLs[cnt], resultSet.getString(2));
           assertEquals(schemaReplicaFactors[cnt], resultSet.getInt(3));
@@ -116,6 +122,9 @@ public class IoTDBDatabaseIT {
               showDBDetailsColumnHeaders.get(i).getColumnName(), 
metaData.getColumnName(i + 1));
         }
         while (resultSet.next()) {
+          if (resultSet.getString(1).equals("information_schema")) {
+            continue;
+          }
           assertEquals(databaseNames[cnt], resultSet.getString(1));
           assertEquals(TTLs[cnt], resultSet.getString(2));
           assertEquals(schemaReplicaFactors[cnt], resultSet.getInt(3));
@@ -141,6 +150,8 @@ public class IoTDBDatabaseIT {
       // drop
       statement.execute("drop database test");
       try (final ResultSet resultSet = statement.executeQuery("SHOW 
DATABASES")) {
+        // Information_schema
+        assertTrue(resultSet.next());
         assertFalse(resultSet.next());
       }
 
@@ -171,6 +182,9 @@ public class IoTDBDatabaseIT {
           assertEquals(showDBColumnHeaders.get(i).getColumnName(), 
metaData.getColumnName(i + 1));
         }
         while (resultSet.next()) {
+          if (resultSet.getString(1).equals("information_schema")) {
+            continue;
+          }
           assertEquals(databaseNames[cnt], resultSet.getString(1));
           assertEquals(TTLs[cnt], resultSet.getString(2));
           assertEquals(schemaReplicaFactors[cnt], resultSet.getInt(3));
@@ -270,6 +284,9 @@ public class IoTDBDatabaseIT {
 
       try (final ResultSet resultSet = statement.executeQuery("SHOW 
DATABASES")) {
         assertTrue(resultSet.next());
+        if (resultSet.getString(1).equals("information_schema")) {
+          assertTrue(resultSet.next());
+        }
         assertEquals("````x", resultSet.getString(1));
         assertFalse(resultSet.next());
       }
@@ -305,4 +322,87 @@ public class IoTDBDatabaseIT {
           Collections.singleton("1,4,"));
     }
   }
+
+  @Test
+  public void testInformationSchema() throws SQLException {
+    try (final Connection connection =
+            EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT);
+        final Statement statement = connection.createStatement()) {
+      // Test unsupported write plans
+      final Set<String> writeSQLs =
+          new HashSet<>(
+              Arrays.asList(
+                  "create database information_schema",
+                  "drop database information_schema",
+                  "create table information_schema.tableA ()",
+                  "alter table information_schema.tableA add column a id",
+                  "alter table information_schema.tableA set properties 
ttl=default",
+                  "insert into information_schema.tables (database) 
values('db')",
+                  "update information_schema.tables set status='RUNNING'"));
+
+      for (final String writeSQL : writeSQLs) {
+        try {
+          statement.execute(writeSQL);
+          fail("information_schema does not support write");
+        } catch (final SQLException e) {
+          assertEquals(
+              "701: The database 'information_schema' can only be queried", 
e.getMessage());
+        }
+      }
+
+      statement.execute("use information_schema");
+
+      TestUtils.assertResultSetEqual(
+          statement.executeQuery("show databases"),
+          
"Database,TTL(ms),SchemaReplicationFactor,DataReplicationFactor,TimePartitionInterval,",
+          Collections.singleton("information_schema,INF,null,null,null,"));
+      TestUtils.assertResultSetEqual(
+          statement.executeQuery("show tables"),
+          "TableName,TTL(ms),",
+          new HashSet<>(
+              Arrays.asList("databases,INF,", "tables,INF,", "columns,INF,", 
"queries,INF,")));
+
+      TestUtils.assertResultSetEqual(
+          statement.executeQuery("desc databases"),
+          "ColumnName,DataType,Category,",
+          new HashSet<>(
+              Arrays.asList(
+                  "database,STRING,ID,",
+                  "ttl(ms),STRING,ATTRIBUTE,",
+                  "schema_replication_factor,INT32,ATTRIBUTE,",
+                  "data_replication_factor,INT32,ATTRIBUTE,",
+                  "time_partition_interval,INT64,ATTRIBUTE,",
+                  "model,STRING,ATTRIBUTE,")));
+      TestUtils.assertResultSetEqual(
+          statement.executeQuery("desc tables"),
+          "ColumnName,DataType,Category,",
+          new HashSet<>(
+              Arrays.asList(
+                  "database,STRING,ID,",
+                  "table_name,STRING,ID,",
+                  "ttl(ms),STRING,ATTRIBUTE,",
+                  "status,STRING,ATTRIBUTE,")));
+      TestUtils.assertResultSetEqual(
+          statement.executeQuery("desc columns"),
+          "ColumnName,DataType,Category,",
+          new HashSet<>(
+              Arrays.asList(
+                  "database,STRING,ID,",
+                  "table_name,STRING,ID,",
+                  "column_name,STRING,ID,",
+                  "datatype,STRING,ATTRIBUTE,",
+                  "category,STRING,ATTRIBUTE,",
+                  "status,STRING,ATTRIBUTE,")));
+      TestUtils.assertResultSetEqual(
+          statement.executeQuery("desc queries"),
+          "ColumnName,DataType,Category,",
+          new HashSet<>(
+              Arrays.asList(
+                  "query_id,STRING,ID,",
+                  "start_time,TIMESTAMP,ATTRIBUTE,",
+                  "datanode_id,INT32,ATTRIBUTE,",
+                  "elapsed_time,FLOAT,ATTRIBUTE,",
+                  "statement,STRING,ATTRIBUTE,")));
+    }
+  }
 }
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/ainode/GetAINodeConfigurationPlan.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/ainode/GetAINodeConfigurationPlan.java
index 7222a8f53f8..303f30b9c77 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/ainode/GetAINodeConfigurationPlan.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/ainode/GetAINodeConfigurationPlan.java
@@ -22,10 +22,6 @@ package 
org.apache.iotdb.confignode.consensus.request.read.ainode;
 import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
 import 
org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan;
 
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
 public class GetAINodeConfigurationPlan extends ConfigPhysicalReadPlan {
 
   // if aiNodeId is set to -1, return all AINode configurations.
@@ -44,17 +40,6 @@ public class GetAINodeConfigurationPlan extends 
ConfigPhysicalReadPlan {
     return aiNodeId;
   }
 
-  @Override
-  protected void serializeImpl(DataOutputStream stream) throws IOException {
-    stream.writeShort(getType().getPlanType());
-    stream.writeInt(aiNodeId);
-  }
-
-  @Override
-  protected void deserializeImpl(ByteBuffer buffer) throws IOException {
-    this.aiNodeId = buffer.getInt();
-  }
-
   @Override
   public boolean equals(final Object o) {
     if (this == o) {
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/GetModelInfoPlan.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/GetModelInfoPlan.java
index 9c33c267882..dd79910e51f 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/GetModelInfoPlan.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/GetModelInfoPlan.java
@@ -23,11 +23,6 @@ import 
org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
 import 
org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan;
 import org.apache.iotdb.confignode.rpc.thrift.TGetModelInfoReq;
 
-import org.apache.tsfile.utils.ReadWriteIOUtils;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.Objects;
 
 public class GetModelInfoPlan extends ConfigPhysicalReadPlan {
@@ -47,17 +42,6 @@ public class GetModelInfoPlan extends ConfigPhysicalReadPlan 
{
     return modelId;
   }
 
-  @Override
-  protected void serializeImpl(DataOutputStream stream) throws IOException {
-    stream.writeShort(getType().getPlanType());
-    ReadWriteIOUtils.write(modelId, stream);
-  }
-
-  @Override
-  protected void deserializeImpl(ByteBuffer buffer) throws IOException {
-    this.modelId = ReadWriteIOUtils.readString(buffer);
-  }
-
   @Override
   public boolean equals(final Object o) {
     if (this == o) {
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/ShowModelPlan.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/ShowModelPlan.java
index df924c97f5b..16bc16bc872 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/ShowModelPlan.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/ShowModelPlan.java
@@ -23,11 +23,6 @@ import 
org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
 import 
org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan;
 import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq;
 
-import org.apache.tsfile.utils.ReadWriteIOUtils;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.Objects;
 
 public class ShowModelPlan extends ConfigPhysicalReadPlan {
@@ -53,21 +48,6 @@ public class ShowModelPlan extends ConfigPhysicalReadPlan {
     return modelName;
   }
 
-  @Override
-  protected void serializeImpl(DataOutputStream stream) throws IOException {
-    stream.writeShort(getType().getPlanType());
-    ReadWriteIOUtils.write(modelName != null, stream);
-    ReadWriteIOUtils.write(modelName, stream);
-  }
-
-  @Override
-  protected void deserializeImpl(ByteBuffer buffer) throws IOException {
-    boolean isSetModelId = ReadWriteIOUtils.readBool(buffer);
-    if (isSetModelId) {
-      this.modelName = ReadWriteIOUtils.readString(buffer);
-    }
-  }
-
   @Override
   public boolean equals(final Object o) {
     if (this == o) {
diff --git 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionMetrics.java
 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionMetrics.java
index 38e64dbd2f7..b6b709cd801 100644
--- 
a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionMetrics.java
+++ 
b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionMetrics.java
@@ -274,7 +274,8 @@ public class PartitionMetrics implements IMetricSet {
         Metric.DATABASE_NUM.toString(),
         MetricLevel.CORE,
         clusterSchemaManager,
-        c -> c.getDatabaseNames(null).size());
+        // Add 1 for information schema
+        c -> c.getDatabaseNames(null).size() + 1);
 
     List<String> databases = clusterSchemaManager.getDatabaseNames(null);
     for (String database : databases) {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java
index cef49d96c3f..f81b3c4d8a0 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java
@@ -66,6 +66,7 @@ import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTablet
 import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode;
 import 
org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertTabletNode;
 import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode;
+import 
org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.TableNodeLocationAddNode;
 import org.apache.iotdb.db.schemaengine.SchemaEngine;
 import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion;
 import org.apache.iotdb.db.schemaengine.template.ClusterTemplateManager;
@@ -165,7 +166,7 @@ public class RegionWriteExecutor {
     public RegionExecutionResult visitPlan(
         final PlanNode node, final WritePlanNodeExecutionContext context) {
 
-      if (CommonDescriptor.getInstance().getConfig().isReadOnly()) {
+      if (CommonDescriptor.getInstance().getConfig().isReadOnly() && 
!isForceExecutedPlan(node)) {
         return RegionExecutionResult.create(
             false,
             "Fail to do non-query operations because system is read-only.",
@@ -190,6 +191,10 @@ public class RegionWriteExecutor {
       }
     }
 
+    private boolean isForceExecutedPlan(final PlanNode node) {
+      return node instanceof TableNodeLocationAddNode;
+    }
+
     private TSStatus executePlanNodeInConsensusLayer(
         final ConsensusGroupId groupId, final PlanNode planNode) throws 
ConsensusException {
       if (groupId instanceof DataRegionId) {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java
index 0f091ab5e78..41abb73f54e 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java
@@ -16,8 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.queryengine.execution.operator.source.relational;
 
+import org.apache.iotdb.commons.schema.table.InformationSchema;
 import org.apache.iotdb.db.protocol.session.IClientSession;
 import org.apache.iotdb.db.queryengine.plan.Coordinator;
 import org.apache.iotdb.db.queryengine.plan.execution.IQueryExecution;
@@ -32,13 +34,12 @@ import org.apache.tsfile.utils.BytesUtils;
 import java.util.Iterator;
 import java.util.List;
 
-import static 
org.apache.iotdb.commons.schema.table.InformationSchemaTable.QUERIES;
-
 public class InformationSchemaContentSupplierFactory {
   private InformationSchemaContentSupplierFactory() {}
 
-  public static Iterator<TsBlock> getSupplier(String tableName, 
List<TSDataType> dataTypes) {
-    if (tableName.equals(QUERIES.getSchemaTableName())) {
+  public static Iterator<TsBlock> getSupplier(
+      final String tableName, final List<TSDataType> dataTypes) {
+    if (tableName.equals(InformationSchema.QUERIES)) {
       return new Iterator<TsBlock>() {
         private final TsBlockBuilder resultBuilder = new 
TsBlockBuilder(dataTypes);
         private final ColumnBuilder[] columnBuilders = 
resultBuilder.getValueColumnBuilders();
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeUtils.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeUtils.java
index a59c614902d..98621bdb121 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeUtils.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeUtils.java
@@ -51,6 +51,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowStatement;
 import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowsStatement;
 import 
org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement;
 import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache;
+import org.apache.iotdb.db.schemaengine.table.InformationSchemaUtils;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.DeletionPredicate;
 import org.apache.iotdb.db.storageengine.dataregion.modification.IDPredicate;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.IDPredicate.And;
@@ -342,9 +343,9 @@ public class AnalyzeUtils {
   }
 
   @SuppressWarnings("java:S3655") // optional is checked
-  private static void validateSchema(Delete node, MPPQueryContext 
queryContext) {
-    String tableName = node.getTable().getName().getSuffix();
-    String databaseName;
+  private static void validateSchema(final Delete node, final MPPQueryContext 
queryContext) {
+    final String tableName = node.getTable().getName().getSuffix();
+    final String databaseName;
     if (node.getTable().getName().getPrefix().isPresent()) {
       databaseName = node.getTable().getName().getPrefix().get().toString();
     } else if (queryContext.getDatabaseName().isPresent()) {
@@ -352,9 +353,10 @@ public class AnalyzeUtils {
     } else {
       throw new SemanticException(DATABASE_NOT_SPECIFIED);
     }
+    InformationSchemaUtils.checkDBNameInWrite(databaseName);
     node.setDatabaseName(databaseName);
 
-    TsTable table = DataNodeTableCache.getInstance().getTable(databaseName, 
tableName);
+    final TsTable table = 
DataNodeTableCache.getInstance().getTable(databaseName, tableName);
     if (table == null) {
       throw new SemanticException("Table " + tableName + " not found");
     }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/SchemaValidator.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/SchemaValidator.java
index 9533cca1bd6..aaf5d08f43c 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/SchemaValidator.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/SchemaValidator.java
@@ -62,13 +62,15 @@ public class SchemaValidator {
   }
 
   public static void validate(
-      Metadata metadata, WrappedInsertStatement insertStatement, 
MPPQueryContext context) {
+      final Metadata metadata,
+      final WrappedInsertStatement insertStatement,
+      final MPPQueryContext context) {
     try {
       insertStatement.toLowerCase();
       insertStatement.validateTableSchema(metadata, context);
       insertStatement.updateAfterSchemaValidation(context);
       insertStatement.validateDeviceSchema(metadata, context);
-    } catch (QueryProcessException e) {
+    } catch (final QueryProcessException e) {
       throw new SemanticException(e.getMessage());
     }
   }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java
index 0b5c6f2379b..0d82c7e0c89 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java
@@ -150,6 +150,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.metadata.ShowClusterStatem
 import 
org.apache.iotdb.db.queryengine.plan.statement.metadata.ShowRegionStatement;
 import org.apache.iotdb.db.queryengine.plan.statement.sys.FlushStatement;
 import 
org.apache.iotdb.db.queryengine.plan.statement.sys.SetConfigurationStatement;
+import org.apache.iotdb.db.schemaengine.table.InformationSchemaUtils;
 
 import org.apache.tsfile.common.conf.TSFileConfig;
 import org.apache.tsfile.enums.TSDataType;
@@ -296,6 +297,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   @Override
   protected IConfigTask visitDropDB(final DropDB node, final MPPQueryContext 
context) {
     context.setQueryType(QueryType.WRITE);
+    InformationSchemaUtils.checkDBNameInWrite(node.getDbName().getValue());
     accessControl.checkCanDropDatabase(
         context.getSession().getUserName(), node.getDbName().getValue());
     return new DropDBTask(node);
@@ -311,7 +313,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
             accessControl.checkCanShowOrUseDatabase(
                 context.getSession().getUserName(), databaseName);
             return true;
-          } catch (AccessControlException e) {
+          } catch (final AccessControlException e) {
             return false;
           }
         });
@@ -378,7 +380,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   @Override
   protected IConfigTask visitCreateTable(final CreateTable node, final 
MPPQueryContext context) {
     context.setQueryType(QueryType.WRITE);
-    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getName());
+    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getName(), true);
 
     final TsTable table = new TsTable(databaseTablePair.getRight());
 
@@ -425,7 +427,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   @Override
   protected IConfigTask visitRenameTable(final RenameTable node, final 
MPPQueryContext context) {
     context.setQueryType(QueryType.WRITE);
-    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getSource());
+    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getSource(), true);
 
     final String oldName = databaseTablePair.getRight();
     final String newName = node.getTarget().getValue();
@@ -444,7 +446,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   @Override
   protected IConfigTask visitAddColumn(final AddColumn node, final 
MPPQueryContext context) {
     context.setQueryType(QueryType.WRITE);
-    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTableName());
+    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTableName(), true);
 
     final ColumnDefinition definition = node.getColumn();
     return new AlterTableAddColumnTask(
@@ -463,7 +465,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   @Override
   protected IConfigTask visitRenameColumn(final RenameColumn node, final 
MPPQueryContext context) {
     context.setQueryType(QueryType.WRITE);
-    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTable());
+    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTable(), true);
 
     final String oldName = node.getSource().getValue();
     final String newName = node.getTarget().getValue();
@@ -484,7 +486,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   @Override
   protected IConfigTask visitDropColumn(final DropColumn node, final 
MPPQueryContext context) {
     context.setQueryType(QueryType.WRITE);
-    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTable());
+    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTable(), true);
 
     return new AlterTableDropColumnTask(
         databaseTablePair.getLeft(),
@@ -499,7 +501,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   protected IConfigTask visitSetProperties(
       final SetProperties node, final MPPQueryContext context) {
     context.setQueryType(QueryType.WRITE);
-    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getName());
+    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getName(), true);
 
     return new AlterTableSetPropertiesTask(
         databaseTablePair.getLeft(),
@@ -510,6 +512,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   }
 
   public static void validateDatabaseName(final String dbName) throws 
SemanticException {
+    InformationSchemaUtils.checkDBNameInWrite(dbName);
     // Check database length here
     // We need to calculate the database name without "root."
     if (dbName.contains(PATH_SEPARATOR)
@@ -524,7 +527,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
     }
   }
 
-  public Pair<String, String> splitQualifiedName(final QualifiedName name) {
+  public Pair<String, String> splitQualifiedName(final QualifiedName name, 
final boolean isWrite) {
     String database = clientSession.getDatabaseName();
     if (name.getPrefix().isPresent()) {
       database = name.getPrefix().get().toString();
@@ -532,6 +535,9 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
     if (database == null) {
       throw new SemanticException(DATABASE_NOT_SPECIFIED);
     }
+    if (isWrite) {
+      InformationSchemaUtils.checkDBNameInWrite(database);
+    }
     return new Pair<>(database, name.getSuffix());
   }
 
@@ -575,7 +581,7 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   @Override
   protected IConfigTask visitDropTable(final DropTable node, final 
MPPQueryContext context) {
     context.setQueryType(QueryType.WRITE);
-    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTableName());
+    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTableName(), true);
     return new DropTableTask(
         databaseTablePair.getLeft(),
         databaseTablePair.getRight(),
@@ -614,17 +620,10 @@ public class TableConfigTaskVisitor extends 
AstVisitor<IConfigTask, MPPQueryCont
   protected IConfigTask visitDescribeTable(
       final DescribeTable node, final MPPQueryContext context) {
     context.setQueryType(QueryType.READ);
-    String database = clientSession.getDatabaseName();
-    if (node.getTable().getPrefix().isPresent()) {
-      database = node.getTable().getPrefix().get().toString();
-    }
-    if (database == null) {
-      throw new SemanticException(DATABASE_NOT_SPECIFIED);
-    }
-    final String tableName = node.getTable().getSuffix();
+    final Pair<String, String> databaseTablePair = 
splitQualifiedName(node.getTable(), false);
     return node.isDetails()
-        ? new DescribeTableDetailsTask(database, tableName)
-        : new DescribeTableTask(database, tableName);
+        ? new DescribeTableDetailsTask(databaseTablePair.getLeft(), 
databaseTablePair.getRight())
+        : new DescribeTableTask(databaseTablePair.getLeft(), 
databaseTablePair.getRight());
   }
 
   @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
index b19a6ded9bc..9677da95e8f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
@@ -255,6 +255,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.sys.quota.ShowSpaceQuotaSt
 import 
org.apache.iotdb.db.queryengine.plan.statement.sys.quota.ShowThrottleQuotaStatement;
 import org.apache.iotdb.db.schemaengine.SchemaEngine;
 import org.apache.iotdb.db.schemaengine.rescon.DataNodeSchemaQuotaManager;
+import org.apache.iotdb.db.schemaengine.table.InformationSchemaUtils;
 import org.apache.iotdb.db.schemaengine.template.ClusterTemplateManager;
 import org.apache.iotdb.db.schemaengine.template.Template;
 import org.apache.iotdb.db.schemaengine.template.TemplateAlterOperationType;
@@ -310,7 +311,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.TreeMap;
-import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 import static 
org.apache.iotdb.commons.conf.IoTDBConstant.MAX_DATABASE_NAME_LENGTH;
@@ -3041,7 +3042,7 @@ public class ClusterConfigTaskExecutor implements 
IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> showDatabases(
-      final ShowDB showDB, final Function<String, Boolean> canSeenDB) {
+      final ShowDB showDB, final Predicate<String> canSeenDB) {
     final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     // Construct request using statement
     final List<String> databasePathPattern = Arrays.asList(ALL_RESULT_NODES);
@@ -3072,6 +3073,10 @@ public class ClusterConfigTaskExecutor implements 
IConfigTaskExecutor {
   public SettableFuture<ConfigTaskResult> useDatabase(
       final Use useDB, final IClientSession clientSession) {
     final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+
+    if (InformationSchemaUtils.mayUseDB(useDB.getDatabaseId().getValue(), 
clientSession, future)) {
+      return future;
+    }
     // Construct request using statement
     final List<String> databasePathPattern = Arrays.asList(ROOT, 
useDB.getDatabaseId().getValue());
     try (final ConfigNodeClient client =
@@ -3251,6 +3256,9 @@ public class ClusterConfigTaskExecutor implements 
IConfigTaskExecutor {
       final String database, final String tableName, final boolean isDetails) {
     final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
 
+    if (InformationSchemaUtils.mayDescribeTable(database, tableName, 
isDetails, future)) {
+      return future;
+    }
     try (final ConfigNodeClient configNodeClient =
         
CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       final TDescTableResp resp = configNodeClient.describeTable(database, 
tableName, isDetails);
@@ -3277,6 +3285,9 @@ public class ClusterConfigTaskExecutor implements 
IConfigTaskExecutor {
       final String database, final boolean isDetails) {
     final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
 
+    if (InformationSchemaUtils.mayShowTable(database, isDetails, future)) {
+      return future;
+    }
     try (final ConfigNodeClient configNodeClient =
         
CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       final TShowTableResp resp = configNodeClient.showTables(database, 
isDetails);
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java
index ad81c6469ee..179d2455037 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java
@@ -98,7 +98,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
-import java.util.function.Function;
+import java.util.function.Predicate;
 
 public interface IConfigTaskExecutor {
 
@@ -284,7 +284,7 @@ public interface IConfigTaskExecutor {
   // =============================== table syntax 
=========================================
 
   SettableFuture<ConfigTaskResult> showDatabases(
-      ShowDB showDB, Function<String, Boolean> canSeenDB);
+      final ShowDB showDB, final Predicate<String> canSeenDB);
 
   SettableFuture<ConfigTaskResult> showCluster(ShowCluster showCluster);
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/ShowDBTask.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/ShowDBTask.java
index 8873e34272b..a15ed225971 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/ShowDBTask.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/ShowDBTask.java
@@ -22,6 +22,7 @@ package 
org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relationa
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.schema.column.ColumnHeader;
 import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant;
+import org.apache.iotdb.commons.utils.PathUtils;
 import org.apache.iotdb.confignode.rpc.thrift.TDatabaseInfo;
 import org.apache.iotdb.db.queryengine.common.header.DatasetHeader;
 import org.apache.iotdb.db.queryengine.common.header.DatasetHeaderFactory;
@@ -29,6 +30,7 @@ import 
org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult;
 import org.apache.iotdb.db.queryengine.plan.execution.config.IConfigTask;
 import 
org.apache.iotdb.db.queryengine.plan.execution.config.executor.IConfigTaskExecutor;
 import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowDB;
+import org.apache.iotdb.db.schemaengine.table.InformationSchemaUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 
 import com.google.common.util.concurrent.ListenableFuture;
@@ -40,7 +42,7 @@ import org.apache.tsfile.utils.Binary;
 
 import java.util.List;
 import java.util.Map;
-import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 public class ShowDBTask implements IConfigTask {
@@ -48,9 +50,9 @@ public class ShowDBTask implements IConfigTask {
   private final ShowDB node;
 
   // judge whether the specific database can be seen, dbName should be without 
`root.` prefix
-  private final Function<String, Boolean> canSeenDB;
+  private final Predicate<String> canSeenDB;
 
-  public ShowDBTask(final ShowDB node, final Function<String, Boolean> 
canSeenDB) {
+  public ShowDBTask(final ShowDB node, final Predicate<String> canSeenDB) {
     this.node = node;
     this.canSeenDB = canSeenDB;
   }
@@ -65,7 +67,7 @@ public class ShowDBTask implements IConfigTask {
       final Map<String, TDatabaseInfo> storageGroupInfoMap,
       final SettableFuture<ConfigTaskResult> future,
       final boolean isDetails,
-      final Function<String, Boolean> canSeenDB) {
+      final Predicate<String> canSeenDB) {
     if (isDetails) {
       buildTSBlockForDetails(storageGroupInfoMap, future, canSeenDB);
     } else {
@@ -76,16 +78,17 @@ public class ShowDBTask implements IConfigTask {
   private static void buildTSBlockForNonDetails(
       final Map<String, TDatabaseInfo> storageGroupInfoMap,
       final SettableFuture<ConfigTaskResult> future,
-      final Function<String, Boolean> canSeenDB) {
+      final Predicate<String> canSeenDB) {
     final List<TSDataType> outputDataTypes =
         ColumnHeaderConstant.showDBColumnHeaders.stream()
             .map(ColumnHeader::getColumnType)
             .collect(Collectors.toList());
 
     final TsBlockBuilder builder = new TsBlockBuilder(outputDataTypes);
+    InformationSchemaUtils.buildDatabaseTsBlock(canSeenDB, builder, false);
     for (final Map.Entry<String, TDatabaseInfo> entry : 
storageGroupInfoMap.entrySet()) {
-      final String dbName = entry.getKey().substring(5);
-      if (!canSeenDB.apply(dbName)) {
+      final String dbName = PathUtils.unQualifyDatabaseName(entry.getKey());
+      if (Boolean.FALSE.equals(canSeenDB.test(dbName))) {
         continue;
       }
       final TDatabaseInfo storageGroupInfo = entry.getValue();
@@ -115,16 +118,17 @@ public class ShowDBTask implements IConfigTask {
   private static void buildTSBlockForDetails(
       final Map<String, TDatabaseInfo> storageGroupInfoMap,
       final SettableFuture<ConfigTaskResult> future,
-      final Function<String, Boolean> canSeenDB) {
+      final Predicate<String> canSeenDB) {
     final List<TSDataType> outputDataTypes =
         ColumnHeaderConstant.showDBDetailsColumnHeaders.stream()
             .map(ColumnHeader::getColumnType)
             .collect(Collectors.toList());
 
     final TsBlockBuilder builder = new TsBlockBuilder(outputDataTypes);
+    InformationSchemaUtils.buildDatabaseTsBlock(canSeenDB, builder, true);
     for (final Map.Entry<String, TDatabaseInfo> entry : 
storageGroupInfoMap.entrySet()) {
       final String dbName = entry.getKey().substring(5);
-      if (!canSeenDB.apply(dbName)) {
+      if (!canSeenDB.test(dbName)) {
         continue;
       }
       final TDatabaseInfo storageGroupInfo = entry.getValue();
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalDeleteDataNode.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalDeleteDataNode.java
index 05e46cf895d..008e7ca5e1c 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalDeleteDataNode.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalDeleteDataNode.java
@@ -62,35 +62,38 @@ public class RelationalDeleteDataNode extends 
AbstractDeleteDataNode {
 
   private Collection<TRegionReplicaSet> replicaSets;
 
-  public RelationalDeleteDataNode(PlanNodeId id, Delete delete) {
+  public RelationalDeleteDataNode(final PlanNodeId id, final Delete delete) {
     super(id);
     this.modEntries = delete.getTableDeletionEntries();
     this.replicaSets = delete.getReplicaSets();
   }
 
-  public RelationalDeleteDataNode(PlanNodeId id, TableDeletionEntry entry) {
+  public RelationalDeleteDataNode(final PlanNodeId id, final 
TableDeletionEntry entry) {
     super(id);
     this.modEntries = Collections.singletonList(entry);
   }
 
-  public RelationalDeleteDataNode(PlanNodeId id, List<TableDeletionEntry> 
entries) {
+  public RelationalDeleteDataNode(final PlanNodeId id, final 
List<TableDeletionEntry> entries) {
     super(id);
     this.modEntries = entries;
   }
 
-  public RelationalDeleteDataNode(PlanNodeId id, Delete delete, ProgressIndex 
progressIndex) {
+  public RelationalDeleteDataNode(
+      final PlanNodeId id, final Delete delete, final ProgressIndex 
progressIndex) {
     this(id, delete);
     this.progressIndex = progressIndex;
   }
 
   public RelationalDeleteDataNode(
-      PlanNodeId id, Delete delete, TRegionReplicaSet regionReplicaSet) {
+      final PlanNodeId id, final Delete delete, final TRegionReplicaSet 
regionReplicaSet) {
     this(id, delete);
     this.regionReplicaSet = regionReplicaSet;
   }
 
   public RelationalDeleteDataNode(
-      PlanNodeId id, TableDeletionEntry delete, TRegionReplicaSet 
regionReplicaSet) {
+      final PlanNodeId id,
+      final TableDeletionEntry delete,
+      final TRegionReplicaSet regionReplicaSet) {
     this(id, delete);
     this.regionReplicaSet = regionReplicaSet;
   }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java
index f5f769a3d28..9bb89c71fba 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.SchemaPartition;
+import org.apache.iotdb.commons.schema.table.InformationSchema;
 import org.apache.iotdb.db.queryengine.common.MPPQueryContext;
 import org.apache.iotdb.db.queryengine.common.header.DatasetHeader;
 import org.apache.iotdb.db.queryengine.plan.analyze.IAnalysis;
@@ -103,7 +104,6 @@ import static java.util.Collections.unmodifiableMap;
 import static java.util.Collections.unmodifiableSet;
 import static java.util.Objects.requireNonNull;
 import static org.apache.iotdb.commons.partition.DataPartition.NOT_ASSIGNED;
-import static 
org.apache.iotdb.commons.schema.table.InformationSchemaTable.QUERIES;
 
 public class Analysis implements IAnalysis {
 
@@ -781,7 +781,7 @@ public class Analysis implements IAnalysis {
   @Override
   public boolean needSetHighestPriority() {
     return root instanceof ShowStatement
-        && ((ShowStatement) 
root).getTableName().equals(QUERIES.getSchemaTableName());
+        && ((ShowStatement) 
root).getTableName().equals(InformationSchema.QUERIES);
   }
 
   @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java
index 0412ede205a..5ec2fda249a 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java
@@ -143,6 +143,7 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WrappedInsertStat
 import org.apache.iotdb.db.queryengine.plan.statement.component.FillPolicy;
 import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertBaseStatement;
 import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache;
+import org.apache.iotdb.db.schemaengine.table.InformationSchemaUtils;
 import org.apache.iotdb.db.storageengine.load.config.LoadTsFileConfigurator;
 import org.apache.iotdb.db.storageengine.load.metrics.LoadTsFileCostMetricsSet;
 import org.apache.iotdb.rpc.RpcUtils;
@@ -408,6 +409,7 @@ public class StatementAnalyzer {
     protected Scope visitUpdate(final Update node, final Optional<Scope> 
context) {
       queryContext.setQueryType(QueryType.WRITE);
       final TranslationMap translationMap = analyzeTraverseDevice(node, 
context, true);
+      InformationSchemaUtils.checkDBNameInWrite(node.getDatabase());
       final TsTable table =
           DataNodeTableCache.getInstance().getTable(node.getDatabase(), 
node.getTableName());
       node.parseRawExpression(
@@ -456,6 +458,7 @@ public class StatementAnalyzer {
       // Actually write, but will return the result
       queryContext.setQueryType(QueryType.READ);
       node.parseTable(sessionContext);
+      InformationSchemaUtils.checkDBNameInWrite(node.getDatabase());
       final TsTable table =
           DataNodeTableCache.getInstance().getTable(node.getDatabase(), 
node.getTableName());
       if (Objects.isNull(table)) {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java
index 3bdcbf94c48..16286a8e65f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java
@@ -22,7 +22,6 @@ package 
org.apache.iotdb.db.queryengine.plan.relational.metadata;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.schema.table.InformationSchemaTable;
 import org.apache.iotdb.commons.schema.table.TsTable;
 import 
org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinAggregationFunction;
 import 
org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction;
@@ -65,12 +64,12 @@ import org.apache.tsfile.read.common.type.TypeFactory;
 import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.stream.Collectors;
 
 import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_ROOT;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_SEPARATOR;
-import static 
org.apache.iotdb.commons.schema.table.InformationSchemaTable.INFORMATION_SCHEMA;
 import static org.apache.tsfile.read.common.type.BinaryType.TEXT;
 import static org.apache.tsfile.read.common.type.BooleanType.BOOLEAN;
 import static org.apache.tsfile.read.common.type.DateType.DATE;
@@ -91,34 +90,28 @@ public class TableMetadataImpl implements Metadata {
   private final DataNodeTableCache tableCache = 
DataNodeTableCache.getInstance();
 
   @Override
-  public boolean tableExists(QualifiedObjectName name) {
+  public boolean tableExists(final QualifiedObjectName name) {
     return tableCache.getTable(name.getDatabaseName(), name.getObjectName()) 
!= null;
   }
 
   @Override
-  public Optional<TableSchema> getTableSchema(SessionInfo session, 
QualifiedObjectName name) {
-    String databaseName = name.getDatabaseName();
-    String tableName = name.getObjectName();
-
-    // TODO Recover this line after put InformationSchema Table into cache
-    TsTable table =
-        databaseName.equals(INFORMATION_SCHEMA)
-            ? InformationSchemaTable.getTableFromStringValue(tableName)
-            : tableCache.getTable(databaseName, tableName);
-    if (table == null) {
-      return Optional.empty();
-    }
-    List<ColumnSchema> columnSchemaList =
-        table.getColumnList().stream()
-            .map(
-                o ->
-                    new ColumnSchema(
-                        o.getColumnName(),
-                        TypeFactory.getType(o.getDataType()),
-                        false,
-                        o.getColumnCategory()))
-            .collect(Collectors.toList());
-    return Optional.of(new TableSchema(table.getTableName(), 
columnSchemaList));
+  public Optional<TableSchema> getTableSchema(
+      final SessionInfo session, final QualifiedObjectName name) {
+    final TsTable table = tableCache.getTable(name.getDatabaseName(), 
name.getObjectName());
+    return Objects.isNull(table)
+        ? Optional.empty()
+        : Optional.of(
+            new TableSchema(
+                table.getTableName(),
+                table.getColumnList().stream()
+                    .map(
+                        o ->
+                            new ColumnSchema(
+                                o.getColumnName(),
+                                TypeFactory.getType(o.getDataType()),
+                                false,
+                                o.getColumnCategory()))
+                    .collect(Collectors.toList())));
   }
 
   @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableSchema.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableSchema.java
index b5c2997206e..f67fed6ca80 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableSchema.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableSchema.java
@@ -41,7 +41,7 @@ public class TableSchema {
 
   private final List<ColumnSchema> columns;
 
-  public TableSchema(String tableName, List<ColumnSchema> columns) {
+  public TableSchema(final String tableName, final List<ColumnSchema> columns) 
{
     this.tableName = tableName;
     this.columns = columns;
   }
@@ -55,7 +55,8 @@ public class TableSchema {
   }
 
   /** Get the column with the specified name and category, return null if not 
found. */
-  public ColumnSchema getColumn(String columnName, TsTableColumnCategory 
columnCategory) {
+  public ColumnSchema getColumn(
+      final String columnName, final TsTableColumnCategory columnCategory) {
     for (final ColumnSchema column : columns) {
       if (column.getName().equals(columnName) && column.getColumnCategory() == 
columnCategory) {
         return column;
@@ -68,9 +69,9 @@ public class TableSchema {
    * Given the name of an ID column, return the index of this column among all 
ID columns, return -1
    * if not found.
    */
-  public int getIndexAmongIdColumns(String idColumnName) {
+  public int getIndexAmongIdColumns(final String idColumnName) {
     int index = 0;
-    for (ColumnSchema column : getIdColumns()) {
+    for (final ColumnSchema column : getIdColumns()) {
       if (column.getName().equals(idColumnName)) {
         return index;
       }
@@ -79,10 +80,10 @@ public class TableSchema {
     return -1;
   }
 
-  public static TableSchema of(TsTable tsTable) {
-    String tableName = tsTable.getTableName();
-    List<ColumnSchema> columns = new ArrayList<>();
-    for (TsTableColumnSchema tsTableColumnSchema : tsTable.getColumnList()) {
+  public static TableSchema of(final TsTable tsTable) {
+    final String tableName = tsTable.getTableName();
+    final List<ColumnSchema> columns = new ArrayList<>();
+    for (final TsTableColumnSchema tsTableColumnSchema : 
tsTable.getColumnList()) {
       columns.add(ColumnSchema.ofTsColumnSchema(tsTableColumnSchema));
     }
     return new TableSchema(tableName, columns);
@@ -128,9 +129,9 @@ public class TableSchema {
   private static final Logger LOGGER = 
org.slf4j.LoggerFactory.getLogger(TableSchema.class);
 
   public static TableSchema fromTsFileTableSchema(
-      String tableName, org.apache.tsfile.file.metadata.TableSchema 
tsFileTableSchema) {
+      final String tableName, final 
org.apache.tsfile.file.metadata.TableSchema tsFileTableSchema) {
     try {
-      List<ColumnSchema> columns = new ArrayList<>();
+      final List<ColumnSchema> columns = new ArrayList<>();
       for (int i = 0; i < tsFileTableSchema.getColumnSchemas().size(); i++) {
         final String columnName = 
tsFileTableSchema.getColumnSchemas().get(i).getMeasurementName();
         if (columnName == null || columnName.isEmpty()) {
@@ -156,7 +157,7 @@ public class TableSchema {
                 TsTableColumnCategory.fromTsFileColumnType(columnType)));
       }
       return new TableSchema(tableName, columns);
-    } catch (Exception e) {
+    } catch (final Exception e) {
       LOGGER.warn(
           "Cannot convert tsfile table schema to iotdb table schema, table 
name: {}, tsfile table schema: {}",
           tableName,
@@ -167,14 +168,14 @@ public class TableSchema {
   }
 
   @Override
-  public boolean equals(Object o) {
+  public boolean equals(final Object o) {
     if (this == o) {
       return true;
     }
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    TableSchema that = (TableSchema) o;
+    final TableSchema that = (TableSchema) o;
     return Objects.equals(tableName, that.tableName) && 
Objects.equals(columns, that.columns);
   }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableHeaderSchemaValidator.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableHeaderSchemaValidator.java
index 1a8e7318ee5..70850f6712b 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableHeaderSchemaValidator.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableHeaderSchemaValidator.java
@@ -40,6 +40,7 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema;
 import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema;
 import 
org.apache.iotdb.db.queryengine.plan.relational.type.InternalTypeManager;
 import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache;
+import org.apache.iotdb.db.schemaengine.table.InformationSchemaUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 
 import com.google.common.util.concurrent.ListenableFuture;
@@ -86,6 +87,8 @@ public class TableHeaderSchemaValidator {
       final MPPQueryContext context,
       final boolean allowCreateTable,
       final boolean isStrictIdColumn) {
+    InformationSchemaUtils.checkDBNameInWrite(database);
+
     // The schema cache R/W and fetch operation must be locked together thus 
the cache clean
     // operation executed by delete timeSeries will be effective.
     DataNodeSchemaLockManager.getInstance()
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java
index 33bf0a6d5be..fd533fa9a78 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java
@@ -94,7 +94,7 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import static java.util.Objects.requireNonNull;
-import static 
org.apache.iotdb.commons.schema.table.InformationSchemaTable.INFORMATION_SCHEMA;
+import static 
org.apache.iotdb.commons.schema.table.InformationSchema.INFORMATION_DATABASE;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.planner.PlanBuilder.newPlanBuilder;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.planner.QueryPlanner.coerce;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.planner.QueryPlanner.coerceIfNecessary;
@@ -201,7 +201,7 @@ public class RelationPlanner extends 
AstVisitor<RelationPlan, Void> {
     Map<Symbol, ColumnSchema> tableColumnSchema = symbolToColumnSchema.build();
     analysis.addTableSchema(qualifiedObjectName, tableColumnSchema);
     TableScanNode tableScanNode =
-        qualifiedObjectName.getDatabaseName().equals(INFORMATION_SCHEMA)
+        qualifiedObjectName.getDatabaseName().equals(INFORMATION_DATABASE)
             ? new InformationSchemaTableScanNode(
                 idAllocator.genPlanNodeId(), qualifiedObjectName, 
outputSymbols, tableColumnSchema)
             : new DeviceTableScanNode(
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java
index 2d93e873cf0..2c33d30f8f5 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java
@@ -22,6 +22,7 @@ package 
org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations;
 import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
 import org.apache.iotdb.commons.exception.IoTDBRuntimeException;
+import org.apache.iotdb.commons.schema.table.InformationSchema;
 import org.apache.iotdb.confignode.rpc.thrift.TGetDataNodeLocationsResp;
 import org.apache.iotdb.db.protocol.client.ConfigNodeClient;
 import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager;
@@ -32,7 +33,6 @@ import org.apache.thrift.TException;
 
 import java.util.List;
 
-import static 
org.apache.iotdb.commons.schema.table.InformationSchemaTable.QUERIES;
 import static org.apache.iotdb.rpc.TSStatusCode.QUERY_PROCESS_ERROR;
 
 public class DataNodeLocationSupplierFactory {
@@ -60,9 +60,9 @@ public class DataNodeLocationSupplierFactory {
     }
 
     private List<TDataNodeLocation> getRunningDataNodeLocations() {
-      try (ConfigNodeClient client =
+      try (final ConfigNodeClient client =
           
ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID))
 {
-        TGetDataNodeLocationsResp showDataNodesResp = 
client.getRunningDataNodeLocations();
+        final TGetDataNodeLocationsResp showDataNodesResp = 
client.getRunningDataNodeLocations();
         if (showDataNodesResp.getStatus().getCode()
             != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
           throw new IoTDBRuntimeException(
@@ -71,7 +71,7 @@ public class DataNodeLocationSupplierFactory {
               QUERY_PROCESS_ERROR.getStatusCode());
         }
         return showDataNodesResp.getDataNodeLocationList();
-      } catch (ClientManagerException | TException e) {
+      } catch (final ClientManagerException | TException e) {
         throw new IoTDBRuntimeException(
             "An error occurred when executing getRunningDataNodeLocations():" 
+ e.getMessage(),
             QUERY_PROCESS_ERROR.getStatusCode());
@@ -79,8 +79,8 @@ public class DataNodeLocationSupplierFactory {
     }
 
     @Override
-    public List<TDataNodeLocation> getDataNodeLocations(String tableName) {
-      if (tableName.equals(QUERIES.getSchemaTableName())) {
+    public List<TDataNodeLocation> getDataNodeLocations(final String 
tableName) {
+      if (tableName.equals(InformationSchema.QUERIES)) {
         return getRunningDataNodeLocations();
       } else {
         throw new UnsupportedOperationException("Unknown table: " + tableName);
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/AccessControl.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/AccessControl.java
index fe439a19a8c..6b9dcf457b9 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/AccessControl.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/AccessControl.java
@@ -58,7 +58,7 @@ public interface AccessControl {
    * @param databaseName without `root.` prefix, like db
    * @throws AccessDeniedException if not allowed
    */
-  void checkCanShowOrUseDatabase(String userName, String databaseName);
+  void checkCanShowOrUseDatabase(final String userName, final String 
databaseName);
 
   /**
    * Check if user is allowed to create the specified table.
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AbstractTraverseDevice.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AbstractTraverseDevice.java
index dea404dd92b..0c68c5b92cf 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AbstractTraverseDevice.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AbstractTraverseDevice.java
@@ -41,6 +41,7 @@ import java.util.Optional;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AbstractQueryDeviceWithCache.getDeviceColumnHeaderList;
 
 // TODO table metadata: reuse query distinct logic
+// Show, Count, Update, Delete Devices
 public abstract class AbstractTraverseDevice extends Statement {
 
   protected String database;
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java
index fcf65be1eb3..170aa6ec060 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.schema.cache.CacheClearOptions;
+import org.apache.iotdb.commons.schema.table.InformationSchema;
 import org.apache.iotdb.commons.schema.table.TsTable;
 import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory;
 import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema;
@@ -223,7 +224,6 @@ import java.util.function.Function;
 import static com.google.common.collect.ImmutableList.toImmutableList;
 import static java.util.Objects.requireNonNull;
 import static java.util.stream.Collectors.toList;
-import static 
org.apache.iotdb.commons.schema.table.InformationSchemaTable.QUERIES;
 import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME;
 import static 
org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.ATTRIBUTE;
 import static 
org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.ID;
@@ -1231,7 +1231,7 @@ public class AstBuilder extends 
RelationalSqlBaseVisitor<Node> {
 
     return new ShowStatement(
         getLocation(ctx),
-        QUERIES.getSchemaTableName(),
+        InformationSchema.QUERIES,
         visitIfPresent(ctx.where, Expression.class),
         orderBy,
         offset,
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/rewrite/ShowRewrite.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/rewrite/ShowRewrite.java
index 85942d59a2e..ff0a426657a 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/rewrite/ShowRewrite.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/rewrite/ShowRewrite.java
@@ -39,7 +39,7 @@ import java.util.Map;
 import java.util.Optional;
 
 import static java.util.Objects.requireNonNull;
-import static 
org.apache.iotdb.commons.schema.table.InformationSchemaTable.INFORMATION_SCHEMA;
+import static 
org.apache.iotdb.commons.schema.table.InformationSchema.INFORMATION_DATABASE;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.util.QueryUtil.selectList;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.util.QueryUtil.simpleQuery;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.sql.util.QueryUtil.table;
@@ -86,7 +86,7 @@ public final class ShowRewrite implements 
StatementRewrite.Rewrite {
 
       return simpleQuery(
           selectList(new AllColumns()),
-          from(INFORMATION_SCHEMA, showStatement.getTableName()),
+          from(INFORMATION_DATABASE, showStatement.getTableName()),
           showStatement.getWhere(),
           Optional.empty(),
           Optional.empty(),
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java
index 7c30cd29836..a15c13706b7 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java
@@ -54,7 +54,7 @@ public class ShowDatabaseStatement extends ShowStatement 
implements IConfigState
   private final PartialPath pathPattern;
   private boolean isDetailed;
 
-  public ShowDatabaseStatement(PartialPath pathPattern) {
+  public ShowDatabaseStatement(final PartialPath pathPattern) {
     super();
     this.pathPattern = pathPattern;
     this.isDetailed = false;
@@ -68,7 +68,7 @@ public class ShowDatabaseStatement extends ShowStatement 
implements IConfigState
     return isDetailed;
   }
 
-  public void setDetailed(boolean detailed) {
+  public void setDetailed(final boolean detailed) {
     isDetailed = detailed;
   }
 
@@ -120,7 +120,7 @@ public class ShowDatabaseStatement extends ShowStatement 
implements IConfigState
   }
 
   @Override
-  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
+  public <R, C> R accept(final StatementVisitor<R, C> visitor, C context) {
     return visitor.visitShowStorageGroup(this, context);
   }
 
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 3d49790fc4f..ed1fa4aa2c7 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
@@ -373,9 +373,13 @@ public class DataNodeTableCache implements ITableCache {
   private TsTable getTableInCache(final String database, final String 
tableName) {
     readWriteLock.readLock().lock();
     try {
-      return databaseTableMap.containsKey(database)
-          ? databaseTableMap.get(database).get(tableName)
-          : null;
+      final TsTable result =
+          databaseTableMap.containsKey(database)
+              ? databaseTableMap.get(database).get(tableName)
+              : null;
+      return Objects.nonNull(result)
+          ? result
+          : InformationSchemaUtils.mayGetTable(database, tableName);
     } finally {
       readWriteLock.readLock().unlock();
     }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/InformationSchemaUtils.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/InformationSchemaUtils.java
new file mode 100644
index 00000000000..111b883ce51
--- /dev/null
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/InformationSchemaUtils.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.schemaengine.table;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.commons.schema.column.ColumnHeader;
+import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant;
+import org.apache.iotdb.commons.schema.table.TsTable;
+import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema;
+import org.apache.iotdb.db.exception.metadata.table.TableNotExistsException;
+import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.protocol.session.IClientSession;
+import org.apache.iotdb.db.queryengine.common.header.DatasetHeaderFactory;
+import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.tsfile.common.conf.TSFileConfig;
+import org.apache.tsfile.enums.TSDataType;
+import org.apache.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.tsfile.utils.Binary;
+
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import static 
org.apache.iotdb.commons.schema.table.InformationSchema.INFORMATION_DATABASE;
+import static 
org.apache.iotdb.commons.schema.table.InformationSchema.getSchemaTables;
+
+public class InformationSchemaUtils {
+
+  public static void checkDBNameInWrite(final String dbName) {
+    if (dbName.equals(INFORMATION_DATABASE)) {
+      throw new SemanticException(
+          new IoTDBException(
+              "The database 'information_schema' can only be queried",
+              TSStatusCode.SEMANTIC_ERROR.getStatusCode()));
+    }
+  }
+
+  public static void buildDatabaseTsBlock(
+      final Predicate<String> canSeenDB, final TsBlockBuilder builder, final 
boolean details) {
+    if (!canSeenDB.test(INFORMATION_DATABASE)) {
+      return;
+    }
+    builder.getTimeColumnBuilder().writeLong(0L);
+    builder
+        .getColumnBuilder(0)
+        .writeBinary(new Binary(INFORMATION_DATABASE, 
TSFileConfig.STRING_CHARSET));
+    builder
+        .getColumnBuilder(1)
+        .writeBinary(new Binary(IoTDBConstant.TTL_INFINITE, 
TSFileConfig.STRING_CHARSET));
+
+    builder.getColumnBuilder(2).appendNull();
+    builder.getColumnBuilder(3).appendNull();
+    builder.getColumnBuilder(4).appendNull();
+    if (details) {
+      builder.getColumnBuilder(5).writeBinary(new Binary("TABLE", 
TSFileConfig.STRING_CHARSET));
+    }
+    builder.declarePosition();
+  }
+
+  public static TsTable mayGetTable(final String database, final String 
tableName) {
+    return INFORMATION_DATABASE.equals(database) ? 
getSchemaTables().get(tableName) : null;
+  }
+
+  public static boolean mayUseDB(
+      final String database,
+      final IClientSession clientSession,
+      final SettableFuture<ConfigTaskResult> future) {
+    if (!database.equals(INFORMATION_DATABASE)) {
+      return false;
+    }
+    clientSession.setDatabaseName(INFORMATION_DATABASE);
+    future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+    return true;
+  }
+
+  public static boolean mayShowTable(
+      final String database,
+      final boolean isDetails,
+      final SettableFuture<ConfigTaskResult> future) {
+    if (!database.equals(INFORMATION_DATABASE)) {
+      return false;
+    }
+    final List<TSDataType> outputDataTypes =
+        (isDetails
+                ? ColumnHeaderConstant.showTablesDetailsColumnHeaders
+                : ColumnHeaderConstant.showTablesColumnHeaders)
+            
.stream().map(ColumnHeader::getColumnType).collect(Collectors.toList());
+
+    final TsBlockBuilder builder = new TsBlockBuilder(outputDataTypes);
+    for (final String schemaTable : getSchemaTables().keySet()) {
+      builder.getTimeColumnBuilder().writeLong(0L);
+      builder.getColumnBuilder(0).writeBinary(new Binary(schemaTable, 
TSFileConfig.STRING_CHARSET));
+      builder.getColumnBuilder(1).writeBinary(new Binary("INF", 
TSFileConfig.STRING_CHARSET));
+      if (isDetails) {
+        builder.getColumnBuilder(2).writeBinary(new Binary("USING", 
TSFileConfig.STRING_CHARSET));
+      }
+      builder.declarePosition();
+    }
+
+    future.set(
+        new ConfigTaskResult(
+            TSStatusCode.SUCCESS_STATUS,
+            builder.build(),
+            isDetails
+                ? DatasetHeaderFactory.getShowTablesDetailsHeader()
+                : DatasetHeaderFactory.getShowTablesHeader()));
+    return true;
+  }
+
+  public static boolean mayDescribeTable(
+      final String database,
+      final String tableName,
+      final boolean isDetails,
+      final SettableFuture<ConfigTaskResult> future) {
+    if (!database.equals(INFORMATION_DATABASE)) {
+      return false;
+    }
+    if (!getSchemaTables().containsKey(tableName)) {
+      final TableNotExistsException exception =
+          new TableNotExistsException(INFORMATION_DATABASE, tableName);
+      future.setException(new IoTDBException(exception.getMessage(), 
exception.getErrorCode()));
+      return true;
+    }
+    final TsTable table = getSchemaTables().get(tableName);
+
+    final List<TSDataType> outputDataTypes =
+        (isDetails
+                ? ColumnHeaderConstant.describeTableDetailsColumnHeaders
+                : ColumnHeaderConstant.describeTableColumnHeaders)
+            
.stream().map(ColumnHeader::getColumnType).collect(Collectors.toList());
+
+    final TsBlockBuilder builder = new TsBlockBuilder(outputDataTypes);
+    for (final TsTableColumnSchema columnSchema : table.getColumnList()) {
+      builder.getTimeColumnBuilder().writeLong(0L);
+      builder
+          .getColumnBuilder(0)
+          .writeBinary(new Binary(columnSchema.getColumnName(), 
TSFileConfig.STRING_CHARSET));
+      builder
+          .getColumnBuilder(1)
+          .writeBinary(new Binary(columnSchema.getDataType().name(), 
TSFileConfig.STRING_CHARSET));
+      builder
+          .getColumnBuilder(2)
+          .writeBinary(
+              new Binary(columnSchema.getColumnCategory().name(), 
TSFileConfig.STRING_CHARSET));
+      if (isDetails) {
+        builder.getColumnBuilder(3).writeBinary(new Binary("USING", 
TSFileConfig.STRING_CHARSET));
+      }
+      builder.declarePosition();
+    }
+
+    future.set(
+        new ConfigTaskResult(
+            TSStatusCode.SUCCESS_STATUS,
+            builder.build(),
+            isDetails
+                ? DatasetHeaderFactory.getDescribeTableDetailsHeader()
+                : DatasetHeaderFactory.getDescribeTableHeader()));
+    return true;
+  }
+
+  private InformationSchemaUtils() {
+    // Util class
+  }
+}
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java
index 6a6257fb226..1b49488972e 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java
@@ -24,7 +24,6 @@ import 
org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaNodeManagementPartition;
 import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.commons.path.PathPatternTree;
-import org.apache.iotdb.commons.schema.table.InformationSchemaTable;
 import org.apache.iotdb.commons.schema.table.TsTable;
 import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory;
 import org.apache.iotdb.commons.udf.builtin.BuiltinAggregationFunction;
@@ -49,6 +48,7 @@ import 
org.apache.iotdb.db.queryengine.plan.relational.type.InternalTypeManager;
 import org.apache.iotdb.db.queryengine.plan.relational.type.TypeManager;
 import 
org.apache.iotdb.db.queryengine.plan.relational.type.TypeNotFoundException;
 import org.apache.iotdb.db.queryengine.plan.relational.type.TypeSignature;
+import org.apache.iotdb.db.schemaengine.table.InformationSchemaUtils;
 import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq;
 
 import org.apache.tsfile.file.metadata.IDeviceID;
@@ -65,7 +65,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.stream.Collectors;
 
-import static 
org.apache.iotdb.commons.schema.table.InformationSchemaTable.INFORMATION_SCHEMA;
+import static 
org.apache.iotdb.commons.schema.table.InformationSchema.INFORMATION_DATABASE;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.MockTableModelDataPartition.DEVICE_1;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.MockTableModelDataPartition.DEVICE_1_ATTRIBUTES;
 import static 
org.apache.iotdb.db.queryengine.plan.relational.analyzer.MockTableModelDataPartition.DEVICE_2;
@@ -123,8 +123,10 @@ public class TestMatadata implements Metadata {
 
   @Override
   public Optional<TableSchema> getTableSchema(SessionInfo session, 
QualifiedObjectName name) {
-    if (name.getDatabaseName().equals(INFORMATION_SCHEMA)) {
-      TsTable table = 
InformationSchemaTable.getTableFromStringValue(name.getObjectName());
+    if (name.getDatabaseName().equals(INFORMATION_DATABASE)) {
+      TsTable table =
+          InformationSchemaUtils.mayGetTable(
+              INFORMATION_DATABASE, 
name.getObjectName().toLowerCase(Locale.ENGLISH));
       if (table == null) {
         return Optional.empty();
       }
diff --git 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java
index fa00a614c32..6dfd8081ea3 100644
--- 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java
+++ 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java
@@ -199,10 +199,18 @@ public class ColumnHeaderConstant {
   public static final String STATEMENT = "Statement";
 
   public static final String QUERY_ID_TABLE_MODEL = "query_id";
+  public static final String QUERY_ID_START_TIME_TABLE_MODEL = "start_time";
   public static final String DATA_NODE_ID_TABLE_MODEL = "datanode_id";
   public static final String START_TIME_TABLE_MODEL = "start_time";
   public static final String ELAPSED_TIME_TABLE_MODEL = "elapsed_time";
 
+  public static final String TABLE_NAME_TABLE_MODEL = "table_name";
+  public static final String COLUMN_NAME_TABLE_MODEL = "column_name";
+
+  public static final String SCHEMA_REPLICATION_FACTOR_TABLE_MODEL = 
"schema_replication_factor";
+  public static final String DATA_REPLICATION_FACTOR_TABLE_MODEL = 
"data_replication_factor";
+  public static final String TIME_PARTITION_INTERVAL_TABLE_MODEL = 
"time_partition_interval";
+
   // column names for show space quota
   public static final String QUOTA_TYPE = "QuotaType";
   public static final String LIMIT = "Limit";
diff --git 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java
new file mode 100644
index 00000000000..93105f847f4
--- /dev/null
+++ 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.commons.schema.table;
+
+import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant;
+import org.apache.iotdb.commons.schema.table.column.AttributeColumnSchema;
+import org.apache.iotdb.commons.schema.table.column.IdColumnSchema;
+
+import org.apache.tsfile.enums.TSDataType;
+
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+
+public class InformationSchema {
+  public static final String INFORMATION_DATABASE = "information_schema";
+  private static final Map<String, TsTable> schemaTables = new HashMap<>();
+
+  public static final String QUERIES = "queries";
+  public static final String DATABASES = "databases";
+  public static final String TABLES = "tables";
+  public static final String COLUMNS = "columns";
+
+  static {
+    final TsTable queriesTable = new TsTable(QUERIES);
+    queriesTable.addColumnSchema(
+        new IdColumnSchema(ColumnHeaderConstant.QUERY_ID_TABLE_MODEL, 
TSDataType.STRING));
+    queriesTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.QUERY_ID_START_TIME_TABLE_MODEL, 
TSDataType.TIMESTAMP));
+    queriesTable.addColumnSchema(
+        new 
AttributeColumnSchema(ColumnHeaderConstant.DATA_NODE_ID_TABLE_MODEL, 
TSDataType.INT32));
+    queriesTable.addColumnSchema(
+        new 
AttributeColumnSchema(ColumnHeaderConstant.ELAPSED_TIME_TABLE_MODEL, 
TSDataType.FLOAT));
+    queriesTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.STATEMENT.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    queriesTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME);
+    schemaTables.put(QUERIES, queriesTable);
+
+    final TsTable databaseTable = new TsTable(DATABASES);
+    databaseTable.addColumnSchema(
+        new IdColumnSchema(
+            ColumnHeaderConstant.DATABASE.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    databaseTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.COLUMN_TTL.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    databaseTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.SCHEMA_REPLICATION_FACTOR_TABLE_MODEL, 
TSDataType.INT32));
+    databaseTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.DATA_REPLICATION_FACTOR_TABLE_MODEL, 
TSDataType.INT32));
+    databaseTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.TIME_PARTITION_INTERVAL_TABLE_MODEL, 
TSDataType.INT64));
+    databaseTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.MODEL.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    databaseTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME);
+    schemaTables.put(DATABASES, databaseTable);
+
+    final TsTable tableTable = new TsTable(TABLES);
+    tableTable.addColumnSchema(
+        new IdColumnSchema(
+            ColumnHeaderConstant.DATABASE.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    tableTable.addColumnSchema(
+        new IdColumnSchema(ColumnHeaderConstant.TABLE_NAME_TABLE_MODEL, 
TSDataType.STRING));
+    tableTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.COLUMN_TTL.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    tableTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.STATUS.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    tableTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME);
+    schemaTables.put(TABLES, tableTable);
+
+    final TsTable columnTable = new TsTable(COLUMNS);
+    columnTable.addColumnSchema(
+        new IdColumnSchema(
+            ColumnHeaderConstant.DATABASE.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    columnTable.addColumnSchema(
+        new IdColumnSchema(ColumnHeaderConstant.TABLE_NAME_TABLE_MODEL, 
TSDataType.STRING));
+    columnTable.addColumnSchema(
+        new IdColumnSchema(ColumnHeaderConstant.COLUMN_NAME_TABLE_MODEL, 
TSDataType.STRING));
+    columnTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.DATATYPE.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    columnTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.COLUMN_CATEGORY.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    columnTable.addColumnSchema(
+        new AttributeColumnSchema(
+            ColumnHeaderConstant.STATUS.toLowerCase(Locale.ENGLISH), 
TSDataType.STRING));
+    columnTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME);
+    schemaTables.put(COLUMNS, columnTable);
+  }
+
+  public static Map<String, TsTable> getSchemaTables() {
+    return schemaTables;
+  }
+
+  private InformationSchema() {
+    // Utils
+  }
+}
diff --git 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchemaTable.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchemaTable.java
deleted file mode 100644
index 004408fe616..00000000000
--- 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchemaTable.java
+++ /dev/null
@@ -1,91 +0,0 @@
-package org.apache.iotdb.commons.schema.table;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant;
-import org.apache.iotdb.commons.schema.table.column.MeasurementColumnSchema;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.tsfile.enums.TSDataType;
-import org.apache.tsfile.file.metadata.enums.CompressionType;
-import org.apache.tsfile.file.metadata.enums.TSEncoding;
-
-import java.util.Locale;
-
-public enum InformationSchemaTable {
-  QUERIES(
-      new TsTable(
-          "queries",
-          ImmutableList.of(
-              new MeasurementColumnSchema(
-                  ColumnHeaderConstant.QUERY_ID_TABLE_MODEL,
-                  TSDataType.STRING,
-                  TSEncoding.PLAIN,
-                  CompressionType.UNCOMPRESSED),
-              new MeasurementColumnSchema(
-                  ColumnHeaderConstant.START_TIME_TABLE_MODEL,
-                  TSDataType.TIMESTAMP,
-                  TSEncoding.PLAIN,
-                  CompressionType.UNCOMPRESSED),
-              new MeasurementColumnSchema(
-                  ColumnHeaderConstant.DATA_NODE_ID_TABLE_MODEL,
-                  TSDataType.INT32,
-                  TSEncoding.PLAIN,
-                  CompressionType.UNCOMPRESSED),
-              new MeasurementColumnSchema(
-                  ColumnHeaderConstant.ELAPSED_TIME_TABLE_MODEL,
-                  TSDataType.FLOAT,
-                  TSEncoding.PLAIN,
-                  CompressionType.UNCOMPRESSED),
-              new MeasurementColumnSchema(
-                  ColumnHeaderConstant.STATEMENT,
-                  TSDataType.STRING,
-                  TSEncoding.PLAIN,
-                  CompressionType.UNCOMPRESSED))));
-
-  public static final String INFORMATION_SCHEMA = "information_schema";
-
-  private final TsTable table;
-
-  InformationSchemaTable(TsTable table) {
-    this.table = table;
-  }
-
-  public TsTable getTableMetadata() {
-    return table;
-  }
-
-  public String getSchemaTableName() {
-    return table.getTableName();
-  }
-
-  public InformationSchemaTable fromStringValue(String value) {
-    return valueOf(value);
-  }
-
-  public static TsTable getTableFromStringValue(String value) {
-    try {
-      return valueOf(value.toUpperCase(Locale.ENGLISH)).table;
-    } catch (IllegalArgumentException e) {
-      // No matched table
-      return null;
-    }
-  }
-}
diff --git 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/IdColumnSchema.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/IdColumnSchema.java
index db5d97e7f5a..87b2f3c77fe 100644
--- 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/IdColumnSchema.java
+++ 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/IdColumnSchema.java
@@ -28,11 +28,12 @@ import java.nio.ByteBuffer;
 import java.util.Map;
 
 public class IdColumnSchema extends TsTableColumnSchema {
-  public IdColumnSchema(String columnName, TSDataType dataType) {
+  public IdColumnSchema(final String columnName, final TSDataType dataType) {
     super(columnName, dataType);
   }
 
-  public IdColumnSchema(String columnName, TSDataType dataType, Map<String, 
String> props) {
+  public IdColumnSchema(
+      final String columnName, final TSDataType dataType, final Map<String, 
String> props) {
     super(columnName, dataType, props);
   }
 
@@ -41,14 +42,14 @@ public class IdColumnSchema extends TsTableColumnSchema {
     return TsTableColumnCategory.ID;
   }
 
-  static IdColumnSchema deserialize(InputStream stream) throws IOException {
+  static IdColumnSchema deserialize(final InputStream stream) throws 
IOException {
     String columnName = ReadWriteIOUtils.readString(stream);
     TSDataType dataType = ReadWriteIOUtils.readDataType(stream);
     Map<String, String> props = ReadWriteIOUtils.readMap(stream);
     return new IdColumnSchema(columnName, dataType, props);
   }
 
-  static IdColumnSchema deserialize(ByteBuffer buffer) {
+  static IdColumnSchema deserialize(final ByteBuffer buffer) {
     String columnName = ReadWriteIOUtils.readString(buffer);
     TSDataType dataType = ReadWriteIOUtils.readDataType(buffer);
     Map<String, String> props = ReadWriteIOUtils.readMap(buffer);
diff --git 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/MeasurementColumnSchema.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/MeasurementColumnSchema.java
index 1ffec222a7e..b5ff592dff6 100644
--- 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/MeasurementColumnSchema.java
+++ 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/MeasurementColumnSchema.java
@@ -38,19 +38,29 @@ public class MeasurementColumnSchema extends 
TsTableColumnSchema {
 
   private final CompressionType compressor;
 
+  // Only for information schema
+  public MeasurementColumnSchema(final String columnName, final TSDataType 
dataType) {
+    super(columnName, dataType);
+    this.encoding = TSEncoding.PLAIN;
+    this.compressor = CompressionType.UNCOMPRESSED;
+  }
+
   public MeasurementColumnSchema(
-      String columnName, TSDataType dataType, TSEncoding encoding, 
CompressionType compressor) {
+      final String columnName,
+      final TSDataType dataType,
+      final TSEncoding encoding,
+      final CompressionType compressor) {
     super(columnName, dataType);
     this.encoding = encoding;
     this.compressor = compressor;
   }
 
   public MeasurementColumnSchema(
-      String columnName,
-      TSDataType dataType,
-      TSEncoding encoding,
-      CompressionType compressor,
-      Map<String, String> props) {
+      final String columnName,
+      final TSDataType dataType,
+      final TSEncoding encoding,
+      final CompressionType compressor,
+      final Map<String, String> props) {
     super(columnName, dataType, props);
     this.encoding = encoding;
     this.compressor = compressor;
@@ -74,7 +84,7 @@ public class MeasurementColumnSchema extends 
TsTableColumnSchema {
   }
 
   @Override
-  void serialize(OutputStream outputStream) throws IOException {
+  void serialize(final OutputStream outputStream) throws IOException {
     ReadWriteIOUtils.write(columnName, outputStream);
     ReadWriteIOUtils.write(dataType, outputStream);
     ReadWriteIOUtils.write(encoding, outputStream);
@@ -82,21 +92,21 @@ public class MeasurementColumnSchema extends 
TsTableColumnSchema {
     ReadWriteIOUtils.write(props, outputStream);
   }
 
-  static MeasurementColumnSchema deserialize(InputStream stream) throws 
IOException {
-    String columnName = ReadWriteIOUtils.readString(stream);
-    TSDataType dataType = ReadWriteIOUtils.readDataType(stream);
-    TSEncoding encoding = ReadWriteIOUtils.readEncoding(stream);
-    CompressionType compressor = ReadWriteIOUtils.readCompressionType(stream);
-    Map<String, String> props = ReadWriteIOUtils.readMap(stream);
+  static MeasurementColumnSchema deserialize(final InputStream stream) throws 
IOException {
+    final String columnName = ReadWriteIOUtils.readString(stream);
+    final TSDataType dataType = ReadWriteIOUtils.readDataType(stream);
+    final TSEncoding encoding = ReadWriteIOUtils.readEncoding(stream);
+    final CompressionType compressor = 
ReadWriteIOUtils.readCompressionType(stream);
+    final Map<String, String> props = ReadWriteIOUtils.readMap(stream);
     return new MeasurementColumnSchema(columnName, dataType, encoding, 
compressor, props);
   }
 
-  static MeasurementColumnSchema deserialize(ByteBuffer buffer) {
-    String columnName = ReadWriteIOUtils.readString(buffer);
-    TSDataType dataType = ReadWriteIOUtils.readDataType(buffer);
-    TSEncoding encoding = ReadWriteIOUtils.readEncoding(buffer);
-    CompressionType compressor = ReadWriteIOUtils.readCompressionType(buffer);
-    Map<String, String> props = ReadWriteIOUtils.readMap(buffer);
+  static MeasurementColumnSchema deserialize(final ByteBuffer buffer) {
+    final String columnName = ReadWriteIOUtils.readString(buffer);
+    final TSDataType dataType = ReadWriteIOUtils.readDataType(buffer);
+    final TSEncoding encoding = ReadWriteIOUtils.readEncoding(buffer);
+    final CompressionType compressor = 
ReadWriteIOUtils.readCompressionType(buffer);
+    final Map<String, String> props = ReadWriteIOUtils.readMap(buffer);
     return new MeasurementColumnSchema(columnName, dataType, encoding, 
compressor, props);
   }
 }

Reply via email to