You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2021/08/17 05:34:37 UTC

[iotdb] 01/03: [IOTDB-1565] Add sql: set system to readonly/writable

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

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

commit c1f3ae60c4c98383fc97ade3713c19ad21a29ece
Author: HTHou <hh...@outlook.com>
AuthorDate: Tue Aug 17 12:58:40 2021 +0800

    [IOTDB-1565] Add sql: set system to readonly/writable
---
 RELEASE_NOTES.md                                   |   1 +
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |  20 +++
 .../IoTDB-SQL-Language/Maintenance-Command.md      |  27 ++--
 .../IoTDB-SQL-Language/Maintenance-Command.md      |  28 ++--
 .../main/java/org/apache/iotdb/db/qp/Planner.java  |   1 +
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |   2 +
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |   8 ++
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   3 +-
 .../db/qp/logical/sys/SetReadOnlyOperator.java     |  36 +++++
 .../iotdb/db/qp/physical/sys/SetReadOnlyPlan.java  |  45 +++++++
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    | 150 ++-------------------
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |   5 +
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   3 +-
 .../writelog/manager/MultiFileLogNodeManager.java  |   7 +-
 14 files changed, 177 insertions(+), 159 deletions(-)

diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md
index 402dabc..994283b 100644
--- a/RELEASE_NOTES.md
+++ b/RELEASE_NOTES.md
@@ -28,6 +28,7 @@
 * [IOTDB-1491] UDTF query supported in cluster
 * [IOTDB-1536] Support fuzzy query
 * [IOTDB-1561] Support fill by specific value
+* [IOTDB-1565] Add sql: set system to readonly/writable
 * TTL can be set to the prefix path of storage group
 * add JMX monitor to all ThreadPools in the server module 
 
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index 49585ea..d79c0ff 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -86,6 +86,8 @@ statement
     | KILL QUERY INT? #killQuery
     | TRACING ON #tracingOn
     | TRACING OFF #tracingOff
+    | SET SYSTEM TO READONLY #setSystemToReadOnly
+    | SET SYSTEM TO WRITABLE #setSystemToWritable
     | COUNT TIMESERIES prefixPath? (GROUP BY LEVEL OPERATOR_EQ INT)? #countTimeseries
     | COUNT DEVICES prefixPath? #countDevices
     | COUNT STORAGE GROUP prefixPath? #countStorageGroup
@@ -541,6 +543,9 @@ nodeName
     | SCHEMA
     | TRACING
     | OFF
+    | SYSTEM
+    | READONLY
+    | WRITABLE
     | (ID | OPERATOR_IN)? LS_BRACKET INT? ID? RS_BRACKET? ID?
     | compressor
     | GLOBAL
@@ -652,6 +657,9 @@ nodeNameWithoutStar
     | SCHEMA
     | TRACING
     | OFF
+    | SYSTEM
+    | READONLY
+    | WRITABLE
     | (ID | OPERATOR_IN)? LS_BRACKET INT? ID? RS_BRACKET? ID?
     | compressor
     | GLOBAL
@@ -991,6 +999,18 @@ OFF
     : O F F
     ;
 
+SYSTEM
+    : S Y S T E M
+    ;
+
+READONLY
+    : R E A D O N L Y
+    ;
+
+WRITABLE
+    : W R I T A B L E
+    ;
+
 DROP
     : D R O P
     ;
diff --git a/docs/UserGuide/IoTDB-SQL-Language/Maintenance-Command.md b/docs/UserGuide/IoTDB-SQL-Language/Maintenance-Command.md
index 6df1c17..41e0944 100644
--- a/docs/UserGuide/IoTDB-SQL-Language/Maintenance-Command.md
+++ b/docs/UserGuide/IoTDB-SQL-Language/Maintenance-Command.md
@@ -23,7 +23,7 @@
 
 Persist all the data points in the memory table of the storage group to the disk, and seal the data file.
 
-```
+```sql
 IoTDB> FLUSH 
 IoTDB> FLUSH root.ln
 IoTDB> FLUSH root.sg1,root.sg2
