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 2020/12/01 02:54:24 UTC

[iotdb] branch mem_control_op created (now 313eb0c)

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

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


      at 313eb0c  fix review

This branch includes the following new commits:

     new 5b7ea3d  change reject error log to warn
     new 4d2dbda  add some properties discription
     new 96013e7  use synchronized to fix mem control concurrent issue
     new 313eb0c  fix review

The 4 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.



[iotdb] 02/04: add some properties discription

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

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

commit 4d2dbda2c7728453c40ecb03fc077364f799e976
Author: HTHou <hh...@outlook.com>
AuthorDate: Tue Nov 24 23:25:18 2020 +0800

    add some properties discription
---
 server/src/assembly/resources/conf/iotdb-engine.properties | 10 ++++++++++
 1 file changed, 10 insertions(+)

diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index aae3444..fe5f7bb 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -174,6 +174,7 @@ timestamp_precision=ms
 wal_buffer_size=16777216
 
 # When a TsFile's file size (in byte) exceeds this, the TsFile is forced closed.
+# It may cause memTable size smaller, if it sets a large value
 tsfile_size_threshold=1
 
 # When a memTable's size (in byte) exceeds this, the memtable is flushed to disk. The default threshold is 1 GB.
@@ -224,18 +225,23 @@ enable_mem_control=true
 
 # Memory Allocation Ratio: Write, Read, Schema and Free Memory.
 # The parameter form is a:b:c:d, where a, b, c and d are integers. for example: 1:1:1:1 , 6:2:1:1
+# If you have high level of writing pressure and low level of reading pressure, please adjust it to for example 6:1:1:2 
 write_read_schema_free_memory_proportion=4:3:1:2
 
 # primitive array size (length of each array) in array pool
 primitive_array_size=128
 
 # Ratio of write memory for invoking flush disk, 0.4 by default
+# If you have extremely high level of writing pressure (like batch=1000) and the physical memory size is large enough, 
+# it can be set lower than the default value like 0.2
 flush_proportion=0.4
 
 # Ratio of write memory allocated for buffered arrays, 0.6 by default
 buffered_arrays_memory_proportion=0.6
 
 # Ratio of write memory for rejecting insertion, 0.8 by default
+# If you have extremely high level of writing pressure (like batch=1000) and the physical memory size is large enough, 
+# it can be set higher than the default value like 0.9
 reject_proportion=0.8
 
 # If memory (in byte) of storage group increased more than this threshold, report to system. The default value is 16MB
@@ -246,9 +252,13 @@ storage_group_report_threshold=16777216
 max_deduplicated_path_num=1000
 
 # When an inserting is rejected, waiting time (in ms) to check system again, 0 by default.
+# If you have high level of writing pressure and low level of reading pressure,
+# it can be set larger than 0, like 10. 
 waiting_time_when_insert_blocked=0
 
 # When the waiting time (in ms) of an inserting exceeds this, throw an exception. 0 by default.
+# If you have high level of writing pressure and low level of reading pressure,
+# it can be set larger than 0, like 10000. 
 max_waiting_time_when_insert_blocked=0
 
 # estimated metadata size (in byte) of one timeseries in Mtree


[iotdb] 01/04: change reject error log to warn

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

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

commit 5b7ea3d370cc3500446baabc55b94f7851aa4527
Author: HTHou <hh...@outlook.com>
AuthorDate: Tue Nov 24 22:58:50 2020 +0800

    change reject error log to warn
---
 .../engine/storagegroup/StorageGroupProcessor.java |  5 +++-
 .../db/engine/storagegroup/TsFileProcessor.java    | 19 +++++++-----
 .../db/exception/WriteProcessRejectException.java  | 35 ++++++++++++++++++++++
 .../org/apache/iotdb/db/rescon/SystemInfo.java     | 19 ++++++------
 4 files changed, 60 insertions(+), 18 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index 69aec66..ffa4002 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -68,6 +68,7 @@ import org.apache.iotdb.db.exception.LoadFileException;
 import org.apache.iotdb.db.exception.StorageGroupProcessorException;
 import org.apache.iotdb.db.exception.TsFileProcessorException;
 import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.exception.WriteProcessRejectException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.OutOfTTLException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
