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 2020/11/03 15:37:19 UTC

[iotdb] branch master updated: [IOTDB-776] Control the memory usage of flushing the memtable (#1524)

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

qiaojialin 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 04d3ac0  [IOTDB-776] Control the memory usage of flushing the memtable (#1524)
04d3ac0 is described below

commit 04d3ac0ba0110aa7027396036ba5d9780ecd1412
Author: Haonan <hh...@outlook.com>
AuthorDate: Tue Nov 3 23:37:01 2020 +0800

    [IOTDB-776] Control the memory usage of flushing the memtable (#1524)
    
    * add TsFileProcessorInfo
    * Add interface for System module and ArrayManager
    * Add timedCollectSchemaDataTypeNumThread for collect data type num
    * Add SystemInfo
    * Add PrimitiveArrayManagement buffer replacement strategy
    * remove MemTablePool
    * Update server/src/assembly/resources/conf/iotdb-engine.properties
    * [IOTDB-834] Remove DynamicAdapter
    * don't get array from array pool for sorting and fix flush small memtable when closing tsfile
    * [IOTDB-961] Remove ActiveTimeSeriesCounter
    Co-authored-by: samperson1997 <sz...@mails.tsinghua.edu.cn>
    Co-authored-by: Xiangwei Wei <52...@qq.com>
    Co-authored-by: Xiangwei Wei <34...@users.noreply.github.com>
    Co-authored-by: qiaojialin <64...@qq.com>
    Co-authored-by: EJTTianyu <16...@qq.com>
---
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |  14 -
 docs/UserGuide/Server/Config Manual.md             |  10 +-
 .../System Tools/Memory Estimation Tool.md         |  84 ------
 docs/zh/UserGuide/Server/Config Manual.md          |  19 +-
 .../System Tools/Memory Estimation Tool.md         |  88 ------
 .../resources/conf/iotdb-engine.properties         |  49 ++--
 .../src/assembly/resources/tools/memory-tool.bat   |  65 -----
 server/src/assembly/resources/tools/memory-tool.sh |  47 ---
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 131 +++++++--
 .../org/apache/iotdb/db/conf/IoTDBConstant.java    |   1 -
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  47 ++-
 .../db/conf/adapter/ActiveTimeSeriesCounter.java   | 159 ----------
 .../iotdb/db/conf/adapter/CompressionRatio.java    |  17 --
 .../db/conf/adapter/IActiveTimeSeriesCounter.java  |  61 ----
 .../iotdb/db/conf/adapter/IDynamicAdapter.java     |  47 ---
 .../db/conf/adapter/IoTDBConfigDynamicAdapter.java | 300 -------------------
 .../db/conf/adapter/ManageDynamicParameters.java   | 135 ---------
 .../conf/adapter/ManageDynamicParametersMBean.java |  46 ---
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   6 +-
 .../iotdb/db/engine/cache/ChunkMetadataCache.java  |   2 -
 .../iotdb/db/engine/flush/MemTableFlushTask.java   |  13 +-
 .../iotdb/db/engine/memtable/AbstractMemTable.java |  53 +++-
 .../apache/iotdb/db/engine/memtable/IMemTable.java |  29 +-
 .../db/engine/memtable/IWritableMemChunk.java      |  13 +-
 .../db/engine/memtable/PrimitiveMemTable.java      |   3 +
 .../iotdb/db/engine/memtable/WritableMemChunk.java |  31 --
 .../db/engine/storagegroup/StorageGroupInfo.java   | 105 +++++++
 .../engine/storagegroup/StorageGroupProcessor.java |  66 ++++-
 .../db/engine/storagegroup/TsFileProcessor.java    | 257 +++++++++++++---
 .../engine/storagegroup/TsFileProcessorInfo.java   |  65 +++++
 .../db/engine/storagegroup/TsFileResource.java     |  31 ++
 .../db/exception/ConfigAdjusterException.java      |  31 --
 .../org/apache/iotdb/db/metadata/MManager.java     | 217 +++++++-------
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |   3 -
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  51 ----
 .../apache/iotdb/db/qp/physical/sys/ShowPlan.java  |   2 +-
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |   6 -
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |   2 -
 .../org/apache/iotdb/db/rescon/MemTablePool.java   | 114 --------
 .../iotdb/db/rescon/PrimitiveArrayManager.java     | 322 +++++++++++++++++++++
 .../apache/iotdb/db/rescon/PrimitiveArrayPool.java | 166 -----------
 .../org/apache/iotdb/db/rescon/SystemInfo.java     | 185 ++++++++++++
 .../apache/iotdb/db/rescon/TVListAllocator.java    |   4 +-
 .../java/org/apache/iotdb/db/service/IoTDB.java    |  11 +-
 .../org/apache/iotdb/db/service/ServiceType.java   |   1 -
 .../org/apache/iotdb/db/service/StaticResps.java   |   5 -
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   2 -
 .../iotdb/db/tools/memestimation/MemEstTool.java   |  92 ------
 .../db/tools/memestimation/MemEstToolCmd.java      | 120 --------
 .../java/org/apache/iotdb/db/utils/MemUtils.java   |  29 +-
 .../iotdb/db/utils/datastructure/BinaryTVList.java |  52 +---
 .../db/utils/datastructure/BooleanTVList.java      |  53 +---
 .../iotdb/db/utils/datastructure/DoubleTVList.java |  53 +---
 .../iotdb/db/utils/datastructure/FloatTVList.java  |  52 +---
 .../iotdb/db/utils/datastructure/IntTVList.java    |  52 +---
 .../iotdb/db/utils/datastructure/LongTVList.java   |  52 +---
 .../iotdb/db/utils/datastructure/TVList.java       |  64 ++--
 .../db/writelog/node/ExclusiveWriteLogNode.java    |   3 +-
 .../conf/adapter/ActiveTimeSeriesCounterTest.java  | 125 --------
 .../db/conf/adapter/CompressionRatioTest.java      |   1 -
 .../iotdb/db/conf/adapter/HyperLogLogTest.java     |  65 -----
 .../adapter/IoTDBConfigDynamicAdapterTest.java     | 138 ---------
 .../db/engine/cache/ChunkMetadataCacheTest.java    |   2 -
 .../db/engine/memtable/MemTableFlushTaskTest.java  |   2 -
 .../iotdb/db/engine/memtable/MemTablePoolTest.java |  82 ------
 .../db/engine/memtable/MemtableBenchmark.java      |   3 +-
 .../db/engine/memtable/PrimitiveMemTableTest.java  |   1 +
 .../engine/modification/DeletionFileNodeTest.java  |   2 -
 .../db/engine/modification/DeletionQueryTest.java  |   2 -
 .../engine/modification/ModificationFileTest.java  |   2 -
 .../storagegroup/StorageGroupProcessorTest.java    |  14 +-
 .../engine/storagegroup/TsFileProcessorTest.java   |  37 ++-
 .../iotdb/db/metadata/MManagerBasicTest.java       |  26 --
 .../iotdb/db/qp/plan/LogicalPlanSmallTest.java     |   3 +-
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  |   8 -
 .../iotdb/db/query/reader/ReaderTestHelper.java    |  69 -----
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |  19 +-
 .../db/utils/datastructure/LongTVListTest.java     |  18 +-
 .../recover/RecoverResourceFromReaderTest.java     |   3 -
 .../db/writelog/recover/SeqTsFileRecoverTest.java  |   3 -
 .../writelog/recover/UnseqTsFileRecoverTest.java   |   3 -
 .../iotdb/session/IoTDBSessionComplexIT.java       |   2 +-
 site/src/main/.vuepress/config.js                  |   2 -
 .../apache/iotdb/spark/db/EnvironmentUtils.java    |   8 -
 .../iotdb/tsfile/file/metadata/ChunkMetadata.java  |   5 +
 .../tsfile/file/metadata/enums/TSDataType.java     |  18 ++
 .../file/metadata/statistics/BinaryStatistics.java |   4 +-
 .../metadata/statistics/BooleanStatistics.java     |   2 +-
 .../file/metadata/statistics/DoubleStatistics.java |   2 +-
 .../file/metadata/statistics/FloatStatistics.java  |   2 +-
 .../metadata/statistics/IntegerStatistics.java     |   2 +-
 .../file/metadata/statistics/LongStatistics.java   |   2 +-
 .../file/metadata/statistics/Statistics.java       |  19 ++
 93 files changed, 1579 insertions(+), 2855 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
index ef27c2c..ec29bb5 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
@@ -73,7 +73,6 @@ statement
     | SHOW TTL ON prefixPath (COMMA prefixPath)* #showTTLStatement
     | SHOW ALL TTL #showAllTTLStatement
     | SHOW FLUSH TASK INFO #showFlushTaskInfo
-    | SHOW DYNAMIC PARAMETER #showDynamicParameter
     | SHOW VERSION #showVersion
     | SHOW LATEST? TIMESERIES prefixPath? showWhereClause? limitClause? #showTimeseries
     | SHOW STORAGE GROUP prefixPath? #showStorageGroup
@@ -473,8 +472,6 @@ nodeName
     | FLUSH
     | TASK
     | INFO
-    | DYNAMIC
-    | PARAMETER
     | VERSION
     | REMOVE
     | MOVE
@@ -583,8 +580,6 @@ nodeNameWithoutStar
     | FLUSH
     | TASK
     | INFO
-    | DYNAMIC
-    | PARAMETER
     | VERSION
     | REMOVE
     | MOVE
@@ -996,15 +991,6 @@ INFO
     : I N F O
     ;
 
-DYNAMIC
-    : D Y N A M I C
-    ;
-
-PARAMETER
-    : P A R A M E T E R
-    ;
-
-
 VERSION
     : V E R S I O N
     ;
diff --git a/docs/UserGuide/Server/Config Manual.md b/docs/UserGuide/Server/Config Manual.md
index 34f1886..4915ae2 100644
--- a/docs/UserGuide/Server/Config Manual.md	
+++ b/docs/UserGuide/Server/Config Manual.md	
@@ -272,11 +272,11 @@ The permission definitions are in ${IOTDB\_CONF}/conf/jmx.access.
 |Default| true |
 |Effective|Trigger|
 
-* enable\_parameter\_adapter
+* enable\_mem\_control
 
-|Name| enable\_parameter\_adapter |
+|Name| enable\_mem\_control |
 |:---:|:---|
-|Description| enable dynamically adjusting system to avoid OOM|
+|Description| enable memory control to avoid OOM|
 |Type|Bool|
 |Default| true |
 |Effective|After restart system|
@@ -288,7 +288,7 @@ The permission definitions are in ${IOTDB\_CONF}/conf/jmx.access.
 |Description| max memtable size|
 |Type|Long|
 |Default| 1073741824 |
-|Effective| when enable\_parameter\_adapter is false & After restart system|
+|Effective| when enable\_mem\_control is false & After restart system|
 
 * avg\_series\_point\_number\_threshold
 
@@ -306,7 +306,7 @@ The permission definitions are in ${IOTDB\_CONF}/conf/jmx.access.
 |Description| max tsfile size|
 |Type|Long|
 |Default| 536870912 |
-|Effective| when enable\_parameter\_adapter is false & After restart system|
+|Effective| After restart system|
 
 * enable\_partition
 
diff --git a/docs/UserGuide/System Tools/Memory Estimation Tool.md b/docs/UserGuide/System Tools/Memory Estimation Tool.md
deleted file mode 100644
index 99aaf38..0000000
--- a/docs/UserGuide/System Tools/Memory Estimation Tool.md	
+++ /dev/null
@@ -1,84 +0,0 @@
-<!--
-
-    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.
-
--->
-
-# Memory Estimation Tool
-
-## Introduction
-
-This tool calculates the minimum memory for writing to meet specific workload through a number of parameters input by users. (Memory in IoTDB is divided into three parts: write memory, read memory and reserve memory. Write memory is used for data write allocation. The ratio of the three can be set in the configuration file) The unit of result is in GB.
-
-## Input parameters
-
-When using this tool, the parameters needed to be input are as follows:
-
-<table>
-   <tr>
-      <td>Parameter</td>
-      <td>Parameter Description</td>
-      <td>Example</td>
-      <td>Necessary</td>
-   </tr>
-   <tr>
-      <td>-sg | --storagegroup &lt;storage group number&gt;</td>
-      <td>storage group number</td>
-      <td>-sg 20</td>
-      <td>true</td>
-   </tr>
-   <tr>
-      <td>-ts | --timeseries &lt;total timeseries number&gt;</td>
-      <td>total timeseries number</td>
-      <td>-ts 10000</td>
-      <td>true</td>
-   </tr>
-   <tr>
-      <td>-mts | --maxtimeseries &lt;max timeseries&gt;</td>
-      <td>maximum number of timeseries among storage groups.If the time series are evenly distributed in the storage group, this parameter may not be set.</td>
-      <td>-mts 10000</td>
-      <td>false</td>
-   </tr>
-
-</table>
-
-In memory estimation, if the calculation takes a long time, the tool will show the running progress below, which is convenient for users to master the progress.
-
-## Usage
-
-Users can use the tool using scripts under the `IOTDB_HOME/bin`folder.
-For Linux and Mac OS X users:
-* Assume that there are 20 storage groups, 100k timeseries and timeseries are evenly distributed in the storage groups:
-```
-  Shell >$IOTDB_HOME/bin/memory-tool.sh calmem -sg 20 -ts 100000
-```
-* Assume that there are 20 storage groups, 100k timeseries and maximum timeseries number among storage groups is 50000:
-```
-  Shell >$IOTDB_HOME/bin/memory-tool.sh calmem -sg 20 -ts 100000 -mts -50000
-```
-
-For Windows users:
-* Assume that there are 20 storage groups, 100k timeseries and timeseries are evenly distributed in the storage groups:
-```
-  Shell >$IOTDB_HOME\bin\memory-tool.bat calmem -sg 20 -ts 100000
-```
-* Assume that there are 20 storage groups, 100k timeseries and maximum timeseries number among storage groups is 50000:
-```
-  Shell >$IOTDB_HOME\bin\memory-tool.bat calmem -sg 20 -ts 100000 -mts -50000
-```
-
diff --git a/docs/zh/UserGuide/Server/Config Manual.md b/docs/zh/UserGuide/Server/Config Manual.md
index a4067e5..c192658 100644
--- a/docs/zh/UserGuide/Server/Config Manual.md	
+++ b/docs/zh/UserGuide/Server/Config Manual.md	
@@ -271,11 +271,20 @@
 |默认值| 0 |
 |改后生效方式|重启服务器生效|
 
-* enable\_parameter\_adapter
+* enable\_mem\_comtrol
 
-|Name| enable\_parameter\_adapter |
+|Name| enable\_mem\_control |
 |:---:|:---|
-|Description| 开启自动调整系统参数,避免爆内存|
+|Description| 开启内存控制,避免爆内存|
+|Type|Bool|
+|Default| true |
+|Effective|重启服务器生效|
+
+* enable\_mem\_comtrol
+
+|Name| enable\_mem\_control |
+|:---:|:---|
+|Description| 开启内存控制,避免爆内存|
 |Type|Bool|
 |Default| true |
 |Effective|重启服务器生效|
@@ -287,7 +296,7 @@
 |Description| 内存缓冲区 memtable 阈值|
 |Type|Long|
 |Default| 1073741824 |
-|Effective|enable\_parameter\_adapter为false时生效、重启服务器生效|
+|Effective|enable\_mem\_control为false时生效、重启服务器生效|
 
 * avg\_series\_point\_number\_threshold
 
@@ -305,7 +314,7 @@
 |Description| 每个 tsfile 大小|
 |Type|Long|
 |Default| 536870912 |
-|Effective|enable\_parameter\_adapter为false时生效、重启服务器生效|
+|Effective| 重启服务器生效|
 
 * enable\_partition
 
diff --git a/docs/zh/UserGuide/System Tools/Memory Estimation Tool.md b/docs/zh/UserGuide/System Tools/Memory Estimation Tool.md
deleted file mode 100644
index 3e64f07..0000000
--- a/docs/zh/UserGuide/System Tools/Memory Estimation Tool.md	
+++ /dev/null
@@ -1,88 +0,0 @@
-<!--
-
-    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.
-
--->
-
-# 内存预估
-<!-- TOC -->
-
-- [内存预估工具](#内存预估工具)
-- [介绍](#介绍)
-- [输入参数](#输入参数)
-- [使用方式](#使用方式)
-
-<!-- /TOC -->
-
-# 介绍
-本工具通过用户输入的若干参数,计算出IoTDB运行此负载的最小写内存。(IoTDB中的内存分为三部分:写内存,读内存,预留内存。写内存是用于数据写入分配的内存,三者的比例可在配置文件中设置),结果以GB为单位。
-
-# 输入参数
-本工具使用时,需要输入的参数如下:
-<table>
-   <tr>
-      <td>参数名</td>
-      <td>参数说明</td>
-      <td>示例</td>
-      <td>是否必需</td>
-   </tr>
-   <tr>
-      <td>-sg | --storagegroup &lt;storage group number&gt;</td>
-      <td>存储组数量</td>
-      <td>-sg 20</td>
-      <td>是</td>
-   </tr>
-   <tr>
-      <td>-ts | --timeseries &lt;total timeseries number&gt;</td>
-      <td>总时间序列数量</td>
-      <td>-ts 10000</td>
-      <td>是</td>
-   </tr>
-   <tr>
-      <td>-mts | --maxtimeseries &lt;max timeseries&gt;</td>
-      <td>存储组中的最大时间序列的数量,如果时间序列均匀分配在存储组中,本参数可以不设置</td>
-      <td>-mts 10000</td>
-      <td>否</td>
-   </tr>
-</table>
-
-在内存预估时,若工具计算需要较长的时间,则会在下方显示出运行进度,便于用户掌握进度。
-
-# 使用方式
-
-用户可以使用```$IOTDB_HOME/bin```文件夹下的脚本使用该工具
-Linux系统与MacOS系统启动命令如下:
-* 以20个存储组,共10w条时间序列,时间序列在存储组中均分为例:
-```
-  Shell >$IOTDB_HOME/bin/memory-tool.sh calmem -sg 20 -ts 100000
-```
-* 以20个存储组,共10w条时间序列,存储组中最大时间序列数为50000为例:
-```
-  Shell >$IOTDB_HOME/bin/memory-tool.sh calmem -sg 20 -ts 100000 -mts -50000
-```
-
-Windows系统启动命令如下:
-* 以20个存储组,共10w条时间序列,时间序列在存储组中均分为例:
-```
-  Shell >$IOTDB_HOME\bin\memory-tool.bat calmem -sg 20 -ts 100000
-```
-* 以20个存储组,共10w条时间序列,存储组中最大时间序列数为50000为例:
-```
-  Shell >$IOTDB_HOME\bin\memory-tool.bat calmem -sg 20 -ts 100000 -mts -50000
-```
-
diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index fab0fa1..681475a 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -173,13 +173,14 @@ timestamp_precision=ms
 # If it sets a value smaller than 0, use the default value 16777216
 wal_buffer_size=16777216
 
-# When a TsFile's file size (in byte) exceeds this, the TsFile is forced closed. The default threshold is 512 MB.
-tsfile_size_threshold=536870912
+# When a TsFile's file size (in byte) exceeds this, the TsFile is forced closed. The default threshold is 0.
+tsfile_size_threshold=0
 
 # When a memTable's size (in byte) exceeds this, the memtable is flushed to disk. The default threshold is 1 GB.
 memtable_size_threshold=1073741824
 
-avg_series_point_number_threshold=100000
+# When the average point number of timeseries in memtable exceeds this, the memtable is flushed to disk. The default threshold is 10000.
+avg_series_point_number_threshold=10000
 
 # How many threads can concurrently flush. When <= 0, use CPU core number.
 concurrent_flush_thread=0
@@ -218,28 +219,41 @@ mtree_snapshot_threshold_time=3600
 ### Memory Control Configuration
 ####################
 
-# This adapter could adjust the system memory to avoid OOM.
-# It will refuse to create time series or add storage groups under high system load.
-#
-# Attention!!!
-# If disable this parameter, you need to set some parameters according to your system load:
-# Normal scenario: 1-50 storage groups. <100k devices, <10M time series
-# memtable_size_threshold = tsfile_size_threshold = IoTDB memory allocation in byte / 2 / num of storage group / 4
-# Make sure the total num of time series in system * primitive_array_size * 16 <= IoTDB memory allocation in byte / 2 / 4
-# you could reduce the primitive_array_size in very high workload.
-enable_parameter_adapter=true
-
-# Memory Allocation Ratio: Write, Read, and Free Memory.
-# The parameter form is a:b:c, where a, b and c are integers. for example: 1:1:1 , 6:3:1
-write_read_free_memory_proportion=6:3:1
+# Whether to enable memory control
+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
+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.3 by default
+flush_proportion=0.3
+
+# 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
+reject_proportion=0.8
+
+# If memory (in byte) of storage group increased more than this threshold, report to system. The default value is 16MB
+storage_group_report_threshold=16777216
+
 # allowed max numbers of deduplicated path in one query
 # it's just an advised value, the real limitation will be the smaller one between this and the one we calculated
 max_deduplicated_path_num=1000
 
+# When an inserting is rejected, waiting time (in ms) to check system again.
+waiting_time_when_insert_blocked=10
+
+# When the waiting time of an inserting exceeds this, throw an exception.
+max_waiting_time_when_insert_blocked=10000
+
+# estimated metadata size (in byte) of one timeseries in Mtree
+estimated_series_size=300
+
 ####################
 ### Upgrade Configurations
 ####################
@@ -565,4 +579,3 @@ authorizer_provider_class=org.apache.iotdb.db.auth.authorizer.LocalFileAuthorize
 #If OpenIdAuthorizer is enabled, then openID_url must be set.
 
 #openID_url=
-
diff --git a/server/src/assembly/resources/tools/memory-tool.bat b/server/src/assembly/resources/tools/memory-tool.bat
deleted file mode 100755
index ff6aa7d..0000000
--- a/server/src/assembly/resources/tools/memory-tool.bat
+++ /dev/null
@@ -1,65 +0,0 @@
-@REM
-@REM Licensed to the Apache Software Foundation (ASF) under one
-@REM or more contributor license agreements.  See the NOTICE file
-@REM distributed with this work for additional information
-@REM regarding copyright ownership.  The ASF licenses this file
-@REM to you under the Apache License, Version 2.0 (the
-@REM "License"); you may not use this file except in compliance
-@REM with the License.  You may obtain a copy of the License at
-@REM
-@REM     http://www.apache.org/licenses/LICENSE-2.0
-@REM
-@REM Unless required by applicable law or agreed to in writing,
-@REM software distributed under the License is distributed on an
-@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-@REM KIND, either express or implied.  See the License for the
-@REM specific language governing permissions and limitations
-@REM under the License.
-@REM
-
-
-@echo off
-if "%OS%" == "Windows_NT" setlocal
-
-pushd %~dp0..
-if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%CD%
-popd
-
-set IOTDB_CONF=%IOTDB_HOME%\conf
-
-if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.db.tools.memestimation.MemEstTool
-if NOT DEFINED JAVA_HOME goto :err
-
-@REM -----------------------------------------------------------------------------
-@REM JVM Opts we'll use in legacy run or installation
-set JAVA_OPTS=-ea^
- -Dlogback.configurationFile="%IOTDB_CONF%\logback-tool.xml"^
- -DIOTDB_HOME=%IOTDB_HOME%
-
-@REM ***** CLASSPATH library setting *****
-@REM Ensure that any user defined CLASSPATH variables are not used on startup
-set CLASSPATH="%IOTDB_HOME%\lib\*"
-
-goto okClasspath
-
-:append
-set CLASSPATH=%CLASSPATH%;%1
-goto :eof
-
-@REM -----------------------------------------------------------------------------
-:okClasspath
-
-"%JAVA_HOME%\bin\java" %JAVA_OPTS% %JAVA_OPTS% -cp "%CLASSPATH%" %MAIN_CLASS% %*
-
-goto finally
-
-
-:err
-echo JAVA_HOME environment variable must be set!
-pause
-
-
-@REM -----------------------------------------------------------------------------
-:finally
-
-ENDLOCAL
\ No newline at end of file
diff --git a/server/src/assembly/resources/tools/memory-tool.sh b/server/src/assembly/resources/tools/memory-tool.sh
deleted file mode 100755
index 52d7578..0000000
--- a/server/src/assembly/resources/tools/memory-tool.sh
+++ /dev/null
@@ -1,47 +0,0 @@
-#!/bin/sh
-#
-# 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.
-#
-
-if [ -z "${IOTDB_HOME}" ]; then
-  export IOTDB_HOME="$(cd "`dirname "$0"`"/..; pwd)"
-fi
-
-IOTDB_CONF=${IOTDB_HOME}/conf
-
-CLASSPATH=""
-for f in ${IOTDB_HOME}/lib/*.jar; do
-  CLASSPATH=${CLASSPATH}":"$f
-done
-
-MAIN_CLASS=org.apache.iotdb.db.tools.memestimation.MemEstTool
-
-if [ -n "$JAVA_HOME" ]; then
-    for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
-        if [ -x "$java" ]; then
-            JAVA="$java"
-            break
-        fi
-    done
-else
-    JAVA=java
-fi
-
-iotdb_parms="-Dlogback.configurationFile=${IOTDB_CONF}/logback-tool.xml"
-
-exec "$JAVA" $iotdb_parms -cp "$CLASSPATH" "$MAIN_CLASS" "$@"
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 97957fe..671dab6 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -120,16 +120,21 @@ public class IoTDBConfig {
   private int rpcMaxConcurrentClientNum = 65535;
 
   /**
-   * Memory allocated for the read process
+   * Memory allocated for the write process
    */
-  private long allocateMemoryForWrite = Runtime.getRuntime().maxMemory() * 6 / 10;
+  private long allocateMemoryForWrite = Runtime.getRuntime().maxMemory() * 4 / 10;
 
   /**
-   * Memory allocated for the write process
+   * Memory allocated for the read process
    */
   private long allocateMemoryForRead = Runtime.getRuntime().maxMemory() * 3 / 10;
 
   /**
+   * Memory allocated for the mtree
+   */
+  private long allocateMemoryForSchema = Runtime.getRuntime().maxMemory() * 1 / 10;
+
+  /**
    * Memory allocated for the read process besides cache
    */
   private long allocateMemoryForReadWithoutCache = Runtime.getRuntime().maxMemory() * 9 / 100;
@@ -137,11 +142,35 @@ public class IoTDBConfig {
   private volatile int maxQueryDeduplicatedPathNum = 1000;
 
   /**
-   * Is dynamic parameter adapter enable.
+   * Ratio of memory allocated for buffered arrays
+   */
+  private double bufferedArraysMemoryProportion = 0.6;
+
+  /**
+   * Flush proportion for system
+   */
+  private double flushProportion = 0.3;
+
+  /**
+   * Reject proportion for system
+   */
+  private double rejectProportion = 0.8;
+
+  /**
+   * If storage group increased more than this threshold, report to system.
    */
-  private boolean enableParameterAdapter = true;
+  private long storageGroupSizeReportThreshold = 16 * 1024 * 1024L;
 
   /**
+   * When inserting rejected, waiting this time to check system again
+   */
+  private int waitingTimeWhenInsertBlockedInMs = 10;
+
+  /**
+   * When inserting rejected exceeds this, throw an exception
+   */
+  private int maxWaitingTimeWhenInsertBlockedInMs = 10000; 
+  /**
    * Is the write ahead log enable.
    */
   private boolean enableWal = true;
@@ -173,6 +202,8 @@ public class IoTDBConfig {
    */
   private int walBufferSize = 16 * 1024 * 1024;
 
+  private int estimatedSeriesSize = 300;
+
   /**
    * default base dir, stores all IoTDB runtime files
    */
@@ -221,11 +252,6 @@ public class IoTDBConfig {
   private String walDir = DEFAULT_BASE_DIR + File.separator + "wal";
 
   /**
-   * Maximum MemTable number in MemTable pool.
-   */
-  private int maxMemtableNumber = 20;
-
-  /**
    * The amount of data iterate each time in server
    */
   private int batchSize = 100000;
@@ -241,6 +267,11 @@ public class IoTDBConfig {
   private int concurrentQueryThread = Runtime.getRuntime().availableProcessors();
 
   /**
+   * Is the write mem control for writing enable.
+   */
+  private boolean enableMemControl = true;
+
+  /**
    * When a TsFile's file size (in byte) exceed this, the TsFile is forced closed.
    */
   private long tsFileSizeThreshold = 512 * 1024 * 1024L;
@@ -1019,14 +1050,6 @@ public class IoTDBConfig {
     this.batchSize = batchSize;
   }
 
-  public int getMaxMemtableNumber() {
-    return maxMemtableNumber;
-  }
-
-  public void setMaxMemtableNumber(int maxMemtableNumber) {
-    this.maxMemtableNumber = maxMemtableNumber;
-  }
-
   public int getConcurrentFlushThread() {
     return concurrentFlushThread;
   }
@@ -1171,6 +1194,14 @@ public class IoTDBConfig {
     this.walBufferSize = walBufferSize;
   }
 
+  public int getEstimatedSeriesSize() {
+    return estimatedSeriesSize;
+  }
+
+  public void setEstimatedSeriesSize(int estimatedSeriesSize) {
+    this.estimatedSeriesSize = estimatedSeriesSize;
+  }
+  
   public boolean isChunkBufferPoolEnable() {
     return chunkBufferPoolEnable;
   }
@@ -1211,12 +1242,36 @@ public class IoTDBConfig {
     this.mergeIntervalSec = mergeIntervalSec;
   }
 
-  public boolean isEnableParameterAdapter() {
-    return enableParameterAdapter;
+  public double getBufferedArraysMemoryProportion() {
+    return bufferedArraysMemoryProportion;
+  }
+
+  public void setBufferedArraysMemoryProportion(double bufferedArraysMemoryProportion) {
+    this.bufferedArraysMemoryProportion = bufferedArraysMemoryProportion;
+  }
+
+  public double getFlushProportion() {
+    return flushProportion;
+  }
+
+  public void setFlushProportion(double flushProportion) {
+    this.flushProportion = flushProportion;
+  }
+
+  public double getRejectProportion() {
+    return rejectProportion;
+  }
+
+  public void setRejectProportion(double rejectProportion) {
+    this.rejectProportion = rejectProportion;
+  }
+
+  public long getStorageGroupSizeReportThreshold() {
+    return storageGroupSizeReportThreshold;
   }
 
-  public void setEnableParameterAdapter(boolean enableParameterAdapter) {
-    this.enableParameterAdapter = enableParameterAdapter;
+  public void setStorageGroupSizeReportThreshold(long storageGroupSizeReportThreshold) {
+    this.storageGroupSizeReportThreshold = storageGroupSizeReportThreshold;
   }
 
   public long getAllocateMemoryForWrite() {
@@ -1227,6 +1282,14 @@ public class IoTDBConfig {
     this.allocateMemoryForWrite = allocateMemoryForWrite;
   }
 
+  public long getAllocateMemoryForSchema() {
+    return allocateMemoryForSchema;
+  }
+
+  void setAllocateMemoryForSchema(long allocateMemoryForSchema) {
+    this.allocateMemoryForSchema = allocateMemoryForSchema;
+  }
+
   long getAllocateMemoryForRead() {
     return allocateMemoryForRead;
   }
@@ -1339,6 +1402,14 @@ public class IoTDBConfig {
     this.mergeReadThroughputMbPerSec = mergeReadThroughputMbPerSec;
   }
 
+  public boolean isEnableMemControl() {
+    return enableMemControl;
+  }
+
+  public void setEnableMemControl(boolean enableMemControl) {
+    this.enableMemControl = enableMemControl;
+  }
+
   public long getMemtableSizeThreshold() {
     return memtableSizeThreshold;
   }
@@ -1940,6 +2011,22 @@ public class IoTDBConfig {
     this.maxQueryDeduplicatedPathNum = maxQueryDeduplicatedPathNum;
   }
 
+  public int getWaitingTimeWhenInsertBlocked() {
+    return waitingTimeWhenInsertBlockedInMs;
+  }
+
+  public void setWaitingTimeWhenInsertBlocked(int waitingTimeWhenInsertBlocked) {
+    this.waitingTimeWhenInsertBlockedInMs = waitingTimeWhenInsertBlocked;
+  }
+
+  public int getMaxWaitingTimeWhenInsertBlocked() {
+    return maxWaitingTimeWhenInsertBlockedInMs;
+  }
+
+  public void setMaxWaitingTimeWhenInsertBlocked(int maxWaitingTimeWhenInsertBlocked) {
+    this.maxWaitingTimeWhenInsertBlockedInMs = maxWaitingTimeWhenInsertBlocked;
+  }
+
   public int getFrequencyIntervalInMinute() {
     return frequencyIntervalInMinute;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
index e8ac636..2ea59a2 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
@@ -58,7 +58,6 @@ public class IoTDBConstant {
 
   // show info
   public static final String COLUMN_ITEM = "                             item";
-  public static final String COLUMN_PARAMETER = "                                          parameter";
   public static final String COLUMN_VALUE = "value";
   public static final String COLUMN_VERSION = "        version";
   public static final String COLUMN_TIMESERIES = "timeseries";
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index c7e7c63..74f4371 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -198,9 +198,19 @@ public class IoTDBDescriptor {
       conf.setTimestampPrecision(properties.getProperty("timestamp_precision",
           conf.getTimestampPrecision()));
 
-      conf.setEnableParameterAdapter(
-          Boolean.parseBoolean(properties.getProperty("enable_parameter_adapter",
-              Boolean.toString(conf.isEnableParameterAdapter()))));
+      conf.setBufferedArraysMemoryProportion(
+          Double.parseDouble(properties.getProperty("buffered_arrays_memory_proportion",
+              Double.toString(conf.getBufferedArraysMemoryProportion()))));
+
+      conf.setFlushProportion(Double.parseDouble(properties.getProperty("flush_proportion",
+          Double.toString(conf.getFlushProportion()))));
+
+      conf.setRejectProportion(Double.parseDouble(properties.getProperty("reject_proportion",
+          Double.toString(conf.getRejectProportion()))));
+
+      conf.setStorageGroupSizeReportThreshold(
+          Long.parseLong(properties.getProperty("storage_group_report_threshold",
+              Long.toString(conf.getStorageGroupSizeReportThreshold()))));
 
       conf.setMetaDataCacheEnable(
           Boolean.parseBoolean(properties.getProperty("meta_data_cache_enable",
@@ -250,6 +260,11 @@ public class IoTDBDescriptor {
       conf.setBatchSize(Integer.parseInt(properties.getProperty("batch_size",
           Integer.toString(conf.getBatchSize()))));
 
+      conf.setEnableMemControl((Boolean
+          .parseBoolean(properties.getProperty("enable_mem_control",
+              Boolean.toString(conf.isEnableMemControl())))));
+      logger.info("IoTDB enable memory control: {}", conf.isEnableMemControl());
+
       long tsfileSizeThreshold = Long.parseLong(properties
           .getProperty("tsfile_size_threshold",
               Long.toString(conf.getTsFileSizeThreshold())).trim());
@@ -268,6 +283,18 @@ public class IoTDBDescriptor {
           .getProperty("avg_series_point_number_threshold",
               Integer.toString(conf.getAvgSeriesPointNumberThreshold()))));
 
+      conf.setWaitingTimeWhenInsertBlocked(Integer.parseInt(properties
+          .getProperty("waiting_time_when_insert_blocked",
+              Integer.toString(conf.getWaitingTimeWhenInsertBlocked()))));
+
+      conf.setMaxWaitingTimeWhenInsertBlocked(Integer.parseInt(properties
+          .getProperty("max_waiting_time_when_insert_blocked",
+              Integer.toString(conf.getMaxWaitingTimeWhenInsertBlocked()))));
+
+      conf.setEstimatedSeriesSize(Integer.parseInt(properties
+          .getProperty("estimated_series_size",
+              Integer.toString(conf.getEstimatedSeriesSize()))));
+
       conf.setMergeChunkPointNumberThreshold(Integer.parseInt(properties
           .getProperty("merge_chunk_point_number",
               Integer.toString(conf.getMergeChunkPointNumberThreshold()))));
@@ -688,18 +715,17 @@ public class IoTDBDescriptor {
       // update WAL conf
       loadWALProps(properties);
 
-      // dynamic parameters
       long tsfileSizeThreshold = Long.parseLong(properties
           .getProperty("tsfile_size_threshold",
               Long.toString(conf.getTsFileSizeThreshold())).trim());
-      if (tsfileSizeThreshold >= 0 && !conf.isEnableParameterAdapter()) {
+      if (tsfileSizeThreshold >= 0) {
         conf.setTsFileSizeThreshold(tsfileSizeThreshold);
       }
 
       long memTableSizeThreshold = Long.parseLong(properties
           .getProperty("memtable_size_threshold",
               Long.toString(conf.getMemtableSizeThreshold())).trim());
-      if (memTableSizeThreshold > 0 && !conf.isEnableParameterAdapter()) {
+      if (memTableSizeThreshold > 0) {
         conf.setMemtableSizeThreshold(memTableSizeThreshold);
       }
 
@@ -749,7 +775,7 @@ public class IoTDBDescriptor {
   }
 
   private void initMemoryAllocate(Properties properties) {
-    String memoryAllocateProportion = properties.getProperty("write_read_free_memory_proportion");
+    String memoryAllocateProportion = properties.getProperty("write_read_schema_free_memory_proportion");
     if (memoryAllocateProportion != null) {
       String[] proportions = memoryAllocateProportion.split(":");
       int proportionSum = 0;
@@ -762,11 +788,14 @@ public class IoTDBDescriptor {
             maxMemoryAvailable * Integer.parseInt(proportions[0].trim()) / proportionSum);
         conf.setAllocateMemoryForRead(
             maxMemoryAvailable * Integer.parseInt(proportions[1].trim()) / proportionSum);
+        conf.setAllocateMemoryForSchema(
+            maxMemoryAvailable * Integer.parseInt(proportions[2].trim()) / proportionSum);
       }
     }
 
-    logger.info("allocateMemoryForRead = " + conf.getAllocateMemoryForRead());
-    logger.info("allocateMemoryForWrite = " + conf.getAllocateMemoryForWrite());
+    logger.info("allocateMemoryForRead = {}", conf.getAllocateMemoryForRead());
+    logger.info("allocateMemoryForWrite = {}", conf.getAllocateMemoryForWrite());
+    logger.info("allocateMemoryForSchema = {}", conf.getAllocateMemoryForSchema());
 
     if (!conf.isMetaDataCacheEnable()) {
       return;
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/adapter/ActiveTimeSeriesCounter.java b/server/src/main/java/org/apache/iotdb/db/conf/adapter/ActiveTimeSeriesCounter.java
deleted file mode 100644
index 0934605..0000000
--- a/server/src/main/java/org/apache/iotdb/db/conf/adapter/ActiveTimeSeriesCounter.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * 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.conf.adapter;
-
-import com.clearspring.analytics.stream.cardinality.HyperLogLog;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ActiveTimeSeriesCounter implements IActiveTimeSeriesCounter {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(ActiveTimeSeriesCounter.class);
-  /**
-   * Map[StorageGroup, HyperLogLogCounter]
-   */
-  private static Map<String, HyperLogLog> storageGroupHllMap = new ConcurrentHashMap<>();
-
-  /**
-   * Map[StorageGroup, ActiveTimeSeriesRatio]
-   */
-  private static Map<String, Double> activeRatioMap = new ConcurrentHashMap<>();
-
-  /**
-   * Map[StorageGroup, ActiveTimeSeriesNumber]
-   */
-  private static Map<String, Long> activeTimeSeriesNumMap = new ConcurrentHashMap<>();
-
-  /**
-   * LOG2M decide the precision of the HyperLogLog algorithm
-   */
-  static final int LOG2M = 13;
-
-  private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-
-  @Override
-  public void init(String storageGroup) {
-    storageGroupHllMap.put(storageGroup, new HyperLogLog(LOG2M));
-    activeRatioMap.put(storageGroup, 0D);
-    activeTimeSeriesNumMap.put(storageGroup, 0L);
-  }
-
-  @Override
-  public void offer(String storageGroup, String device, String measurement) {
-    String path = device + IoTDBConstant.PATH_SEPARATOR + measurement;
-    try {
-      HyperLogLog log = storageGroupHllMap.get(storageGroup);
-      if (log != null) {
-        log.offer(path);
-      } else {
-        LOGGER.warn(
-            "Unknown SG {} is calling HyperLogLog.offer(). It can be ignored the sg is deleted recently",
-            storageGroup);
-      }
-    } catch (Exception e) {
-      //we do not register it if there is no the storage group.
-      LOGGER.error("Storage group {} registers active time series {} failed", storageGroup, path,
-          e);
-    }
-  }
-
-  @Override
-  public void updateActiveRatio(String storageGroup) {
-    lock.writeLock().lock();
-    try {
-      HyperLogLog log = storageGroupHllMap.get(storageGroup);
-      if (log == null) {
-        LOGGER.warn(
-            "Unknown SG {} is calling updateActiveRatio. It can be ignored the sg is deleted recently",
-            storageGroup);
-        return;
-      }
-      long activeTimeSeriesNum = storageGroupHllMap.get(storageGroup).cardinality();
-      if (activeTimeSeriesNum != activeTimeSeriesNumMap.get(storageGroup)) {
-        // update the active time series number in the newest memtable to be flushed
-        activeTimeSeriesNumMap.put(storageGroup, activeTimeSeriesNum);
-
-        double totalActiveTsNum = 0;
-        LOGGER.debug("{}: updating active ratio", Thread.currentThread().getName());
-        for (double number : activeTimeSeriesNumMap.values()) {
-          totalActiveTsNum += number;
-        }
-        for (Map.Entry<String, Long> entry : activeTimeSeriesNumMap.entrySet()) {
-          double activeRatio = 0;
-          if (totalActiveTsNum > 0) {
-            activeRatio = entry.getValue() / totalActiveTsNum;
-          }
-          activeRatioMap.put(entry.getKey(), activeRatio);
-          LOGGER.debug("{}: storage group {} has an active ratio: {}",
-              Thread.currentThread().getName(),
-              entry.getKey(), activeRatio);
-        }
-      }
-      // initialize the HLL counter
-      storageGroupHllMap.put(storageGroup, new HyperLogLog(LOG2M));
-    } catch (Exception e) {
-      LOGGER.error("Update {} active ratio failed", storageGroup, e);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  @Override
-  public double getActiveRatio(String storageGroup) {
-    lock.readLock().lock();
-    double ratio;
-    try {
-      ratio = activeRatioMap.get(storageGroup);
-    } catch (Exception e) {
-      ratio = 0;
-      LOGGER.error("Get active ratio failed", e);
-    } finally {
-      lock.readLock().unlock();
-    }
-    return ratio;
-  }
-
-  @Override
-  public void delete(String storageGroup) {
-    storageGroupHllMap.remove(storageGroup);
-    activeRatioMap.remove(storageGroup);
-    activeTimeSeriesNumMap.remove(storageGroup);
-  }
-
-  private static class ActiveTimeSeriesCounterHolder {
-    private static final ActiveTimeSeriesCounter INSTANCE = new ActiveTimeSeriesCounter();
-  }
-
-  public static ActiveTimeSeriesCounter getInstance() {
-    return ActiveTimeSeriesCounterHolder.INSTANCE;
-  }
-
-  /**
-   * this method is for test
-   */
-  public static void clear() {
-    storageGroupHllMap = new ConcurrentHashMap<>();
-    activeRatioMap = new ConcurrentHashMap<>();
-    activeTimeSeriesNumMap = new ConcurrentHashMap<>();
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/adapter/CompressionRatio.java b/server/src/main/java/org/apache/iotdb/db/conf/adapter/CompressionRatio.java
index a2a626e..2afa56b 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/adapter/CompressionRatio.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/adapter/CompressionRatio.java
@@ -94,23 +94,6 @@ public class CompressionRatio {
     if (LOGGER.isInfoEnabled()) {
       LOGGER.info("Compression ratio is {}", compressionRatio.get());
     }
-    if (CONFIG.isEnableParameterAdapter()) {
-      if (LOGGER.isInfoEnabled()) {
-        LOGGER.info(
-            "After updating compression ratio, trying to adjust parameters, the original parameters: "
-                + "MemTableSize threshold is {}B, TsfileSize threshold is {}B, MemTableNumber is {}",
-            CONFIG.getMemtableSizeThreshold(), CONFIG.getTsFileSizeThreshold(),
-            CONFIG.getMaxMemtableNumber());
-      }
-      IoTDBConfigDynamicAdapter.getInstance().tryToAdaptParameters();
-      if(LOGGER.isInfoEnabled()) {
-        LOGGER.info(
-            "After updating compression ratio, trying to adjust parameters, the modified parameters: "
-                + "MemTableSize threshold is {}B, TsfileSize threshold is {}B, MemTableNumber is {}",
-            CONFIG.getMemtableSizeThreshold(), CONFIG.getTsFileSizeThreshold(),
-            CONFIG.getMaxMemtableNumber());
-      }
-    }
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/adapter/IActiveTimeSeriesCounter.java b/server/src/main/java/org/apache/iotdb/db/conf/adapter/IActiveTimeSeriesCounter.java
deleted file mode 100644
index 43bc10c..0000000
--- a/server/src/main/java/org/apache/iotdb/db/conf/adapter/IActiveTimeSeriesCounter.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.conf.adapter;
-
-public interface IActiveTimeSeriesCounter {
-
-  /**
-   * Initialize the counter by adding a new HyperLogLog counter for the given storage group
-   *
-   * @param storageGroup the given storage group to be initialized
-   */
-  void init(String storageGroup);
-
-  /**
-   * Register a time series to the active time series counter
-   *
-   * @param storageGroup the storage group name of the time series
-   * @param device the device name of the time series
-   * @param measurement the sensor name of the time series
-   */
-  void offer(String storageGroup, String device, String measurement);
-
-  /**
-   * Update the ActiveRatioMap
-   *
-   * @param storageGroup whose counter will be refreshed after the update
-   */
-  void updateActiveRatio(String storageGroup);
-
-  /**
-   * Get the active time series number proportion of the given storage group
-   *
-   * @param storageGroup the storage group to be calculated
-   * @return the active time series number proportion of the given storage group
-   */
-  double getActiveRatio(String storageGroup);
-
-  /**
-   * Delete the counter for the given storage group
-   *
-   * @param storageGroup whose counter will be removed
-   */
-  void delete(String storageGroup);
-
-}
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/adapter/IDynamicAdapter.java b/server/src/main/java/org/apache/iotdb/db/conf/adapter/IDynamicAdapter.java
deleted file mode 100644
index 1f6b58f..0000000
--- a/server/src/main/java/org/apache/iotdb/db/conf/adapter/IDynamicAdapter.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.conf.adapter;
-
-import org.apache.iotdb.db.exception.ConfigAdjusterException;
-
-public interface IDynamicAdapter {
-
-  /**
-   * Adjust parameters of maxMemtableNumber, memtableSizeThreshold and tsFileSizeThreshold.
-   * @return {@code true} if it has appropriate parameters and adjust them successfully
-   *         {@code false} adjusting parameters failed
-   */
-  boolean tryToAdaptParameters();
-
-  /**
-   * Add or delete storage groups
-   *
-   * @param diff it's positive if add new storage groups; it's negative if delete old storage
-   * groups.
-   */
-  void addOrDeleteStorageGroup(int diff) throws ConfigAdjusterException;
-
-  /**
-   * Add or delete timeseries
-   *
-   * @param diff it's positive if create new timeseries; it's negative if delete old timeseris.
-   */
-  void addOrDeleteTimeSeries(int diff) throws ConfigAdjusterException;
-
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapter.java b/server/src/main/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapter.java
deleted file mode 100644
index d66f054..0000000
--- a/server/src/main/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapter.java
+++ /dev/null
@@ -1,300 +0,0 @@
-/*
- * 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.conf.adapter;
-
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.ConfigAdjusterException;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
-import org.apache.iotdb.db.service.IoTDB;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is to dynamically adjust some important parameters of the system, determine the speed
- * of MenTable brushing disk, the speed of file sealing and so on, with the continuous change of
- * load in the process of system operation.
- * <p>
- * There are three dynamically adjustable parameters: maxMemTableNum, memtableSize and
- * tsFileSizeThreshold.
- * <p>
- * 1. maxMemTableNum. This parameter represents the size of the MemTable available in the MemTable
- * pool, which is closely related to the number of storage groups. When adding or deleting a storage
- * group, the parameter also adds or deletes four MemTables. The reason why adding or deleting four
- * MemTables is that when the system is running stably, the speed of the flush operation is faster
- * than that of data writing, so one is used for the Flush process and the other is used for data
- * writing. Otherwise, the system should limit the speed of data writing to maintain stability. And
- * two for sequence data, two for unsequence data.
- * <p>
- * 2. memtableSize. This parameter determines the threshold value for the MemTable in memory to be
- * flushed into disk. When the system load increases, the parameter should be set smaller so that
- * the data in memory can be flushed into disk as soon as possible.
- * <p>
- * 3. tsFileSizeThreshold. This parameter determines the speed of the tsfile seal, and then
- * determines the maximum size of metadata information maintained in memory. When the system load
- * increases, the parameter should be smaller to seal the file as soon as possible, release the
- * memory occupied by the corresponding metadata information as soon as possible.
- * <p>
- * The following equation is used to adjust the dynamic parameters of the data:
- * <p>
- * *
- *
- * Abbreviation of parameters:
- * 1 memtableSize: m
- * 2 maxMemTableNum: Nm
- * 3 maxSeriesNumberAmongStorageGroup: Ns
- * 4 tsFileSizeThreshold: Sf
- * 5 CompressionRatio: c
- * 6 chunk metadata size: a
- * 7 static memory: b
- * 8 allocate memory for write: S
- *
- * The equation: m * Nm + Nm * Ns * Sf * a * c / m + b = S
- * Namely: MemTable data memory size + chunk metadata memory size + static memory size = memory size for write
- *
- */
-public class IoTDBConfigDynamicAdapter implements IDynamicAdapter {
-
-  public static final String CREATE_STORAGE_GROUP = "create storage group";
-  public static final String ADD_TIMESERIES = "add timeseries";
-  /**
-   * Average queue length in memtable pool
-   */
-  static final int MEM_TABLE_AVERAGE_QUEUE_LEN = 5;
-  private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBConfigDynamicAdapter.class);
-
-  // static parameter section
-  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
-  /**
-   * Metadata size of per timeseries, the default value is 2KB.
-   */
-  private static final long TIMESERIES_METADATA_SIZE_IN_BYTE = 2L * 1024;
-  private static final double WAL_MEMORY_RATIO = 0.1;
-  public static final int MEMTABLE_NUM_FOR_EACH_PARTITION = 4;
-  /**
-   * Maximum amount of memory allocated for write process.
-   */
-  private static long allocateMemoryForWrite = CONFIG.getAllocateMemoryForWrite();
-  /**
-   * Metadata size of per chunk, the default value is 1.5 KB.
-   */
-  private static long CHUNK_METADATA_SIZE_IN_BYTE = 1536L;
-
-  // static memory section
-  /**
-   * Static memory, includes all timeseries metadata, which equals to
-   * TIMESERIES_METADATA_SIZE_IN_BYTE * totalTimeseriesNum, the unit is byte.
-   * <p>
-   * Currently, we think that static memory only consists of time series metadata information. We
-   * ignore the memory occupied by the tsfile information maintained in memory, because we think
-   * that this part occupies very little memory.
-   */
-  private long staticMemory;
-
-  private int totalStorageGroup;
-
-  private int totalTimeseries;
-
-  // MemTable section
-
-  private int maxMemTableNum = MEM_TABLE_AVERAGE_QUEUE_LEN;
-
-  private long currentMemTableSize;
-
-  // Adapter section
-
-  private boolean initialized = false;
-
-  private IoTDBConfigDynamicAdapter() {
-  }
-
-  public static void setChunkMetadataSizeInByte(long chunkMetadataSizeInByte) {
-    CHUNK_METADATA_SIZE_IN_BYTE = chunkMetadataSizeInByte;
-  }
-
-  public static IoTDBConfigDynamicAdapter getInstance() {
-    return IoTDBConfigAdapterHolder.INSTANCE;
-  }
-
-  @Override
-  public synchronized boolean tryToAdaptParameters() {
-    if (!CONFIG.isEnableParameterAdapter()) {
-      return true;
-    }
-    boolean canAdjust = true;
-    double ratio = CompressionRatio.getInstance().getRatio();
-    long memtableSizeInByte = calcMemTableSize(ratio);
-    long memTableSizeFloorThreshold = getMemTableSizeFloorThreshold();
-    long tsFileSizeThreshold = CONFIG.getTsFileSizeThreshold();
-    if (memtableSizeInByte < memTableSizeFloorThreshold) {
-      if (LOGGER.isDebugEnabled() && initialized) {
-        LOGGER.debug("memtableSizeInByte {} is smaller than memTableSizeFloorThreshold {}",
-            memtableSizeInByte, memTableSizeFloorThreshold);
-      }
-      tsFileSizeThreshold = calcTsFileSizeThreshold(memTableSizeFloorThreshold, ratio);
-      if ((long) (tsFileSizeThreshold * ratio) < memTableSizeFloorThreshold) {
-        canAdjust = false;
-      } else {
-        // memtableSizeInByte need to be larger than memTableSizeFloorThreshold
-        memtableSizeInByte = Math.max(memTableSizeFloorThreshold,
-            memTableSizeFloorThreshold + (
-                ((long) (tsFileSizeThreshold * ratio) - memTableSizeFloorThreshold)
-                    >> 1));
-      }
-    }
-
-    if (canAdjust) {
-      CONFIG.setMaxMemtableNumber(maxMemTableNum);
-      CONFIG.setWalBufferSize(
-          (int) Math
-              .min(Integer.MAX_VALUE, allocateMemoryForWrite * WAL_MEMORY_RATIO / maxMemTableNum));
-      CONFIG.setTsFileSizeThreshold(tsFileSizeThreshold);
-      CONFIG.setMemtableSizeThreshold(memtableSizeInByte);
-      if (LOGGER.isDebugEnabled() && initialized) {
-        LOGGER.debug(
-            "After adjusting, max memTable num is {}, tsFile threshold is {}, memtableSize is {}, memTableSizeFloorThreshold is {}, storage group = {}, total timeseries = {}, the max number of timeseries among storage groups = {}",
-            maxMemTableNum, tsFileSizeThreshold, memtableSizeInByte, memTableSizeFloorThreshold,
-            totalStorageGroup, totalTimeseries,
-            IoTDB.metaManager.getMaximalSeriesNumberAmongStorageGroups());
-      }
-      currentMemTableSize = memtableSizeInByte;
-    }
-    if (!initialized) {
-      CONFIG.setMaxMemtableNumber(maxMemTableNum);
-      return true;
-    }
-    return canAdjust;
-  }
-
-  /**
-   * Calculate appropriate MemTable size. Computing method refers to class annotations.
-   *
-   * @return MemTable byte size. If the value is -1, there is no valid solution.
-   */
-  private long calcMemTableSize(double ratio) {
-    // when unit is byte, it's likely to cause Long type overflow.
-    // so when b is larger than Integer.MAC_VALUE use the unit KB.
-    double a = maxMemTableNum;
-    double b = allocateMemoryForWrite * (1 - WAL_MEMORY_RATIO) - staticMemory;
-    int magnification = b > Integer.MAX_VALUE ? 1024 : 1;
-    b /= magnification;
-    double c =
-        (double) CONFIG.getTsFileSizeThreshold() * maxMemTableNum * CHUNK_METADATA_SIZE_IN_BYTE
-            * IoTDB.metaManager.getMaximalSeriesNumberAmongStorageGroups() * ratio
-            / magnification / magnification;
-    double tempValue = b * b - 4 * a * c;
-    double memTableSize = ((b + Math.sqrt(tempValue)) / (2 * a));
-    return tempValue < 0 ? -1 : (long) (memTableSize * magnification);
-  }
-
-  /**
-   * Calculate appropriate Tsfile size based on MemTable size. Computing method refers to class
-   * annotations.
-   *
-   * @param memTableSize MemTable size
-   * @return Tsfile byte threshold
-   */
-  private long calcTsFileSizeThreshold(long memTableSize, double ratio) {
-    return (long) ((allocateMemoryForWrite * (1 - WAL_MEMORY_RATIO) - maxMemTableNum * memTableSize
-        - staticMemory) * memTableSize / (ratio * maxMemTableNum * CHUNK_METADATA_SIZE_IN_BYTE
-        * IoTDB.metaManager.getMaximalSeriesNumberAmongStorageGroups()));
-  }
-
-  /**
-   * Get the floor threshold MemTable size. For Primitive Array, we think that the maximum memory
-   * occupied by each value is 8 bytes. The reason for multiplying 2 is that the timestamp also
-   * takes 8 bytes.
-   */
-  private long getMemTableSizeFloorThreshold() {
-    return IoTDB.metaManager.getMaximalSeriesNumberAmongStorageGroups()
-        * PrimitiveArrayPool.ARRAY_SIZE * Long.BYTES * 2;
-  }
-
-  @Override
-  public void addOrDeleteStorageGroup(int diff) throws ConfigAdjusterException {
-    totalStorageGroup += diff;
-    maxMemTableNum +=
-        MEMTABLE_NUM_FOR_EACH_PARTITION * IoTDBDescriptor.getInstance().getConfig().getConcurrentWritingTimePartition() * diff
-            + diff;
-    if (!CONFIG.isEnableParameterAdapter()) {
-      CONFIG.setMaxMemtableNumber(maxMemTableNum);
-      return;
-    }
-
-    if (!tryToAdaptParameters()) {
-      totalStorageGroup -= diff;
-      maxMemTableNum -=
-          MEMTABLE_NUM_FOR_EACH_PARTITION * IoTDBDescriptor.getInstance().getConfig().getConcurrentWritingTimePartition() * diff
-              + diff;
-      throw new ConfigAdjusterException(CREATE_STORAGE_GROUP);
-    }
-  }
-
-  @Override
-  public void addOrDeleteTimeSeries(int diff) throws ConfigAdjusterException {
-    if (!CONFIG.isEnableParameterAdapter()) {
-      return;
-    }
-    totalTimeseries += diff;
-    staticMemory += diff * TIMESERIES_METADATA_SIZE_IN_BYTE;
-    if (!tryToAdaptParameters()) {
-      totalTimeseries -= diff;
-      staticMemory -= diff * TIMESERIES_METADATA_SIZE_IN_BYTE;
-      throw new ConfigAdjusterException(ADD_TIMESERIES);
-    }
-  }
-
-  public void setInitialized(boolean initialized) {
-    this.initialized = initialized;
-  }
-
-  long getCurrentMemTableSize() {
-    return currentMemTableSize;
-  }
-
-  public int getTotalTimeseries() {
-    return totalTimeseries;
-  }
-
-  public int getTotalStorageGroup() {
-    return totalStorageGroup;
-  }
-
-  /**
-   * Only for test
-   */
-  public void reset() {
-    totalTimeseries = 0;
-    staticMemory = 0;
-    maxMemTableNum = MEM_TABLE_AVERAGE_QUEUE_LEN;
-    allocateMemoryForWrite = CONFIG.getAllocateMemoryForWrite();
-    initialized = false;
-  }
-
-  private static class IoTDBConfigAdapterHolder {
-
-    private static final IoTDBConfigDynamicAdapter INSTANCE = new IoTDBConfigDynamicAdapter();
-
-    private IoTDBConfigAdapterHolder() {
-
-    }
-
-  }
-
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/adapter/ManageDynamicParameters.java b/server/src/main/java/org/apache/iotdb/db/conf/adapter/ManageDynamicParameters.java
deleted file mode 100644
index 395214f..0000000
--- a/server/src/main/java/org/apache/iotdb/db/conf/adapter/ManageDynamicParameters.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * 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.conf.adapter;
-
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.StartupException;
-import org.apache.iotdb.db.service.IService;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.service.JMXService;
-import org.apache.iotdb.db.service.ServiceType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is to get and set dynamic parameters through JMX.
- */
-public class ManageDynamicParameters implements ManageDynamicParametersMBean, IService {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(ManageDynamicParameters.class);
-
-  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
-
-  private final String mbeanName = String
-      .format("%s:%s=%s", IoTDBConstant.IOTDB_PACKAGE, IoTDBConstant.JMX_TYPE,
-          getID().getJmxName());
-
-  private ManageDynamicParameters() {
-
-  }
-
-  public static ManageDynamicParameters getInstance() {
-    return ManageDynamicParametersHolder.INSTANCE;
-  }
-
-  @Override
-  public void showDynamicParameters() {
-    LOGGER.info(
-        "Memtable size threshold: {}B, Memtable number: {}, Tsfile size threshold: {}B, Compression ratio: {}, "
-            + "Storage group number: {}, Timeseries number: {}, Maximal timeseries number among storage groups: {}",
-        CONFIG.getMemtableSizeThreshold(), CONFIG.getMaxMemtableNumber(),
-        CONFIG.getTsFileSizeThreshold(), CompressionRatio.getInstance().getRatio(),
-        IoTDBConfigDynamicAdapter.getInstance().getTotalStorageGroup(),
-        IoTDBConfigDynamicAdapter.getInstance().getTotalTimeseries(),
-        IoTDB.metaManager.getMaximalSeriesNumberAmongStorageGroups());
-  }
-
-  @Override
-  public boolean isEnableDynamicAdapter() {
-    return CONFIG.isEnableParameterAdapter();
-  }
-
-  @Override
-  public void setEnableDynamicAdapter(boolean enableDynamicAdapter) {
-    CONFIG.setEnableParameterAdapter(enableDynamicAdapter);
-  }
-
-  @Override
-  public long getMemTableSizeThreshold() {
-    return CONFIG.getMemtableSizeThreshold();
-  }
-
-  @Override
-  public void setMemTableSizeThreshold(long memTableSizeThreshold) {
-    CONFIG.setMemtableSizeThreshold(memTableSizeThreshold);
-  }
-
-  @Override
-  public int getMemTableNumber() {
-    return CONFIG.getMaxMemtableNumber();
-  }
-
-  @Override
-  public void setMemTableNumber(int memTableNumber) {
-    CONFIG.setMaxMemtableNumber(memTableNumber);
-  }
-
-  @Override
-  public long getTsfileSizeThreshold() {
-    return CONFIG.getTsFileSizeThreshold();
-  }
-
-  @Override
-  public void setTsfileSizeThreshold(long tsfileSizeThreshold) {
-    CONFIG.setTsFileSizeThreshold(tsfileSizeThreshold);
-  }
-
-  @Override
-  public void start() throws StartupException {
-    try {
-      JMXService.registerMBean(getInstance(), mbeanName);
-      LOGGER.info("{}: start {}...", IoTDBConstant.GLOBAL_DB_NAME, this.getID().getName());
-    } catch (Exception e) {
-      LOGGER.error("Failed to start {} because: ", this.getID().getName(), e);
-      throw new StartupException(e);
-    }
-  }
-
-  @Override
-  public void stop() {
-    JMXService.deregisterMBean(mbeanName);
-    LOGGER.info("{}: stop {}...", IoTDBConstant.GLOBAL_DB_NAME, this.getID().getName());
-  }
-
-  @Override
-  public ServiceType getID() {
-    return ServiceType.MANAGE_DYNAMIC_PARAMETERS_SERVICE;
-  }
-
-  private static class ManageDynamicParametersHolder {
-
-    private static final ManageDynamicParameters INSTANCE = new ManageDynamicParameters();
-
-    private ManageDynamicParametersHolder() {
-
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/adapter/ManageDynamicParametersMBean.java b/server/src/main/java/org/apache/iotdb/db/conf/adapter/ManageDynamicParametersMBean.java
deleted file mode 100644
index 92e0da5..0000000
--- a/server/src/main/java/org/apache/iotdb/db/conf/adapter/ManageDynamicParametersMBean.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.conf.adapter;
-
-public interface ManageDynamicParametersMBean {
-
-  /**
-   * Show all dynamic parameters, including memtable size threshold, memtable number, tsfile size
-   * threshold, compression ratio, storage group number, timeseries number, max timeseries number
-   * among storage groups.
-   */
-  void showDynamicParameters();
-
-  boolean isEnableDynamicAdapter();
-
-  void setEnableDynamicAdapter(boolean enableDynamicAdapter);
-
-  long getMemTableSizeThreshold();
-
-  void setMemTableSizeThreshold(long memTableSizeThreshold);
-
-  int getMemTableNumber();
-
-  void setMemTableNumber(int memTableNumber);
-
-  long getTsfileSizeThreshold();
-
-  void setTsfileSizeThreshold(long tsfileSizeThreshold);
-
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index 9e1279e..b6d5ace 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -428,7 +428,7 @@ public class StorageEngine implements IService {
       } else {
         // to avoid concurrent modification problem, we need a new array list
         for (TsFileProcessor tsfileProcessor : new ArrayList<>(
-            processor.getWorkUnsequenceTsFileProcessor())) {
+            processor.getWorkUnsequenceTsFileProcessors())) {
           if (isSync) {
             processor.syncCloseOneTsFileProcessor(false, tsfileProcessor);
           } else {
@@ -459,7 +459,7 @@ public class StorageEngine implements IService {
       // to avoid concurrent modification problem, we need a new array list
       List<TsFileProcessor> processors = isSeq ?
           new ArrayList<>(processor.getWorkSequenceTsFileProcessors()) :
-          new ArrayList<>(processor.getWorkUnsequenceTsFileProcessor());
+          new ArrayList<>(processor.getWorkUnsequenceTsFileProcessors());
       try {
         for (TsFileProcessor tsfileProcessor : processors) {
           if (tsfileProcessor.getTimeRangeId() == partitionId) {
@@ -746,7 +746,7 @@ public class StorageEngine implements IService {
         partitionIdList.add(tmpPair);
       }
 
-      for (TsFileProcessor tsFileProcessor : processor.getWorkUnsequenceTsFileProcessor()) {
+      for (TsFileProcessor tsFileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
         Pair<Long, Boolean> tmpPair = new Pair<>(tsFileProcessor.getTimeRangeId(), false);
         partitionIdList.add(tmpPair);
       }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkMetadataCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkMetadataCache.java
index d76ec25..f8fd5f3 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkMetadataCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkMetadataCache.java
@@ -28,7 +28,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.db.utils.TestOnly;
@@ -79,7 +78,6 @@ public class ChunkMetadataCache {
         if (count < 10) {
           long currentSize = value.get(0).calculateRamSize();
           averageSize = ((averageSize * count) + currentSize) / (++count);
-          IoTDBConfigDynamicAdapter.setChunkMetadataSizeInByte(averageSize);
           entrySize = RamUsageEstimator.sizeOf(key)
               + (currentSize + RamUsageEstimator.NUM_BYTES_OBJECT_REF) * value.size()
               + RamUsageEstimator.shallowSizeOf(value);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java b/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
index b05e348..e9f4c92 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
@@ -23,8 +23,6 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.engine.flush.pool.FlushSubTaskPoolManager;
 import org.apache.iotdb.db.engine.memtable.IMemTable;
 import org.apache.iotdb.db.engine.memtable.IWritableMemChunk;
@@ -78,6 +76,10 @@ public class MemTableFlushTask {
    */
   public void syncFlushMemTable()
       throws ExecutionException, InterruptedException, IOException {
+    logger.info("The memTable size of SG {} is {}, the avg series points num in chunk is {} ",
+        storageGroup,
+        memTable.memSize(),
+        memTable.getTotalPointsNum() / memTable.getSeriesNumber());
     long start = System.currentTimeMillis();
     long sortTime = 0;
 
@@ -90,16 +92,9 @@ public class MemTableFlushTask {
         TVList tvList = series.getSortedTVList();
         sortTime += System.currentTimeMillis() - startTime;
         encodingTaskQueue.add(new Pair<>(tvList, desc));
-        // register active time series to the ActiveTimeSeriesCounter
-        if (IoTDBDescriptor.getInstance().getConfig().isEnableParameterAdapter()) {
-          ActiveTimeSeriesCounter.getInstance().offer(storageGroup, deviceId, measurementId);
-        }
       }
       encodingTaskQueue.add(new EndChunkGroupIoTask());
     }
-    if (IoTDBDescriptor.getInstance().getConfig().isEnableParameterAdapter()) {
-      ActiveTimeSeriesCounter.getInstance().updateActiveRatio(storageGroup);
-    }
     noMoreEncodingTask = true;
     logger.debug(
         "Storage group {} memtable {}, flushing into disk: data sort time cost {} ms.",
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
index eabedcb..5960d43 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
@@ -54,8 +54,23 @@ public abstract class AbstractMemTable implements IMemTable {
   private int avgSeriesPointNumThreshold = IoTDBDescriptor.getInstance().getConfig()
       .getAvgSeriesPointNumberThreshold();
 
+  /**
+   * memory size of data points, including TEXT values
+   */
   private long memSize = 0;
 
+  /**
+   * memory usage of all TVLists memory usage regardless of whether these TVLists are full,
+   * including TEXT values
+   */
+  private long tvListRamCost = 0;
+
+  /**
+   * The initial value is true because we want calculate the text data size when recover
+   * memTable!!
+   */
+  protected boolean disableMemControl = true;
+
   private int seriesNumber = 0;
 
   private long totalPointsNum = 0;
@@ -114,7 +129,8 @@ public abstract class AbstractMemTable implements IMemTable {
       }
 
       Object value = insertRowPlan.getValues()[i];
-      memSize += MemUtils.getRecordSize(insertRowPlan.getMeasurementMNodes()[i].getSchema().getType(), value);
+      memSize += MemUtils.getRecordSize(insertRowPlan.getMeasurementMNodes()[i].getSchema().getType(), value,
+          disableMemControl);
 
       write(insertRowPlan.getDeviceId().getFullPath(), insertRowPlan.getMeasurements()[i],
           insertRowPlan.getMeasurementMNodes()[i].getSchema(), insertRowPlan.getTime(), value);
@@ -129,11 +145,11 @@ public abstract class AbstractMemTable implements IMemTable {
     updatePlanIndexes(insertTabletPlan.getIndex());
     try {
       write(insertTabletPlan, start, end);
-      memSize += MemUtils.getRecordSize(insertTabletPlan, start, end);
+      memSize += MemUtils.getRecordSize(insertTabletPlan, start, end, disableMemControl);
       totalPointsNum += (insertTabletPlan.getMeasurements().length - insertTabletPlan.getFailedMeasurementNumber())
         * (end - start);
     } catch (RuntimeException e) {
-      throw new WriteProcessException(e.getMessage());
+      throw new WriteProcessException(e);
     }
   }
 
@@ -159,6 +175,21 @@ public abstract class AbstractMemTable implements IMemTable {
     }
   }
 
+  @Override
+  public boolean checkIfChunkDoesNotExist(String deviceId, String measurement) {
+    if (!memTableMap.containsKey(deviceId)) {
+      return true;
+    }
+    Map<String, IWritableMemChunk> memSeries = memTableMap.get(deviceId);
+    return !memSeries.containsKey(measurement);
+  }
+
+  @Override
+  public int getCurrentChunkPointNum(String deviceId, String measurement) {
+    Map<String, IWritableMemChunk> memSeries = memTableMap.get(deviceId);
+    IWritableMemChunk memChunk = memSeries.get(measurement);
+    return memChunk.getTVList().size();
+  }
 
   public int getSeriesNumber() {
     return seriesNumber;
@@ -200,6 +231,7 @@ public abstract class AbstractMemTable implements IMemTable {
     seriesNumber = 0;
     totalPointsNum = 0;
     totalPointsNumThreshold = 0;
+    tvListRamCost = 0;
     maxPlanIndex = 0;
   }
 
@@ -276,6 +308,21 @@ public abstract class AbstractMemTable implements IMemTable {
   }
 
   @Override
+  public void addTVListRamCost(long cost) {
+    this.tvListRamCost += cost;
+  }
+
+  @Override
+  public long getTVListsRamCost() {
+    return tvListRamCost;
+  }
+
+  @Override
+  public void addTextDataSize(long testDataSize) {
+    this.memSize += testDataSize;
+  }
+
+  @Override
   public void release() {
     for (Entry<String, Map<String, IWritableMemChunk>> entry : memTableMap.entrySet()) {
       for (Entry<String, IWritableMemChunk> subEntry : entry.getValue().entrySet()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
index fb3d293..81435c9 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
@@ -59,6 +59,17 @@ public interface IMemTable {
   long memSize();
 
   /**
+   * only used when mem control enabled
+   */
+  void addTVListRamCost(long cost);
+
+  /**
+   * only used when mem control enabled
+   */
+  long getTVListsRamCost();
+
+  /**
+   * only used when mem control enabled
    * @return whether the average number of points in each WritableChunk reaches the threshold
    */
   boolean reachTotalPointNumThreshold();
@@ -68,7 +79,7 @@ public interface IMemTable {
   long getTotalPointsNum();
 
 
-  void insert(InsertRowPlan insertRowPlan) throws WriteProcessException;
+  void insert(InsertRowPlan insertRowPlan);
 
   /**
    * [start, end)
@@ -121,6 +132,22 @@ public interface IMemTable {
 
   void release();
 
+  /**
+   * must guarantee the device exists in the work memtable
+   * only used when mem control enabled
+   */
+  boolean checkIfChunkDoesNotExist(String deviceId, String measurement);
+
+  /**
+   * only used when mem control enabled
+   */
+  int getCurrentChunkPointNum(String deviceId, String measurement);
+
+  /**
+   * only used when mem control enabled
+   */
+  void addTextDataSize(long textDataIncrement);
+
   long getMaxPlanIndex();
 
   long getMinPlanIndex();
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
index aa8a59a..9dc19fd 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
@@ -37,18 +37,6 @@ public interface IWritableMemChunk {
 
   void putBoolean(long t, boolean v);
 
-  void putLongs(long[] t, long[] v);
-
-  void putInts(long[] t, int[] v);
-
-  void putFloats(long[] t, float[] v);
-
-  void putDoubles(long[] t, double[] v);
-
-  void putBinaries(long[] t, Binary[] v);
-
-  void putBooleans(long[] t, boolean[] v);
-
   void putLongs(long[] t, long[] v, int start, int end);
 
   void putInts(long[] t, int[] v, int start, int end);
@@ -94,4 +82,5 @@ public interface IWritableMemChunk {
    * @return how many points are deleted
    */
   int delete(long lowerBound, long upperBound);
+
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
index dcb5ca0..b68422f 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
@@ -29,6 +29,9 @@ public class PrimitiveMemTable extends AbstractMemTable {
   public PrimitiveMemTable() {
   }
 
+  public PrimitiveMemTable(boolean enableMemControl) {
+    this.disableMemControl = !enableMemControl;
+  }
   public PrimitiveMemTable(Map<String, Map<String, IWritableMemChunk>> memTableMap) {
     super(memTableMap);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
index 3ac190f..3d981d0 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
@@ -92,7 +92,6 @@ public class WritableMemChunk implements IWritableMemChunk {
     }
   }
 
-
   @Override
   public void putLong(long t, long v) {
     list.putLong(t, v);
@@ -124,36 +123,6 @@ public class WritableMemChunk implements IWritableMemChunk {
   }
 
   @Override
-  public void putLongs(long[] t, long[] v) {
-    list.putLongs(t, v);
-  }
-
-  @Override
-  public void putInts(long[] t, int[] v) {
-    list.putInts(t, v);
-  }
-
-  @Override
-  public void putFloats(long[] t, float[] v) {
-    list.putFloats(t, v);
-  }
-
-  @Override
-  public void putDoubles(long[] t, double[] v) {
-    list.putDoubles(t, v);
-  }
-
-  @Override
-  public void putBinaries(long[] t, Binary[] v) {
-    list.putBinaries(t, v);
-  }
-
-  @Override
-  public void putBooleans(long[] t, boolean[] v) {
-    list.putBooleans(t, v);
-  }
-
-  @Override
   public void putLongs(long[] t, long[] v, int start, int end) {
     list.putLongs(t, v, start, end);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupInfo.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupInfo.java
new file mode 100644
index 0000000..4d60c52
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupInfo.java
@@ -0,0 +1,105 @@
+/*
+ * 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.engine.storagegroup;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.rescon.SystemInfo;
+
+/**
+ * The storageGroupInfo records the total memory cost of the Storage Group.
+ */
+public class StorageGroupInfo {
+
+  private StorageGroupProcessor storageGroupProcessor;
+
+  /**
+   * The total Storage group memory cost
+   */
+  private AtomicLong memoryCost;
+
+  /**
+   * The threshold of reporting it's size to SystemInfo
+   */
+  private long storageGroupSizeReportThreshold = 
+      IoTDBDescriptor.getInstance().getConfig().getStorageGroupSizeReportThreshold();
+
+  private long lastReportedSize = 0L;
+
+  /**
+   * A set of all unclosed TsFileProcessors in this SG
+   */
+  private Set<TsFileProcessor> reportedTsps = new HashSet<>();
+
+  public StorageGroupInfo(StorageGroupProcessor storageGroupProcessor) {
+    this.storageGroupProcessor = storageGroupProcessor;
+    memoryCost = new AtomicLong();
+  }
+
+  public StorageGroupProcessor getStorageGroupProcessor() {
+    return storageGroupProcessor;
+  }
+
+  /**
+   * When create a new TsFileProcessor, call this method
+   */
+  public void initTsFileProcessorInfo(TsFileProcessor tsFileProcessor) {
+    if (reportedTsps.add(tsFileProcessor)) {
+      memoryCost.getAndAdd(IoTDBDescriptor.getInstance().getConfig().getWalBufferSize());
+    }
+  }
+
+  public void addStorageGroupMemCost(long cost) {
+    memoryCost.getAndAdd(cost);
+  }
+
+  public void releaseStorageGroupMemCost(long cost) {
+    memoryCost.getAndAdd(-cost);
+  }
+
+  public long getSgMemCost() {
+    return memoryCost.get();
+  }
+
+  public Set<TsFileProcessor> getAllReportedTsp() {
+    return reportedTsps;
+  }
+
+  public boolean needToReportToSystem() {
+    return memoryCost.get() - lastReportedSize > storageGroupSizeReportThreshold;
+  }
+
+  public void setLastReportedSize(long size) {
+    lastReportedSize = size;
+  }
+
+  /**
+   * When a TsFileProcessor is closing, remove it from reportedTsps, and report to systemInfo
+   * to update SG cost.
+   * 
+   * @param tsFileProcessor
+   */
+  public void closeTsFileProcessorAndReportToSystem(TsFileProcessor tsFileProcessor) {
+    reportedTsps.remove(tsFileProcessor);
+    SystemInfo.getInstance().resetStorageGroupStatus(this);
+  }
+}
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 4d72060..a263a23 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
@@ -43,6 +43,7 @@ import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
@@ -84,6 +85,7 @@ import org.apache.iotdb.db.writelog.recover.TsFileRecoverPerformer;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -127,6 +129,8 @@ public class StorageGroupProcessor {
 
   private static final Logger logger = LoggerFactory.getLogger(StorageGroupProcessor.class);
 
+  private final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private final boolean enableMemControl = config.isEnableMemControl();
   /**
    * indicating the file to be loaded already exists locally.
    */
@@ -236,6 +240,8 @@ public class StorageGroupProcessor {
    */
   private Map<Long, Long> partitionMaxFileVersions = new HashMap<>();
 
+  private StorageGroupInfo storageGroupInfo = new StorageGroupInfo(this);
+
   public boolean isReady() {
     return isReady;
   }
@@ -339,7 +345,7 @@ public class StorageGroupProcessor {
         mergingMods.delete();
       }
 
-      updateLastestFlushedTime();
+      updateLatestFlushedTime();
     } catch (IOException | MetadataException e) {
       throw new StorageGroupProcessorException(e);
     }
@@ -374,7 +380,7 @@ public class StorageGroupProcessor {
    * partitionLatestFlushedTimeForEachDevice and timePartitionIdVersionControllerMap
    */
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void updateLastestFlushedTime() throws IOException {
+  private void updateLatestFlushedTime() throws IOException {
 
     VersionController versionController = new SimpleFileVersionController(
         storageGroupSysDir.getPath());
@@ -573,22 +579,48 @@ public class StorageGroupProcessor {
         // the last file is not closed, continue writing to in
         TsFileProcessor tsFileProcessor;
         if (isSeq) {
-          tsFileProcessor = new TsFileProcessor(storageGroupName, tsFileResource,
+          tsFileProcessor = new TsFileProcessor(storageGroupName, storageGroupInfo, tsFileResource,
               getVersionControllerByTimePartitionId(timePartitionId),
               this::closeUnsealedTsFileProcessorCallBack, this::updateLatestFlushTimeCallback,
               true, writer);
+          if (enableMemControl) {
+            TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(storageGroupInfo);
+            tsFileProcessor.setTsFileProcessorInfo(tsFileProcessorInfo);
+            this.storageGroupInfo.initTsFileProcessorInfo(tsFileProcessor);
+            tsFileProcessorInfo.addTSPMemCost(tsFileProcessor
+                .getTsFileResource().calculateRamSize());
+          }
           workSequenceTsFileProcessors.put(timePartitionId, tsFileProcessor);
         } else {
-          tsFileProcessor = new TsFileProcessor(storageGroupName, tsFileResource,
+          tsFileProcessor = new TsFileProcessor(storageGroupName, storageGroupInfo, tsFileResource,
               getVersionControllerByTimePartitionId(timePartitionId),
               this::closeUnsealedTsFileProcessorCallBack, this::unsequenceFlushCallback, false,
               writer);
+          if (enableMemControl) {
+            TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(storageGroupInfo);
+            tsFileProcessor.setTsFileProcessorInfo(tsFileProcessorInfo);
+            this.storageGroupInfo.initTsFileProcessorInfo(tsFileProcessor);
+            tsFileProcessorInfo.addTSPMemCost(tsFileProcessor
+                .getTsFileResource().calculateRamSize());
+          }
           workUnsequenceTsFileProcessors.put(timePartitionId, tsFileProcessor);
         }
         tsFileResource.setProcessor(tsFileProcessor);
         tsFileResource.removeResourceFile();
         tsFileProcessor.setTimeRangeId(timePartitionId);
         writer.makeMetadataVisible();
+        if (enableMemControl) {
+          // get chunkMetadata size
+          long chunkMetadataSize = 0;
+          for (Map<String, List<ChunkMetadata>> metaMap : writer.getMetadatasForQuery().values()) {
+            for (List<ChunkMetadata> metadatas : metaMap.values()) {
+              for (ChunkMetadata chunkMetadata: metadatas) {
+                chunkMetadataSize += chunkMetadata.calculateRamSize();
+              }
+            }
+          }
+          tsFileProcessor.getTsFileProcessorInfo().addTSPMemCost(chunkMetadataSize);
+        }
       }
       tsFileManagement.add(tsFileResource, isSeq);
     }
@@ -967,14 +999,28 @@ public class StorageGroupProcessor {
     VersionController versionController = getVersionControllerByTimePartitionId(timePartitionId);
     if (sequence) {
       tsFileProcessor = new TsFileProcessor(storageGroupName,
-          fsFactory.getFileWithParent(filePath),
+          fsFactory.getFileWithParent(filePath), storageGroupInfo,
           versionController, this::closeUnsealedTsFileProcessorCallBack,
           this::updateLatestFlushTimeCallback, true);
+      if (enableMemControl) {
+        TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(storageGroupInfo);
+        tsFileProcessor.setTsFileProcessorInfo(tsFileProcessorInfo);
+        this.storageGroupInfo.initTsFileProcessorInfo(tsFileProcessor);
+        tsFileProcessorInfo.addTSPMemCost(tsFileProcessor
+            .getTsFileResource().calculateRamSize());
+      }
     } else {
       tsFileProcessor = new TsFileProcessor(storageGroupName,
-          fsFactory.getFileWithParent(filePath),
+          fsFactory.getFileWithParent(filePath), storageGroupInfo,
           versionController, this::closeUnsealedTsFileProcessorCallBack,
           this::unsequenceFlushCallback, false);
+      if (enableMemControl) {
+        TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(storageGroupInfo);
+        tsFileProcessor.setTsFileProcessorInfo(tsFileProcessorInfo);
+        this.storageGroupInfo.initTsFileProcessorInfo(tsFileProcessor);
+        tsFileProcessorInfo.addTSPMemCost(tsFileProcessor
+            .getTsFileResource().calculateRamSize());
+      }
     }
     tsFileProcessor.addCloseFileListeners(customCloseFileListeners);
     tsFileProcessor.addFlushListeners(customFlushListeners);
@@ -2272,7 +2318,7 @@ public class StorageGroupProcessor {
   }
 
 
-  public Collection<TsFileProcessor> getWorkUnsequenceTsFileProcessor() {
+  public Collection<TsFileProcessor> getWorkUnsequenceTsFileProcessors() {
     return workUnsequenceTsFileProcessors.values();
   }
 
@@ -2293,6 +2339,10 @@ public class StorageGroupProcessor {
     return storageGroupName;
   }
 
+  public StorageGroupInfo getStorageGroupInfo() {
+    return storageGroupInfo;
+  }
+
   /**
    * Check if the data of "tsFileResource" all exist locally by comparing the historical versions in
    * the partition of "partitionNumber". This is available only when the IoTDB instances which
@@ -2316,7 +2366,7 @@ public class StorageGroupProcessor {
         return false;
       }
     }
-    for (TsFileProcessor workUnsequenceTsFileProcessor : getWorkUnsequenceTsFileProcessor()) {
+    for (TsFileProcessor workUnsequenceTsFileProcessor : getWorkUnsequenceTsFileProcessors()) {
       long workingFileVersion = workUnsequenceTsFileProcessor.getTsFileResource().getMaxVersion();
       if (tsFileResource.getHistoricalVersions().contains(workingFileVersion)) {
         return false;
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 3cc67ba..6af591b 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
@@ -18,8 +18,6 @@
  */
 package org.apache.iotdb.db.engine.storagegroup;
 
-import static org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter.MEMTABLE_NUM_FOR_EACH_PARTITION;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -29,20 +27,21 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.conf.adapter.CompressionRatio;
-import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
 import org.apache.iotdb.db.engine.flush.CloseFileListener;
 import org.apache.iotdb.db.engine.flush.FlushListener;
 import org.apache.iotdb.db.engine.flush.FlushManager;
 import org.apache.iotdb.db.engine.flush.MemTableFlushTask;
 import org.apache.iotdb.db.engine.flush.NotifyFlushMemTable;
 import org.apache.iotdb.db.engine.memtable.IMemTable;
+import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
@@ -57,8 +56,11 @@ import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.rescon.MemTablePool;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.db.rescon.SystemInfo;
+import org.apache.iotdb.db.utils.MemUtils;
 import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.db.utils.datastructure.TVList;
 import org.apache.iotdb.db.writelog.WALFlushListener;
 import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
 import org.apache.iotdb.db.writelog.node.WriteLogNode;
@@ -68,6 +70,7 @@ import org.apache.iotdb.service.rpc.thrift.TSStatus;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -75,9 +78,15 @@ import org.slf4j.LoggerFactory;
 public class TsFileProcessor {
 
   private static final Logger logger = LoggerFactory.getLogger(TsFileProcessor.class);
+
   private final String storageGroupName;
 
   private final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private final boolean enableMemControl = config.isEnableMemControl();
+  private final int waitingTimeWhenInsertBlocked = config.getWaitingTimeWhenInsertBlocked();
+  private final int maxWaitingTimeWhenInsertBlocked = config.getMaxWaitingTimeWhenInsertBlocked();
+  private StorageGroupInfo storageGroupInfo;
+  private TsFileProcessorInfo tsFileProcessorInfo;
 
   /**
    * sync this object in query() and asyncTryToFlush()
@@ -108,6 +117,7 @@ public class TsFileProcessor {
   private WriteLogNode logNode;
   private final boolean sequence;
   private long totalMemTableSize;
+  private boolean shouldFlush = false;
 
   private static final String FLUSH_QUERY_WRITE_LOCKED = "{}: {} get flushQueryLock write lock";
   private static final String FLUSH_QUERY_WRITE_RELEASE = "{}: {} get flushQueryLock write lock released";
@@ -116,12 +126,16 @@ public class TsFileProcessor {
   private List<FlushListener> flushListeners = new ArrayList<>();
 
   TsFileProcessor(String storageGroupName, File tsfile,
+      StorageGroupInfo storageGroupInfo,
       VersionController versionController,
       CloseFileListener closeTsFileCallback,
       UpdateEndTimeCallBack updateLatestFlushTimeCallback, boolean sequence)
       throws IOException {
     this.storageGroupName = storageGroupName;
     this.tsFileResource = new TsFileResource(tsfile, this);
+    if (enableMemControl) {
+      this.storageGroupInfo = storageGroupInfo;
+    }
     this.versionController = versionController;
     this.writer = new RestorableTsFileIOWriter(tsfile);
     this.updateLatestFlushTimeCallback = updateLatestFlushTimeCallback;
@@ -134,12 +148,15 @@ public class TsFileProcessor {
     closeFileListeners.add(closeTsFileCallback);
   }
 
-  public TsFileProcessor(String storageGroupName, TsFileResource tsFileResource,
+  public TsFileProcessor(String storageGroupName, StorageGroupInfo storageGroupInfo, TsFileResource tsFileResource,
       VersionController versionController, CloseFileListener closeUnsealedTsFileProcessor,
       UpdateEndTimeCallBack updateLatestFlushTimeCallback, boolean sequence,
       RestorableTsFileIOWriter writer) {
     this.storageGroupName = storageGroupName;
     this.tsFileResource = tsFileResource;
+    if (enableMemControl) {
+      this.storageGroupInfo = storageGroupInfo;
+    }
     this.versionController = versionController;
     this.writer = writer;
     this.updateLatestFlushTimeCallback = updateLatestFlushTimeCallback;
@@ -157,12 +174,14 @@ public class TsFileProcessor {
   public void insert(InsertRowPlan insertRowPlan) throws WriteProcessException {
 
     if (workMemTable == null) {
-      workMemTable = MemTablePool.getInstance().getAvailableMemTable(this);
+      workMemTable = new PrimitiveMemTable(enableMemControl);
+    }
+    if (enableMemControl) {
+      blockInsertionIfReject();
+      checkMemCostAndAddToTspInfo(insertRowPlan);
     }
 
-    // insert insertRowPlan to the work memtable
     workMemTable.insert(insertRowPlan);
-
     if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
       try {
         getLogNode().write(insertRowPlan);
@@ -197,10 +216,13 @@ public class TsFileProcessor {
       TSStatus[] results) throws WriteProcessException {
 
     if (workMemTable == null) {
-      workMemTable = MemTablePool.getInstance().getAvailableMemTable(this);
+      workMemTable = new PrimitiveMemTable(enableMemControl);
+    }
+    if (enableMemControl) {
+      blockInsertionIfReject();
+      checkMemCostAndAddToTspInfo(insertTabletPlan, start, end);
     }
 
-    // insert insertRowPlan to the work memtable
     try {
       workMemTable.insertTablet(insertTabletPlan, start, end);
       if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
@@ -214,15 +236,12 @@ public class TsFileProcessor {
       }
       throw new WriteProcessException(e);
     }
-
     for (int i = start; i < end; i++) {
       results[i] = RpcUtils.SUCCESS_STATUS;
     }
-
     tsFileResource
         .updateStartTime(insertTabletPlan.getDeviceId().getFullPath(),
             insertTabletPlan.getTimes()[start]);
-
     //for sequence tsfile, we update the endTime only when the file is prepared to be closed.
     //for unsequence tsfile, we have to update the endTime for each insertion.
     if (!sequence) {
@@ -233,6 +252,134 @@ public class TsFileProcessor {
     tsFileResource.updatePlanIndexes(insertTabletPlan.getIndex());
   }
 
+  private void checkMemCostAndAddToTspInfo(InsertRowPlan insertRowPlan) throws WriteProcessException {
+    // memory of increased PrimitiveArray and TEXT values, e.g., add a long[128], add 128*8
+    long memTableIncrement = 0L;
+    long textDataIncrement = 0L;
+    long chunkMetadataIncrement = 0L;
+    String deviceId = insertRowPlan.getDeviceId().getFullPath();
+    long unsealedResourceIncrement = 
+        tsFileResource.estimateRamIncrement(deviceId);
+    for (int i = 0; i < insertRowPlan.getDataTypes().length; i++) {
+      // skip failed Measurements
+      if (insertRowPlan.getDataTypes()[i] == null) {
+        continue;
+      }
+      if (workMemTable.checkIfChunkDoesNotExist(deviceId, insertRowPlan.getMeasurements()[i])) {
+        // ChunkMetadataIncrement
+        chunkMetadataIncrement += ChunkMetadata.calculateRamSize(insertRowPlan.getMeasurements()[i],
+            insertRowPlan.getDataTypes()[i]);
+        memTableIncrement += TVList.tvListArrayMemSize(insertRowPlan.getDataTypes()[i]);
+      }
+      else {
+        // here currentChunkPointNum >= 1
+        int currentChunkPointNum = workMemTable.getCurrentChunkPointNum(deviceId,
+            insertRowPlan.getMeasurements()[i]);
+        memTableIncrement += (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE) == 0 ? TVList
+            .tvListArrayMemSize(insertRowPlan.getDataTypes()[i]) : 0;
+      }
+      // TEXT data mem size
+      if (insertRowPlan.getDataTypes()[i] == TSDataType.TEXT) {
+        textDataIncrement += MemUtils.getBinarySize((Binary) insertRowPlan.getValues()[i]);
+      }
+    }
+    memTableIncrement += textDataIncrement;
+    storageGroupInfo.addStorageGroupMemCost(memTableIncrement);
+    tsFileProcessorInfo.addTSPMemCost(unsealedResourceIncrement + chunkMetadataIncrement);
+    if (storageGroupInfo.needToReportToSystem()) {
+      SystemInfo.getInstance().reportStorageGroupStatus(storageGroupInfo);
+      try {
+        blockInsertionIfReject();
+      } catch (WriteProcessException e) {
+        storageGroupInfo.releaseStorageGroupMemCost(memTableIncrement);
+        tsFileProcessorInfo.releaseTSPMemCost(unsealedResourceIncrement + chunkMetadataIncrement);
+        SystemInfo.getInstance().resetStorageGroupStatus(storageGroupInfo);
+        throw e;
+      }
+    }
+    workMemTable.addTVListRamCost(memTableIncrement);
+    workMemTable.addTextDataSize(textDataIncrement);
+  }
+
+  private void checkMemCostAndAddToTspInfo(InsertTabletPlan insertTabletPlan, int start, int end)
+      throws WriteProcessException {
+    if (start >= end) {
+      return;
+    }
+    long memTableIncrement = 0L;
+    long textDataIncrement = 0L;
+    long chunkMetadataIncrement = 0L;
+    String deviceId = insertTabletPlan.getDeviceId().getFullPath();
+    long unsealedResourceIncrement = tsFileResource.estimateRamIncrement(deviceId);
+
+    for (int i = 0; i < insertTabletPlan.getDataTypes().length; i++) {
+      // skip failed Measurements
+      TSDataType dataType = insertTabletPlan.getDataTypes()[i];
+      String measurement = insertTabletPlan.getMeasurements()[i];
+      if (dataType == null) {
+        continue;
+      }
+
+      if (workMemTable.checkIfChunkDoesNotExist(deviceId, measurement)) {
+        // ChunkMetadataIncrement
+        chunkMetadataIncrement += ChunkMetadata.calculateRamSize(measurement, dataType);
+        memTableIncrement += ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1)
+            * TVList.tvListArrayMemSize(dataType);
+      }
+      else {
+        int currentChunkPointNum = workMemTable
+            .getCurrentChunkPointNum(deviceId, measurement);
+        if (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE == 0) {
+          memTableIncrement += ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1)
+              * TVList.tvListArrayMemSize(dataType);
+        }
+        else {
+          int acquireArray =
+              (end - start - 1 + (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE))
+                  / PrimitiveArrayManager.ARRAY_SIZE;
+          memTableIncrement += acquireArray == 0 ? 0
+              : acquireArray * TVList.tvListArrayMemSize(dataType);
+        }
+      }
+      // TEXT data size
+      if (dataType == TSDataType.TEXT) {
+        Binary[] column = (Binary[]) insertTabletPlan.getColumns()[i];
+        textDataIncrement += MemUtils.getBinaryColumnSize(column, start, end);
+      }
+    }
+    memTableIncrement += textDataIncrement;
+    storageGroupInfo.addStorageGroupMemCost(memTableIncrement);
+    tsFileProcessorInfo.addTSPMemCost(unsealedResourceIncrement + chunkMetadataIncrement);
+    if (storageGroupInfo.needToReportToSystem()) {
+      SystemInfo.getInstance().reportStorageGroupStatus(storageGroupInfo);
+      try {
+        blockInsertionIfReject();
+      } catch (WriteProcessException e) {
+        storageGroupInfo.releaseStorageGroupMemCost(memTableIncrement);
+        tsFileProcessorInfo.releaseTSPMemCost(unsealedResourceIncrement + chunkMetadataIncrement);
+        SystemInfo.getInstance().resetStorageGroupStatus(storageGroupInfo);
+        throw e;
+      }
+    }
+    workMemTable.addTVListRamCost(memTableIncrement);
+    workMemTable.addTextDataSize(textDataIncrement);
+  }
+
+  private void blockInsertionIfReject() throws WriteProcessException {
+    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");
+        }
+      } catch (InterruptedException e) {
+        logger.error("Failed when waiting for getting memory for insertion ", e);
+        Thread.currentThread().interrupt();
+      }
+    }
+  }
+
   /**
    * Delete data which belongs to the timeseries `deviceId.measurementId` and the timestamp of which
    * <= 'timestamp' in the deletion. <br/>
@@ -269,46 +416,29 @@ public class TsFileProcessor {
     return tsFileResource;
   }
 
-
-  boolean shouldFlush() {
+  public boolean shouldFlush() {
     if (workMemTable == null) {
       return false;
     }
-
-    if (workMemTable.memSize() >= getMemtableSizeThresholdBasedOnSeriesNum()) {
+    if (shouldFlush) {
+      return true;
+    }
+    if (!enableMemControl && workMemTable.memSize() >= getMemtableSizeThresholdBasedOnSeriesNum()) {
       logger.info("The memtable size {} of tsfile {} reaches the threshold",
           workMemTable.memSize(), tsFileResource.getTsFile().getAbsolutePath());
       return true;
     }
-
     if (workMemTable.reachTotalPointNumThreshold()) {
       logger.info("The avg series points num {} of tsfile {} reaches the threshold",
           workMemTable.getTotalPointsNum() / workMemTable.getSeriesNumber(),
           tsFileResource.getTsFile().getAbsolutePath());
       return true;
     }
-
     return false;
   }
 
-  /**
-   * <p>In the dynamic parameter adjustment module{@link IoTDBConfigDynamicAdapter}, it calculated
-   * the average size of each metatable{@link IoTDBConfigDynamicAdapter#tryToAdaptParameters()}.
-   * However, considering that the number of timeseries between storage groups may vary greatly,
-   * it's inappropriate to judge whether to flush the memtable according to the average memtable
-   * size. We need to adjust it according to the number of timeseries in a specific storage group.
-   */
-  @SuppressWarnings("squid:S2184") //Suppress math operands cast warning
   private long getMemtableSizeThresholdBasedOnSeriesNum() {
-    if (!config.isEnableParameterAdapter()) {
-      return config.getMemtableSizeThreshold();
-    }
-    long memTableSize = (long) (config.getMemtableSizeThreshold() * config.getMaxMemtableNumber()
-        / IoTDBDescriptor.getInstance().getConfig().getConcurrentWritingTimePartition()
-        / MEMTABLE_NUM_FOR_EACH_PARTITION
-        * ActiveTimeSeriesCounter.getInstance()
-        .getActiveRatio(storageGroupName));
-    return Math.max(memTableSize, config.getMemtableSizeThreshold());
+    return config.getMemtableSizeThreshold();
   }
 
   public boolean shouldClose() {
@@ -344,7 +474,6 @@ public class TsFileProcessor {
                 FlushManager.getInstance()
             );
           }
-
         }
       } catch (InterruptedException e) {
         logger.error("{}: {} wait close interrupted", storageGroupName,
@@ -389,13 +518,15 @@ public class TsFileProcessor {
       // is set true, we need to generate a NotifyFlushMemTable as a signal task and submit it to
       // the FlushManager.
 
-      //we have to add the memtable into flushingList first and then set the shouldClose tag.
+      // we have to add the memtable into flushingList first and then set the shouldClose tag.
       // see https://issues.apache.org/jira/browse/IOTDB-510
-      IMemTable tmpMemTable = workMemTable == null || workMemTable.memSize() == 0
-          ? new NotifyFlushMemTable()
+      IMemTable tmpMemTable = workMemTable == null || workMemTable.memSize() == 0 
+          ? new NotifyFlushMemTable() 
           : workMemTable;
 
       try {
+        // When invoke closing TsFile after insert data to memTable, we shouldn't flush until invoke
+        // flushing memTable in System module.
         addAMemtableIntoFlushingList(tmpMemTable);
         shouldClose = true;
         tsFileResource.setCloseFlag();
@@ -473,7 +604,7 @@ public class TsFileProcessor {
       }
       addAMemtableIntoFlushingList(workMemTable);
     } catch (Exception e) {
-      logger.error("{}: {} add a memtable into flushing listfailed", storageGroupName,
+      logger.error("{}: {} add a memtable into flushing list failed", storageGroupName,
           tsFileResource.getTsFile().getName(), e);
     } finally {
       flushQueryLock.writeLock().unlock();
@@ -515,6 +646,7 @@ public class TsFileProcessor {
       totalMemTableSize += tobeFlushed.memSize();
     }
     workMemTable = null;
+    shouldFlush = false;
     FlushManager.getInstance().registerTsFileProcessor(this);
   }
 
@@ -541,7 +673,13 @@ public class TsFileProcessor {
             memTable.isSignalMemTable(), flushingMemTables.size());
       }
       memTable.release();
-      MemTablePool.getInstance().putBack(memTable, storageGroupName);
+      if (enableMemControl) {
+        // For text type data, reset the mem cost in tsFileProcessorInfo
+        storageGroupInfo.releaseStorageGroupMemCost(memTable.getTVListsRamCost());
+        // report to System
+        SystemInfo.getInstance().resetStorageGroupStatus(storageGroupInfo);
+      }
+      memTable = null;
       if (logger.isDebugEnabled()) {
         logger.debug("{}: {} flush finished, remove a memtable from flushing list, "
                 + "flushing memtable list size: {}", storageGroupName,
@@ -676,6 +814,10 @@ public class TsFileProcessor {
       closeFileListener.onClosed(this);
     }
 
+    if (enableMemControl) {
+      tsFileProcessorInfo.clear();
+      storageGroupInfo.closeTsFileProcessorAndReportToSystem(this);
+    }
     if (logger.isInfoEnabled()) {
       long closeEndTime = System.currentTimeMillis();
       logger.info("Storage group {} close the file {}, TsFile size is {}, "
@@ -688,7 +830,6 @@ public class TsFileProcessor {
     writer = null;
   }
 
-
   public boolean isManagedByFlushManager() {
     return managedByFlushManager;
   }
@@ -807,7 +948,7 @@ public class TsFileProcessor {
   public void putMemTableBackAndClose() throws TsFileProcessorException {
     if (workMemTable != null) {
       workMemTable.release();
-      MemTablePool.getInstance().putBack(workMemTable, storageGroupName);
+      workMemTable = null;
     }
     try {
       writer.close();
@@ -816,6 +957,32 @@ public class TsFileProcessor {
     }
   }
 
+  public TsFileProcessorInfo getTsFileProcessorInfo() {
+    return tsFileProcessorInfo;
+  }
+
+  public void setTsFileProcessorInfo(TsFileProcessorInfo tsFileProcessorInfo) {
+    this.tsFileProcessorInfo = tsFileProcessorInfo;
+  }
+
+  public long getWorkMemTableRamCost() {
+    return workMemTable != null ? workMemTable.getTVListsRamCost() : 0;
+  }
+
+  public boolean isSequence() {
+    return sequence;
+  }
+
+  public void startClose() {
+    storageGroupInfo.getStorageGroupProcessor().asyncCloseOneTsFileProcessor(sequence, this);
+    logger.info("Async close tsfile: {}",
+        getTsFileResource().getTsFile().getAbsolutePath());
+  }
+
+  public void setFlush() {
+    shouldFlush = true;
+  }
+
   public void addFlushListener(FlushListener listener) {
     flushListeners.add(listener);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorInfo.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorInfo.java
new file mode 100644
index 0000000..52bc863
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorInfo.java
@@ -0,0 +1,65 @@
+/*
+ * 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.engine.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+
+/**
+ * The TsFileProcessorInfo records the memory cost of this TsFileProcessor.
+ */
+public class TsFileProcessorInfo {
+
+  /**
+   * Once tspInfo updated, report to storageGroupInfo that this TSP belongs to.
+   */
+  private StorageGroupInfo storageGroupInfo;
+
+  // unsealed TsFileResource, ChunkMetadata, WAL
+  private long memCost;
+
+
+  public TsFileProcessorInfo(StorageGroupInfo storageGroupInfo) {
+    this.storageGroupInfo = storageGroupInfo;
+    this.memCost = IoTDBDescriptor.getInstance().getConfig().getWalBufferSize();
+  }
+
+  /**
+   * called in each insert
+   */
+  public void addTSPMemCost(long cost) {
+    memCost += cost;
+    storageGroupInfo.addStorageGroupMemCost(cost);
+  }
+
+  /**
+   * called when meet exception
+   */
+  public void releaseTSPMemCost(long cost) {
+    storageGroupInfo.releaseStorageGroupMemCost(cost);
+    memCost -= cost;
+  }
+
+  /**
+   * called when closing TSP
+   */
+  public void clear() {
+    storageGroupInfo.releaseStorageGroupMemCost(memCost);
+    memCost = 0;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
index b295e90..58f37b3 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
@@ -55,6 +55,7 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+import org.apache.iotdb.tsfile.utils.RamUsageEstimator;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -833,6 +834,36 @@ public class TsFileResource {
     return maxVersion;
   }
 
+  /**
+   * @return initial resource map size
+   */
+  public long calculateRamSize() {
+    return RamUsageEstimator.sizeOf(deviceToIndex) + RamUsageEstimator.sizeOf(startTimes) + 
+        RamUsageEstimator.sizeOf(endTimes);
+  }
+
+  /**
+   * Calculate the resource ram increment when insert data in TsFileProcessor
+   * 
+   * @return ramIncrement
+   */
+  public long estimateRamIncrement(String deviceToBeChecked) {
+    long ramIncrement = 0L;
+    if (!containsDevice(deviceToBeChecked)) {
+      // 80 is the Map.Entry header ram size
+      if (deviceToIndex.isEmpty()) {
+        ramIncrement += 80;
+      }
+      // Map.Entry ram size
+      ramIncrement += RamUsageEstimator.sizeOf(deviceToBeChecked) + 16;
+      // if needs to extend the startTimes and endTimes arrays
+      if (deviceToIndex.size() >= startTimes.length) {
+        ramIncrement += startTimes.length * Long.BYTES;
+      }
+    }
+    return ramIncrement;
+  }
+
   public void delete() throws IOException {
     if (file.exists()) {
       Files.delete(file.toPath());
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/ConfigAdjusterException.java b/server/src/main/java/org/apache/iotdb/db/exception/ConfigAdjusterException.java
deleted file mode 100644
index 62f6dd6..0000000
--- a/server/src/main/java/org/apache/iotdb/db/exception/ConfigAdjusterException.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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 ConfigAdjusterException extends IoTDBException {
-
-  private static final long serialVersionUID = 3502288856999147687L;
-  public static final String ERROR_MSG_FORMAT = "IoTDB system load is too large to %s, please increase memory or disable the enable_parameter_adapter in iotdb-engine.properties";
-
-  public ConfigAdjusterException(String action) {
-    super(String.format(ERROR_MSG_FORMAT, action), TSStatusCode.CONFIG_ADJUSTER.getStatusCode());
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 728b950..7c34f5c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -39,16 +39,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
-import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
-import org.apache.iotdb.db.exception.ConfigAdjusterException;
 import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
 import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
@@ -70,9 +70,9 @@ import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.utils.RandomDeleteCache;
 import org.apache.iotdb.db.utils.SchemaUtils;
-import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.db.utils.TypeInferenceUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.exception.cache.CacheException;
@@ -100,6 +100,8 @@ public class MManager {
   private static final String DEBUG_MSG_1 = "%s: TimeSeries %s's tag info has been removed from tag inverted index ";
   private static final String PREVIOUS_CONDITION = "before deleting it, tag key is %s, tag value is %s, tlog offset is %d, contains key %b";
 
+  private static final int UPDATE_SCHEMA_MAP_IN_ARRAYPOOL_THRESHOLD = 5000;
+
   private static final Logger logger = LoggerFactory.getLogger(MManager.class);
 
   /**
@@ -108,7 +110,6 @@ public class MManager {
   private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
   private final int mtreeSnapshotInterval;
   private final long mtreeSnapshotThresholdTime;
-  protected IoTDBConfig config;
   // the log file seriesPath
   private String logFilePath;
   private String mtreeSnapshotPath;
@@ -121,15 +122,37 @@ public class MManager {
   private RandomDeleteCache<PartialPath, MNode> mNodeCache;
   // tag key -> tag value -> LeafMNode
   private Map<String, Map<String, Set<MeasurementMNode>>> tagIndex = new HashMap<>();
-  // storage group name -> the series number
-  private Map<String, Integer> seriesNumberInStorageGroups = new HashMap<>();
-  private long maxSeriesNumberAmongStorageGroup;
+
+  // data type -> number
+  private Map<TSDataType, Integer> schemaDataTypeNumMap = new ConcurrentHashMap<>();
+  // reported total series number
+  private long reportedDataTypeTotalNum;
+  private AtomicLong totalSeriesNumber = new AtomicLong();
   private boolean initialized;
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
   private File logFile;
   private ScheduledExecutorService timedCreateMTreeSnapshotThread;
 
+  /**
+   * threshold total size of MTree
+   */
+  private static final long MTREE_SIZE_THRESHOLD = config.getAllocateMemoryForSchema();
+
+  private boolean allowToCreateNewSeries = true;
+
+  private static final int ESTIMATED_SERIES_SIZE = config.getEstimatedSeriesSize();
+
+  private static class MManagerHolder {
+
+    private MManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final MManager INSTANCE = new MManager();
+  }
+
   protected MManager() {
-    config = IoTDBDescriptor.getInstance().getConfig();
     mtreeSnapshotInterval = config.getMtreeSnapshotInterval();
     mtreeSnapshotThresholdTime = config.getMtreeSnapshotThresholdTime() * 1000L;
     String schemaDir = config.getSchemaDir();
@@ -190,15 +213,10 @@ public class MManager {
 
       isRecovering = true;
       int lineNumber = initFromLog(logFile);
-
-      if (config.isEnableParameterAdapter()) {
-        List<PartialPath> storageGroups = mtree.getAllStorageGroupPaths();
-        for (PartialPath sg : storageGroups) {
-          MNode node = mtree.getNodeByPath(sg);
-          seriesNumberInStorageGroups.put(sg.getFullPath(), node.getLeafCount());
-        }
-        maxSeriesNumberAmongStorageGroup =
-            seriesNumberInStorageGroups.values().stream().max(Integer::compareTo).orElse(0);
+      List<PartialPath> storageGroups = mtree.getAllStorageGroupPaths();
+      for (PartialPath sg : storageGroups) {
+        MNode node = mtree.getNodeByPath(sg);
+        totalSeriesNumber.addAndGet(node.getLeafCount());
       }
 
       logWriter = new MLogWriter(config.getSchemaDir(), MetadataConstant.METADATA_LOG);
@@ -208,6 +226,7 @@ public class MManager {
       mtree = new MTree();
       logger.error("Cannot read MTree from file, using an empty new one", e);
     }
+    reportedDataTypeTotalNum = 0L;
     initialized = true;
   }
 
@@ -266,8 +285,7 @@ public class MManager {
       this.mtree = new MTree();
       this.mNodeCache.clear();
       this.tagIndex.clear();
-      this.seriesNumberInStorageGroups.clear();
-      this.maxSeriesNumberAmongStorageGroup = 0;
+      this.totalSeriesNumber.set(0);
       if (logWriter != null) {
         logWriter.close();
         logWriter = null;
@@ -276,6 +294,8 @@ public class MManager {
         tagLogFile.close();
         tagLogFile = null;
       }
+      this.schemaDataTypeNumMap.clear();
+      this.reportedDataTypeTotalNum = 0L;
       initialized = false;
       if (config.isEnableMTreeSnapshot() && timedCreateMTreeSnapshotThread != null) {
         timedCreateMTreeSnapshotThread.shutdownNow();
@@ -359,30 +379,29 @@ public class MManager {
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public void createTimeseries(CreateTimeSeriesPlan plan, long offset) throws MetadataException {
+    if (!allowToCreateNewSeries) {
+      throw new MetadataException("IoTDB system load is too large to create timeseries, "
+          + "please increase MAX_HEAP_SIZE in iotdb-env.sh/bat and restart");
+    }
     try {
       PartialPath path = plan.getPath();
       SchemaUtils.checkDataTypeWithEncoding(plan.getDataType(), plan.getEncoding());
-      /*
-       * get the storage group with auto create schema
-       */
-      PartialPath storageGroupPath;
+
       try {
-        storageGroupPath = mtree.getStorageGroupPath(path);
+        mtree.getStorageGroupPath(path);
       } catch (StorageGroupNotSetException e) {
         if (!config.isAutoCreateSchemaEnabled()) {
           throw e;
         }
-        storageGroupPath =
+        PartialPath storageGroupPath =
             MetaUtils.getStorageGroupPathByLevel(path, config.getDefaultStorageGroupLevel());
         setStorageGroup(storageGroupPath);
       }
 
-      // check memory
-      IoTDBConfigDynamicAdapter.getInstance().addOrDeleteTimeSeries(1);
-
+      TSDataType type = plan.getDataType();
       // create time series in MTree
       MeasurementMNode leafMNode = mtree
-          .createTimeseries(path, plan.getDataType(), plan.getEncoding(), plan.getCompressor(),
+          .createTimeseries(path, type, plan.getEncoding(), plan.getCompressor(),
               plan.getProps(), plan.getAlias());
 
       // update tag index
@@ -394,14 +413,13 @@ public class MManager {
         }
       }
 
-      // update statistics
-      if (config.isEnableParameterAdapter()) {
-        int size = seriesNumberInStorageGroups.get(storageGroupPath.getFullPath());
-        seriesNumberInStorageGroups.put(storageGroupPath.getFullPath(), size + 1);
-        if (size + 1 > maxSeriesNumberAmongStorageGroup) {
-          maxSeriesNumberAmongStorageGroup = size + 1L;
-        }
+      // update statistics and schemaDataTypeNumMap
+      totalSeriesNumber.addAndGet(1);
+      if (totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE >= MTREE_SIZE_THRESHOLD) {
+        logger.warn("Current series number {} is too large...", totalSeriesNumber);
+        allowToCreateNewSeries = false;
       }
+      updateSchemaDataTypeNumMap(type, 1);
 
       // write log
       if (!isRecovering) {
@@ -413,7 +431,8 @@ public class MManager {
         logWriter.createTimeseries(plan, offset);
       }
       leafMNode.setOffset(offset);
-    } catch (IOException | ConfigAdjusterException e) {
+
+    } catch (IOException e) {
       throw new MetadataException(e.getMessage());
     }
   }
@@ -425,8 +444,6 @@ public class MManager {
    * @param dataType   the dateType {@code DataType} of the timeseries
    * @param encoding   the encoding function {@code Encoding} of the timeseries
    * @param compressor the compressor function {@code Compressor} of the time series
-   * @return whether the measurement occurs for the first time in this storage group (if true, the
-   * measurement should be registered to the StorageEngine too)
    */
   public void createTimeseries(PartialPath path, TSDataType dataType, TSEncoding encoding,
       CompressionType compressor, Map<String, String> props) throws MetadataException {
@@ -447,17 +464,6 @@ public class MManager {
    */
   public String deleteTimeseries(PartialPath prefixPath) throws MetadataException {
     if (isStorageGroup(prefixPath)) {
-
-      if (config.isEnableParameterAdapter()) {
-        int size = seriesNumberInStorageGroups.get(prefixPath.getFullPath());
-        seriesNumberInStorageGroups.put(prefixPath.getFullPath(), 0);
-        if (size == maxSeriesNumberAmongStorageGroup) {
-          seriesNumberInStorageGroups.values().stream()
-              .max(Integer::compareTo)
-              .ifPresent(val -> maxSeriesNumberAmongStorageGroup = val);
-        }
-      }
-
       mNodeCache.clear();
     }
     try {
@@ -500,9 +506,7 @@ public class MManager {
         if (tagIndex.containsKey(entry.getKey()) && tagIndex.get(entry.getKey())
             .containsKey(entry.getValue())) {
           if (logger.isDebugEnabled()) {
-            logger.debug(String.format(
-                DEBUG_MSG, "Delete"
-                    + TAG_FORMAT,
+            logger.debug(String.format(DEBUG_MSG, "Delete" + TAG_FORMAT,
                 node.getFullPath(), entry.getKey(), entry.getValue(), node.getOffset()));
           }
           tagIndex.get(entry.getKey()).get(entry.getValue()).remove(node);
@@ -514,9 +518,7 @@ public class MManager {
           }
         } else {
           if (logger.isDebugEnabled()) {
-            logger.debug(String.format(
-                DEBUG_MSG_1, "Delete"
-                    + PREVIOUS_CONDITION,
+            logger.debug(String.format(DEBUG_MSG_1, "Delete" + PREVIOUS_CONDITION,
                 node.getFullPath(), entry.getKey(), entry.getValue(), node.getOffset(),
                 tagIndex.containsKey(entry.getKey())));
           }
@@ -536,48 +538,33 @@ public class MManager {
     removeFromTagInvertedIndex(pair.right);
     PartialPath storageGroupPath = pair.left;
 
+    // update statistics in schemaDataTypeNumMap
+    updateSchemaDataTypeNumMap(pair.right.getSchema().getType(), -1);
+
     // TODO: delete the path node and all its ancestors
     mNodeCache.clear();
-    try {
-      IoTDBConfigDynamicAdapter.getInstance().addOrDeleteTimeSeries(-1);
-    } catch (ConfigAdjusterException e) {
-      throw new MetadataException(e);
-    }
-
-    if (config.isEnableParameterAdapter()) {
-      PartialPath storageGroup = getStorageGroupPath(path);
-      int size = seriesNumberInStorageGroups.get(storageGroup.getFullPath());
-      seriesNumberInStorageGroups.put(storageGroup.getFullPath(), size - 1);
-      if (size == maxSeriesNumberAmongStorageGroup) {
-        seriesNumberInStorageGroups.values().stream().max(Integer::compareTo)
-            .ifPresent(val -> maxSeriesNumberAmongStorageGroup = val);
-      }
+    totalSeriesNumber.addAndGet(-1);
+    if (!allowToCreateNewSeries && 
+        totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE < MTREE_SIZE_THRESHOLD) {
+      logger.info("Current series number {} come back to normal level", totalSeriesNumber);
+      allowToCreateNewSeries = true;
     }
     return storageGroupPath;
   }
 
   /**
-   * Set storage group of the given path to MTree. Check
+   * Set storage group of the given path to MTree.
    *
    * @param storageGroup root.node.(node)*
    */
   public void setStorageGroup(PartialPath storageGroup) throws MetadataException {
     try {
       mtree.setStorageGroup(storageGroup);
-      IoTDBConfigDynamicAdapter.getInstance().addOrDeleteStorageGroup(1);
-
-      if (config.isEnableParameterAdapter()) {
-        ActiveTimeSeriesCounter.getInstance().init(storageGroup.getFullPath());
-        seriesNumberInStorageGroups.put(storageGroup.getFullPath(), 0);
-      }
       if (!isRecovering) {
         logWriter.setStorageGroup(storageGroup.getFullPath());
       }
     } catch (IOException e) {
       throw new MetadataException(e.getMessage());
-    } catch (ConfigAdjusterException e) {
-      mtree.deleteStorageGroup(storageGroup);
-      throw new MetadataException(e);
     }
   }
 
@@ -589,40 +576,58 @@ public class MManager {
   public void deleteStorageGroups(List<PartialPath> storageGroups) throws MetadataException {
     try {
       for (PartialPath storageGroup : storageGroups) {
-
+        totalSeriesNumber.addAndGet(mtree.getAllTimeseriesCount(storageGroup));
         // clear cached MNode
+        if (!allowToCreateNewSeries && 
+            totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE < MTREE_SIZE_THRESHOLD) {
+          logger.info("Current series number {} come back to normal level", totalSeriesNumber);
+          allowToCreateNewSeries = true;
+        }
         mNodeCache.clear();
 
         // try to delete storage group
         List<MeasurementMNode> leafMNodes = mtree.deleteStorageGroup(storageGroup);
         for (MeasurementMNode leafMNode : leafMNodes) {
           removeFromTagInvertedIndex(leafMNode);
+          // update statistics in schemaDataTypeNumMap
+          updateSchemaDataTypeNumMap(leafMNode.getSchema().getType(), -1);
         }
 
-        if (config.isEnableParameterAdapter()) {
-          IoTDBConfigDynamicAdapter.getInstance().addOrDeleteStorageGroup(-1);
-          int size = seriesNumberInStorageGroups.get(storageGroup.getFullPath());
-          IoTDBConfigDynamicAdapter.getInstance().addOrDeleteTimeSeries(size * -1);
-          ActiveTimeSeriesCounter.getInstance().delete(storageGroup.getFullPath());
-          seriesNumberInStorageGroups.remove(storageGroup.getFullPath());
-          if (size == maxSeriesNumberAmongStorageGroup) {
-            maxSeriesNumberAmongStorageGroup =
-                seriesNumberInStorageGroups.values().stream().max(Integer::compareTo).orElse(0);
-          }
-        }
         // if success
         if (!isRecovering) {
           logWriter.deleteStorageGroup(storageGroup.getFullPath());
         }
       }
-    } catch (ConfigAdjusterException e) {
-      throw new MetadataException(e);
     } catch (IOException e) {
       throw new MetadataException(e.getMessage());
     }
   }
 
   /**
+   * update statistics in schemaDataTypeNumMap
+   *
+   * @param type data type
+   * @param num  1 for creating timeseries and -1 for deleting timeseries
+   */
+  private synchronized void updateSchemaDataTypeNumMap(TSDataType type, int num) {
+    // add an array of the series type
+    schemaDataTypeNumMap.put(type, schemaDataTypeNumMap.getOrDefault(type, 0) + num);
+    // add an array of time
+    schemaDataTypeNumMap.put(TSDataType.INT64,
+        schemaDataTypeNumMap.getOrDefault(TSDataType.INT64, 0) + num);
+
+    // total current DataType Total Num (twice of number of time series)
+    // used in primitive array manager
+    long currentDataTypeTotalNum = totalSeriesNumber.get() * 2;
+
+    if (num > 0 && currentDataTypeTotalNum - reportedDataTypeTotalNum
+        >= UPDATE_SCHEMA_MAP_IN_ARRAYPOOL_THRESHOLD) {
+      PrimitiveArrayManager.updateSchemaDataTypeNum(schemaDataTypeNumMap, currentDataTypeTotalNum);
+      reportedDataTypeTotalNum = currentDataTypeTotalNum;
+    }
+  }
+
+  /**
    * Check if the given path is storage group or not.
    *
    * @param path Format: root.node.(node)*
@@ -709,7 +714,7 @@ public class MManager {
 
   public List<PartialPath> searchAllRelatedStorageGroups(PartialPath path)
       throws MetadataException {
-      return mtree.searchAllRelatedStorageGroups(path);
+    return mtree.searchAllRelatedStorageGroups(path);
   }
 
   /**
@@ -990,7 +995,7 @@ public class MManager {
    */
   public MNode getDeviceNodeWithAutoCreate(
       PartialPath path, boolean autoCreateSchema, int sgLevel) throws MetadataException {
-    MNode node = null;
+    MNode node;
     boolean shouldSetStorageGroup;
     try {
       node = mNodeCache.get(path);
@@ -1065,15 +1070,6 @@ public class MManager {
     return TIME_SERIES_TREE_HEADER + mtree.toString();
   }
 
-  @TestOnly
-  public void setMaxSeriesNumberAmongStorageGroup(long maxSeriesNumberAmongStorageGroup) {
-    this.maxSeriesNumberAmongStorageGroup = maxSeriesNumberAmongStorageGroup;
-  }
-
-  public long getMaximalSeriesNumberAmongStorageGroups() {
-    return maxSeriesNumberAmongStorageGroup;
-  }
-
   public void setTTL(PartialPath storageGroup, long dataTTL) throws MetadataException, IOException {
     getStorageGroupNodeByStorageGroupPath(storageGroup).setDataTTL(dataTTL);
     if (!isRecovering) {
@@ -1860,13 +1856,4 @@ public class MManager {
     boolean satisfy(String storageGroup);
   }
 
-  private static class MManagerHolder {
-
-    private static final MManager INSTANCE = new MManager();
-
-    private MManagerHolder() {
-      // allowed to do nothing
-    }
-  }
-
 }
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 63adc95..8976733 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
@@ -126,8 +126,6 @@ public class SQLConstant {
   public static final int TOK_LOAD_CONFIGURATION = 66;
 
   public static final int TOK_FLUSH_TASK_INFO = 67;
-  public static final int TOK_DYNAMIC_PARAMETER = 68;
-
   public static final int TOK_LOAD_FILES = 69;
   public static final int TOK_REMOVE_FILE = 70;
   public static final int TOK_MOVE_FILE = 71;
@@ -230,7 +228,6 @@ public class SQLConstant {
 
     tokenNames.put(TOK_LOAD_CONFIGURATION, "TOK_LOAD_CONFIGURATION");
     tokenNames.put(TOK_FLUSH_TASK_INFO, "TOK_FLUSH_TASK_INFO");
-    tokenNames.put(TOK_DYNAMIC_PARAMETER, "TOK_DYNAMIC_PARAMETER");
 
     tokenNames.put(TOK_LOAD_FILES, "TOK_LOAD_FILES");
     tokenNames.put(TOK_REMOVE_FILE, "TOK_REMOVE_FILE");
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 0933877..da4bb4f 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
@@ -26,7 +26,6 @@ import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_CREATED_TIME;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_DEVICES;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_DONE;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ITEM;
-import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PARAMETER;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PRIVILEGE;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PROGRESS;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ROLE;
@@ -58,8 +57,6 @@ import org.apache.iotdb.db.auth.entity.Role;
 import org.apache.iotdb.db.auth.entity.User;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.CompressionRatio;
-import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.ChunkMetadataCache;
@@ -386,8 +383,6 @@ public class PlanExecutor implements IPlanExecutor {
     switch (showPlan.getShowContentType()) {
       case TTL:
         return processShowTTLQuery((ShowTTLPlan) showPlan);
-      case DYNAMIC_PARAMETER:
-        return processShowDynamicParameterQuery();
       case FLUSH_TASK_INFO:
         return processShowFlushTaskInfo();
       case VERSION:
@@ -638,52 +633,6 @@ public class PlanExecutor implements IPlanExecutor {
     return singleDataSet;
   }
 
-  private QueryDataSet processShowDynamicParameterQuery() {
-    ListDataSet listDataSet =
-        new ListDataSet(
-            Arrays.asList(new PartialPath(COLUMN_PARAMETER, false),
-                new PartialPath(COLUMN_VALUE, false)),
-            Arrays.asList(TSDataType.TEXT, TSDataType.TEXT));
-
-    int timestamp = 0;
-    addRowRecordForShowQuery(
-        listDataSet,
-        timestamp++,
-        "memtable size threshold",
-        IoTDBDescriptor.getInstance().getConfig().getMemtableSizeThreshold() + "B");
-    addRowRecordForShowQuery(
-        listDataSet,
-        timestamp++,
-        "memtable number",
-        IoTDBDescriptor.getInstance().getConfig().getMaxMemtableNumber() + "B");
-    addRowRecordForShowQuery(
-        listDataSet,
-        timestamp++,
-        "tsfile size threshold",
-        IoTDBDescriptor.getInstance().getConfig().getTsFileSizeThreshold() + "B");
-    addRowRecordForShowQuery(
-        listDataSet,
-        timestamp++,
-        "compression ratio",
-        Double.toString(CompressionRatio.getInstance().getRatio()));
-    addRowRecordForShowQuery(
-        listDataSet,
-        timestamp++,
-        "storage group number",
-        Integer.toString(IoTDB.metaManager.getAllStorageGroupPaths().size()));
-    addRowRecordForShowQuery(
-        listDataSet,
-        timestamp++,
-        "timeseries number",
-        Integer.toString(IoTDBConfigDynamicAdapter.getInstance().getTotalTimeseries()));
-    addRowRecordForShowQuery(
-        listDataSet,
-        timestamp,
-        "maximal timeseries number among storage groups",
-        Long.toString(IoTDB.metaManager.getMaximalSeriesNumberAmongStorageGroups()));
-    return listDataSet;
-  }
-
   private QueryDataSet processShowFlushTaskInfo() {
     ListDataSet listDataSet =
         new ListDataSet(
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
index 31d3b98..dabb949 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
@@ -50,7 +50,7 @@ public class ShowPlan extends PhysicalPlan {
   }
 
   public enum ShowContentType {
-    DYNAMIC_PARAMETER, FLUSH_TASK_INFO, TTL, VERSION, TIMESERIES, STORAGE_GROUP, CHILD_PATH, DEVICES,
+    FLUSH_TASK_INFO, TTL, VERSION, TIMESERIES, STORAGE_GROUP, CHILD_PATH, DEVICES,
     COUNT_TIMESERIES, COUNT_NODE_TIMESERIES, COUNT_NODES, MERGE_STATUS, COUNT_DEVICES, COUNT_STORAGE_GROUP
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index fae8a3b..2826b3b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -394,12 +394,6 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   }
 
   @Override
-  public void enterShowDynamicParameter(SqlBaseParser.ShowDynamicParameterContext ctx) {
-    super.enterShowDynamicParameter(ctx);
-    initializedOperator = new ShowOperator(SQLConstant.TOK_DYNAMIC_PARAMETER);
-  }
-
-  @Override
   public void enterShowFlushTaskInfo(SqlBaseParser.ShowFlushTaskInfoContext ctx) {
     super.enterShowFlushTaskInfo(ctx);
     initializedOperator = new ShowOperator(SQLConstant.TOK_FLUSH_TASK_INFO);
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 0359584..b45fa95 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
@@ -223,8 +223,6 @@ public class PhysicalGenerator {
         return generateLoadConfigurationPlan(type);
       case SHOW:
         switch (operator.getTokenIntType()) {
-          case SQLConstant.TOK_DYNAMIC_PARAMETER:
-            return new ShowPlan(ShowContentType.DYNAMIC_PARAMETER);
           case SQLConstant.TOK_FLUSH_TASK_INFO:
             return new ShowPlan(ShowContentType.FLUSH_TASK_INFO);
           case SQLConstant.TOK_VERSION:
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java b/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java
deleted file mode 100644
index 6fefb14..0000000
--- a/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * 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.rescon;
-
-import java.util.ArrayDeque;
-import java.util.Deque;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MemTablePool {
-
-  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
-
-  private static final Logger logger = LoggerFactory.getLogger(MemTablePool.class);
-
-  private static final Deque<IMemTable> availableMemTables = new ArrayDeque<>();
-  private static final int WAIT_TIME = 2000;
-  private int size = 0;
-
-  private MemTablePool() {
-  }
-
-  public static MemTablePool getInstance() {
-    return InstanceHolder.INSTANCE;
-  }
-
-  // TODO change the impl of getAvailableMemTable to non-blocking
-  public IMemTable getAvailableMemTable(Object applier) {
-    synchronized (availableMemTables) {
-      if (availableMemTables.isEmpty() && size < CONFIG.getMaxMemtableNumber()) {
-        size++;
-        logger.info("generated a new memtable for {}, system memtable size: {}, stack size: {}",
-            applier, size, availableMemTables.size());
-        return new PrimitiveMemTable();
-      } else if (!availableMemTables.isEmpty()) {
-        logger
-            .debug(
-                "system memtable size: {}, stack size: {}, then get a memtable from stack for {}",
-                size, availableMemTables.size(), applier);
-        return availableMemTables.pop();
-      }
-
-      // wait until some one has released a memtable
-      int waitCount = 1;
-      while (true) {
-        if (!availableMemTables.isEmpty()) {
-          logger.debug(
-              "system memtable size: {}, stack size: {}, then get a memtable from stack for {}",
-              size, availableMemTables.size(), applier);
-          return availableMemTables.pop();
-        }
-        try {
-          availableMemTables.wait(WAIT_TIME);
-        } catch (InterruptedException e) {
-          logger.error("{} fails to wait fot memtables {}, continue to wait", applier, e);
-          Thread.currentThread().interrupt();
-        }
-        logger.info("{} has waited for a memtable for {}ms", applier, waitCount++ * WAIT_TIME);
-      }
-    }
-  }
-
-  public void putBack(IMemTable memTable, String storageGroup) {
-    if (memTable.isSignalMemTable()) {
-      return;
-    }
-    synchronized (availableMemTables) {
-      // because of dynamic parameter adjust, the max number of memtable may decrease.
-      if (size > CONFIG.getMaxMemtableNumber()) {
-        logger.debug(
-            "Currently the size of available MemTables is {}, the maxmin size of MemTables is {}, discard this MemTable.",
-            CONFIG.getMaxMemtableNumber(), size);
-        size--;
-        return;
-      }
-      memTable.clear();
-      availableMemTables.push(memTable);
-      availableMemTables.notifyAll();
-      logger.debug("{} return a memtable, stack size {}", storageGroup, availableMemTables.size());
-    }
-  }
-
-  public int getSize() {
-    return size;
-  }
-
-  private static class InstanceHolder {
-
-    private static final MemTablePool INSTANCE = new MemTablePool();
-
-    private InstanceHolder() {
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayManager.java b/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayManager.java
new file mode 100644
index 0000000..20063fd
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayManager.java
@@ -0,0 +1,322 @@
+/*
+ * 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.rescon;
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Manage all primitive data list in memory, including get and release operation.
+ */
+public class PrimitiveArrayManager {
+
+  /**
+   * data type -> ArrayDeque<Array>
+   */
+  private static final Map<TSDataType, ArrayDeque<Object>> bufferedArraysMap = new EnumMap<>(
+      TSDataType.class);
+
+  /**
+   * data type -> current number of buffered arrays
+   */
+  private static final Map<TSDataType, Integer> bufferedArraysNumMap = new EnumMap<>(
+      TSDataType.class);
+
+  /**
+   * data type -> ratio of data type in schema, which could be seen as recommended ratio
+   */
+  private static final Map<TSDataType, Double> bufferedArraysNumRatio = new EnumMap<>(
+      TSDataType.class);
+
+  private static final Logger logger = LoggerFactory.getLogger(PrimitiveArrayManager.class);
+
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  public static final int ARRAY_SIZE = config.getPrimitiveArraySize();
+
+  /**
+   * threshold total size of arrays for all data types
+   */
+  private static final double BUFFERED_ARRAY_SIZE_THRESHOLD =
+      config.getAllocateMemoryForWrite() * config.getBufferedArraysMemoryProportion();
+
+  /**
+   * total size of buffered arrays
+   */
+  private static AtomicLong bufferedArraysRamSize = new AtomicLong();
+
+  /**
+   * total size of out of buffer arrays
+   */
+  private static AtomicLong outOfBufferArraysRamSize = new AtomicLong();
+
+  static {
+    bufferedArraysMap.put(TSDataType.BOOLEAN, new ArrayDeque<>());
+    bufferedArraysMap.put(TSDataType.INT32, new ArrayDeque<>());
+    bufferedArraysMap.put(TSDataType.INT64, new ArrayDeque<>());
+    bufferedArraysMap.put(TSDataType.FLOAT, new ArrayDeque<>());
+    bufferedArraysMap.put(TSDataType.DOUBLE, new ArrayDeque<>());
+    bufferedArraysMap.put(TSDataType.TEXT, new ArrayDeque<>());
+  }
+
+  private PrimitiveArrayManager() {
+    logger.info("BufferedArraySizeThreshold is {}", BUFFERED_ARRAY_SIZE_THRESHOLD);
+  }
+
+  /**
+   * Get primitive data lists according to type
+   *
+   * @param dataType data type
+   * @return an array
+   */
+  public static Object getPrimitiveArraysByType(TSDataType dataType) {
+    // check memory of buffered array, if already full, generate OOB
+    if (bufferedArraysRamSize.get() + ARRAY_SIZE * dataType.getDataTypeSize()
+        > BUFFERED_ARRAY_SIZE_THRESHOLD) {
+      // return an out of buffer array
+      outOfBufferArraysRamSize.addAndGet((long) ARRAY_SIZE * dataType.getDataTypeSize());
+      return createPrimitiveArray(dataType);
+    }
+
+    synchronized (bufferedArraysMap.get(dataType)) {
+      // try to get a buffered array
+      Object dataArray = bufferedArraysMap.get(dataType).poll();
+      if (dataArray != null) {
+        return dataArray;
+      }
+      // no buffered array, create one
+      bufferedArraysNumMap.put(dataType, bufferedArraysNumMap.getOrDefault(dataType, 0) + 1);
+      bufferedArraysRamSize.addAndGet((long) ARRAY_SIZE * dataType.getDataTypeSize());
+    }
+
+    return createPrimitiveArray(dataType);
+  }
+
+  private static Object createPrimitiveArray(TSDataType dataType) {
+    Object dataArray;
+    switch (dataType) {
+      case BOOLEAN:
+        dataArray = new boolean[ARRAY_SIZE];
+        break;
+      case INT32:
+        dataArray = new int[ARRAY_SIZE];
+        break;
+      case INT64:
+        dataArray = new long[ARRAY_SIZE];
+        break;
+      case FLOAT:
+        dataArray = new float[ARRAY_SIZE];
+        break;
+      case DOUBLE:
+        dataArray = new double[ARRAY_SIZE];
+        break;
+      case TEXT:
+        dataArray = new Binary[ARRAY_SIZE];
+        break;
+      default:
+        throw new UnSupportedDataTypeException(dataType.toString());
+    }
+
+    return dataArray;
+  }
+
+  /**
+   * Get primitive data lists according to data type and size, only for TVList's sorting
+   *
+   * @param dataType data type
+   * @param size     needed capacity
+   * @return an array of primitive data arrays
+   */
+  public static synchronized Object createDataListsByType(TSDataType dataType, int size) {
+    int arrayNumber = (int) Math.ceil((float) size / (float) ARRAY_SIZE);
+    switch (dataType) {
+      case BOOLEAN:
+        boolean[][] booleans = new boolean[arrayNumber][];
+        for (int i = 0; i < arrayNumber; i++) {
+          booleans[i] = new boolean[ARRAY_SIZE];
+        }
+        return booleans;
+      case INT32:
+        int[][] ints = new int[arrayNumber][];
+        for (int i = 0; i < arrayNumber; i++) {
+          ints[i] = new int[ARRAY_SIZE];
+        }
+        return ints;
+      case INT64:
+        long[][] longs = new long[arrayNumber][];
+        for (int i = 0; i < arrayNumber; i++) {
+          longs[i] = new long[ARRAY_SIZE];
+        }
+        return longs;
+      case FLOAT:
+        float[][] floats = new float[arrayNumber][];
+        for (int i = 0; i < arrayNumber; i++) {
+          floats[i] = new float[ARRAY_SIZE];
+        }
+        return floats;
+      case DOUBLE:
+        double[][] doubles = new double[arrayNumber][];
+        for (int i = 0; i < arrayNumber; i++) {
+          doubles[i] = new double[ARRAY_SIZE];
+        }
+        return doubles;
+      case TEXT:
+        Binary[][] binaries = new Binary[arrayNumber][];
+        for (int i = 0; i < arrayNumber; i++) {
+          binaries[i] = new Binary[ARRAY_SIZE];
+        }
+        return binaries;
+      default:
+        return null;
+    }
+  }
+
+  /**
+   * This method is called when bringing back data array
+   *
+   * @param dataArray data array
+   */
+  public static void release(Object dataArray) {
+    TSDataType dataType;
+    if (dataArray instanceof boolean[]) {
+      dataType = TSDataType.BOOLEAN;
+    } else if (dataArray instanceof int[]) {
+      dataType = TSDataType.INT32;
+    } else if (dataArray instanceof long[]) {
+      dataType = TSDataType.INT64;
+    } else if (dataArray instanceof float[]) {
+      dataType = TSDataType.FLOAT;
+    } else if (dataArray instanceof double[]) {
+      dataType = TSDataType.DOUBLE;
+    } else if (dataArray instanceof Binary[]) {
+      Arrays.fill((Binary[]) dataArray, null);
+      dataType = TSDataType.TEXT;
+    } else {
+      throw new UnSupportedDataTypeException("Unknown data array type");
+    }
+
+    // Check out of buffer array num
+    if (outOfBufferArraysRamSize.get() > 0 && isCurrentDataTypeExceeded(dataType)) {
+      // release an out of buffer array
+      bringBackOOBArray(dataType, ARRAY_SIZE);
+    } else if (outOfBufferArraysRamSize.get() > 0 && !isCurrentDataTypeExceeded(dataType)) {
+      // if the ratio of buffered arrays of this data type does not exceed the schema ratio,
+      // choose one replaced array who has larger ratio than schema recommended ratio
+      TSDataType replacedDataType = null;
+      for (Map.Entry<TSDataType, Integer> entry : bufferedArraysNumMap.entrySet()) {
+        if (isCurrentDataTypeExceeded(entry.getKey())) {
+          replacedDataType = entry.getKey();
+          // bring back the replaced array as OOB array
+          bringBackOOBArray(replacedDataType, ARRAY_SIZE);
+          break;
+        }
+      }
+      if (replacedDataType != null) {
+        // if we find a replaced array, bring back the original array as a buffered array
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "The ratio of {} in buffered array has not reached the schema ratio. Replaced by {}",
+              dataType, replacedDataType);
+        }
+        bringBackBufferedArray(dataType, dataArray);
+      } else {
+        // or else bring back the original array as OOB array
+        bringBackOOBArray(dataType, ARRAY_SIZE);
+      }
+    } else {
+      // if there is no out of buffer array, bring back as buffered array directly
+      bringBackBufferedArray(dataType, dataArray);
+    }
+  }
+
+  /**
+   * Bring back a buffered array
+   *
+   * @param dataType  data type
+   * @param dataArray data array
+   */
+  private static void bringBackBufferedArray(TSDataType dataType, Object dataArray) {
+    synchronized (bufferedArraysMap.get(dataType)) {
+      bufferedArraysMap.get(dataType).add(dataArray);
+      bufferedArraysNumMap.put(dataType, bufferedArraysNumMap.getOrDefault(dataType, 0) + 1);
+    }
+    bufferedArraysRamSize.addAndGet((long) -ARRAY_SIZE * dataType.getDataTypeSize());
+  }
+
+  /**
+   * Bring back out of buffered array
+   *
+   * @param dataType data type
+   * @param size     capacity
+   */
+  private static void bringBackOOBArray(TSDataType dataType, int size) {
+    outOfBufferArraysRamSize.addAndGet((long) -size * dataType.getDataTypeSize());
+  }
+
+  /**
+   * @param schemaDataTypeNumMap schema DataType Num Map (for each series, increase a long and a specific type)
+   * @param total current DataType Total Num (twice of number of time series)
+   */
+  public static void updateSchemaDataTypeNum(Map<TSDataType, Integer> schemaDataTypeNumMap,
+      long total) {
+    for (Map.Entry<TSDataType, Integer> entry : schemaDataTypeNumMap.entrySet()) {
+      TSDataType dataType = entry.getKey();
+      bufferedArraysNumRatio.put(dataType, (double) schemaDataTypeNumMap.get(dataType) / total);
+    }
+  }
+
+  /**
+   * check whether the ratio of buffered array of specific data type reaches the ratio in schema (as
+   * recommended ratio)
+   *
+   * @param dataType data type
+   * @return true if the buffered array ratio exceeds the recommend ratio
+   */
+  private static boolean isCurrentDataTypeExceeded(TSDataType dataType) {
+    int total = 0;
+    for (int num : bufferedArraysNumMap.values()) {
+      total += num;
+    }
+    return total != 0 &&
+        ((double) bufferedArraysNumMap.getOrDefault(dataType, 0) / total >
+            bufferedArraysNumRatio.getOrDefault(dataType, 0.0));
+  }
+
+  public static void close() {
+    for (ArrayDeque<Object> dataListQueue : bufferedArraysMap.values()) {
+      dataListQueue.clear();
+    }
+
+    bufferedArraysNumMap.clear();
+    bufferedArraysNumRatio.clear();
+
+    bufferedArraysRamSize.set(0);
+    outOfBufferArraysRamSize.set(0);
+  }
+}
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayPool.java b/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayPool.java
deleted file mode 100644
index 2284be2..0000000
--- a/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayPool.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*
- * 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.rescon;
-
-import java.util.ArrayDeque;
-import java.util.Arrays;
-import java.util.EnumMap;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.Binary;
-
-/**
- * Manage all primitive data list in memory, including get and release operation.
- */
-public class PrimitiveArrayPool {
-
-  /**
-   * data type -> Array<PrimitiveArray>
-   */
-  private static final EnumMap<TSDataType, ArrayDeque<Object>> primitiveArraysMap = new EnumMap<>(TSDataType.class);
-
-  public static final int ARRAY_SIZE =
-      IoTDBDescriptor.getInstance().getConfig().getPrimitiveArraySize();
-
-  static {
-    primitiveArraysMap.put(TSDataType.BOOLEAN, new ArrayDeque<>());
-    primitiveArraysMap.put(TSDataType.INT32, new ArrayDeque<>());
-    primitiveArraysMap.put(TSDataType.INT64, new ArrayDeque<>());
-    primitiveArraysMap.put(TSDataType.FLOAT, new ArrayDeque<>());
-    primitiveArraysMap.put(TSDataType.DOUBLE, new ArrayDeque<>());
-    primitiveArraysMap.put(TSDataType.TEXT, new ArrayDeque<>());
-  }
-
-  public static PrimitiveArrayPool getInstance() {
-    return INSTANCE;
-  }
-
-  private static final PrimitiveArrayPool INSTANCE = new PrimitiveArrayPool();
-
-
-  private PrimitiveArrayPool() {}
-
-  public synchronized Object getPrimitiveDataListByType(TSDataType dataType) {
-    ArrayDeque<Object> dataListQueue = primitiveArraysMap.computeIfAbsent(dataType, k ->new ArrayDeque<>());
-    Object dataArray = dataListQueue.poll();
-    switch (dataType) {
-      case BOOLEAN:
-        if (dataArray == null) {
-          dataArray = new boolean[ARRAY_SIZE];
-        }
-        break;
-      case INT32:
-        if (dataArray == null) {
-          dataArray = new int[ARRAY_SIZE];
-        }
-        break;
-      case INT64:
-        if (dataArray == null) {
-          dataArray = new long[ARRAY_SIZE];
-        }
-        break;
-      case FLOAT:
-        if (dataArray == null) {
-          dataArray = new float[ARRAY_SIZE];
-        }
-        break;
-      case DOUBLE:
-        if (dataArray == null) {
-          dataArray = new double[ARRAY_SIZE];
-        }
-        break;
-      case TEXT:
-        if (dataArray == null) {
-          dataArray = new Binary[ARRAY_SIZE];
-        }
-        break;
-      default:
-        throw new UnSupportedDataTypeException("DataType: " + dataType);
-    }
-    return dataArray;
-  }
-
-
-  public synchronized void release(Object dataArray) {
-    if (dataArray instanceof boolean[]) {
-      primitiveArraysMap.get(TSDataType.BOOLEAN).add(dataArray);
-    } else if (dataArray instanceof int[]) {
-      primitiveArraysMap.get(TSDataType.INT32).add(dataArray);
-    } else if (dataArray instanceof long[]){
-      primitiveArraysMap.get(TSDataType.INT64).add(dataArray);
-    } else if (dataArray instanceof float[]) {
-      primitiveArraysMap.get(TSDataType.FLOAT).add(dataArray);
-    } else if (dataArray instanceof double[]) {
-      primitiveArraysMap.get(TSDataType.DOUBLE).add(dataArray);
-    } else if (dataArray instanceof Binary[]) {
-      Arrays.fill((Binary[]) dataArray, null);
-      primitiveArraysMap.get(TSDataType.TEXT).add(dataArray);
-    }
-  }
-
-  /**
-   * @param size needed capacity
-   * @return an array of primitive data arrays
-   */
-  public synchronized Object getDataListsByType(TSDataType dataType, int size) {
-    int arrayNumber = (int) Math.ceil((float) size / (float)ARRAY_SIZE);
-    switch (dataType) {
-      case BOOLEAN:
-        boolean[][] booleans = new boolean[arrayNumber][];
-        for (int i = 0; i < arrayNumber; i++) {
-          booleans[i] = (boolean[]) getPrimitiveDataListByType(dataType);
-        }
-        return booleans;
-      case INT32:
-        int[][] ints = new int[arrayNumber][];
-        for (int i = 0; i < arrayNumber; i++) {
-          ints[i] = (int[]) getPrimitiveDataListByType(dataType);
-        }
-        return ints;
-      case INT64:
-        long[][] longs = new long[arrayNumber][];
-        for (int i = 0; i < arrayNumber; i++) {
-          longs[i] = (long[]) getPrimitiveDataListByType(dataType);
-        }
-        return longs;
-      case FLOAT:
-        float[][] floats = new float[arrayNumber][];
-        for (int i = 0; i < arrayNumber; i++) {
-          floats[i] = (float[]) getPrimitiveDataListByType(dataType);
-        }
-        return floats;
-      case DOUBLE:
-        double[][] doubles = new double[arrayNumber][];
-        for (int i = 0; i < arrayNumber; i++) {
-          doubles[i] = (double[]) getPrimitiveDataListByType(dataType);
-        }
-        return doubles;
-      case TEXT:
-        Binary[][] binaries = new Binary[arrayNumber][];
-        for (int i = 0; i < arrayNumber; i++) {
-          binaries[i] = (Binary[]) getPrimitiveDataListByType(dataType);
-        }
-        return binaries;
-      default:
-        return null;
-    }
-  }
-
-}
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
new file mode 100644
index 0000000..82d2903
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.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.rescon;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.flush.FlushManager;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupInfo;
+import org.apache.iotdb.db.engine.storagegroup.TsFileProcessor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SystemInfo {
+
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private static final Logger logger = LoggerFactory.getLogger(SystemInfo.class);
+
+  private long totalSgMemCost;
+  private volatile boolean rejected = false;
+
+  private Map<StorageGroupInfo, Long> reportedSgMemCostMap = new HashMap<>();
+
+  private static final double FLUSH_PROPORTION = config.getFlushProportion();
+  private static final double REJECT_PROPORTION = config.getRejectProportion();
+
+  /**
+   * Report current mem cost of storage group to system. Called when the memory of
+   * storage group newly accumulates to IoTDBConfig.getStorageGroupSizeReportThreshold()
+   *
+   * @param storageGroupInfo storage group
+   */
+  public synchronized void reportStorageGroupStatus(StorageGroupInfo storageGroupInfo) {
+    long delta = storageGroupInfo.getSgMemCost() -
+        reportedSgMemCostMap.getOrDefault(storageGroupInfo, 0L);
+    totalSgMemCost += delta;
+    logger.debug("Report Storage Group Status to the system. "
+          + "After adding {}, current sg mem cost is {}.", delta, totalSgMemCost);
+    reportedSgMemCostMap.put(storageGroupInfo, storageGroupInfo.getSgMemCost());
+    storageGroupInfo.setLastReportedSize(storageGroupInfo.getSgMemCost());
+    if (totalSgMemCost >= config.getAllocateMemoryForWrite() * FLUSH_PROPORTION) {
+      logger.debug("The total storage group mem costs are too large, call for flushing. "
+          + "Current sg cost is {}", totalSgMemCost);
+      flush();
+    }
+    if (totalSgMemCost >= config.getAllocateMemoryForWrite() * REJECT_PROPORTION) {
+      logger.info("Change system to reject status...");
+      rejected = true;
+    }
+  }
+
+  /**
+   * Report resetting the mem cost of sg to system.
+   * It will be invoked after flushing, closing and failed to insert
+   *
+   * @param storageGroupInfo storage group
+   */
+  public synchronized void resetStorageGroupStatus(StorageGroupInfo storageGroupInfo) {
+    if (reportedSgMemCostMap.containsKey(storageGroupInfo)) {
+      this.totalSgMemCost -= reportedSgMemCostMap.get(storageGroupInfo)
+          - storageGroupInfo.getSgMemCost();
+      storageGroupInfo.setLastReportedSize(storageGroupInfo.getSgMemCost());
+      if (totalSgMemCost > config.getAllocateMemoryForWrite() * FLUSH_PROPORTION) {
+        logger.debug("Some sg memory released but still exceeding flush proportion, call flush.");
+        logCurrentTotalSGMemory();
+        forceFlush();
+      }
+      if (totalSgMemCost < config.getAllocateMemoryForWrite() * REJECT_PROPORTION) {
+        logger.debug("Some sg memory released, set system to normal status.");
+        logCurrentTotalSGMemory();
+        rejected = false;
+      } else {
+        logger.warn("Some sg memory released, but system is still in reject status.");
+        logCurrentTotalSGMemory();
+        rejected = true;
+      }
+      reportedSgMemCostMap.put(storageGroupInfo, storageGroupInfo.getSgMemCost());
+    }
+  }
+
+  private void logCurrentTotalSGMemory() {
+    logger.debug("Current Sg cost is {}", totalSgMemCost);
+  }
+
+  /**
+   * Flush the tsfileProcessor in SG with the max mem cost. If the queue size of flushing >
+   * threshold, it's identified as flushing is in progress.
+   */
+  public void flush() {
+
+    if (FlushManager.getInstance().getNumberOfWorkingTasks() > 0) {
+      return;
+    }
+    // If invoke flush by replaying logs, do not flush now!
+    if (reportedSgMemCostMap.size() == 0) {
+      return;
+    }
+    // get the tsFile processors which has the max work MemTable size
+    List<TsFileProcessor> processors = getTsFileProcessorsToFlush();
+    for (TsFileProcessor processor : processors) {
+      if (processor != null) {
+        processor.setFlush();
+      }
+    }
+  }
+
+  public void forceFlush() {
+    if (FlushManager.getInstance().getNumberOfWorkingTasks() > 0) {
+      return;
+    }
+    List<TsFileProcessor> processors = getTsFileProcessorsToFlush();
+    for (TsFileProcessor processor : processors) {
+      if (processor != null) {
+        if (processor.shouldClose()) {
+          processor.startClose();
+        } else {
+          processor.asyncFlush();
+        }
+      }
+    }
+  }
+
+  private List<TsFileProcessor> getTsFileProcessorsToFlush() {
+    PriorityQueue<TsFileProcessor> tsps = new PriorityQueue<>(
+        (o1, o2) -> Long.compare(o2.getWorkMemTableRamCost(), o1.getWorkMemTableRamCost()));
+    for (StorageGroupInfo sgInfo : reportedSgMemCostMap.keySet()) {
+      tsps.addAll(sgInfo.getAllReportedTsp());
+    }
+    List<TsFileProcessor> processors = new ArrayList<>();
+    long memCost = 0;
+    while (totalSgMemCost - memCost > config.getAllocateMemoryForWrite() *
+        FLUSH_PROPORTION / 2) {
+      if (tsps.isEmpty() || tsps.peek().getWorkMemTableRamCost() == 0) {
+        return processors;
+      }
+      processors.add(tsps.peek());
+      memCost += tsps.peek().getWorkMemTableRamCost();
+      tsps.poll();
+    }
+    return processors;
+  }
+
+  public boolean isRejected() {
+    return rejected;
+  }
+
+  public void close() {
+    reportedSgMemCostMap.clear();
+    totalSgMemCost = 0;
+    rejected = false;
+  }
+
+  public static SystemInfo getInstance() {
+    return InstanceHolder.instance;
+  }
+
+  private static class InstanceHolder {
+
+    private InstanceHolder() {
+    }
+
+    private static SystemInfo instance = new SystemInfo();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/TVListAllocator.java b/server/src/main/java/org/apache/iotdb/db/rescon/TVListAllocator.java
index b09c05a..c6fda52 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/TVListAllocator.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/TVListAllocator.java
@@ -100,7 +100,9 @@ public class TVListAllocator implements TVListAllocatorMBean, IService {
   @Override
   public void stop() {
     JMXService.deregisterMBean(mbeanName);
-    tvListCache.clear();
+    for (Queue<TVList> queue : tvListCache.values()) {
+      queue.clear();
+    }
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
index 4ba42d2..68af06c 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
@@ -23,8 +23,6 @@ import org.apache.iotdb.db.concurrent.IoTDBDefaultThreadExceptionHandler;
 import org.apache.iotdb.db.conf.IoTDBConfigCheck;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
-import org.apache.iotdb.db.conf.adapter.ManageDynamicParameters;
 import org.apache.iotdb.db.cost.statistic.Measurement;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.CacheHitRatioMonitor;
@@ -35,6 +33,8 @@ import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.monitor.StatMonitor;
 import org.apache.iotdb.db.query.control.TracingManager;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.rescon.TVListAllocator;
 import org.apache.iotdb.db.sync.receiver.SyncServerManager;
 import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
@@ -104,7 +104,6 @@ public class IoTDB implements IoTDBMBean {
     registerManager.register(Monitor.getInstance());
     registerManager.register(StatMonitor.getInstance());
     registerManager.register(Measurement.INSTANCE);
-    registerManager.register(ManageDynamicParameters.getInstance());
     registerManager.register(TVListAllocator.getInstance());
     registerManager.register(CacheHitRatioMonitor.getInstance());
     JMXService.registerMBean(getInstance(), mbeanName);
@@ -157,10 +156,8 @@ public class IoTDB implements IoTDBMBean {
     IoTDB.metaManager.init();
     long end = System.currentTimeMillis() - time;
     logger.info("spend {}ms to recover schema.", end);
-    IoTDBConfigDynamicAdapter.getInstance().setInitialized(true);
     logger.info(
-        "After initializing, max memTable num is {}, tsFile threshold is {}, memtableSize is {}",
-        IoTDBDescriptor.getInstance().getConfig().getMaxMemtableNumber(),
+        "After initializing, tsFile threshold is {}, memtableSize is {}",
         IoTDBDescriptor.getInstance().getConfig().getTsFileSizeThreshold(),
         IoTDBDescriptor.getInstance().getConfig().getMemtableSizeThreshold());
 
@@ -176,6 +173,8 @@ public class IoTDB implements IoTDBMBean {
     IoTDB.metaManager.clear();
     TracingManager.getInstance().close();
     registerManager.shutdownAll();
+    PrimitiveArrayManager.close();
+    SystemInfo.getInstance().close();
     JMXService.deregisterMBean(mbeanName);
     logger.info("IoTDB is deactivated.");
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java b/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
index 9ceac71..91ca2a8 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
@@ -39,7 +39,6 @@ public enum ServiceType {
   MERGE_SERVICE("Merge Manager", "Merge Manager"),
   HOT_COMPACTION_SERVICE("Hot Compaction Manager", "Hot Compaction Manager"),
   PERFORMANCE_STATISTIC_SERVICE("PERFORMANCE_STATISTIC_SERVICE", "PERFORMANCE_STATISTIC_SERVICE"),
-  MANAGE_DYNAMIC_PARAMETERS_SERVICE("Manage Dynamic Parameters", "Manage Dynamic Parameters"),
   TVLIST_ALLOCATOR_SERVICE("TVList Allocator", ""),
   CACHE_HIT_RATIO_DISPLAY_SERVICE("CACHE_HIT_RATIO_DISPLAY_SERVICE",
       generateJmxName(IoTDBConstant.IOTDB_PACKAGE, "Cache Hit Ratio")),
diff --git a/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java b/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java
index 3bfe9c7..53f442d 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java
@@ -27,7 +27,6 @@ import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_CREATED_TIME;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_DEVICES;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_DONE;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ITEM;
-import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PARAMETER;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PRIVILEGE;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PROGRESS;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ROLE;
@@ -67,10 +66,6 @@ class StaticResps {
       Arrays.asList(COLUMN_ITEM, COLUMN_VALUE),
       Arrays.asList(TSDataType.TEXT.toString(), TSDataType.TEXT.toString()));
 
-  static final TSExecuteStatementResp DYNAMIC_PARAMETER_RESP = getNoTimeExecuteResp(
-      Arrays.asList(COLUMN_PARAMETER, COLUMN_VALUE),
-      Arrays.asList(TSDataType.TEXT.toString(), TSDataType.TEXT.toString()));
-
   static final TSExecuteStatementResp SHOW_VERSION_RESP = getNoTimeExecuteResp(
       Collections.singletonList(COLUMN_VERSION),
       Collections.singletonList(TSDataType.TEXT.toString()));
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 28043f6..bcbcd29 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
@@ -765,8 +765,6 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
         return StaticResps.TTL_RESP;
       case FLUSH_TASK_INFO:
         return StaticResps.FLUSH_INFO_RESP;
-      case DYNAMIC_PARAMETER:
-        return StaticResps.DYNAMIC_PARAMETER_RESP;
       case VERSION:
         return StaticResps.SHOW_VERSION_RESP;
       case TIMESERIES:
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/memestimation/MemEstTool.java b/server/src/main/java/org/apache/iotdb/db/tools/memestimation/MemEstTool.java
deleted file mode 100644
index e723aeb..0000000
--- a/server/src/main/java/org/apache/iotdb/db/tools/memestimation/MemEstTool.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.tools.memestimation;
-
-import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
-import io.airlift.airline.Cli;
-import io.airlift.airline.Help;
-import io.airlift.airline.ParseArgumentsMissingException;
-import io.airlift.airline.ParseArgumentsUnexpectedException;
-import io.airlift.airline.ParseCommandMissingException;
-import io.airlift.airline.ParseCommandUnrecognizedException;
-import io.airlift.airline.ParseOptionConversionException;
-import io.airlift.airline.ParseOptionMissingException;
-import io.airlift.airline.ParseOptionMissingValueException;
-import java.io.IOException;
-import java.util.List;
-import org.apache.commons.io.FileUtils;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
-
-public class MemEstTool {
-
-  public static void main(String... args) throws IOException {
-    List<Class<? extends Runnable>> commands = Lists.newArrayList(
-        Help.class,
-        MemEstToolCmd.class
-    );
-    Cli.CliBuilder<Runnable> builder = Cli.builder("memory-tool");
-
-    builder.withDescription("Estimate memory for writing")
-        .withDefaultCommand(Help.class)
-        .withCommands(commands);
-
-    Cli<Runnable> parser = builder.build();
-
-    int status = 0;
-    try {
-      Runnable parse = parser.parse(args);
-      parse.run();
-    } catch (IllegalArgumentException |
-        IllegalStateException |
-        ParseArgumentsMissingException |
-        ParseArgumentsUnexpectedException |
-        ParseOptionConversionException |
-        ParseOptionMissingException |
-        ParseOptionMissingValueException |
-        ParseCommandMissingException |
-        ParseCommandUnrecognizedException e) {
-      badUse(e);
-      status = 1;
-    } catch (Exception e) {
-      err(Throwables.getRootCause(e));
-      status = 2;
-    }
-
-    FileUtils.deleteDirectory(SystemFileFactory.INSTANCE.getFile(IoTDBDescriptor.getInstance().getConfig().getSystemDir()));
-    FileUtils.deleteDirectory(SystemFileFactory.INSTANCE.getFile(IoTDBDescriptor.getInstance().getConfig().getWalDir()));
-    for(int i=0; i < IoTDBDescriptor.getInstance().getConfig().getDataDirs().length; i++){
-      FileUtils.deleteDirectory(SystemFileFactory.INSTANCE.getFile(IoTDBDescriptor.getInstance().getConfig().getDataDirs()[i]));
-    }
-
-    System.exit(status);
-  }
-
-  private static void badUse(Exception e) {
-    System.out.println("memory-tool: " + e.getMessage());
-    System.out.println("See 'memory-tool help' or 'memory-tool help <command>'.");
-  }
-
-  private static void err(Throwable e) {
-    System.err.println("error: " + e.getMessage());
-    System.err.println("-- StackTrace --");
-    System.err.println(Throwables.getStackTraceAsString(e));
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/memestimation/MemEstToolCmd.java b/server/src/main/java/org/apache/iotdb/db/tools/memestimation/MemEstToolCmd.java
deleted file mode 100644
index a12f33b..0000000
--- a/server/src/main/java/org/apache/iotdb/db/tools/memestimation/MemEstToolCmd.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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.tools.memestimation;
-
-import io.airlift.airline.Command;
-import io.airlift.airline.Option;
-import org.apache.iotdb.db.concurrent.WrappedRunnable;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
-import org.apache.iotdb.db.exception.ConfigAdjusterException;
-import org.apache.iotdb.db.service.IoTDB;
-
-@Command(name = "calmem", description = "calculate minimum memory required for writing based on the number of storage groups and timeseries")
-public class MemEstToolCmd extends WrappedRunnable {
-
-  @Option(title = "storage group number", name = {"-sg",
-      "--storagegroup"}, description = "Storage group number")
-  private String sgNumString = "10";
-
-  @Option(title = "total timeseries number", name = {"-ts",
-      "--timeseries"}, description = "Total timeseries number")
-  private String tsNumString = "1000";
-
-  @Option(title = "max timeseries", name = {"-mts",
-      "--maxtimeseries"}, description = "Maximum timeseries number among storage groups, make sure that it's smaller than total timeseries number")
-  private String maxTsNumString = "0";
-
-  @Override
-  public void runMayThrow() {
-    // backup origin config parameters
-    IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-    long memTableSize = config.getMemtableSizeThreshold();
-    int maxMemtableNumber = config.getMaxMemtableNumber();
-    long tsFileSize = config.getTsFileSizeThreshold();
-
-    // parse input parameters
-    long sgNum = Long.parseLong(sgNumString);
-    long tsNum = Long.parseLong(tsNumString);
-    long maxTsNum = Long.parseLong(maxTsNumString);
-
-    // tool parameters
-    long stepMemory = calStepMemory(tsNum) * IoTDBConstant.GB;
-    long currentMemory = stepMemory;
-    long maxTsNumValid = maxTsNum;
-    long maxProcess = 0;
-    long start = System.currentTimeMillis();
-    while (true) {
-
-      // recover config parameter
-      config.setAllocateMemoryForWrite(currentMemory);
-      config.setMemtableSizeThreshold(memTableSize);
-      config.setMaxMemtableNumber(maxMemtableNumber);
-      config.setTsFileSizeThreshold(tsFileSize);
-      IoTDBConfigDynamicAdapter.getInstance().reset();
-      IoTDBConfigDynamicAdapter.getInstance().setInitialized(true);
-      IoTDB.metaManager.clear();
-
-      long sgCnt = 1;
-      long tsCnt = 0;
-      try {
-        for (; sgCnt <= sgNum; sgCnt++) {
-          IoTDBConfigDynamicAdapter.getInstance().addOrDeleteStorageGroup(1);
-        }
-        for (; tsCnt < tsNum; tsCnt++) {
-          IoTDBConfigDynamicAdapter.getInstance().addOrDeleteTimeSeries(1);
-          if (maxTsNum == 0) {
-            maxTsNumValid = tsCnt / sgNum + 1;
-          } else {
-            maxTsNumValid = Math.min(tsCnt, maxTsNum);
-            maxTsNumValid = Math.max(maxTsNumValid, tsCnt / sgNum + 1);
-          }
-          IoTDB.metaManager.setMaxSeriesNumberAmongStorageGroup(maxTsNumValid);
-        }
-
-      } catch (ConfigAdjusterException e) {
-        if (sgCnt > sgNum) {
-          maxProcess = Math.max(maxProcess, (tsCnt + 1) * 100 / tsNum);
-          System.out
-              .print(String.format("Memory estimation progress : %d%%\r", maxProcess));
-        }
-        currentMemory += stepMemory;
-        continue;
-      }
-      break;
-    }
-    System.out.println(String
-        .format("Memory for writing: %dGB, SG: %d, TS: %d, MTS: %d", currentMemory / IoTDBConstant.GB,
-            sgNum, tsNum, maxTsNumValid));
-    System.out.println(String.format("Calculating memory for writing consumes: %dms",
-        (System.currentTimeMillis() - start)));
-  }
-
-  private long calStepMemory(long maxTimeseriesNumber) {
-    maxTimeseriesNumber /= 10000000;
-    int step = 1;
-    while (maxTimeseriesNumber > 0) {
-      maxTimeseriesNumber /= 10;
-      step *= 10;
-    }
-    return step;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
index affe70d..740ff10 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
@@ -45,8 +45,11 @@ public class MemUtils {
 
   /**
    * function for getting the value size.
+   * If mem control enabled, do not add text data size here, the size will be added to memtable
+   * before inserting.
    */
-  public static long getRecordSize(TSDataType dataType, Object value) {
+  public static long getRecordSize(TSDataType dataType, Object value,
+      boolean addingTextDataSize) {
     switch (dataType) {
       case INT32:
         return 8L + 4L;
@@ -59,7 +62,7 @@ public class MemUtils {
       case BOOLEAN:
         return 8L + 1L;
       case TEXT:
-        return 8L + getBinarySize((Binary) value);
+        return 8L + (addingTextDataSize ? getBinarySize((Binary) value) : 0);
       default:
         return 8L + 8L;
     }
@@ -70,7 +73,21 @@ public class MemUtils {
         .sizeOf(value.getValues());
   }
 
-  public static long getRecordSize(InsertTabletPlan insertTabletPlan, int start, int end) {
+  public static long getBinaryColumnSize(Binary[] column, int start, int end) {
+    long memSize = 0;
+    memSize += (end-start) * RamUsageEstimator.NUM_BYTES_OBJECT_HEADER;
+    for (int i = start; i < end; i++) {
+      memSize += RamUsageEstimator.sizeOf(column[i].getValues());
+    }
+    return memSize;
+  }
+
+  /**
+   * If mem control enabled, do not add text data size here, the size will be added to memtable
+   * before inserting.
+   */
+  public static long getRecordSize(InsertTabletPlan insertTabletPlan, int start, int end,
+      boolean addingTextDataSize) {
     if (start >= end) {
       return 0L;
     }
@@ -92,8 +109,10 @@ public class MemUtils {
           memSize += (end - start) * (8L + 1L); break;
         case TEXT:
           memSize += (end - start) * 8L;
-          for (int j = start; j < end; j++) {
-            memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[i])[j]);
+          if (addingTextDataSize) {
+            for (int j = start; j < end; j++) {
+              memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[i])[j]);
+            }
           }
           break;
         default:
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java
index b1adc1e..ae24c26 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java
@@ -18,11 +18,11 @@
  */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.rescon.PrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -94,12 +94,11 @@ public class BinaryTVList extends TVList {
 
   public void sort() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
-      sortedTimestamps = (long[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.INT64, size);
+      sortedTimestamps = (long[][]) PrimitiveArrayManager
+          .createDataListsByType(TSDataType.INT64, size);
     }
     if (sortedValues == null || sortedValues.length < size) {
-      sortedValues = (Binary[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.TEXT, size);
+      sortedValues = (Binary[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.TEXT, size);
     }
     sort(0, size);
     clearSortedValue();
@@ -111,7 +110,7 @@ public class BinaryTVList extends TVList {
   void clearValue() {
     if (values != null) {
       for (Binary[] dataArray : values) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
+        PrimitiveArrayManager.release(dataArray);
       }
       values.clear();
     }
@@ -120,9 +119,6 @@ public class BinaryTVList extends TVList {
   @Override
   void clearSortedValue() {
     if (sortedValues != null) {
-      for (Binary[] dataArray : sortedValues) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
-      }
       sortedValues = null;
     }
   }
@@ -158,8 +154,7 @@ public class BinaryTVList extends TVList {
 
   @Override
   protected void expandValues() {
-    values.add((Binary[]) PrimitiveArrayPool
-        .getInstance().getPrimitiveDataListByType(TSDataType.TEXT));
+    values.add((Binary[]) getPrimitiveArraysByType(TSDataType.TEXT));
   }
 
   @Override
@@ -187,38 +182,7 @@ public class BinaryTVList extends TVList {
 
   @Override
   protected void releaseLastValueArray() {
-    PrimitiveArrayPool.getInstance().release(values.remove(values.size() - 1));
-  }
-
-  @Override
-  public void putBinaries(long[] time, Binary[] value) {
-    checkExpansion();
-    int idx = 0;
-    int length = time.length;
-
-    updateMinTimeAndSorted(time);
-
-    while (idx < length) {
-      int inputRemaining = length - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
-      int internalRemaining = ARRAY_SIZE - elementIdx;
-      if (internalRemaining >= inputRemaining) {
-        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
-        break;
-      } else {
-        // the remaining inputs cannot fit the last array, fill the last array and create a new
-        // one and enter the next loop
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-        idx += internalRemaining;
-        size += internalRemaining;
-        checkExpansion();
-      }
-    }
+    PrimitiveArrayManager.release(values.remove(values.size() - 1));
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java
index ba7d262..5d9b014 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java
@@ -18,11 +18,11 @@
  */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.rescon.PrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -93,12 +93,12 @@ public class BooleanTVList extends TVList {
 
   public void sort() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
-      sortedTimestamps = (long[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.INT64, size);
+      sortedTimestamps = (long[][]) PrimitiveArrayManager
+          .createDataListsByType(TSDataType.INT64, size);
     }
     if (sortedValues == null || sortedValues.length < size) {
-      sortedValues = (boolean[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.BOOLEAN, size);
+      sortedValues = (boolean[][]) PrimitiveArrayManager
+          .createDataListsByType(TSDataType.BOOLEAN, size);
     }
     sort(0, size);
     clearSortedValue();
@@ -110,7 +110,7 @@ public class BooleanTVList extends TVList {
   void clearValue() {
     if (values != null) {
       for (boolean[] dataArray : values) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
+        PrimitiveArrayManager.release(dataArray);
       }
       values.clear();
     }
@@ -119,9 +119,6 @@ public class BooleanTVList extends TVList {
   @Override
   void clearSortedValue() {
     if (sortedValues != null) {
-      for (boolean[] dataArray : sortedValues) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
-      }
       sortedValues = null;
     }
   }
@@ -157,8 +154,7 @@ public class BooleanTVList extends TVList {
 
   @Override
   protected void expandValues() {
-    values.add((boolean[]) PrimitiveArrayPool
-        .getInstance().getPrimitiveDataListByType(TSDataType.BOOLEAN));
+    values.add((boolean[]) getPrimitiveArraysByType(TSDataType.BOOLEAN));
   }
 
   @Override
@@ -187,38 +183,7 @@ public class BooleanTVList extends TVList {
 
   @Override
   protected void releaseLastValueArray() {
-    PrimitiveArrayPool.getInstance().release(values.remove(values.size() - 1));
-  }
-
-  @Override
-  public void putBooleans(long[] time, boolean[] value) {
-    checkExpansion();
-    int idx = 0;
-    int length = time.length;
-
-    updateMinTimeAndSorted(time);
-
-    while (idx < length) {
-      int inputRemaining = length - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
-      int internalRemaining = ARRAY_SIZE - elementIdx;
-      if (internalRemaining >= inputRemaining) {
-        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
-        break;
-      } else {
-        // the remaining inputs cannot fit the last array, fill the last array and create a new
-        // one and enter the next loop
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-        idx += internalRemaining;
-        size += internalRemaining;
-        checkExpansion();
-      }
-    }
+    PrimitiveArrayManager.release(values.remove(values.size() - 1));
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java
index e141b75..817c6d1 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java
@@ -18,11 +18,11 @@
  */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.rescon.PrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.utils.MathUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -94,12 +94,12 @@ public class DoubleTVList extends TVList {
 
   public void sort() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
-      sortedTimestamps = (long[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.INT64, size);
+      sortedTimestamps = (long[][]) PrimitiveArrayManager
+          .createDataListsByType(TSDataType.INT64, size);
     }
     if (sortedValues == null || sortedValues.length < size) {
-      sortedValues = (double[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.DOUBLE, size);
+      sortedValues = (double[][]) PrimitiveArrayManager
+          .createDataListsByType(TSDataType.DOUBLE, size);
     }
     sort(0, size);
     clearSortedValue();
@@ -111,7 +111,7 @@ public class DoubleTVList extends TVList {
   void clearValue() {
     if (values != null) {
       for (double[] dataArray : values) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
+        PrimitiveArrayManager.release(dataArray);
       }
       values.clear();
     }
@@ -120,9 +120,6 @@ public class DoubleTVList extends TVList {
   @Override
   void clearSortedValue() {
     if (sortedValues != null) {
-      for (double[] dataArray : sortedValues) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
-      }
       sortedValues = null;
     }
   }
@@ -158,8 +155,7 @@ public class DoubleTVList extends TVList {
 
   @Override
   protected void expandValues() {
-    values.add((double[]) PrimitiveArrayPool
-        .getInstance().getPrimitiveDataListByType(TSDataType.DOUBLE));
+    values.add((double[]) getPrimitiveArraysByType(TSDataType.DOUBLE));
   }
 
   @Override
@@ -191,38 +187,7 @@ public class DoubleTVList extends TVList {
 
   @Override
   protected void releaseLastValueArray() {
-    PrimitiveArrayPool.getInstance().release(values.remove(values.size() - 1));
-  }
-
-  @Override
-  public void putDoubles(long[] time, double[] value) {
-    checkExpansion();
-    int idx = 0;
-    int length = time.length;
-
-    updateMinTimeAndSorted(time);
-
-    while (idx < length) {
-      int inputRemaining = length - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
-      int internalRemaining = ARRAY_SIZE - elementIdx;
-      if (internalRemaining >= inputRemaining) {
-        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
-        break;
-      } else {
-        // the remaining inputs cannot fit the last array, fill the last array and create a new
-        // one and enter the next loop
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-        idx += internalRemaining;
-        size += internalRemaining;
-        checkExpansion();
-      }
-    }
+    PrimitiveArrayManager.release(values.remove(values.size() - 1));
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java
index 178a4f7..8b02238 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java
@@ -18,11 +18,12 @@
  */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.rescon.PrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
+
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.utils.MathUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -94,12 +95,10 @@ public class FloatTVList extends TVList {
 
   public void sort() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
-      sortedTimestamps = (long[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.INT64, size);
+      sortedTimestamps = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
     }
     if (sortedValues == null || sortedValues.length < size) {
-      sortedValues = (float[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.FLOAT, size);
+      sortedValues = (float[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.FLOAT, size);
     }
     sort(0, size);
     clearSortedValue();
@@ -111,7 +110,7 @@ public class FloatTVList extends TVList {
   void clearValue() {
     if (values != null) {
       for (float[] dataArray : values) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
+        PrimitiveArrayManager.release(dataArray);
       }
       values.clear();
     }
@@ -120,9 +119,6 @@ public class FloatTVList extends TVList {
   @Override
   void clearSortedValue() {
     if (sortedValues != null) {
-      for (float[] dataArray : sortedValues) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
-      }
       sortedValues = null;
     }
   }
@@ -158,8 +154,7 @@ public class FloatTVList extends TVList {
 
   @Override
   protected void expandValues() {
-    values.add((float[]) PrimitiveArrayPool
-        .getInstance().getPrimitiveDataListByType(TSDataType.FLOAT));
+    values.add((float[]) getPrimitiveArraysByType(TSDataType.FLOAT));
   }
 
   @Override
@@ -191,38 +186,7 @@ public class FloatTVList extends TVList {
 
   @Override
   protected void releaseLastValueArray() {
-    PrimitiveArrayPool.getInstance().release(values.remove(values.size() - 1));
-  }
-
-  @Override
-  public void putFloats(long[] time, float[] value) {
-    checkExpansion();
-    int idx = 0;
-    int length = time.length;
-
-    updateMinTimeAndSorted(time);
-
-    while (idx < length) {
-      int inputRemaining = length - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
-      int internalRemaining = ARRAY_SIZE - elementIdx;
-      if (internalRemaining >= inputRemaining) {
-        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
-        break;
-      } else {
-        // the remaining inputs cannot fit the last array, fill the last array and create a new
-        // one and enter the next loop
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-        idx += internalRemaining;
-        size += internalRemaining;
-        checkExpansion();
-      }
-    }
+    PrimitiveArrayManager.release(values.remove(values.size() - 1));
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java
index 18c7c0f..183877f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java
@@ -18,11 +18,11 @@
  */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.rescon.PrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -93,12 +93,11 @@ public class IntTVList extends TVList {
 
   public void sort() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
-      sortedTimestamps = (long[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.INT64, size);
+      sortedTimestamps = (long[][]) PrimitiveArrayManager
+          .createDataListsByType(TSDataType.INT64, size);
     }
     if (sortedValues == null || sortedValues.length < size) {
-      sortedValues = (int[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.INT32, size);
+      sortedValues = (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, size);
     }
     sort(0, size);
     clearSortedValue();
@@ -110,7 +109,7 @@ public class IntTVList extends TVList {
   void clearValue() {
     if (values != null) {
       for (int[] dataArray : values) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
+        PrimitiveArrayManager.release(dataArray);
       }
       values.clear();
     }
@@ -119,9 +118,6 @@ public class IntTVList extends TVList {
   @Override
   void clearSortedValue() {
     if (sortedValues != null) {
-      for (int[] dataArray : sortedValues) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
-      }
       sortedValues = null;
     }
   }
@@ -157,8 +153,7 @@ public class IntTVList extends TVList {
 
   @Override
   protected void expandValues() {
-    values.add((int[]) PrimitiveArrayPool
-        .getInstance().getPrimitiveDataListByType(TSDataType.INT32));
+    values.add((int[]) getPrimitiveArraysByType(TSDataType.INT32));
   }
 
   @Override
@@ -187,38 +182,7 @@ public class IntTVList extends TVList {
 
   @Override
   protected void releaseLastValueArray() {
-    PrimitiveArrayPool.getInstance().release(values.remove(values.size() - 1));
-  }
-
-  @Override
-  public void putInts(long[] time, int[] value) {
-    checkExpansion();
-    int idx = 0;
-    int length = time.length;
-
-    updateMinTimeAndSorted(time);
-
-    while (idx < length) {
-      int inputRemaining = length - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
-      int internalRemaining = ARRAY_SIZE - elementIdx;
-      if (internalRemaining >= inputRemaining) {
-        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
-        break;
-      } else {
-        // the remaining inputs cannot fit the last array, fill the last array and create a new
-        // one and enter the next loop
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-        idx += internalRemaining;
-        size += internalRemaining;
-        checkExpansion();
-      }
-    }
+    PrimitiveArrayManager.release(values.remove(values.size() - 1));
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
index c5c198f..47efefd 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
@@ -18,11 +18,11 @@
  */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.rescon.PrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -93,12 +93,11 @@ public class LongTVList extends TVList {
 
   public void sort() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
-      sortedTimestamps = (long[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.INT64, size);
+      sortedTimestamps = (long[][]) PrimitiveArrayManager
+          .createDataListsByType(TSDataType.INT64, size);
     }
     if (sortedValues == null || sortedValues.length < size) {
-      sortedValues = (long[][]) PrimitiveArrayPool
-          .getInstance().getDataListsByType(TSDataType.INT64, size);
+      sortedValues = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
     }
     sort(0, size);
     clearSortedValue();
@@ -110,7 +109,7 @@ public class LongTVList extends TVList {
   void clearValue() {
     if (values != null) {
       for (long[] dataArray : values) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
+        PrimitiveArrayManager.release(dataArray);
       }
       values.clear();
     }
@@ -119,9 +118,6 @@ public class LongTVList extends TVList {
   @Override
   void clearSortedValue() {
     if (sortedValues != null) {
-      for (long[] dataArray : sortedValues) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
-      }
       sortedValues = null;
     }
   }
@@ -157,8 +153,7 @@ public class LongTVList extends TVList {
 
   @Override
   protected void expandValues() {
-    values.add((long[]) PrimitiveArrayPool
-        .getInstance().getPrimitiveDataListByType(TSDataType.INT64));
+    values.add((long[]) getPrimitiveArraysByType(TSDataType.INT64));
   }
 
   @Override
@@ -186,38 +181,7 @@ public class LongTVList extends TVList {
 
   @Override
   protected void releaseLastValueArray() {
-    PrimitiveArrayPool.getInstance().release(values.remove(values.size() - 1));
-  }
-
-  @Override
-  public void putLongs(long[] time, long[] value) {
-    checkExpansion();
-    int idx = 0;
-    int length = time.length;
-
-    updateMinTimeAndSorted(time);
-
-    while (idx < length) {
-      int inputRemaining = length - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
-      int internalRemaining = ARRAY_SIZE - elementIdx;
-      if (internalRemaining >= inputRemaining) {
-        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
-        break;
-      } else {
-        // the remaining inputs cannot fit the last array, fill the last array and create a new
-        // one and enter the next loop
-        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-        idx += internalRemaining;
-        size += internalRemaining;
-        checkExpansion();
-      }
-    }
+    PrimitiveArrayManager.release(values.remove(values.size() - 1));
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
index 7def4d0..13b4766 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
@@ -19,12 +19,13 @@
 
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.rescon.PrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
+
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -58,7 +59,7 @@ public abstract class TVList {
   public TVList() {
     timestamps = new ArrayList<>();
     size = 0;
-    minTime = Long.MIN_VALUE;
+    minTime = Long.MAX_VALUE;
   }
 
   public int size() {
@@ -98,30 +99,6 @@ public abstract class TVList {
     throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
   }
 
-  public void putLongs(long[] time, long[] value) {
-    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
-  }
-
-  public void putInts(long[] time, int[] value) {
-    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
-  }
-
-  public void putFloats(long[] time, float[] value) {
-    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
-  }
-
-  public void putDoubles(long[] time, double[] value) {
-    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
-  }
-
-  public void putBinaries(long[] time, Binary[] value) {
-    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
-  }
-
-  public void putBooleans(long[] time, boolean[] value) {
-    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
-  }
-
   public void putLongs(long[] time, long[] value, int start, int end) {
     throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
   }
@@ -200,7 +177,7 @@ public abstract class TVList {
   protected abstract void releaseLastValueArray();
 
   protected void releaseLastTimeArray() {
-    PrimitiveArrayPool.getInstance().release(timestamps.remove(timestamps.size() - 1));
+    PrimitiveArrayManager.release(timestamps.remove(timestamps.size() - 1));
   }
 
   public int delete(long lowerBound, long upperBound) {
@@ -239,7 +216,7 @@ public abstract class TVList {
   public void clear() {
     size = 0;
     sorted = true;
-    minTime = Long.MIN_VALUE;
+    minTime = Long.MAX_VALUE;
     clearTime();
     clearSortedTime();
 
@@ -253,7 +230,7 @@ public abstract class TVList {
   protected void clearTime() {
     if (timestamps != null) {
       for (long[] dataArray : timestamps) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
+        PrimitiveArrayManager.release(dataArray);
       }
       timestamps.clear();
     }
@@ -261,25 +238,29 @@ public abstract class TVList {
 
   protected void clearSortedTime() {
     if (sortedTimestamps != null) {
-      for (long[] dataArray : sortedTimestamps) {
-        PrimitiveArrayPool.getInstance().release(dataArray);
-      }
       sortedTimestamps = null;
     }
   }
 
   abstract void clearValue();
 
+  /**
+   * The arrays for sorting are not including in write memory now, 
+   * the memory usage is considered as temporary memory.
+   */
   abstract void clearSortedValue();
 
   protected void checkExpansion() {
     if ((size % ARRAY_SIZE) == 0) {
       expandValues();
-      timestamps.add(
-          (long[]) PrimitiveArrayPool.getInstance().getPrimitiveDataListByType(TSDataType.INT64));
+      timestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64));
     }
   }
 
+  protected Object getPrimitiveArraysByType(TSDataType dataType) {
+    return PrimitiveArrayManager.getPrimitiveArraysByType(dataType);
+  }
+
   protected long[] cloneTime(long[] array) {
     long[] cloneArray = new long[array.length];
     System.arraycopy(array, 0, cloneArray, 0, array.length);
@@ -340,6 +321,8 @@ public abstract class TVList {
         return new DoubleTVList();
       case BOOLEAN:
         return new BooleanTVList();
+      default:
+        break;
     }
     return null;
   }
@@ -494,6 +477,17 @@ public abstract class TVList {
     return new Ite(floatPrecision, encoding);
   }
 
+  public static long tvListArrayMemSize(TSDataType type) {
+    long size = 0;
+    // time size
+    size +=
+        PrimitiveArrayManager.ARRAY_SIZE * 8;
+    // value size
+    size +=
+        PrimitiveArrayManager.ARRAY_SIZE * type.getDataTypeSize();
+    return size;
+  }
+
   private class Ite implements IPointReader {
 
     private TimeValuePair cachedTimeValuePair;
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java b/server/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
index f206395..680ac65 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/node/ExclusiveWriteLogNode.java
@@ -89,8 +89,7 @@ public class ExclusiveWriteLogNode implements WriteLogNode, Comparable<Exclusive
       }
     } catch (BufferOverflowException e) {
       throw new IOException(
-          "Log cannot fit into buffer, if you don't enable Dynamic Parameter Adapter, please increase wal_buffer_size;"
-              + "otherwise, please increase the JVM memory", e);
+          "Log cannot fit into the buffer, please increase wal_buffer_size", e);
     } finally {
       lock.writeLock().unlock();
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/conf/adapter/ActiveTimeSeriesCounterTest.java b/server/src/test/java/org/apache/iotdb/db/conf/adapter/ActiveTimeSeriesCounterTest.java
deleted file mode 100644
index 0cd020f..0000000
--- a/server/src/test/java/org/apache/iotdb/db/conf/adapter/ActiveTimeSeriesCounterTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.conf.adapter;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.iotdb.db.concurrent.WrappedRunnable;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * ActiveTimeSeriesCounter Tester.
- */
-public class ActiveTimeSeriesCounterTest {
-
-  private static final String TEST_SG_PREFIX = "root.sg_";
-  private static int testStorageGroupNum = 10;
-  private static String[] storageGroups = new String[testStorageGroupNum];
-  private static int[] measurementNum = new int[testStorageGroupNum];
-  private static double totalSeriesNum = 0;
-
-  static {
-    for (int i = 0; i < testStorageGroupNum; i++) {
-      storageGroups[i] = TEST_SG_PREFIX + i;
-      measurementNum[i] = i + 1;
-      totalSeriesNum += measurementNum[i];
-    }
-  }
-
-  @Before
-  public void before() throws Exception {
-    ActiveTimeSeriesCounter.clear();
-    for (String storageGroup : storageGroups) {
-      ActiveTimeSeriesCounter.getInstance().init(storageGroup);
-    }
-  }
-
-  @After
-  public void after() throws Exception {
-    for (String storageGroup : storageGroups) {
-      ActiveTimeSeriesCounter.getInstance().delete(storageGroup);
-    }
-  }
-
-  /**
-   * Method: init(String storageGroup)
-   */
-  @Test
-  public void testInit() throws Exception {
-    for (int i = 0; i < testStorageGroupNum; i++) {
-      assertEquals(0D, ActiveTimeSeriesCounter.getInstance().getActiveRatio(storageGroups[i]), 0.0);
-    }
-  }
-
-  /**
-   * Method: updateActiveRatio(String storageGroup)
-   */
-  @Test
-  public void testUpdateActiveRatio() throws Exception {
-    ExecutorService service = Executors.newFixedThreadPool(storageGroups.length);
-    CountDownLatch finished = new CountDownLatch(storageGroups.length);
-    for (int i = 0; i < storageGroups.length; i++) {
-      service.submit(new OfferThreads(storageGroups[i], measurementNum[i], finished));
-    }
-    finished.await();
-    for (String storageGroup : storageGroups) {
-      ActiveTimeSeriesCounter.getInstance().updateActiveRatio(storageGroup);
-      double sum = 0;
-      for (String s : storageGroups) {
-        sum += ActiveTimeSeriesCounter.getInstance().getActiveRatio(s);
-      }
-      assertEquals(1.0, sum, 0.001);
-    }
-    for (int i = 0; i < storageGroups.length; i++) {
-      double r = ActiveTimeSeriesCounter.getInstance().getActiveRatio(storageGroups[i]);
-      assertEquals(measurementNum[i] / totalSeriesNum, r, 0.001);
-    }
-  }
-
-  private static class OfferThreads extends WrappedRunnable {
-    private int sensorNum;
-    private String storageGroup;
-    private CountDownLatch finished;
-
-    private OfferThreads(String storageGroup, int sensorNum, CountDownLatch finished) {
-      this.sensorNum = sensorNum;
-      this.storageGroup = storageGroup;
-      this.finished = finished;
-    }
-
-    @Override
-    public void runMayThrow() {
-      try {
-        for (int j = 0; j < sensorNum; j++) {
-          ActiveTimeSeriesCounter.getInstance().offer(storageGroup, "device_0", "sensor_" + j);
-        }
-      }finally {
-        finished.countDown();
-      }
-    }
-  }
-
-
-} 
diff --git a/server/src/test/java/org/apache/iotdb/db/conf/adapter/CompressionRatioTest.java b/server/src/test/java/org/apache/iotdb/db/conf/adapter/CompressionRatioTest.java
index 4269f8f..688762d 100644
--- a/server/src/test/java/org/apache/iotdb/db/conf/adapter/CompressionRatioTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/conf/adapter/CompressionRatioTest.java
@@ -49,7 +49,6 @@ public class CompressionRatioTest {
     FileUtils.forceMkdir(new File(directory));
     compressionRatio.reset();
     compressionRatio.restore();
-    IoTDBDescriptor.getInstance().getConfig().setEnableParameterAdapter(true);
   }
 
   @After
diff --git a/server/src/test/java/org/apache/iotdb/db/conf/adapter/HyperLogLogTest.java b/server/src/test/java/org/apache/iotdb/db/conf/adapter/HyperLogLogTest.java
deleted file mode 100644
index 09da1b1..0000000
--- a/server/src/test/java/org/apache/iotdb/db/conf/adapter/HyperLogLogTest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.conf.adapter;
-
-import static org.junit.Assert.assertEquals;
-
-import com.clearspring.analytics.stream.cardinality.HyperLogLog;
-import com.clearspring.analytics.stream.cardinality.ICardinality;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Random;
-import java.util.Set;
-import org.junit.Ignore;
-import org.junit.Test;
-
-public class HyperLogLogTest {
-
-  @Test
-  public void testStreamLibHll() {
-    final int seed = 12345;
-    // data on which to calculate distinct count
-    Random random = new Random(seed);
-    int sampleSize = 10000;
-    double[] floor = {0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.2, 1.4, 1.6, 1.8, 2.0};
-    String[] avgErrors = new String[floor.length];
-    int testNum = 1000;
-
-    for (double v : floor) {
-      double errorSum = 0;
-      for (int testIndex = 0; testIndex < testNum; testIndex++) {
-        final ArrayList<Integer> list = new ArrayList<>();
-        for (int i = 0; i < sampleSize; i++) {
-          list.add(random.nextInt((int) (sampleSize * v)));
-        }
-        Set<Integer> set = new HashSet<>();
-        ICardinality card = new HyperLogLog(ActiveTimeSeriesCounter.LOG2M);
-        for (int a : list) {
-          set.add(a);
-          card.offer(a);
-        }
-        double p = (card.cardinality() - set.size()) / (double) set.size();
-        errorSum += Math.abs(p);
-      }
-      // allow average error rate less than 1%
-      assertEquals(0, errorSum / testNum, 0.01);
-    }
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapterTest.java b/server/src/test/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapterTest.java
deleted file mode 100644
index 0b1ba26..0000000
--- a/server/src/test/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapterTest.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.conf.adapter;
-
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.ConfigAdjusterException;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter.MEMTABLE_NUM_FOR_EACH_PARTITION;
-import static org.junit.Assert.assertEquals;
-
-public class IoTDBConfigDynamicAdapterTest {
-
-  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
-
-  private long oldTsFileThreshold = CONFIG.getTsFileSizeThreshold();
-
-  private int oldMaxMemTableNumber = CONFIG.getMaxMemtableNumber();
-
-  private long oldGroupSizeInByte = CONFIG.getMemtableSizeThreshold();
-
-  @Before
-  public void setUp() throws Exception {
-    EnvironmentUtils.closeStatMonitor();
-    EnvironmentUtils.envSetUp();
-    IoTDBDescriptor.getInstance().getConfig().setEnableParameterAdapter(true);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    EnvironmentUtils.cleanEnv();
-    CONFIG.setMaxMemtableNumber(oldMaxMemTableNumber);
-    CONFIG.setTsFileSizeThreshold(oldTsFileThreshold);
-    CONFIG.setMemtableSizeThreshold(oldGroupSizeInByte);
-    IoTDB.metaManager.setMaxSeriesNumberAmongStorageGroup(0);
-    IoTDBConfigDynamicAdapter.getInstance().reset();
-  }
-
-  @Test
-  public void addOrDeleteStorageGroup() throws ConfigAdjusterException {
-    int memTableNum = IoTDBConfigDynamicAdapter.MEM_TABLE_AVERAGE_QUEUE_LEN;
-    for (int i = 1; i < 100; i++) {
-      IoTDBConfigDynamicAdapter.getInstance().addOrDeleteTimeSeries(1);
-    }
-    IoTDB.metaManager.setMaxSeriesNumberAmongStorageGroup(100);
-    for (int i = 1; i < 1000000; i++) {
-      try {
-        IoTDBConfigDynamicAdapter.getInstance().addOrDeleteStorageGroup(1);
-        memTableNum += IoTDBDescriptor.getInstance().getConfig().getConcurrentWritingTimePartition() * MEMTABLE_NUM_FOR_EACH_PARTITION + 1;
-        assertEquals(IoTDBConfigDynamicAdapter.getInstance().getCurrentMemTableSize(),
-            CONFIG.getMemtableSizeThreshold());
-        assertEquals(CONFIG.getMaxMemtableNumber(), memTableNum);
-      } catch (ConfigAdjusterException e) {
-        assertEquals(String.format(ConfigAdjusterException.ERROR_MSG_FORMAT,
-            IoTDBConfigDynamicAdapter.CREATE_STORAGE_GROUP), e.getMessage());
-        assertEquals(CONFIG.getMaxMemtableNumber(), memTableNum);
-        break;
-      }
-    }
-  }
-
-  @Test
-  public void addOrDeleteTimeSeries() throws ConfigAdjusterException {
-    int totalTimeseries = 0;
-    for (int i = 1; i < 100; i++) {
-      IoTDBConfigDynamicAdapter.getInstance().addOrDeleteStorageGroup(1);
-    }
-    IoTDB.metaManager.setMaxSeriesNumberAmongStorageGroup(100);
-    for (int i = 1; i < 1000000; i++) {
-      try {
-        IoTDBConfigDynamicAdapter.getInstance().addOrDeleteTimeSeries(1);
-
-        if (i % 10 == 0) {
-          IoTDB.metaManager.setMaxSeriesNumberAmongStorageGroup(i);
-        }
-        totalTimeseries += 1;
-        assertEquals(IoTDBConfigDynamicAdapter.getInstance().getCurrentMemTableSize(),
-            CONFIG.getMemtableSizeThreshold());
-        assertEquals(IoTDBConfigDynamicAdapter.getInstance().getTotalTimeseries(),
-            totalTimeseries);
-      } catch (ConfigAdjusterException e) {
-        assertEquals(String.format(ConfigAdjusterException.ERROR_MSG_FORMAT,
-            IoTDBConfigDynamicAdapter.ADD_TIMESERIES), e.getMessage());
-        assertEquals(IoTDBConfigDynamicAdapter.getInstance().getTotalTimeseries(),
-            totalTimeseries);
-        break;
-      }
-    }
-  }
-
-  @Test
-  public void addOrDeleteTimeSeriesSyso() throws ConfigAdjusterException {
-    int sgNum = 1;
-    for (int i = 0; i < 30; i++) {
-      IoTDBConfigDynamicAdapter.getInstance().addOrDeleteStorageGroup(sgNum);
-    }
-    int i = 1;
-    try {
-      for (; i <= 280 * 3200; i++) {
-        IoTDBConfigDynamicAdapter.getInstance().addOrDeleteTimeSeries(1);
-        IoTDB.metaManager.setMaxSeriesNumberAmongStorageGroup(i / 30 + 1);
-      }
-    } catch (ConfigAdjusterException e) {
-      assertEquals(String.format(ConfigAdjusterException.ERROR_MSG_FORMAT,
-          IoTDBConfigDynamicAdapter.ADD_TIMESERIES), e.getMessage());
-    }
-    try {
-      while (true) {
-        IoTDBConfigDynamicAdapter.getInstance().addOrDeleteTimeSeries(1);
-        IoTDB.metaManager.setMaxSeriesNumberAmongStorageGroup(IoTDB.metaManager.getMaximalSeriesNumberAmongStorageGroups() + 1);
-      }
-    } catch (ConfigAdjusterException e ) {
-      assertEquals(String.format(ConfigAdjusterException.ERROR_MSG_FORMAT,
-          IoTDBConfigDynamicAdapter.ADD_TIMESERIES), e.getMessage());
-    }
-  }
-}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/cache/ChunkMetadataCacheTest.java b/server/src/test/java/org/apache/iotdb/db/engine/cache/ChunkMetadataCacheTest.java
index 6608b95..abf89e6 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/cache/ChunkMetadataCacheTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/cache/ChunkMetadataCacheTest.java
@@ -22,7 +22,6 @@ import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
 import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy.DirectFlushPolicy;
@@ -67,7 +66,6 @@ public class ChunkMetadataCacheTest {
   public void setUp() throws Exception {
     EnvironmentUtils.envSetUp();
     MetadataManagerHelper.initMetadata();
-    ActiveTimeSeriesCounter.getInstance().init(storageGroup);
     storageGroupProcessor = new StorageGroupProcessor(systemDir, storageGroup,
         new DirectFlushPolicy());
     insertData();
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
index 3dc173a..d175cac 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.concurrent.ExecutionException;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
 import org.apache.iotdb.db.engine.flush.MemTableFlushTask;
@@ -49,7 +48,6 @@ public class MemTableFlushTaskTest {
   public void setUp() throws Exception {
     EnvironmentUtils.envSetUp();
     MetadataManagerHelper.initMetadata();
-    ActiveTimeSeriesCounter.getInstance().init(storageGroup);
     writer = new RestorableTsFileIOWriter(FSFactoryProducer.getFSFactory().getFile(filePath));
     memTable = new PrimitiveMemTable();
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTablePoolTest.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTablePoolTest.java
deleted file mode 100644
index 5c59d54..0000000
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTablePoolTest.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.engine.memtable;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-import org.apache.iotdb.db.rescon.MemTablePool;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class MemTablePoolTest {
-
-  private ConcurrentLinkedQueue<IMemTable> memTables;
-  private Thread thread = new ReturnThread();
-  private volatile boolean isFinished = false;
-
-  @Before
-  public void setUp() throws Exception {
-    memTables = new ConcurrentLinkedQueue();
-    thread.start();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    isFinished = true;
-    thread.join();
-  }
-
-  @Test
-  public void testGetAndRelease() {
-    long time = System.currentTimeMillis();
-    for (int i = 0; i < 10; i++) {
-      IMemTable memTable = MemTablePool.getInstance().getAvailableMemTable("test case");
-      memTables.add(memTable);
-    }
-    time -= System.currentTimeMillis();
-    System.out.println("memtable pool use deque and synchronized consume:" + time);
-  }
-
-  class ReturnThread extends Thread {
-
-    @Override
-    public void run() {
-      while (true) {
-        if (isInterrupted()) {
-          break;
-        }
-        IMemTable memTable = memTables.poll();
-        if (memTable == null) {
-          if (isFinished) {
-            break;
-          }
-          try {
-            Thread.sleep(10);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-          }
-          continue;
-        }
-        memTables.remove(memTable);
-        MemTablePool.getInstance().putBack(memTable, "test case");
-      }
-    }
-  }
-
-}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemtableBenchmark.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemtableBenchmark.java
index 5d0e640..6432dcd 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemtableBenchmark.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemtableBenchmark.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.engine.memtable;
 
+import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
@@ -40,7 +41,7 @@ public class MemtableBenchmark {
     }
   }
 
-  public static void main(String[] args) {
+  public static void main(String[] args) throws WriteProcessException {
     IMemTable memTable = new PrimitiveMemTable();
     final long startTime = System.currentTimeMillis();
     // cpu not locality
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
index 8eaef71..94c6378 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Random;
+
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
index a605e91..61c66c0 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
@@ -40,8 +40,6 @@ import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.metadata.mnode.MNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.service.IoTDB;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
index ffc4871..189d910 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
@@ -32,8 +32,6 @@ import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.metadata.mnode.MNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
 import org.apache.iotdb.db.query.executor.QueryRouter;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java b/server/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java
index edeb36f..0137403 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java
@@ -28,8 +28,6 @@ import java.util.List;
 
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class ModificationFileTest {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
index ed7c16b..9d9172a 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
@@ -24,7 +24,6 @@ import java.util.Collections;
 import java.util.List;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
 import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
@@ -70,7 +69,6 @@ public class StorageGroupProcessorTest {
         .setTsFileManagementStrategy(TsFileManagementStrategy.NORMAL_STRATEGY);
     MetadataManagerHelper.initMetadata();
     EnvironmentUtils.envSetUp();
-    ActiveTimeSeriesCounter.getInstance().init(storageGroup);
     processor = new DummySGP(systemDir, storageGroup);
     MergeManager.getINSTANCE().start();
   }
@@ -106,7 +104,7 @@ public class StorageGroupProcessorTest {
       processor.insert(new InsertRowPlan(record));
     }
 
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessor()) {
+    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
       tsfileProcessor.syncFlush();
     }
 
@@ -119,7 +117,7 @@ public class StorageGroupProcessorTest {
     processor.delete(new PartialPath(deviceId, measurementId), 0, 15L, -1);
 
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessor()) {
+    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
       tsfileProcessor
           .query(deviceId, measurementId, TSDataType.INT32, TSEncoding.RLE, Collections.emptyMap(),
               new QueryContext(), tsfileResourcesForQuery);
@@ -290,7 +288,7 @@ public class StorageGroupProcessorTest {
 
     processor.syncCloseAllWorkingTsFileProcessors();
 
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessor()) {
+    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
       tsfileProcessor.syncFlush();
     }
 
@@ -373,7 +371,7 @@ public class StorageGroupProcessorTest {
     processor.asyncCloseAllWorkingTsFileProcessors();
     processor.syncCloseAllWorkingTsFileProcessors();
 
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessor()) {
+    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
       tsfileProcessor.syncFlush();
     }
 
@@ -456,7 +454,7 @@ public class StorageGroupProcessorTest {
     processor.asyncCloseAllWorkingTsFileProcessors();
     processor.syncCloseAllWorkingTsFileProcessors();
 
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessor()) {
+    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
       tsfileProcessor.syncFlush();
     }
 
@@ -539,7 +537,7 @@ public class StorageGroupProcessorTest {
     processor.asyncCloseAllWorkingTsFileProcessors();
     processor.syncCloseAllWorkingTsFileProcessors();
 
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessor()) {
+    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
       tsfileProcessor.syncFlush();
     }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
index f3d3f0b..a1151c6 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
@@ -28,7 +28,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
@@ -36,10 +35,10 @@ import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.engine.version.SysTimeVersionController;
 import org.apache.iotdb.db.exception.TsFileProcessorException;
 import org.apache.iotdb.db.exception.WriteProcessException;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -59,6 +58,7 @@ public class TsFileProcessorTest {
 
   private TsFileProcessor processor;
   private String storageGroup = "storage_group1";
+  private StorageGroupInfo sgInfo = new StorageGroupInfo(null);
   private String filePath = TestConstant.OUTPUT_DATA_DIR
       .concat("testUnsealedTsFileProcessor.tsfile");
   private String deviceId = "root.vehicle.d0";
@@ -73,7 +73,6 @@ public class TsFileProcessorTest {
   public void setUp() throws Exception {
     EnvironmentUtils.envSetUp();
     MetadataManagerHelper.initMetadata();
-    ActiveTimeSeriesCounter.getInstance().init(storageGroup);
     context = EnvironmentUtils.TEST_QUERY_CONTEXT;
   }
 
@@ -86,10 +85,16 @@ public class TsFileProcessorTest {
   @Test
   public void testWriteAndFlush() throws IOException, WriteProcessException, MetadataException {
     logger.info("testWriteAndFlush begin..");
-    processor = new TsFileProcessor(storageGroup, SystemFileFactory.INSTANCE.getFile(filePath),
+    processor = new TsFileProcessor(storageGroup, SystemFileFactory.INSTANCE.getFile(filePath), sgInfo,
         SysTimeVersionController.INSTANCE, this::closeTsFileProcessor,
         (tsFileProcessor) -> true, true);
 
+    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
+    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
+    this.sgInfo.initTsFileProcessorInfo(processor);
+    tsFileProcessorInfo.addTSPMemCost(processor
+        .getTsFileResource().calculateRamSize());
+    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
     processor.query(deviceId, measurementId, dataType, encoding, props, context,
         tsfileResourcesForQuery);
@@ -137,10 +142,16 @@ public class TsFileProcessorTest {
   public void testWriteAndRestoreMetadata()
       throws IOException, WriteProcessException, MetadataException {
     logger.info("testWriteAndRestoreMetadata begin..");
-    processor = new TsFileProcessor(storageGroup, SystemFileFactory.INSTANCE.getFile(filePath),
+    processor = new TsFileProcessor(storageGroup, SystemFileFactory.INSTANCE.getFile(filePath), sgInfo,
         SysTimeVersionController.INSTANCE, this::closeTsFileProcessor,
         (tsFileProcessor) -> true, true);
 
+    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
+    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
+    this.sgInfo.initTsFileProcessorInfo(processor);
+    tsFileProcessorInfo.addTSPMemCost(processor
+        .getTsFileResource().calculateRamSize());
+    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
     processor.query(deviceId, measurementId, dataType, encoding, props, context,
         tsfileResourcesForQuery);
@@ -214,10 +225,16 @@ public class TsFileProcessorTest {
   @Test
   public void testMultiFlush() throws IOException, WriteProcessException, MetadataException {
     logger.info("testWriteAndRestoreMetadata begin..");
-    processor = new TsFileProcessor(storageGroup, SystemFileFactory.INSTANCE.getFile(filePath),
+    processor = new TsFileProcessor(storageGroup, SystemFileFactory.INSTANCE.getFile(filePath), sgInfo,
         SysTimeVersionController.INSTANCE, this::closeTsFileProcessor,
         (tsFileProcessor) -> true, true);
 
+    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
+    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
+    this.sgInfo.initTsFileProcessorInfo(processor);
+    tsFileProcessorInfo.addTSPMemCost(processor
+        .getTsFileResource().calculateRamSize());
+    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
     processor.query(deviceId, measurementId, dataType, encoding, props, context,
         tsfileResourcesForQuery);
@@ -249,10 +266,16 @@ public class TsFileProcessorTest {
   @Test
   public void testWriteAndClose() throws IOException, WriteProcessException, MetadataException {
     logger.info("testWriteAndRestoreMetadata begin..");
-    processor = new TsFileProcessor(storageGroup, SystemFileFactory.INSTANCE.getFile(filePath),
+    processor = new TsFileProcessor(storageGroup, SystemFileFactory.INSTANCE.getFile(filePath), sgInfo,
         SysTimeVersionController.INSTANCE, this::closeTsFileProcessor,
         (tsFileProcessor) -> true, true);
 
+    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
+    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
+    this.sgInfo.initTsFileProcessorInfo(processor);
+    tsFileProcessorInfo.addTSPMemCost(processor
+        .getTsFileResource().calculateRamSize());
+    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
 
     processor.query(deviceId, measurementId, dataType, encoding, props, context,
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
index c473e2c..f583b46 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
@@ -48,19 +48,15 @@ import org.junit.Test;
 public class MManagerBasicTest {
 
   private CompressionType compressionType;
-  private boolean canAdjust = IoTDBDescriptor.getInstance().getConfig().isEnableParameterAdapter();
 
   @Before
   public void setUp() throws Exception {
-    canAdjust = IoTDBDescriptor.getInstance().getConfig().isEnableParameterAdapter();
     compressionType = TSFileDescriptor.getInstance().getConfig().getCompressor();
     EnvironmentUtils.envSetUp();
-    IoTDBDescriptor.getInstance().getConfig().setEnableParameterAdapter(true);
   }
 
   @After
   public void tearDown() throws Exception {
-    IoTDBDescriptor.getInstance().getConfig().setEnableParameterAdapter(canAdjust);
     EnvironmentUtils.cleanEnv();
   }
 
@@ -338,28 +334,6 @@ public class MManagerBasicTest {
   }
 
   @Test
-  public void testMaximalSeriesNumberAmongStorageGroup() throws MetadataException {
-    MManager manager = IoTDB.metaManager;
-    assertEquals(0, manager.getMaximalSeriesNumberAmongStorageGroups());
-    manager.setStorageGroup(new PartialPath("root.laptop"));
-    assertEquals(0, manager.getMaximalSeriesNumberAmongStorageGroups());
-    manager.createTimeseries(new PartialPath("root.laptop.d1.s1"), TSDataType.INT32, TSEncoding.PLAIN,
-        CompressionType.GZIP, null);
-    manager.createTimeseries(new PartialPath("root.laptop.d1.s2"), TSDataType.INT32, TSEncoding.PLAIN,
-        CompressionType.GZIP, null);
-    assertEquals(2, manager.getMaximalSeriesNumberAmongStorageGroups());
-    manager.setStorageGroup(new PartialPath("root.vehicle"));
-    manager.createTimeseries(new PartialPath("root.vehicle.d1.s1"), TSDataType.INT32, TSEncoding.PLAIN,
-        CompressionType.GZIP, null);
-    assertEquals(2, manager.getMaximalSeriesNumberAmongStorageGroups());
-
-    manager.deleteTimeseries(new PartialPath("root.laptop.d1.s1"));
-    assertEquals(1, manager.getMaximalSeriesNumberAmongStorageGroups());
-    manager.deleteTimeseries(new PartialPath("root.laptop.d1.s2"));
-    assertEquals(1, manager.getMaximalSeriesNumberAmongStorageGroups());
-  }
-
-  @Test
   public void testGetStorageGroupNameByAutoLevel() {
     int level = IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel();
     boolean caughtException;
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java
index 16a0ee2..614117e 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java
@@ -247,7 +247,7 @@ public class LogicalPlanSmallTest {
               + "CONFIGURATION.DESCRIBE.SLIMIT.LIMIT.UNLINK.OFFSET.SOFFSET.FILL.LINEAR.PREVIOUS.PREVIOUSUNTILLAST."
               + "METADATA.TIMESERIES.TIMESTAMP.PROPERTY.WITH.DATATYPE.COMPRESSOR.STORAGE.GROUP.LABEL.ADD."
               + "UPSERT.VALUES.NOW.LINK.INDEX.USING.ON.DROP.MERGE.LIST.USER.PRIVILEGES.ROLE.ALL.OF."
-              + "ALTER.PASSWORD.REVOKE.LOAD.WATERMARK_EMBEDDING.UNSET.TTL.FLUSH.TASK.INFO.DYNAMIC.PARAMETER.VERSION."
+              + "ALTER.PASSWORD.REVOKE.LOAD.WATERMARK_EMBEDDING.UNSET.TTL.FLUSH.TASK.INFO.VERSION."
               + "REMOVE.MOVE.CHILD.PATHS.DEVICES.COUNT.NODES.LEVEL.MIN_TIME.MAX_TIME.MIN_VALUE.MAX_VALUE.AVG."
               + "FIRST_VALUE.SUM.LAST_VALUE.LAST.DISABLE.ALIGN.COMPRESSION.TIME.ATTRIBUTES.TAGS.RENAME.FULL.CLEAR.CACHE."
               + "SNAPSHOT.FOR.SCHEMA.TRACING.OFF where time>=1 and time < 3";
@@ -255,7 +255,6 @@ public class LogicalPlanSmallTest {
       Operator op = parseDriver.parse(sql, ZoneId.systemDefault());
       Assert.assertEquals(DeleteDataOperator.class, op.getClass());
     } catch (ParseCancellationException ignored) {
-
     }
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
index 2896ea7..0194a92 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
@@ -710,14 +710,6 @@ public class PhysicalPlanTest {
   }
 
   @Test
-  public void testShowDynamicParameter() throws QueryProcessException {
-    String metadata = "show dynamic parameter";
-    Planner processor = new Planner();
-    ShowPlan plan = (ShowPlan) processor.parseSQLToPhysicalPlan(metadata);
-    assertEquals("SHOW DYNAMIC_PARAMETER", plan.toString());
-  }
-
-  @Test
   public void testShowFlushInfo() throws QueryProcessException {
     String metadata = "show flush task info";
     Planner processor = new Planner();
diff --git a/server/src/test/java/org/apache/iotdb/db/query/reader/ReaderTestHelper.java b/server/src/test/java/org/apache/iotdb/db/query/reader/ReaderTestHelper.java
deleted file mode 100644
index 28135c1..0000000
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/ReaderTestHelper.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.query.reader;
-
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
-import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.MetadataManagerHelper;
-import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy.DirectFlushPolicy;
-import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.junit.After;
-import org.junit.Before;
-
-import java.io.IOException;
-
-public abstract class ReaderTestHelper {
-
-  private String storageGroup = "root.vehicle";
-  protected String deviceId = "root.vehicle.d0";
-  protected String measurementId = "s0";
-  protected TSDataType dataType = TSDataType.INT32;
-  protected StorageGroupProcessor storageGroupProcessor;
-  private String systemDir = TestConstant.OUTPUT_DATA_DIR.concat("info");
-
-  static {
-    IoTDB.metaManager.init();
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    EnvironmentUtils.envSetUp();
-    MetadataManagerHelper.initMetadata();
-    ActiveTimeSeriesCounter.getInstance().init(storageGroup);
-    storageGroupProcessor = new StorageGroupProcessor(systemDir, storageGroup, new DirectFlushPolicy());
-    insertData();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    FileReaderManager.getInstance().closeAndRemoveAllOpenedReaders();
-    storageGroupProcessor.syncDeleteDataFiles();
-    EnvironmentUtils.cleanEnv();
-    EnvironmentUtils.cleanDir(systemDir);
-  }
-
-  abstract protected void insertData() throws IOException, QueryProcessException;
-
-}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
index b283e16..6bba93d 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
@@ -32,7 +32,6 @@ import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.auth.authorizer.BasicAuthorizer;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.StorageEngine;
@@ -44,6 +43,8 @@ import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.control.TracingManager;
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransport;
@@ -68,8 +69,6 @@ public class EnvironmentUtils {
 
   private static final long oldTsFileThreshold = config.getTsFileSizeThreshold();
 
-  private static final int oldMaxMemTableNumber = config.getMaxMemtableNumber();
-
   private static final long oldGroupSizeInByte = config.getMemtableSizeThreshold();
 
   private static IoTDB daemon;
@@ -151,15 +150,16 @@ public class EnvironmentUtils {
       TracingManager.getInstance().close();
     }
 
+    // close array manager
+    PrimitiveArrayManager.close();
+
+    // clear system info
+    SystemInfo.getInstance().close();
+
     // delete all directory
     cleanAllDir();
-
-    config.setMaxMemtableNumber(oldMaxMemTableNumber);
     config.setTsFileSizeThreshold(oldTsFileThreshold);
     config.setMemtableSizeThreshold(oldGroupSizeInByte);
-    IoTDBConfigDynamicAdapter.getInstance().reset();
-
-    logger.warn("EnvironmentUtil cleanEnv done.");
   }
 
   public static void cleanAllDir() throws IOException {
@@ -214,9 +214,6 @@ public class EnvironmentUtils {
       fail(e.getMessage());
     }
 
-    IoTDBDescriptor.getInstance().getConfig().setEnableParameterAdapter(false);
-    IoTDBConfigDynamicAdapter.getInstance().setInitialized(true);
-
     createAllDir();
     // disable the system monitor
     config.setEnableStatMonitor(false);
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/LongTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/LongTVListTest.java
index 8f2783b..8be3429 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/LongTVListTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/LongTVListTest.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.utils.datastructure;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
+
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsLong;
 import org.junit.Assert;
@@ -73,21 +74,4 @@ public class LongTVListTest {
       Assert.assertEquals(inputs.get((int)i).getValue().getLong(), tvList.getLong((int)i));
     }
   }
-
-
-  @Test
-  public void compareLongTVListSortTime() {
-    for (int j = 0; j < 100; j++) {
-      LongTVList tvList = new LongTVList();
-      for (long i = 0; i < 1000; i++) {
-        tvList.putLong(i, i);
-      }
-      tvList.sort();
-      for (int i = 0; i < tvList.size; i++) {
-        tvList.getLong(i);
-        tvList.getTime(i);
-      }
-    }
-  }
-
 }
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
index 563adce..0283cfd 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
@@ -27,7 +27,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -184,8 +183,6 @@ public class RecoverResourceFromReaderTest {
 
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix, versionController,
         resource, false, false);
-    ActiveTimeSeriesCounter.getInstance()
-        .init(resource.getTsFile().getParentFile().getParentFile().getName());
     performer.recover().close();
     assertEquals(1, resource.getStartTime("root.sg.device99"));
     assertEquals(300, resource.getEndTime("root.sg.device99"));
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
index fe1df10..aaa080c 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
@@ -31,7 +31,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.commons.io.FileUtils;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -172,7 +171,6 @@ public class SeqTsFileRecoverTest {
   public void testNonLastRecovery() throws StorageGroupProcessorException, IOException {
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix, versionController,
         resource, false, false);
-    ActiveTimeSeriesCounter.getInstance().init(storageGroup);
     RestorableTsFileIOWriter writer = performer.recover();
     assertFalse(writer.canWrite());
     writer.close();
@@ -222,7 +220,6 @@ public class SeqTsFileRecoverTest {
   public void testLastRecovery() throws StorageGroupProcessorException, IOException {
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix, versionController,
         resource, false, true);
-    ActiveTimeSeriesCounter.getInstance().init(storageGroup);
     RestorableTsFileIOWriter writer = performer.recover();
 
     writer.makeMetadataVisible();
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
index d6d51fd..e9c411c 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
@@ -26,7 +26,6 @@ import java.io.IOException;
 import java.util.Collections;
 import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -181,8 +180,6 @@ public class UnseqTsFileRecoverTest {
   public void test() throws StorageGroupProcessorException, IOException {
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix,
         versionController, resource, false, false);
-    ActiveTimeSeriesCounter.getInstance()
-        .init(resource.getTsFile().getParentFile().getParentFile().getName());
     performer.recover().close();
 
     assertEquals(1, resource.getStartTime("root.sg.device99"));
diff --git a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
index ff0ebdd..ad59aea 100644
--- a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
+++ b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
@@ -677,7 +677,7 @@ public class IoTDBSessionComplexIT {
     }
   }
 
-  public void deleteStorageGroupTest() throws ClassNotFoundException, SQLException,
+  private void deleteStorageGroupTest() throws ClassNotFoundException, SQLException,
       IoTDBConnectionException, StatementExecutionException {
     try {
       session.deleteStorageGroup("root.sg1.d1.s1");
diff --git a/site/src/main/.vuepress/config.js b/site/src/main/.vuepress/config.js
index 0dedaa7..e1c6c8a 100644
--- a/site/src/main/.vuepress/config.js
+++ b/site/src/main/.vuepress/config.js
@@ -460,7 +460,6 @@ var config = {
 						title: 'System Tools',
 						children: [
 							['System Tools/Sync Tool','Sync Tool'],
-							['System Tools/Memory Estimation Tool','Memory Estimation Tool'],
 							['System Tools/JMX Tool','JMX Tool'],
 							['System Tools/Watermark Tool','Watermark Tool'],
 							['System Tools/Query History Visualization Tool','Query History Visualization Tool'],
@@ -981,7 +980,6 @@ var config = {
 						title: '系统工具',
 						children: [
 							['System Tools/Sync Tool','同步工具'],
-							['System Tools/Memory Estimation Tool','内存预估'],
 							['System Tools/JMX Tool','JMX工具'],
 							['System Tools/Watermark Tool','水印工具'],
 							['System Tools/Query History Visualization Tool','查询历史可视化工具'],
diff --git a/spark-iotdb-connector/src/test/scala/org/apache/iotdb/spark/db/EnvironmentUtils.java b/spark-iotdb-connector/src/test/scala/org/apache/iotdb/spark/db/EnvironmentUtils.java
index f54c6af..afa66ba 100644
--- a/spark-iotdb-connector/src/test/scala/org/apache/iotdb/spark/db/EnvironmentUtils.java
+++ b/spark-iotdb-connector/src/test/scala/org/apache/iotdb/spark/db/EnvironmentUtils.java
@@ -24,7 +24,6 @@ import org.apache.iotdb.db.auth.authorizer.BasicAuthorizer;
 import org.apache.iotdb.db.auth.authorizer.IAuthorizer;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.ChunkMetadataCache;
@@ -99,8 +98,6 @@ public class EnvironmentUtils {
 
   private static long oldTsFileThreshold = config.getTsFileSizeThreshold();
 
-  private static int oldMaxMemTableNumber = config.getMaxMemtableNumber();
-
   private static long oldGroupSizeInByte = config.getMemtableSizeThreshold();
 
   public static void cleanEnv() throws IOException, StorageEngineException {
@@ -131,10 +128,8 @@ public class EnvironmentUtils {
     // delete all directory
     cleanAllDir();
 
-    config.setMaxMemtableNumber(oldMaxMemTableNumber);
     config.setTsFileSizeThreshold(oldTsFileThreshold);
     config.setMemtableSizeThreshold(oldGroupSizeInByte);
-    IoTDBConfigDynamicAdapter.getInstance().reset();
   }
 
   public static void cleanAllDir() throws IOException {
@@ -171,10 +166,7 @@ public class EnvironmentUtils {
    * disable memory control</br> this function should be called before all code in the setup
    */
   public static void envSetUp() throws StartupException, IOException {
-    IoTDBDescriptor.getInstance().getConfig().setEnableParameterAdapter(false);
     IoTDB.metaManager.init();
-    IoTDBConfigDynamicAdapter.getInstance().setInitialized(true);
-
     createAllDir();
     // disable the system monitor
     config.setEnableStatMonitor(false);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
index c880a10..71e2596 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
@@ -259,6 +259,11 @@ public class ChunkMetadata implements Accountable {
         .calculateRamSize();
   }
 
+  public static long calculateRamSize(String measurementId, TSDataType dataType) {
+    return CHUNK_METADATA_FIXED_RAM_SIZE + RamUsageEstimator.sizeOf(measurementId) + Statistics
+        .getSizeByType(dataType);
+  }
+
   public void setRamSize(long size) {
     this.ramSize = size;
   }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
index 882d8dd..505be1b 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.tsfile.file.metadata.enums;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 
 public enum TSDataType {
   BOOLEAN, INT32, INT64, FLOAT, DOUBLE, TEXT;
@@ -98,6 +99,23 @@ public enum TSDataType {
     return enumToByte();
   }
 
+  public int getDataTypeSize() {
+    switch (this) {
+      case BOOLEAN:
+        return 1;
+      case INT32:
+      case FLOAT:
+        return 4;
+        // For text: return the size of reference here
+      case TEXT:
+      case INT64:
+      case DOUBLE:
+        return 8;
+      default:
+        throw new UnSupportedDataTypeException(this.toString());
+    }
+  }
+
   /**
    * @return byte number
    */
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BinaryStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BinaryStatistics.java
index ab6fc6f..607589b 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BinaryStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BinaryStatistics.java
@@ -36,7 +36,9 @@ public class BinaryStatistics extends Statistics<Binary> {
 
   private Binary firstValue = new Binary("");
   private Binary lastValue = new Binary("");
-  private static final String BINARY_STATS_UNSUPPORTED_MSG = "Binary statistics does not support: %s"; 
+  private static final String BINARY_STATS_UNSUPPORTED_MSG = "Binary statistics does not support: %s";
+  static final int BINARY_STATISTICS_FIXED_RAM_SIZE = 32;
+
   @Override
   public TSDataType getType() {
     return TSDataType.TEXT;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BooleanStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BooleanStatistics.java
index c769b80..f7c2d1c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BooleanStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/BooleanStatistics.java
@@ -33,7 +33,7 @@ public class BooleanStatistics extends Statistics<Boolean> {
   private boolean firstValue;
   private boolean lastValue;
 
-  private static final int BOOLEAN_STATISTICS_FIXED_RAM_SIZE = 48;
+  static final int BOOLEAN_STATISTICS_FIXED_RAM_SIZE = 48;
 
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/DoubleStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/DoubleStatistics.java
index 5a8351d..ef7293c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/DoubleStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/DoubleStatistics.java
@@ -35,7 +35,7 @@ public class DoubleStatistics extends Statistics<Double> {
   private double lastValue;
   private double sumValue;
 
-  private static final int DOUBLE_STATISTICS_FIXED_RAM_SIZE = 80;
+  static final int DOUBLE_STATISTICS_FIXED_RAM_SIZE = 80;
 
   @Override
   public TSDataType getType() {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/FloatStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/FloatStatistics.java
index 24a6a98..3080fd7 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/FloatStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/FloatStatistics.java
@@ -38,7 +38,7 @@ public class FloatStatistics extends Statistics<Float> {
   private float lastValue;
   private double sumValue;
 
-  private static final int FLOAT_STATISTICS_FIXED_RAM_SIZE = 64;
+  static final int FLOAT_STATISTICS_FIXED_RAM_SIZE = 64;
 
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/IntegerStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/IntegerStatistics.java
index a499f2e..4ca9e81 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/IntegerStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/IntegerStatistics.java
@@ -38,7 +38,7 @@ public class IntegerStatistics extends Statistics<Integer> {
   private int lastValue;
   private double sumValue;
 
-  private static final int INTEGER_STATISTICS_FIXED_RAM_SIZE = 64;
+  static final int INTEGER_STATISTICS_FIXED_RAM_SIZE = 64;
 
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/LongStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/LongStatistics.java
index 02fc9b3..e4958d3 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/LongStatistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/LongStatistics.java
@@ -35,7 +35,7 @@ public class LongStatistics extends Statistics<Long> {
   private long lastValue;
   private double sumValue;
 
-  private static final int LONG_STATISTICS_FIXED_RAM_SIZE = 80;
+  static final int LONG_STATISTICS_FIXED_RAM_SIZE = 80;
 
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
index ec9cb3c..7a31494 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
@@ -80,6 +80,25 @@ public abstract class Statistics<T> {
     }
   }
 
+  public static int getSizeByType(TSDataType type) {
+    switch (type) {
+      case INT32:
+        return IntegerStatistics.INTEGER_STATISTICS_FIXED_RAM_SIZE;
+      case INT64:
+        return LongStatistics.LONG_STATISTICS_FIXED_RAM_SIZE;
+      case TEXT:
+        return BinaryStatistics.BINARY_STATISTICS_FIXED_RAM_SIZE;
+      case BOOLEAN:
+        return BooleanStatistics.BOOLEAN_STATISTICS_FIXED_RAM_SIZE;
+      case DOUBLE:
+        return DoubleStatistics.DOUBLE_STATISTICS_FIXED_RAM_SIZE;
+      case FLOAT:
+        return FloatStatistics.FLOAT_STATISTICS_FIXED_RAM_SIZE;
+      default:
+        throw new UnknownColumnTypeException(type.toString());
+    }
+  }
+
   public abstract TSDataType getType();
 
   public int getSerializedSize() {