@@ -36,7 +36,7 @@ Merge sequence and unsequence data. Currently IoTDB supports the following two t
 * `MERGE` Only rewrite overlapped Chunks, the merge speed is quick, while there will be redundant data on the disk eventually.
 * `FULL MERGE` Rewrite all data in overlapped files, the merge speed is slow, but there will be no redundant data on the disk eventually.
 
-```
+```sql
 IoTDB> MERGE
 IoTDB> FULL MERGE
 ```
@@ -45,15 +45,24 @@ IoTDB> FULL MERGE
 
 Clear the cache of chunk, chunk metadata and timeseries metadata to release the memory footprint.
 
-```
+```sql
 IoTDB> CLEAR CACHE
 ```
 
+### SET STSTEM TO READONLY / WRITABLE
+
+Manually set IoTDB system to read-only or writable status.
+
+```sql
+IoTDB> SET SYSTEM TO READONLY
+IoTDB> SET SYSTEM TO WRITABLE
+```
+
 ### SCHEMA SNAPSHOT
 
 To speed up restarting of IoTDB, users can create snapshot of schema and avoid recovering schema from mlog file.
 
-```
+```sql
 IoTDB> CREATE SNAPSHOT FOR SCHEMA
 ```
 
@@ -67,7 +76,7 @@ Since version 0.12, IoTDB has provided two solutions for queries with long execu
 
 For queries that take too long to execute, IoTDB will forcibly interrupt the query and throw a timeout exception, as shown in the figure: 
 
-```
+```sql
 IoTDB> select * from root;
 Msg: 701 Current query is time out, please check your statement or modify timeout parameter.
 ```
@@ -76,9 +85,9 @@ The default timeout of the system is 60000 ms,which can be customized in the c
 
 If you use JDBC or Session, we also support setting a timeout for a single query(Unit: ms):
 
-```
-E.g. ((IoTDBStatement) statement).executeQuery(String sql, long timeoutInMS)
-E.g. session.executeQueryStatement(String sql, long timeout)
+```java
+((IoTDBStatement) statement).executeQuery(String sql, long timeoutInMS)
+session.executeQueryStatement(String sql, long timeout)
 ```
 
 If the timeout parameter is not configured or with value 0, the default timeout time will be used.
@@ -87,7 +96,7 @@ If the timeout parameter is not configured or with value 0, the default timeout
 
 In addition to waiting for the query to time out passively, IoTDB also supports stopping the query actively:
 
-```
+```sql
 KILL QUERY <queryId>
 ```
 
diff --git a/docs/zh/UserGuide/IoTDB-SQL-Language/Maintenance-Command.md b/docs/zh/UserGuide/IoTDB-SQL-Language/Maintenance-Command.md
index cb6e6f9..a50ae6f 100644
--- a/docs/zh/UserGuide/IoTDB-SQL-Language/Maintenance-Command.md
+++ b/docs/zh/UserGuide/IoTDB-SQL-Language/Maintenance-Command.md
@@ -24,7 +24,7 @@
 
 将指定存储组的内存缓存区Memory Table的数据持久化到磁盘上,并将数据文件封口。
 
-```
+```sql
 IoTDB> FLUSH 
 IoTDB> FLUSH root.ln
 IoTDB> FLUSH root.sg1,root.sg2
@@ -37,7 +37,7 @@ IoTDB> FLUSH root.sg1,root.sg2
 * `MERGE` 仅重写重复的Chunk,整理速度快,但是最终磁盘会存在多余数据。
 * `FULL MERGE` 将需要合并的顺序和乱序文件的所有数据都重新写一份,整理速度慢,最终磁盘将不存在无用的数据。
 
-```
+```sql
 IoTDB> MERGE
 IoTDB> FULL MERGE
 ```
@@ -45,14 +45,24 @@ IoTDB> FULL MERGE
 ### CLEAR CACHE
 
 手动清除chunk, chunk metadata和timeseries metadata的缓存,在内存资源紧张时,可以通过此命令,释放查询时缓存所占的内存空间。
-```
+
+```sql
 IoTDB> CLEAR CACHE
 ```
 
+### SET STSTEM TO READONLY / WRITABLE
+
+手动设置系统为只读或者可写入状态。
+
+```sql
+IoTDB> SET SYSTEM TO READONLY
+IoTDB> SET SYSTEM TO WRITABLE
+```
+
 ### SCHEMA SNAPSHOT
 
 为了加快 IoTDB 重启速度,用户可以手动触发创建 schema 的快照,从而避免服务器从 mlog 文件中恢复。