@@ -839,6 +840,9 @@ public class StorageGroupProcessor {
 
     try {
       tsFileProcessor.insertTablet(insertTabletPlan, start, end, results);
+    } catch (WriteProcessRejectException e) {
+      logger.warn("insert to TsFileProcessor rejected ", e);
+      return false;
     } catch (WriteProcessException e) {
       logger.error("insert to TsFileProcessor error ", e);
       return false;
@@ -891,7 +895,6 @@ public class StorageGroupProcessor {
       return;
     }
 
-    // insert TsFileProcessor
     tsFileProcessor.insert(insertRowPlan);
 
     // try to update the latest time of the device of this tsRecord
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index 4ed4d31..b6e217b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -48,6 +48,7 @@ import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.UpdateEndTimeCallBack;
 import org.apache.iotdb.db.engine.version.VersionController;
+import org.apache.iotdb.db.exception.WriteProcessRejectException;
 import org.apache.iotdb.db.exception.TsFileProcessorException;
 import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
@@ -171,7 +172,8 @@ public class TsFileProcessor {
    *
    * @param insertRowPlan physical plan of insertion
    */
-  public void insert(InsertRowPlan insertRowPlan) throws WriteProcessException {
+  public void insert(InsertRowPlan insertRowPlan) 
+      throws WriteProcessException, WriteProcessRejectException {
 
     if (workMemTable == null) {
       workMemTable = new PrimitiveMemTable(enableMemControl);
@@ -214,7 +216,7 @@ public class TsFileProcessor {
    * @param results result array
    */
   public void insertTablet(InsertTabletPlan insertTabletPlan, int start, int end,
-      TSStatus[] results) throws WriteProcessException {
+      TSStatus[] results) throws WriteProcessException, WriteProcessRejectException {
 
     if (workMemTable == null) {
       workMemTable = new PrimitiveMemTable(enableMemControl);
@@ -254,7 +256,8 @@ public class TsFileProcessor {
     tsFileResource.updatePlanIndexes(insertTabletPlan.getIndex());
   }
 
-  private void checkMemCostAndAddToTspInfo(InsertRowPlan insertRowPlan) throws WriteProcessException {
+  private void checkMemCostAndAddToTspInfo(InsertRowPlan insertRowPlan) 
+      throws WriteProcessException, WriteProcessRejectException {
     // memory of increased PrimitiveArray and TEXT values, e.g., add a long[128], add 128*8
     long memTableIncrement = 0L;
     long textDataIncrement = 0L;
@@ -292,7 +295,7 @@ public class TsFileProcessor {
       SystemInfo.getInstance().reportStorageGroupStatus(storageGroupInfo);
       try {
         blockInsertionIfReject();
-      } catch (WriteProcessException e) {
+      } catch (WriteProcessRejectException e) {
         storageGroupInfo.releaseStorageGroupMemCost(memTableIncrement);
         tsFileProcessorInfo.releaseTSPMemCost(unsealedResourceIncrement + chunkMetadataIncrement);
         SystemInfo.getInstance().resetStorageGroupStatus(storageGroupInfo, false);
@@ -304,7 +307,7 @@ public class TsFileProcessor {
   }
 
   private void checkMemCostAndAddToTspInfo(InsertTabletPlan insertTabletPlan, int start, int end)
-      throws WriteProcessException {
+      throws WriteProcessException, WriteProcessRejectException {
     if (start >= end) {
       return;
     }
@@ -356,7 +359,7 @@ public class TsFileProcessor {
       SystemInfo.getInstance().reportStorageGroupStatus(storageGroupInfo);
       try {
         blockInsertionIfReject();
-      } catch (WriteProcessException e) {
+      } catch (WriteProcessRejectException e) {
         storageGroupInfo.releaseStorageGroupMemCost(memTableIncrement);
         tsFileProcessorInfo.releaseTSPMemCost(unsealedResourceIncrement + chunkMetadataIncrement);
         SystemInfo.getInstance().resetStorageGroupStatus(storageGroupInfo, false);
@@ -367,13 +370,13 @@ public class TsFileProcessor {
     workMemTable.addTextDataSize(textDataIncrement);
   }
 
-  private void blockInsertionIfReject() throws WriteProcessException {
+  private void blockInsertionIfReject() throws WriteProcessRejectException {
     long startTime = System.currentTimeMillis();
     while (SystemInfo.getInstance().isRejected()) {
       try {
         TimeUnit.MILLISECONDS.sleep(waitingTimeWhenInsertBlocked);
         if (System.currentTimeMillis() - startTime > maxWaitingTimeWhenInsertBlocked) {
-          throw new WriteProcessException("System rejected over " + maxWaitingTimeWhenInsertBlocked + "ms");
+          throw new WriteProcessRejectException("System rejected over " + maxWaitingTimeWhenInsertBlocked + "ms");
         }
       } catch (InterruptedException e) {
         logger.error("Failed when waiting for getting memory for insertion ", e);
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/WriteProcessRejectException.java b/server/src/main/java/org/apache/iotdb/db/exception/WriteProcessRejectException.java
new file mode 100644
index 0000000..2609ab2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/WriteProcessRejectException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.exception;
+
+import org.apache.iotdb.rpc.TSStatusCode;
+
+public class WriteProcessRejectException extends WriteProcessException {
+
+  private static final long serialVersionUID = -4217324287547595610L;
+
+  public WriteProcessRejectException(String message) {
+    super(message, TSStatusCode.WRITE_PROCESS_ERROR.getStatusCode());
+  }
+
+  public WriteProcessRejectException(String message, int errorCode) {
+    super(message, errorCode);
+  }
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java b/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
index 74c00f0..e4f6c84 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
@@ -44,8 +44,10 @@ public class SystemInfo {
 
   private Map<StorageGroupInfo, Long> reportedSgMemCostMap = new ConcurrentHashMap<>();
 
-  private static final double FLUSH_PROPORTION = config.getFlushProportion();
-  private static final double REJECT_PROPORTION = config.getRejectProportion();
+  private static final double FLUSH_THERSHOLD =
+      config.getAllocateMemoryForWrite() * config.getFlushProportion();
+  private static final double REJECT_THERSHOLD = 
+      config.getAllocateMemoryForWrite() * config.getRejectProportion();
 
   /**
    * Report current mem cost of storage group to system. Called when the memory of
@@ -63,12 +65,12 @@ public class SystemInfo {
     }
     reportedSgMemCostMap.put(storageGroupInfo, storageGroupInfo.getMemCost());
     storageGroupInfo.setLastReportedSize(storageGroupInfo.getMemCost());
-    if (totalSgMemCost.get() >= config.getAllocateMemoryForWrite() * FLUSH_PROPORTION) {
+    if (totalSgMemCost.get() >= FLUSH_THERSHOLD) {
       logger.debug("The total storage group mem costs are too large, call for flushing. "
           + "Current sg cost is {}", totalSgMemCost);
       chooseTSPToMarkFlush();
     }
-    if (totalSgMemCost.get() >= config.getAllocateMemoryForWrite() * REJECT_PROPORTION) {
+    if (totalSgMemCost.get() >= REJECT_THERSHOLD) {
       logger.info("Change system to reject status...");
       rejected = true;
     }
@@ -94,8 +96,8 @@ public class SystemInfo {
   }
 
   private void checkSystemToInvokeFlush() {
-    if (totalSgMemCost.get() >= config.getAllocateMemoryForWrite() * FLUSH_PROPORTION &&
-        totalSgMemCost.get() < config.getAllocateMemoryForWrite() * REJECT_PROPORTION) {
+    if (totalSgMemCost.get() >= FLUSH_THERSHOLD &&
+        totalSgMemCost.get() < REJECT_THERSHOLD) {
       logger.debug("Some sg memory released but still exceeding flush proportion, call flush.");
       if (rejected) {
         logger.info("Some sg memory released, set system to normal status.");
@@ -104,7 +106,7 @@ public class SystemInfo {
       rejected = false;
       forceAsyncFlush();
     }
-    else if (totalSgMemCost.get() >= config.getAllocateMemoryForWrite() * REJECT_PROPORTION) {
+    else if (totalSgMemCost.get() >= REJECT_THERSHOLD) {
       logger.warn("Some sg memory released, but system is still in reject status.");
       logCurrentTotalSGMemory();
       rejected = true;
@@ -169,8 +171,7 @@ public class SystemInfo {
     }
     List<TsFileProcessor> processors = new ArrayList<>();
     long memCost = 0;
-    while (totalSgMemCost.get() - memCost > config.getAllocateMemoryForWrite() *
-        FLUSH_PROPORTION / 2) {
+    while (totalSgMemCost.get() - memCost > FLUSH_THERSHOLD / 2) {
       if (tsps.isEmpty() || tsps.peek().getWorkMemTableRamCost() == 0) {
         return processors;
       }


[iotdb] 03/04: use synchronized to fix mem control concurrent issue

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

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

commit 96013e7566ce98b8de164494349295be71ffb54a
Author: HTHou <hh...@outlook.com>
AuthorDate: Wed Nov 25 16:42:25 2020 +0800

    use synchronized to fix mem control concurrent issue
---
 .../org/apache/iotdb/db/rescon/SystemInfo.java     | 30 ++++++++++------------
 1 file changed, 14 insertions(+), 16 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java b/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
index e4f6c84..593e0ce 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
@@ -20,11 +20,10 @@
 package org.apache.iotdb.db.rescon;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.PriorityQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -39,10 +38,10 @@ public class SystemInfo {
   private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
   private static final Logger logger = LoggerFactory.getLogger(SystemInfo.class);
 
-  private AtomicLong totalSgMemCost = new AtomicLong();
+  private long totalSgMemCost = 0L;
   private volatile boolean rejected = false;
 
-  private Map<StorageGroupInfo, Long> reportedSgMemCostMap = new ConcurrentHashMap<>();
+  private Map<StorageGroupInfo, Long> reportedSgMemCostMap = new HashMap<>();
 
   private static final double FLUSH_THERSHOLD =
       config.getAllocateMemoryForWrite() * config.getFlushProportion();
@@ -55,22 +54,22 @@ public class SystemInfo {
    *
    * @param storageGroupInfo storage group
    */
-  public void reportStorageGroupStatus(StorageGroupInfo storageGroupInfo) {
+  public synchronized void reportStorageGroupStatus(StorageGroupInfo storageGroupInfo) {
     long delta = storageGroupInfo.getMemCost() -
         reportedSgMemCostMap.getOrDefault(storageGroupInfo, 0L);
-    totalSgMemCost.addAndGet(delta);
+    totalSgMemCost += delta;
     if (logger.isDebugEnabled()) {
       logger.debug("Report Storage Group Status to the system. "
           + "After adding {}, current sg mem cost is {}.", delta, totalSgMemCost);
     }
     reportedSgMemCostMap.put(storageGroupInfo, storageGroupInfo.getMemCost());
     storageGroupInfo.setLastReportedSize(storageGroupInfo.getMemCost());
-    if (totalSgMemCost.get() >= FLUSH_THERSHOLD) {
+    if (totalSgMemCost >= FLUSH_THERSHOLD) {
       logger.debug("The total storage group mem costs are too large, call for flushing. "
           + "Current sg cost is {}", totalSgMemCost);
       chooseTSPToMarkFlush();
     }
-    if (totalSgMemCost.get() >= REJECT_THERSHOLD) {
+    if (totalSgMemCost >= REJECT_THERSHOLD) {
       logger.info("Change system to reject status...");
       rejected = true;
     }
@@ -82,11 +81,11 @@ public class SystemInfo {
    *
    * @param storageGroupInfo storage group
    */
-  public void resetStorageGroupStatus(StorageGroupInfo storageGroupInfo,
+  public synchronized void resetStorageGroupStatus(StorageGroupInfo storageGroupInfo,
       boolean shouldInvokeFlush) {
     if (reportedSgMemCostMap.containsKey(storageGroupInfo)) {
-      this.totalSgMemCost.addAndGet(storageGroupInfo.getMemCost() -
-          reportedSgMemCostMap.get(storageGroupInfo));
+      this.totalSgMemCost -= (reportedSgMemCostMap.get(storageGroupInfo) -
+          storageGroupInfo.getMemCost());
       storageGroupInfo.setLastReportedSize(storageGroupInfo.getMemCost());
       reportedSgMemCostMap.put(storageGroupInfo, storageGroupInfo.getMemCost());
       if (shouldInvokeFlush) {
@@ -96,8 +95,7 @@ public class SystemInfo {
   }
 
   private void checkSystemToInvokeFlush() {
-    if (totalSgMemCost.get() >= FLUSH_THERSHOLD &&
-        totalSgMemCost.get() < REJECT_THERSHOLD) {
+    if (totalSgMemCost >= FLUSH_THERSHOLD && totalSgMemCost < REJECT_THERSHOLD) {
       logger.debug("Some sg memory released but still exceeding flush proportion, call flush.");
       if (rejected) {
         logger.info("Some sg memory released, set system to normal status.");
@@ -106,7 +104,7 @@ public class SystemInfo {
       rejected = false;
       forceAsyncFlush();
     }
-    else if (totalSgMemCost.get() >= REJECT_THERSHOLD) {
+    else if (totalSgMemCost >= REJECT_THERSHOLD) {
       logger.warn("Some sg memory released, but system is still in reject status.");
       logCurrentTotalSGMemory();
       rejected = true;
@@ -171,7 +169,7 @@ public class SystemInfo {
     }
     List<TsFileProcessor> processors = new ArrayList<>();
     long memCost = 0;
-    while (totalSgMemCost.get() - memCost > FLUSH_THERSHOLD / 2) {
+    while (totalSgMemCost - memCost > FLUSH_THERSHOLD / 2) {
       if (tsps.isEmpty() || tsps.peek().getWorkMemTableRamCost() == 0) {
         return processors;
       }
@@ -188,7 +186,7 @@ public class SystemInfo {
 
   public void close() {
     reportedSgMemCostMap.clear();
-    totalSgMemCost.set(0);
+    totalSgMemCost = 0;
     rejected = false;
   }
 


[iotdb] 04/04: fix review

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

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

commit 313eb0c0e126f93dd39eac7893998642545abbb8
Author: HTHou <hh...@outlook.com>
AuthorDate: Thu Nov 26 23:52:59 2020 +0800

    fix review
---
 server/src/assembly/resources/conf/iotdb-engine.properties  | 13 +++++--------
 .../iotdb/db/engine/storagegroup/TsFileProcessor.java       |  9 ++++-----
 2 files changed, 9 insertions(+), 13 deletions(-)

diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index fe5f7bb..af7938c 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -174,7 +174,7 @@ timestamp_precision=ms
 wal_buffer_size=16777216
 
 # When a TsFile's file size (in byte) exceeds this, the TsFile is forced closed.
-# It may cause memTable size smaller, if it sets a large value
+# It may cause memTable size smaller if it is a large value
 tsfile_size_threshold=1
 
 # When a memTable's size (in byte) exceeds this, the memtable is flushed to disk. The default threshold is 1 GB.
@@ -232,15 +232,14 @@ write_read_schema_free_memory_proportion=4:3:1:2
 primitive_array_size=128
 
 # Ratio of write memory for invoking flush disk, 0.4 by default
-# If you have extremely high level of writing pressure (like batch=1000) and the physical memory size is large enough, 
-# it can be set lower than the default value like 0.2
+# If you have extremely high write load (like batch=1000), it can be set lower than the default value like 0.2
 flush_proportion=0.4
 
 # Ratio of write memory allocated for buffered arrays, 0.6 by default
 buffered_arrays_memory_proportion=0.6
 
 # Ratio of write memory for rejecting insertion, 0.8 by default
-# If you have extremely high level of writing pressure (like batch=1000) and the physical memory size is large enough, 
+# If you have extremely high write load (like batch=1000) and the physical memory size is large enough, 
 # it can be set higher than the default value like 0.9
 reject_proportion=0.8
 
@@ -252,13 +251,11 @@ storage_group_report_threshold=16777216
 max_deduplicated_path_num=1000
 
 # When an inserting is rejected, waiting time (in ms) to check system again, 0 by default.
-# If you have high level of writing pressure and low level of reading pressure,
-# it can be set larger than 0, like 10. 
+# If the insertion has been rejected and the read load is low, it can be set larger than 0, like 10. 
 waiting_time_when_insert_blocked=0
 
 # When the waiting time (in ms) of an inserting exceeds this, throw an exception. 0 by default.
-# If you have high level of writing pressure and low level of reading pressure,
-# it can be set larger than 0, like 10000. 
+# If the insertion has been rejected and the read load is low, it can be set larger than 0, like 10000. 
 max_waiting_time_when_insert_blocked=0
 
 # estimated metadata size (in byte) of one timeseries in Mtree
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index b6e217b..169ed5c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -172,8 +172,7 @@ public class TsFileProcessor {
    *
    * @param insertRowPlan physical plan of insertion
    */
-  public void insert(InsertRowPlan insertRowPlan) 
-      throws WriteProcessException, WriteProcessRejectException {
+  public void insert(InsertRowPlan insertRowPlan) throws WriteProcessException {
 
     if (workMemTable == null) {
       workMemTable = new PrimitiveMemTable(enableMemControl);
@@ -216,7 +215,7 @@ public class TsFileProcessor {
    * @param results result array
    */
   public void insertTablet(InsertTabletPlan insertTabletPlan, int start, int end,
-      TSStatus[] results) throws WriteProcessException, WriteProcessRejectException {
+      TSStatus[] results) throws WriteProcessException {
 
     if (workMemTable == null) {
       workMemTable = new PrimitiveMemTable(enableMemControl);
@@ -257,7 +256,7 @@ public class TsFileProcessor {
   }
 
   private void checkMemCostAndAddToTspInfo(InsertRowPlan insertRowPlan) 
-      throws WriteProcessException, WriteProcessRejectException {
+      throws WriteProcessException {
     // memory of increased PrimitiveArray and TEXT values, e.g., add a long[128], add 128*8
     long memTableIncrement = 0L;
     long textDataIncrement = 0L;
@@ -307,7 +306,7 @@ public class TsFileProcessor {
   }
 
   private void checkMemCostAndAddToTspInfo(InsertTabletPlan insertTabletPlan, int start, int end)
-      throws WriteProcessException, WriteProcessRejectException {
+      throws WriteProcessException {
     if (start >= end) {
       return;
     }