You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2019/07/15 03:21:21 UTC

[incubator-iotdb] branch fix_create_timeseries_check created (now 4c67e79)

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

qiaojialin pushed a change to branch fix_create_timeseries_check
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git.


      at 4c67e79  refactor readonly mode check

This branch includes the following new commits:

     new 4c67e79  refactor readonly mode check

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[incubator-iotdb] 01/01: refactor readonly mode check

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qiaojialin pushed a commit to branch fix_create_timeseries_check
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 4c67e793de905006d4e44a4a68db7c92946bc694
Author: qiaojialin <64...@qq.com>
AuthorDate: Mon Jul 15 11:20:54 2019 +0800

    refactor readonly mode check
---
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 10 +++
 .../org/apache/iotdb/db/engine/StorageEngine.java  | 27 +-------
 .../engine/storagegroup/StorageGroupProcessor.java |  2 +-
 .../db/engine/storagegroup/TsFileProcessor.java    |  4 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |  3 -
 .../iotdb/db/qp/executor/QueryProcessExecutor.java | 62 ++++++++----------
 .../org/apache/iotdb/db/service/TSServiceImpl.java | 73 +++++++++++-----------
 .../writelog/manager/MultiFileLogNodeManager.java  |  2 +-
 .../db/writelog/node/ExclusiveWriteLogNode.java    |  2 +-
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |  1 +
 10 files changed, 80 insertions(+), 106 deletions(-)

diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index fa402d3..e1042ef 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -47,6 +47,8 @@ public class IoTDBConfig {
    */
   private boolean enableWal = true;
 
+  private volatile boolean readOnly = false;
+
   /**
    * When a certain amount of write ahead logs is reached, they will be flushed to the disk. It is
    * possible to lose at most flush_wal_threshold operations.
@@ -478,6 +480,14 @@ public class IoTDBConfig {
     this.cacheFileReaderClearPeriod = cacheFileReaderClearPeriod;
   }
 
+  public boolean isReadOnly() {
+    return readOnly;
+  }
+
+  public void setReadOnly(boolean readOnly) {
+    this.readOnly = readOnly;
+  }
+
   public String getRpcImplClassName() {
     return rpcImplClassName;
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index d27b0a3..6ef6126 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -54,8 +54,6 @@ public class StorageEngine implements IService {
   private static final Logger logger = LoggerFactory.getLogger(StorageEngine.class);
   private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
 
-  private volatile boolean readOnly = false;
-
   /**
    * a folder (system/storage_groups/ by default) that persist system info. Each Storage Processor will have a
    * subfolder under the systemDir.
@@ -145,9 +143,8 @@ public class StorageEngine implements IService {
   /**
    * This function is just for unit test.
    */
-  public synchronized void reset() throws IOException {
+  public synchronized void reset() {
     processorMap.clear();
-    readOnly = false;
   }
 
 
@@ -159,11 +156,6 @@ public class StorageEngine implements IService {
    */
   public boolean insert(InsertPlan insertPlan) throws StorageEngineException {
 
-    if (readOnly) {
-      throw new StorageEngineException(
-          "Current system mode is read only, does not support insertion");
-    }
-
     StorageGroupProcessor storageGroupProcessor;
     try {
       storageGroupProcessor = getProcessor(insertPlan.getDeviceId());
@@ -216,12 +208,6 @@ public class StorageEngine implements IService {
    */
   public void delete(String deviceId, String measurementId, long timestamp)
       throws StorageEngineException {
-
-    if (readOnly) {
-      throw new StorageEngineException(
-          "Current system mode is read only, does not support deletion");
-    }
-
     StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
     try {
       storageGroupProcessor.delete(deviceId, measurementId, timestamp);
@@ -290,23 +276,12 @@ public class StorageEngine implements IService {
     return Collections.emptyList();
   }
 
-  public boolean isReadOnly() {
-    return readOnly;
-  }
-
-  public void setReadOnly(boolean readOnly) {
-    this.readOnly = readOnly;
-  }
-
   /**
    * merge all storage groups.
    *
    * @throws StorageEngineException StorageEngineException
    */
   public void mergeAll() throws StorageEngineException {
-    if (readOnly) {
-      throw new StorageEngineException("Current system mode is read only, does not support merge");
-    }
     // TODO
   }
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index a3d0e9d..ad8f1dd 100755
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -303,7 +303,7 @@ public class StorageGroupProcessor {
       }
     } catch (DiskSpaceInsufficientException e) {
       logger.error("disk space is insufficient", e);
-      StorageEngine.getInstance().setReadOnly(true);
+      IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
       return false;
     }
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index d62e49d..4715afa 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -374,7 +374,7 @@ public class TsFileProcessor {
         writer.mark();
         flushTask.syncFlushMemTable();
       } catch (ExecutionException | InterruptedException | IOException e) {
-        StorageEngine.getInstance().setReadOnly(true);
+        IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
         try {
           logger.error("IOTask meets error, truncate the corrupted data", e);
           writer.reset();
@@ -401,7 +401,7 @@ public class TsFileProcessor {
         writer.mark();
         endFile();
       } catch (IOException | TsFileProcessorException e) {
-        StorageEngine.getInstance().setReadOnly(true);
+        IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
         try {
           writer.reset();
         } catch (IOException e1) {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index c415843..2f98364 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -511,9 +511,6 @@ public class MManager {
    * function for setting storage level of the given path to mTree.
    */
   public void setStorageLevelToMTree(String path) throws MetadataErrorException {
-    if (initialized && StorageEngine.getInstance().isReadOnly()) {
-      throw new MetadataErrorException("Current system mode is read only, does not support creating Storage Group");
-    }
 
     lock.writeLock().lock();
     try {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
index 9a3021b..6d5ee59 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
@@ -77,28 +77,6 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     storageEngine = StorageEngine.getInstance();
   }
 
-  public static String checkValue(TSDataType dataType, String value) throws ProcessorException {
-    if (dataType == TSDataType.BOOLEAN) {
-      value = value.toLowerCase();
-      if (SQLConstant.BOOLEAN_FALSE_NUM.equals(value)) {
-        value = "false";
-      } else if (SQLConstant.BOOLEAN_TRUE_NUM.equals(value)) {
-        value = "true";
-      } else if (!SQLConstant.BOOLEN_TRUE.equals(value) && !SQLConstant.BOOLEN_FALSE
-          .equals(value)) {
-        throw new ProcessorException("The BOOLEAN data type should be true/TRUE or false/FALSE");
-      }
-    } else if (dataType == TSDataType.TEXT) {
-      if ((value.startsWith(SQLConstant.QUOTE) && value.endsWith(SQLConstant.QUOTE))
-          || (value.startsWith(SQLConstant.DQUOTE) && value.endsWith(SQLConstant.DQUOTE))) {
-        value = value.substring(1, value.length() - 1);
-      } else {
-        throw new ProcessorException("The TEXT data type should be covered by \" or '");
-      }
-    }
-    return value;
-  }
-
   @Override
   public boolean processNonQuery(PhysicalPlan plan) throws ProcessorException {
     switch (plan.getOperatorType()) {
@@ -126,14 +104,6 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
       case DELETE_USER:
         AuthorPlan author = (AuthorPlan) plan;
         return operateAuthor(author);
-      case LIST_ROLE:
-      case LIST_USER:
-      case LIST_ROLE_PRIVILEGE:
-      case LIST_ROLE_USERS:
-      case LIST_USER_PRIVILEGE:
-      case LIST_USER_ROLES:
-        throw new ProcessorException(String.format("Author query %s is now allowed"
-            + " in processNonQuery", plan.getOperatorType()));
       case DELETE_TIMESERIES:
       case CREATE_TIMESERIES:
       case SET_STORAGE_GROUP:
@@ -269,6 +239,30 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     return MManager.getInstance().getPaths(originPath);
   }
 
+
+  private static String checkValue(TSDataType dataType, String value) throws ProcessorException {
+    if (dataType == TSDataType.BOOLEAN) {
+      value = value.toLowerCase();
+      if (SQLConstant.BOOLEAN_FALSE_NUM.equals(value)) {
+        value = "false";
+      } else if (SQLConstant.BOOLEAN_TRUE_NUM.equals(value)) {
+        value = "true";
+      } else if (!SQLConstant.BOOLEN_TRUE.equals(value) && !SQLConstant.BOOLEN_FALSE
+          .equals(value)) {
+        throw new ProcessorException("The BOOLEAN data type should be true/TRUE or false/FALSE");
+      }
+    } else if (dataType == TSDataType.TEXT) {
+      if ((value.startsWith(SQLConstant.QUOTE) && value.endsWith(SQLConstant.QUOTE))
+          || (value.startsWith(SQLConstant.DQUOTE) && value.endsWith(SQLConstant.DQUOTE))) {
+        value = value.substring(1, value.length() - 1);
+      } else {
+        throw new ProcessorException("The TEXT data type should be covered by \" or '");
+      }
+    }
+    return value;
+  }
+
+
   private boolean operateAuthor(AuthorPlan author) throws ProcessorException {
     AuthorOperator.AuthorType authorType = author.getAuthorType();
     String userName = author.getUserName();
@@ -346,16 +340,14 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     try {
       switch (namespaceType) {
         case ADD_PATH:
-          boolean isNewMeasurement = mManager.addPathToMTree(path, dataType, encoding, compressor
-              , props);
-          if (isNewMeasurement) {
+          boolean result = mManager.addPathToMTree(path, dataType, encoding, compressor, props);
+          if (result) {
             storageEngine.addTimeSeries(path, dataType, encoding, compressor, props);
           }
           break;
         case DELETE_PATH:
           deleteDataOfTimeSeries(deletePathList);
-          Set<String> emptyStorageGroups =
-              mManager.deletePaths(deletePathList);
+          Set<String> emptyStorageGroups = mManager.deletePaths(deletePathList);
           for (String deleteStorageGroup : emptyStorageGroups) {
             storageEngine.deleteAllDataFilesInOneStorageGroup(deleteStorageGroup);
           }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index 5071eef..459547e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -109,18 +109,18 @@ import org.slf4j.LoggerFactory;
 public class TSServiceImpl implements TSIService.Iface, ServerContext {
 
   private static final Logger logger = LoggerFactory.getLogger(TSServiceImpl.class);
-  protected static final String INFO_NOT_LOGIN = "{}: Not login.";
-  protected static final String ERROR_NOT_LOGIN = "Not login";
+  private static final String INFO_NOT_LOGIN = "{}: Not login.";
+  private static final String ERROR_NOT_LOGIN = "Not login";
 
   protected QueryProcessor processor;
   // Record the username for every rpc connection. Username.get() is null if
   // login is failed.
   protected ThreadLocal<String> username = new ThreadLocal<>();
-  protected ThreadLocal<HashMap<String, PhysicalPlan>> queryStatus = new ThreadLocal<>();
-  protected ThreadLocal<HashMap<String, QueryDataSet>> queryRet = new ThreadLocal<>();
-  protected ThreadLocal<ZoneId> zoneIds = new ThreadLocal<>();
+  private ThreadLocal<HashMap<String, PhysicalPlan>> queryStatus = new ThreadLocal<>();
+  private ThreadLocal<HashMap<String, QueryDataSet>> queryRet = new ThreadLocal<>();
+  private ThreadLocal<ZoneId> zoneIds = new ThreadLocal<>();
   private IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-  protected ThreadLocal<Map<Long, QueryContext>> contextMapLocal = new ThreadLocal<>();
+  private ThreadLocal<Map<Long, QueryContext>> contextMapLocal = new ThreadLocal<>();
 
   private AtomicLong globalStmtId = new AtomicLong(0L);
   // (statementId) -> (statement)
@@ -178,7 +178,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSCloseSessionResp closeSession(TSCloseSessionReq req) throws TException {
+  public TSCloseSessionResp closeSession(TSCloseSessionReq req) {
     logger.info("{}: receive close session", IoTDBConstant.GLOBAL_DB_NAME);
     TS_Status tsStatus;
     if (username.get() == null) {
@@ -198,12 +198,12 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSCancelOperationResp cancelOperation(TSCancelOperationReq req) throws TException {
+  public TSCancelOperationResp cancelOperation(TSCancelOperationReq req) {
     return new TSCancelOperationResp(new TS_Status(TS_StatusCode.SUCCESS_STATUS));
   }
 
   @Override
-  public TSCloseOperationResp closeOperation(TSCloseOperationReq req) throws TException {
+  public TSCloseOperationResp closeOperation(TSCloseOperationReq req) {
     logger.info("{}: receive close operation", IoTDBConstant.GLOBAL_DB_NAME);
     try {
 
@@ -221,7 +221,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return new TSCloseOperationResp(new TS_Status(TS_StatusCode.SUCCESS_STATUS));
   }
 
-  protected void releaseQueryResource(TSCloseOperationReq req) throws StorageEngineException {
+  private void releaseQueryResource(TSCloseOperationReq req) throws StorageEngineException {
     Map<Long, QueryContext> contextMap = contextMapLocal.get();
     if (contextMap == null) {
       return;
@@ -254,7 +254,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSFetchMetadataResp fetchMetadata(TSFetchMetadataReq req) throws TException {
+  public TSFetchMetadataResp fetchMetadata(TSFetchMetadataReq req) {
     TS_Status status;
     if (!checkLogin()) {
       logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
@@ -320,16 +320,16 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return resp;
   }
 
-  protected Set<String> getAllStorageGroups() throws PathErrorException {
+  private Set<String> getAllStorageGroups() throws PathErrorException {
     return MManager.getInstance().getAllStorageGroup();
   }
 
-  protected List<List<String>> getTimeSeriesForPath(String path)
+  private List<List<String>> getTimeSeriesForPath(String path)
       throws PathErrorException {
     return MManager.getInstance().getShowTimeseriesPath(path);
   }
 
-  protected String getMetadataInString() {
+  private String getMetadataInString() {
     return MManager.getInstance().getMetadataInString();
   }
 
@@ -409,8 +409,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSExecuteBatchStatementResp executeBatchStatement(TSExecuteBatchStatementReq req)
-      throws TException {
+  public TSExecuteBatchStatementResp executeBatchStatement(TSExecuteBatchStatementReq req) {
     long t1 = System.currentTimeMillis();
     String currStmt = null;
     List<Integer> result = new ArrayList<>();
@@ -440,9 +439,6 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
             batchErrorMessage.toString(),
             result);
       }
-    } catch (QueryInBatchStmtException e) {
-      return getTSBathExecuteStatementResp(TS_StatusCode.ERROR_STATUS,
-          "statement is query :" + currStmt, result);
     } catch (Exception e) {
       logger.error("{}: error occurs when executing statements", IoTDBConstant.GLOBAL_DB_NAME, e);
       return getTSBathExecuteStatementResp(TS_StatusCode.ERROR_STATUS, e.getMessage(), null);
@@ -454,7 +450,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   // execute one statement of a batch. Currently, query is not allowed in a batch statement and
   // on finding queries in a batch, such query will be ignored and an error will be generated
   private boolean executeStatementInBatch(String statement, StringBuilder batchErrorMessage,
-      List<Integer> result) throws QueryInBatchStmtException {
+      List<Integer> result) {
     try {
       PhysicalPlan physicalPlan = processor.parseSQLToPhysicalPlan(statement, zoneIds.get());
       if (physicalPlan.isQuery()) {
@@ -483,7 +479,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
 
 
   @Override
-  public TSExecuteStatementResp executeStatement(TSExecuteStatementReq req) throws TException {
+  public TSExecuteStatementResp executeStatement(TSExecuteStatementReq req) {
     try {
       if (!checkLogin()) {
         logger.info(INFO_NOT_LOGIN, IoTDBConstant.GLOBAL_DB_NAME);
@@ -520,7 +516,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   /**
    * Set consistency level
    */
-  public boolean execSetConsistencyLevel(String statement) throws SQLException {
+  private boolean execSetConsistencyLevel(String statement) throws SQLException {
     if (statement == null) {
       return false;
     }
@@ -534,7 +530,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSExecuteStatementResp executeQueryStatement(TSExecuteStatementReq req) throws TException {
+  public TSExecuteStatementResp executeQueryStatement(TSExecuteStatementReq req) {
     long t1 = System.currentTimeMillis();
     try {
       if (!checkLogin()) {
@@ -654,12 +650,12 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return resp;
   }
 
-  protected void checkFileLevelSet(List<Path> paths) throws PathErrorException {
+  private void checkFileLevelSet(List<Path> paths) throws PathErrorException {
     MManager.getInstance().checkFileLevel(paths);
   }
 
   @Override
-  public TSFetchResultsResp fetchResults(TSFetchResultsReq req) throws TException {
+  public TSFetchResultsResp fetchResults(TSFetchResultsReq req) {
     try {
       if (!checkLogin()) {
         return getTSFetchResultsResp(TS_StatusCode.ERROR_STATUS, "Not login.");
@@ -697,7 +693,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     }
   }
 
-  protected QueryDataSet createNewDataSet(String statement, TSFetchResultsReq req)
+  private QueryDataSet createNewDataSet(String statement, TSFetchResultsReq req)
       throws PathErrorException, QueryFilterOptimizationException, StorageEngineException,
       ProcessorException, IOException {
     PhysicalPlan physicalPlan = queryStatus.get().get(statement);
@@ -715,7 +711,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return queryDataSet;
   }
 
-  protected void initContextMap() {
+  private void initContextMap() {
     Map<Long, QueryContext> contextMap = contextMapLocal.get();
     if (contextMap == null) {
       contextMap = new HashMap<>();
@@ -724,8 +720,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req)
-      throws TException {
+  public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req) {
     try {
       if (!checkLogin()) {
         return getTSExecuteStatementResp(TS_StatusCode.ERROR_STATUS, ERROR_NOT_LOGIN);
@@ -773,7 +768,11 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return resp;
   }
 
-  protected boolean executeNonQuery(PhysicalPlan plan) throws ProcessorException {
+  private boolean executeNonQuery(PhysicalPlan plan) throws ProcessorException {
+    if (IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
+      throw new ProcessorException(
+          "Current system mode is read only, does not support none query operation");
+    }
     return processor.getExecutor().processNonQuery(plan);
   }
 
@@ -806,11 +805,11 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
    *
    * @return true: If logged in; false: If not logged in
    */
-  protected boolean checkLogin() {
+  private boolean checkLogin() {
     return username.get() != null;
   }
 
-  protected boolean checkAuthorization(List<Path> paths, PhysicalPlan plan) throws AuthException {
+  private boolean checkAuthorization(List<Path> paths, PhysicalPlan plan) throws AuthException {
     String targetUser = null;
     if (plan instanceof AuthorPlan) {
       targetUser = ((AuthorPlan) plan).getUserName();
@@ -831,7 +830,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return resp;
   }
 
-  protected TSExecuteBatchStatementResp getTSBathExecuteStatementResp(TS_StatusCode code,
+  private TSExecuteBatchStatementResp getTSBathExecuteStatementResp(TS_StatusCode code,
       String msg,
       List<Integer> result) {
     TSExecuteBatchStatementResp resp = new TSExecuteBatchStatementResp();
@@ -850,13 +849,13 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return resp;
   }
 
-  protected void handleClientExit() throws TException {
+  void handleClientExit() throws TException {
     closeOperation(null);
     closeSession(null);
   }
 
   @Override
-  public TSGetTimeZoneResp getTimeZone() throws TException {
+  public TSGetTimeZoneResp getTimeZone() {
     TS_Status tsStatus;
     TSGetTimeZoneResp resp;
     try {
@@ -872,7 +871,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public TSSetTimeZoneResp setTimeZone(TSSetTimeZoneReq req) throws TException {
+  public TSSetTimeZoneResp setTimeZone(TSSetTimeZoneReq req) {
     TS_Status tsStatus;
     try {
       String timeZoneID = req.getTimeZone();
@@ -887,7 +886,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   }
 
   @Override
-  public ServerProperties getProperties() throws TException {
+  public ServerProperties getProperties() {
     ServerProperties properties = new ServerProperties();
     properties.setVersion(IoTDBConstant.VERSION);
     properties.setSupportedTimeAggregationOperations(new ArrayList<>());
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/writelog/manager/MultiFileLogNodeManager.java b/iotdb/src/main/java/org/apache/iotdb/db/writelog/manager/MultiFileLogNodeManager.java
index 5e10f75..a67b54b 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/writelog/manager/MultiFileLogNodeManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/writelog/manager/MultiFileLogNodeManager.java
@@ -47,7 +47,7 @@ public class MultiFileLogNodeManager implements WriteLogNodeManager, IService {
 
   private final Runnable forceTask = () -> {
       while (true) {
-        if (StorageEngine.getInstance().isReadOnly()) {
+        if (IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
           return;
         }
         if (Thread.interrupted()) {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java b/iotdb/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
index db02ae6..373290e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
@@ -215,7 +215,7 @@ public class ExclusiveWriteLogNode implements WriteLogNode, Comparable<Exclusive
       try {
         getCurrentFileWriter().write(logBuffer);
       } catch (IOException e) {
-        StorageEngine.getInstance().setReadOnly(true);
+        IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
         logger.error("Log node {} sync failed", identifier, e);
         return;
       }
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/iotdb/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
index 088bdde..60d8d30 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
@@ -73,6 +73,7 @@ public class EnvironmentUtils {
       Assert.fail();
     }
     StorageEngine.getInstance().reset();
+    IoTDBDescriptor.getInstance().getConfig().setReadOnly(false);
 
     StatMonitor.getInstance().close();
     // clean wal