-```
+```sql
 IoTDB> CREATE SNAPSHOT FOR SCHEMA
 ```
 
@@ -66,7 +76,7 @@ IoTDB> CREATE SNAPSHOT FOR SCHEMA
 
 对于执行时间过长的查询,IoTDB 将强行中断该查询,并抛出超时异常,如下所示:
 
-```
+```sql
 IoTDB> select * from root;
 Msg: 701 Current query is time out, please check your statement or modify timeout parameter.
 ```
@@ -75,9 +85,9 @@ Msg: 701 Current query is time out, please check your statement or modify timeou
 
 如果您使用 JDBC 或 Session,还支持对单个查询设置超时时间(单位为 ms):
 
-```
-E.g. ((IoTDBStatement) statement).executeQuery(String sql, long timeoutInMS)
-E.g. session.executeQueryStatement(String sql, long timeout)
+```java
+((IoTDBStatement) statement).executeQuery(String sql, long timeoutInMS)
+session.executeQueryStatement(String sql, long timeout)
 ```
 
 如果不配置超时时间参数或将超时时间设置为 0,将使用服务器端默认的超时时间.
@@ -86,7 +96,7 @@ E.g. session.executeQueryStatement(String sql, long timeout)
 
 除了被动地等待查询超时外,IoTDB 还支持主动地中止查询,命令为:
 
-```
+```sql
 KILL QUERY <queryId>
 ```
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
index dd9abd1..2d6cabe 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
@@ -150,6 +150,7 @@ public class Planner {
       case FLUSH:
       case MERGE:
       case TRACING:
+      case SET_READONLY:
       case CLEAR_CACHE:
       case NULL:
       case SHOW_MERGE_STATUS:
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index 7cbdfff..a1d7b73 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -180,6 +180,8 @@ public class SQLConstant {
 
   public static final int TOK_LOCK_INFO = 105;
 
+  public static final int TOK_SET_READONLY = 106;
+
   public static final Map<Integer, String> tokenSymbol = new HashMap<>();
   public static final Map<Integer, String> tokenNames = new HashMap<>();
   public static final Map<Integer, Integer> reverseWords = new HashMap<>();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index 99fa603..8b86a1c 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -91,6 +91,7 @@ import org.apache.iotdb.db.qp.physical.sys.KillQueryPlan;
 import org.apache.iotdb.db.qp.physical.sys.LoadConfigurationPlan;
 import org.apache.iotdb.db.qp.physical.sys.MergePlan;
 import org.apache.iotdb.db.qp.physical.sys.OperateFilePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetReadOnlyPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowChildNodesPlan;
@@ -308,6 +309,9 @@ public class PlanExecutor implements IPlanExecutor {
       case TRACING:
         operateTracing((TracingPlan) plan);
         return true;
+      case SET_READONLY:
+        operateSetReadOnly((SetReadOnlyPlan) plan);
+        return true;
       case CLEAR_CACHE:
         operateClearCache();
         return true;
@@ -457,6 +461,10 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
+  private void operateSetReadOnly(SetReadOnlyPlan plan) {
+    IoTDBDescriptor.getInstance().getConfig().setReadOnly(plan.isReadOnly());
+  }
+
   private void operateFlush(FlushPlan plan) throws StorageGroupNotSetException {
     if (plan.getPaths().isEmpty()) {
       StorageEngine.getInstance().syncCloseAllProcessor();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index a88d62f..bacb582 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -156,6 +156,7 @@ public abstract class Operator {
     SET_DEVICE_TEMPLATE,
     SET_USING_DEVICE_TEMPLATE,
     AUTO_CREATE_DEVICE_MNODE,
-    LIKE
+    LIKE,
+    SET_READONLY
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/SetReadOnlyOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/SetReadOnlyOperator.java
new file mode 100644
index 0000000..91c1d9f
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/SetReadOnlyOperator.java
@@ -0,0 +1,36 @@
+/*
+ * 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.qp.logical.sys;
+
+import org.apache.iotdb.db.qp.logical.RootOperator;
+
+public class SetReadOnlyOperator extends RootOperator {
+  private boolean isReadOnly;
+
+  public SetReadOnlyOperator(int tokenIntType, boolean isReadOnly) {
+    super(tokenIntType);
+    this.isReadOnly = isReadOnly;
+    operatorType = OperatorType.SET_READONLY;
+  }
+
+  public boolean isReadOnly() {
+    return isReadOnly;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetReadOnlyPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetReadOnlyPlan.java
new file mode 100644
index 0000000..3832e01
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetReadOnlyPlan.java
@@ -0,0 +1,45 @@
+/*
+ * 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.qp.physical.sys;
+
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+
+import java.util.Collections;
+import java.util.List;
+
+public class SetReadOnlyPlan extends PhysicalPlan {
+
+  private boolean isReadOnly;
+
+  public SetReadOnlyPlan(boolean isReadOnly) {
+    super(false, OperatorType.SET_READONLY);
+    this.isReadOnly = isReadOnly;
+  }
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
+
+  public boolean isReadOnly() {
+    return isReadOnly;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
index ec721ea..1135291 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
@@ -63,6 +63,7 @@ import org.apache.iotdb.db.qp.logical.sys.LoadFilesOperator;
 import org.apache.iotdb.db.qp.logical.sys.MergeOperator;
 import org.apache.iotdb.db.qp.logical.sys.MoveFileOperator;
 import org.apache.iotdb.db.qp.logical.sys.RemoveFileOperator;
+import org.apache.iotdb.db.qp.logical.sys.SetReadOnlyOperator;
 import org.apache.iotdb.db.qp.logical.sys.SetStorageGroupOperator;
 import org.apache.iotdb.db.qp.logical.sys.SetTTLOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowChildNodesOperator;
@@ -79,144 +80,7 @@ import org.apache.iotdb.db.qp.logical.sys.ShowTriggersOperator;
 import org.apache.iotdb.db.qp.logical.sys.StartTriggerOperator;
 import org.apache.iotdb.db.qp.logical.sys.StopTriggerOperator;
 import org.apache.iotdb.db.qp.logical.sys.TracingOperator;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AggregationCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AggregationElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AliasClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlignByDeviceClauseOrDisableAlignContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlignByDeviceClauseOrDisableAlignStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AndExpressionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AsClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AsElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AttributeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AttributeClausesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.BuiltInFunctionCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ClearcacheContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ConstantContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountDevicesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountNodesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateFunctionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateIndexContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateSnapshotContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateTriggerContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DateExpressionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DeletePartitionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DeleteStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DeleteStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DeleteTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropFunctionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropIndexContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropTriggerContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FillClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FillStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FlushContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FromClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FullMergeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FullPathContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FunctionAsClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FunctionAsElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantRoleToUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantWatermarkEmbeddingContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByFillClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByFillStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByLevelClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByLevelStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByTimeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByTimeStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.InClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.IndexPredicateClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.IndexWithClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertColumnsSpecContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertValuesSpecContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.KillQueryContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LastClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LastElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LimitClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LimitStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListAllRoleOfUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListAllUserOfRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListPrivilegesRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListPrivilegesUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListRolePrivilegesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListUserPrivilegesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LoadConfigurationStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LoadFilesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LoadStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.MergeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.MoveFileContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.NodeNameContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.NodeNameWithoutStarContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.OffsetClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.OrExpressionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.OrderByTimeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.OrderByTimeStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PredicateContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PrefixPathContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PrivilegesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PropertyContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PropertyValueContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RemoveFileContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeRoleFromUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeWatermarkEmbeddingContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RootOrIdContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SelectStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SequenceClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SetStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SetTTLStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowAllTTLStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowChildNodesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowChildPathsContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowDevicesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowFlushTaskInfoContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowFunctionsContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowLockInfoContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowMergeStatusContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowQueryProcesslistContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowTTLStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowTriggersContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowVersionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowWhereClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SingleStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SlimitClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SlimitStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SoffsetClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SpecialLimitStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.StartTriggerContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.StopTriggerContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.StringLiteralContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SuffixPathContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TableCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TableElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TagClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TimeIntervalContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TracingOffContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TracingOnContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TriggerAttributeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TypeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.UdfAttributeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.UdfCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.UnsetTTLStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.WhereClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.WithoutNullStatementContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.*;
 import org.apache.iotdb.db.qp.utils.DatetimeUtils;
 import org.apache.iotdb.db.query.executor.fill.IFill;
 import org.apache.iotdb.db.query.executor.fill.LinearFill;
@@ -917,6 +781,16 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   }
 
   @Override
+  public Operator visitSetSystemToReadOnly(SetSystemToReadOnlyContext ctx) {
+    return new SetReadOnlyOperator(SQLConstant.TOK_SET_READONLY, true);
+  }
+
+  @Override
+  public Operator visitSetSystemToWritable(SetSystemToWritableContext ctx) {
+    return new SetReadOnlyOperator(SQLConstant.TOK_SET_READONLY, false);
+  }
+
+  @Override
   public Operator visitCountTimeseries(CountTimeseriesContext ctx) {
     PrefixPathContext pathContext = ctx.prefixPath();
     PartialPath path =
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index c9c9d34..abb3f3f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -58,6 +58,7 @@ import org.apache.iotdb.db.qp.logical.sys.LoadDataOperator;
 import org.apache.iotdb.db.qp.logical.sys.LoadFilesOperator;
 import org.apache.iotdb.db.qp.logical.sys.MoveFileOperator;
 import org.apache.iotdb.db.qp.logical.sys.RemoveFileOperator;
+import org.apache.iotdb.db.qp.logical.sys.SetReadOnlyOperator;
 import org.apache.iotdb.db.qp.logical.sys.SetStorageGroupOperator;
 import org.apache.iotdb.db.qp.logical.sys.SetTTLOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowChildNodesOperator;
@@ -109,6 +110,7 @@ import org.apache.iotdb.db.qp.physical.sys.LoadConfigurationPlan.LoadConfigurati
 import org.apache.iotdb.db.qp.physical.sys.LoadDataPlan;
 import org.apache.iotdb.db.qp.physical.sys.MergePlan;
 import org.apache.iotdb.db.qp.physical.sys.OperateFilePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetReadOnlyPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowChildNodesPlan;
@@ -260,6 +262,9 @@ public class PhysicalGenerator {
       case TRACING:
         TracingOperator tracingOperator = (TracingOperator) operator;
         return new TracingPlan(tracingOperator.isTracingOn());
+      case SET_READONLY:
+        SetReadOnlyOperator setReadOnlyOperator = (SetReadOnlyOperator) operator;
+        return new SetReadOnlyPlan(setReadOnlyOperator.isReadOnly());
       case QUERY:
         QueryOperator query = (QueryOperator) operator;
         return transformQuery(query, fetchSize);
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index fccdd04..e28f9cf 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -66,6 +66,7 @@ import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetReadOnlyPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowQueryProcesslistPlan;
@@ -1132,7 +1133,7 @@ public class TSServiceImpl implements TSIService.Iface {
 
   private boolean executeNonQuery(PhysicalPlan plan)
       throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
-    if (IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
+    if (!(plan instanceof SetReadOnlyPlan) && IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
       throw new QueryProcessException(
           "Current system mode is read-only, does not support non-query operation");
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/manager/MultiFileLogNodeManager.java b/server/src/main/java/org/apache/iotdb/db/writelog/manager/MultiFileLogNodeManager.java
index 236ef2f..a9283b1 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/manager/MultiFileLogNodeManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/manager/MultiFileLogNodeManager.java
@@ -50,12 +50,17 @@ public class MultiFileLogNodeManager implements WriteLogNodeManager, IService {
 
   private ScheduledExecutorService executorService;
   private final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private boolean firstReadOnly = true;
 
   private void forceTask() {
     if (IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
-      logger.warn("system mode is read-only, the force flush WAL task is stopped");
+      if (firstReadOnly) {
+        logger.warn("system mode is read-only, the force flush WAL task is stopped");
+        firstReadOnly = false;
+      }
       return;
     }
+    firstReadOnly = true;
     if (Thread.interrupted()) {
       logger.info("WAL force thread exits.");
       return;