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 2022/04/10 12:44:27 UTC

[iotdb] branch master updated: [IOTDB-1614] New WAL (#5320)

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 3c81f567f4 [IOTDB-1614] New WAL (#5320)
3c81f567f4 is described below

commit 3c81f567f4f8adfe8df301c042dbbecb5d5485d1
Author: Alan Choo <43...@users.noreply.github.com>
AuthorDate: Sun Apr 10 20:44:22 2022 +0800

    [IOTDB-1614] New WAL (#5320)
---
 .../iotdb/cluster/metadata/CSchemaProcessor.java   |   7 +-
 .../iotdb/cluster/server/member/BaseMember.java    |   9 +-
 .../iotdb/cluster/utils/SerializeUtilTest.java     |   2 -
 docs/UserGuide/Data-Concept/Time-Partition.md      |   8 +-
 docs/UserGuide/Reference/Config-Manual.md          |  27 -
 docs/zh/UserGuide/Data-Concept/Time-Partition.md   |   2 -
 .../iotdb/session/IoTDBSessionComplexIT.java       |   7 +-
 .../iotdb/commons/concurrent/ThreadName.java       |   9 +-
 .../apache/iotdb/commons/conf/IoTDBConstant.java   |   6 +
 .../resources/conf/iotdb-engine.properties         | 118 ++---
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 235 +++++----
 .../org/apache/iotdb/db/conf/IoTDBConfigCheck.java |  26 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  | 135 ++---
 .../db/conf/directories/DirectoryManager.java      |   4 -
 .../iotdb/db/conf/directories/FolderManager.java   |  65 +++
 .../strategy/DirectoryStrategyType.java}           |  32 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |  84 ++-
 .../apache/iotdb/db/engine/StorageEngineV2.java    |  34 --
 .../org/apache/iotdb/db/engine/cq/CQLogReader.java |   2 +-
 .../org/apache/iotdb/db/engine/cq/CQLogWriter.java |   2 +-
 .../iotdb/db/engine/flush/FlushListener.java       |  17 +-
 .../flush/{FlushListener.java => FlushStatus.java} |  32 +-
 .../iotdb/db/engine/memtable/AbstractMemTable.java | 131 ++++-
 .../engine/memtable/AlignedWritableMemChunk.java   |  51 ++
 .../memtable/AlignedWritableMemChunkGroup.java     |  22 +
 .../apache/iotdb/db/engine/memtable/IMemTable.java |  10 +-
 .../db/engine/memtable/IWritableMemChunk.java      |   3 +-
 .../db/engine/memtable/IWritableMemChunkGroup.java |   3 +-
 .../iotdb/db/engine/memtable/WritableMemChunk.java |  28 +
 .../db/engine/memtable/WritableMemChunkGroup.java  |  37 ++
 .../db/engine/storagegroup/StorageGroupInfo.java   |  39 --
 .../db/engine/storagegroup/TsFileProcessor.java    | 153 +++---
 .../storagegroup/VirtualStorageGroupProcessor.java | 516 ++++++++-----------
 .../virtualSg/StorageGroupManager.java             |  43 +-
 .../engine/trigger/service/TriggerLogReader.java   |   2 +-
 .../engine/trigger/service/TriggerLogWriter.java   |   4 +-
 .../iotdb/db/metadata/logfile/MLogReader.java      |   2 +-
 .../iotdb/db/metadata/logfile/MLogWriter.java      |   2 +-
 .../db/metadata/schemaregion/SchemaRegion.java     |   2 -
 .../db/mpp/sql/parser/StatementGenerator.java      |   4 +-
 .../sql/planner/plan/node/write/InsertRowNode.java |  12 +-
 .../planner/plan/node/write/InsertTabletNode.java  |  14 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |   2 +
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |  61 +--
 .../iotdb/db/qp/physical/crud/DeletePlan.java      |  44 +-
 .../qp/physical/crud/InsertMultiTabletsPlan.java   |   4 +-
 .../iotdb/db/qp/physical/crud/InsertRowPlan.java   | 275 ++++++++--
 .../db/qp/physical/crud/InsertTabletPlan.java      | 462 +++++++++++------
 .../apache/iotdb/db/rescon/MemTableManager.java    |  14 +-
 .../java/org/apache/iotdb/db/service/IoTDB.java    |   4 +-
 .../iotdb/db/service/metrics/MetricsService.java   |  33 +-
 .../db/service/thrift/impl/TSServiceImpl.java      |   4 +-
 .../java/org/apache/iotdb/db/tools/WalChecker.java |  81 +--
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |  27 +-
 .../org/apache/iotdb/db/utils/OpenFileNumUtil.java |   3 +-
 .../apache/iotdb/db/utils/QueryDataSetUtils.java   | 101 +++-
 .../SerializedSize.java}                           |  37 +-
 .../db/utils/datastructure/AlignedTVList.java      | 190 +++++++
 .../iotdb/db/utils/datastructure/BinaryTVList.java |  37 ++
 .../db/utils/datastructure/BooleanTVList.java      |  33 ++
 .../iotdb/db/utils/datastructure/DoubleTVList.java |  32 ++
 .../iotdb/db/utils/datastructure/FloatTVList.java  |  32 ++
 .../iotdb/db/utils/datastructure/IntTVList.java    |  32 ++
 .../iotdb/db/utils/datastructure/LongTVList.java   |  32 ++
 .../iotdb/db/utils/datastructure/TVList.java       |  28 +-
 .../io => utils/writelog}/BatchLogReader.java      |   2 +-
 .../io => utils/writelog}/ILogReader.java          |   2 +-
 .../io => utils/writelog}/ILogWriter.java          |   2 +-
 .../{writelog/io => utils/writelog}/LogWriter.java |   2 +-
 .../io => utils/writelog}/MultiFileLogReader.java  |   2 +-
 .../io => utils/writelog}/SingleFileLogReader.java |   4 +-
 .../java/org/apache/iotdb/db/wal/WALManager.java   | 226 +++++++++
 .../iotdb/db/wal/buffer/AbstractWALBuffer.java     |  76 +++
 .../buffer/IWALBuffer.java}                        |  38 +-
 .../iotdb/db/wal/buffer/IWALByteBufferView.java    |  52 ++
 .../buffer/SignalWALEntry.java}                    |  38 +-
 .../org/apache/iotdb/db/wal/buffer/WALBuffer.java  | 454 +++++++++++++++++
 .../org/apache/iotdb/db/wal/buffer/WALEntry.java   | 219 ++++++++
 .../apache/iotdb/db/wal/buffer/WALEntryType.java   |  55 ++
 .../buffer/WALEntryValue.java}                     |  34 +-
 .../apache/iotdb/db/wal/checkpoint/Checkpoint.java | 102 ++++
 .../iotdb/db/wal/checkpoint/CheckpointManager.java | 257 ++++++++++
 .../checkpoint/CheckpointType.java}                |  48 +-
 .../iotdb/db/wal/checkpoint/MemTableInfo.java      | 113 +++++
 .../exception/WALException.java}                   |  33 +-
 .../exception/WALNodeClosedException.java}         |  29 +-
 .../exception/WALRecoverException.java}            |  31 +-
 .../apache/iotdb/db/wal/io/CheckpointReader.java   |  63 +++
 .../apache/iotdb/db/wal/io/CheckpointWriter.java   |  61 +++
 .../org/apache/iotdb/db/wal/io/ILogWriter.java     |  61 +++
 .../java/org/apache/iotdb/db/wal/io/LogWriter.java |  96 ++++
 .../java/org/apache/iotdb/db/wal/io/WALReader.java | 114 +++++
 .../java/org/apache/iotdb/db/wal/io/WALWriter.java |  60 +++
 .../org/apache/iotdb/db/wal/node/IWALNode.java     |  52 ++
 .../org/apache/iotdb/db/wal/node/WALFakeNode.java  | 118 +++++
 .../java/org/apache/iotdb/db/wal/node/WALNode.java | 382 ++++++++++++++
 .../db/wal/recover/CheckpointRecoverUtils.java     |  78 +++
 .../iotdb/db/wal/recover/WALNodeRecoverTask.java   | 151 ++++++
 .../iotdb/db/wal/recover/WALRecoverManager.java    | 172 +++++++
 .../file/AbstractTsFileRecoverPerformer.java       | 126 +++++
 .../recover/file/SealedTsFileRecoverPerformer.java |  54 ++
 .../db/wal/recover/file/TsFilePlanRedoer.java      | 159 ++++++
 .../file/UnsealedTsFileRecoverPerformer.java       | 271 ++++++++++
 .../FlushListener.java => wal/utils/WALMode.java}  |  43 +-
 .../apache/iotdb/db/wal/utils/WALWriteUtils.java   | 134 +++++
 .../wal/utils/listener/AbstractResultListener.java |  87 ++++
 .../utils/listener/WALFlushListener.java}          |  30 +-
 .../utils/listener/WALRecoverListener.java}        |  33 +-
 .../apache/iotdb/db/writelog/WALFlushListener.java |  50 --
 .../writelog/manager/MultiFileLogNodeManager.java  | 206 --------
 .../db/writelog/manager/WriteLogNodeManager.java   |  49 --
 .../db/writelog/node/ExclusiveWriteLogNode.java    | 435 ----------------
 .../iotdb/db/writelog/node/WriteLogNode.java       | 101 ----
 .../iotdb/db/writelog/recover/LogReplayer.java     | 242 ---------
 .../writelog/recover/TsFileRecoverPerformer.java   | 316 ------------
 .../db/engine/memtable/MemTableTestUtils.java      |   2 -
 .../db/engine/memtable/PrimitiveMemTableTest.java  |  32 +-
 .../storagegroup/StorageGroupProcessorTest.java    |  63 ---
 .../engine/storagegroup/TsFileProcessorTest.java   |   2 -
 .../db/metadata/upgrade/MetadataUpgradeTest.java   |   2 +-
 .../iotdb/db/qp/physical/SerializedSizeTest.java   | 130 +++++
 .../org/apache/iotdb/db/tools/WalCheckerTest.java  | 117 +++--
 .../io => utils/writelog}/LogWriterReaderTest.java |  15 +-
 .../writelog}/MultiFileLogReaderTest.java          |   7 +-
 .../org/apache/iotdb/db/wal/DisableWALTest.java    |  54 ++
 .../org/apache/iotdb/db/wal/WALManagerTest.java    | 107 ++++
 .../iotdb/db/wal/buffer/WALBufferCommonTest.java   | 158 ++++++
 .../apache/iotdb/db/wal/buffer/WALBufferTest.java} |  36 +-
 .../db/wal/checkpoint/CheckpointManagerTest.java   | 154 ++++++
 .../apache/iotdb/db/wal/io/CheckpointFileTest.java | 133 +++++
 .../org/apache/iotdb/db/wal/io/WALFileTest.java    | 222 ++++++++
 .../apache/iotdb/db/wal/node/WALFakeNodeTest.java  |  65 +++
 .../org/apache/iotdb/db/wal/node/WALNodeTest.java  | 262 ++++++++++
 .../db/wal/recover/WALRecoverManagerTest.java      | 466 +++++++++++++++++
 .../file/SealedTsFileRecoverPerformerTest.java     | 307 +++++++++++
 .../db/wal/recover/file/TsFilePlanRedoerTest.java  | 400 +++++++++++++++
 .../file/UnsealedTsFileRecoverPerformerTest.java   | 260 ++++++++++
 .../db/wal/utils/TsFileUtilsForRecoverTest.java}   |  46 +-
 .../iotdb/db/wal/utils/WALByteBufferForTest.java   |  75 +++
 .../iotdb/db/writelog/IoTDBLogFileSizeTest.java    | 220 --------
 .../apache/iotdb/db/writelog/PerformanceTest.java  | 219 --------
 .../iotdb/db/writelog/WriteLogNodeManagerTest.java | 185 -------
 .../apache/iotdb/db/writelog/WriteLogNodeTest.java | 451 ----------------
 .../db/writelog/recover/DeviceStringTest.java      | 132 -----
 .../iotdb/db/writelog/recover/LogReplayerTest.java | 288 -----------
 .../recover/RecoverResourceFromReaderTest.java     | 231 ---------
 .../db/writelog/recover/SeqTsFileRecoverTest.java  | 565 ---------------------
 .../writelog/recover/UnseqTsFileRecoverTest.java   | 277 ----------
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   1 +
 .../apache/iotdb/spark/db/EnvironmentUtils.java    |  14 +-
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |  43 +-
 .../org/apache/iotdb/tsfile/utils/TsFileUtils.java |  20 +
 .../tsfile/write/schema/IMeasurementSchema.java    |   2 +
 .../tsfile/write/schema/MeasurementSchema.java     |  18 +
 .../write/schema/VectorMeasurementSchema.java      |  13 +
 .../write/writer/RestorableTsFileIOWriter.java     |   5 +
 156 files changed, 9163 insertions(+), 5708 deletions(-)

diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CSchemaProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CSchemaProcessor.java
index cccf1bfe37..64ec9d5a7d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CSchemaProcessor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CSchemaProcessor.java
@@ -41,7 +41,6 @@ import org.apache.iotdb.cluster.server.member.MetaGroupMember;
 import org.apache.iotdb.cluster.utils.ClusterQueryUtils;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.storagegroup.VirtualStorageGroupProcessor;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
@@ -114,7 +113,6 @@ import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 import static org.apache.iotdb.cluster.query.ClusterPlanExecutor.LOG_FAIL_CONNECT;
@@ -286,9 +284,8 @@ public class CSchemaProcessor extends LocalSchemaProcessor {
   }
 
   /**
-   * the {@link org.apache.iotdb.db.writelog.recover.LogReplayer#replayLogs(Supplier,
-   * VirtualStorageGroupProcessor)} will call this to get schema after restart we should retry to
-   * get schema util we get the schema.
+   * the {@link org.apache.iotdb.db.wal.recover.file.UnsealedTsFileRecoverPerformer#redoLog} will
+   * call this to get schema after restart we should retry to get schema util we get the schema.
    *
    * @param deviceId the device id.
    * @param measurements the measurements.
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/BaseMember.java b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/BaseMember.java
index 2a96b6881d..382501c64d 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/BaseMember.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/BaseMember.java
@@ -59,6 +59,7 @@ import org.apache.iotdb.db.qp.executor.PlanExecutor;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.wal.utils.WALMode;
 import org.apache.iotdb.metrics.config.MetricConfigDescriptor;
 
 import org.apache.thrift.async.AsyncMethodCallback;
@@ -94,7 +95,7 @@ public class BaseMember {
   private boolean prevUseAsyncServer;
   private int preLogBufferSize;
   private boolean prevUseAsyncApplier;
-  private boolean prevEnableWAL;
+  private WALMode prevWALMode;
 
   private int syncLeaderMaxWait;
   private long heartBeatInterval;
@@ -110,8 +111,8 @@ public class BaseMember {
     ClusterDescriptor.getInstance().getConfig().setRaftLogBufferSize(409600);
     testThreadPool = Executors.newFixedThreadPool(4);
     prevLeaderWait = RaftMember.getWaitLeaderTimeMs();
-    prevEnableWAL = IoTDBDescriptor.getInstance().getConfig().isEnableWal();
-    IoTDBDescriptor.getInstance().getConfig().setEnableWal(false);
+    prevWALMode = IoTDBDescriptor.getInstance().getConfig().getWalMode();
+    IoTDBDescriptor.getInstance().getConfig().setWalMode(WALMode.DISABLE);
     MetricConfigDescriptor.getInstance().getMetricConfig().setEnableMetric(false);
     RaftMember.setWaitLeaderTimeMs(10);
 
@@ -200,7 +201,7 @@ public class BaseMember {
     ClusterDescriptor.getInstance().getConfig().setUseAsyncServer(prevUseAsyncServer);
     ClusterDescriptor.getInstance().getConfig().setRaftLogBufferSize(preLogBufferSize);
     ClusterDescriptor.getInstance().getConfig().setUseAsyncApplier(prevUseAsyncApplier);
-    IoTDBDescriptor.getInstance().getConfig().setEnableWal(prevEnableWAL);
+    IoTDBDescriptor.getInstance().getConfig().setWalMode(prevWALMode);
 
     ClusterConstant.setSyncLeaderMaxWaitMs(syncLeaderMaxWait);
     ClusterConstant.setHeartbeatIntervalMs(heartBeatInterval);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/utils/SerializeUtilTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/utils/SerializeUtilTest.java
index 9c758bf6e3..8a67592e73 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/utils/SerializeUtilTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/utils/SerializeUtilTest.java
@@ -115,8 +115,6 @@ public class SerializeUtilTest {
     tabletPlan.setTimes(times);
     tabletPlan.setColumns(columns);
     tabletPlan.setRowCount(times.length);
-    tabletPlan.setStart(0);
-    tabletPlan.setEnd(4);
 
     Log log = new PhysicalPlanLog(tabletPlan);
     log.setCurrLogTerm(1);
diff --git a/docs/UserGuide/Data-Concept/Time-Partition.md b/docs/UserGuide/Data-Concept/Time-Partition.md
index b77f3c6abe..1ed777b79e 100644
--- a/docs/UserGuide/Data-Concept/Time-Partition.md
+++ b/docs/UserGuide/Data-Concept/Time-Partition.md
@@ -25,8 +25,6 @@
 
 Time partition divides data according to time, and a time partition is used to save all data within a certain time range. The time partition number is represented by a natural number. Number 0 means January 1, 1970, it will increase by one every partition_interval seconds. Time partition number's calculation formula is timestamp / partition_interval. The main configuration items are as follows:
 
-Notice: Currently, it's not recommend to open this function. If open, please calculate appropriate concurrent_writing_time_partition and wal_buffer_size, you can calculate wal_buffer_size = MaxDirectMemorySizeInBytes * 0.3 / (storage_group_num * virtual_storage_group_num) / concurrent_writing_time_partition
-
 * enable\_partition
 
 |Name| enable\_partition |
@@ -57,10 +55,8 @@ Enable time partition and set partition_interval to 86400 (one day), then the da
 
 ## Suggestions
 
-When enabling time partition, it is better to enable timed flush memtable and timed close tsfile, configuration params are detailed in [Config manual for timed flush and timed close](../Reference/Config-Manual.md).
+When enabling time partition, it is better to enable timed flush memtable, configuration params are detailed in [Config manual for timed flush](../Reference/Config-Manual.md).
 
 * enable_timed_flush_unseq_memtable: Whether to enable timed flush unsequence memtable, enabled by default.
 
-* enable_timed_flush_seq_memtable: Whether to enable timed flush sequence memtable, disabled by default. It should be enabled when time partition is enabled, so inactive time partition's memtable can be flushed regularly to prepare for the timed close of TsFileProcessor.
-
-* enable_timed_close_tsfile: Whether to enable timed close tsfile, disabled by default. It should be enabled when time partition is enabled, so inactive time partition's TsFileProcessor can be closed regularly to reduce memory usage.
\ No newline at end of file
+* enable_timed_flush_seq_memtable: Whether to enable timed flush sequence memtable, disabled by default. It should be enabled when time partition is enabled, so inactive time partition's memtable can be flushed regularly.
\ No newline at end of file
diff --git a/docs/UserGuide/Reference/Config-Manual.md b/docs/UserGuide/Reference/Config-Manual.md
index b08da16e32..b2a6b2e20f 100644
--- a/docs/UserGuide/Reference/Config-Manual.md
+++ b/docs/UserGuide/Reference/Config-Manual.md
@@ -372,33 +372,6 @@ The permission definitions are in ${IOTDB\_CONF}/conf/jmx.access.
 |Default| 600000 |
 |Effective| Trigger |
 
-* enable\_timed\_close\_tsfile
-
-|Name| enable\_timed\_close\_tsfile |
-|:---:|:---|
-|Description| whether to timed close tsfiles |
-|Type|Bool|
-|Default| false |
-|Effective| Trigger |
-
-* close\_tsfile\_interval\_after\_flushing\_in\_ms
-
-|Name| close\_tsfile\_interval\_after\_flushing\_in\_ms |
-|:---:|:---|
-|Description| if a TsfileProcessor's last working memtable flush time is older than current time minus this and its working memtable is null, the TsfileProcessor will be closed |
-|Type|Int32|
-|Default| 3600000 |
-|Effective| Trigger |
-
-* close\_tsfile\_check\_interval\_in\_ms
-
-|Name| close\_tsfile\_check\_interval\_in\_ms |
-|:---:|:---|
-|Description| the interval to check whether tsfiles need closing |
-|Type|Int32|
-|Default| 600000 |
-|Effective| Trigger |
-
 * avg\_series\_point\_number\_threshold
 
 |Name| avg\_series\_point\_number\_threshold |
diff --git a/docs/zh/UserGuide/Data-Concept/Time-Partition.md b/docs/zh/UserGuide/Data-Concept/Time-Partition.md
index 8a1321b6b4..6b0dd721bc 100644
--- a/docs/zh/UserGuide/Data-Concept/Time-Partition.md
+++ b/docs/zh/UserGuide/Data-Concept/Time-Partition.md
@@ -25,8 +25,6 @@
 
 时间分区按照时间分割数据,一个时间分区用于保存某个时间范围内的所有数据。时间分区编号使用自然数表示,0 表示 1970 年 1 月 1 日,每隔 partition_interval 秒后加一。数据通过计算 timestamp / partition_interval 得到自己所在的时间分区编号,主要配置项如下所示:
 
-注意:当前不建议开启此功能。 如果打开,请计算合适的 concurrent_writing_time_partition 和 wal_buffer_size, 计算公式如 wal_buffer_size = MaxDirectMemorySizeInBytes * 0.3 / (storage_group_num * virtual_storage_group_num) / concurrent_writing_time_partition
-
 * enable\_partition
 
 |名字| enable\_partition |
diff --git a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
index 746bd5297d..29126d05f3 100644
--- a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
+++ b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.db.engine.trigger.example.Counter;
 import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
 import org.apache.iotdb.db.exception.TriggerManagementException;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.db.wal.utils.WALMode;
 import org.apache.iotdb.itbase.category.LocalStandaloneTest;
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
@@ -471,8 +472,8 @@ public class IoTDBSessionComplexIT {
     session.setStorageGroup("root.sg1");
     String deviceId = "root.sg1.d1";
 
-    boolean isEnableWAL = IoTDBDescriptor.getInstance().getConfig().isEnableWal();
-    IoTDBDescriptor.getInstance().getConfig().setEnableWal(false);
+    WALMode prevWalMode = IoTDBDescriptor.getInstance().getConfig().getWalMode();
+    IoTDBDescriptor.getInstance().getConfig().setWalMode(WALMode.DISABLE);
     createTimeseries();
 
     List<String> measurements = new ArrayList<>();
@@ -528,7 +529,7 @@ public class IoTDBSessionComplexIT {
     }
     Assert.assertEquals(201, count);
 
-    IoTDBDescriptor.getInstance().getConfig().setEnableWal(isEnableWAL);
+    IoTDBDescriptor.getInstance().getConfig().setWalMode(prevWalMode);
     session.close();
   }
 
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
index 4df1111ea7..2c86778dfa 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
@@ -39,10 +39,10 @@ public enum ThreadName {
   COMPACTION_SERVICE("Compaction"),
   COMPACTION_SUB_SERVICE("Sub-Compaction"),
   COMPACTION_SCHEDULE("Compaction_Schedule"),
-  WAL_DAEMON("WAL-Sync"),
-  WAL_FORCE_DAEMON("WAL-Force"),
-  WAL_TRIM("WAL-Trim"),
-  WAL_FLUSH("WAL-Flush"),
+  WAL_SERIALIZE("WAL-Serialize"),
+  WAL_SYNC("WAL-Sync"),
+  WAL_DELETE("WAL-Delete"),
+  WAL_RECOVER("WAL-Recover"),
   INDEX_SERVICE("Index"),
   SYNC_CLIENT("Sync-Client"),
   SYNC_SERVER("Sync"),
@@ -56,7 +56,6 @@ public enum ThreadName {
   TTL_CHECK_SERVICE("TTL-CHECK"),
   TIMED_FlUSH_SEQ_MEMTABLE("Timed-Flush-Seq-Memtable"),
   TIMED_FlUSH_UNSEQ_MEMTABLE("Timed-Flush-Unseq-Memtable"),
-  TIMED_CLOSE_TSFILE("Timed-Close-TsFile"),
   SETTLE_SERVICE("Settle"),
   CONTINUOUS_QUERY_SERVICE("ContinuousQueryTaskPoolManager"),
   CLUSTER_INFO_SERVICE("ClusterInfoClient"),
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
index a4f822b96d..de04e6e0de 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
@@ -138,6 +138,7 @@ public class IoTDBConstant {
   public static final String SDT_COMP_MAX_TIME = "compmaxtime";
 
   // data folder name
+  public static final String DATA_FOLDER_NAME = "data";
   public static final String SEQUENCE_FLODER_NAME = "sequence";
   public static final String UNSEQUENCE_FLODER_NAME = "unsequence";
   public static final String FILE_NAME_SEPARATOR = "-";
@@ -154,6 +155,7 @@ public class IoTDBConstant {
   public static final String UDF_FOLDER_NAME = "udf";
   public static final String TRIGGER_FOLDER_NAME = "trigger";
   public static final String MQTT_FOLDER_NAME = "mqtt";
+  public static final String WAL_FOLDER_NAME = "wal";
 
   // mqtt
   public static final String ENABLE_MQTT = "enable_mqtt_service";
@@ -188,6 +190,10 @@ public class IoTDBConstant {
   // compaction mods of previous version (<0.13)
   public static final String COMPACTION_MODIFICATION_FILE_NAME_FROM_OLD = "merge.mods";
 
+  // write ahead log
+  public static final String WAL_FILE_SUFFIX = ".wal";
+  public static final String WAL_CHECKPOINT_FILE_SUFFIX = ".checkpoint";
+
   // client version number
   public enum ClientVersion {
     V_0_12,
diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index a16b373130..ee675383ed 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -73,25 +73,53 @@ config_nodes=127.0.0.1:22277
 ### Write Ahead Log Configuration
 ####################
 
-# Is insert ahead log enable
-# Datatype: boolean
-# enable_wal=true
+# Write mode of wal
+# The details of these three modes are as follows:
+# 1. DISABLE: the system will disable wal.
+# 2. SYNC: the system will submit wal synchronously, write request will not return until its wal is fsynced to the disk successfully.
+# 3. ASYNC: the system will submit wal asynchronously, write request will return immediately no matter its wal is fsynced to the disk successfully.
+# The write performance order is DISABLE > ASYNC > SYNC, but only SYNC mode can ensure data durability.
+# wal_mode=ASYNC
+
+# Duration a wal flush operation will wait before calling fsync
+# A duration greater than 0 batches multiple wal fsync calls into one. This is useful when disks are slow or WAL write contention exists.
+# Notice: this value affects write performance significantly when wal mode is SYNC. For non-SSD disks, values in the range of 0ms-100ms are recommended.
+# Datatype: long
+# fsync_wal_delay_in_ms=10
 
-# Add a switch to drop ouf-of-order data
-# Out-of-order data will impact the aggregation query a lot. Users may not care about discarding some out-of-order data.
-# Datatype: boolean
-# enable_discard_out_of_order_data=false
+# Max number of wal nodes, each node corresponds to one wal directory
+# The default value 0 means twice the number of wal dirs.
+# Datatype: int
+# max_wal_nodes_num=0
 
-# When a certain amount of insert ahead log is reached, it will be flushed to disk
-# It is possible to lose at most flush_wal_threshold operations
+# Buffer size of each wal node
+# If it's a value smaller than 0, use the default value 16 * 1024 * 1024 bytes (16MB).
 # Datatype: int
-# flush_wal_threshold=10000
+# wal_buffer_size_in_byte=16777216
+
+# Size threshold of each wal file
+# When a wal file's size exceeds this, the wal file will be closed and a new wal file will be created.
+# If it's a value smaller than 0, use the default value 10 * 1024 * 1024 (10MB).
+# Datatype: long
+# wal_file_size_threshold_in_byte=10485760
 
-# The cycle when insert ahead log is periodically forced to be written to disk(in milliseconds)
-# If force_wal_period_in_ms = 0 it means force insert ahead log to be written to disk after each refreshment
-# Set this parameter to 0 may slow down the ingestion on slow disk.
+# TTL of wal file
+# When a wal file's alive time exceeds this, MemTable snapshot or flush will be triggered.
+# Reduce this value when wal occupies too much disk space. But, if this parameter is too small, the write performance may decline.
+# If it's a value smaller than 0, use the default value 24 * 60 * 60 * 1000 ms (24H).
 # Datatype: long
-# force_wal_period_in_ms=100
+# wal_file_ttl_in_ms=86400000
+
+# MemTable size threshold for triggering MemTable snapshot in wal
+# When a memTable's size (in byte) exceeds this, wal can flush this memtable to disk, otherwise wal will snapshot this memtable in wal.
+# If it's a value smaller than 0, use the default value 8 * 1024 * 1024 bytes (8MB).
+# Datatype: long
+# wal_memtable_snapshot_threshold_in_byte=8388608
+
+# The period when outdated wal files are periodically deleted
+# If it's a value smaller than 0, use the default value 10 * 60 * 1000 ms (10 minutes).
+# Datatype: long
+# delete_wal_files_period_in_ms=600000
 
 ####################
 ### Directory Configuration
@@ -113,7 +141,8 @@ config_nodes=127.0.0.1:22277
 # If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/data/data).
 # If it is absolute, system will save the data in exact location it points to.
 # If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder.
-# Note: If data_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
+# If there are more than one directory, please separate them by commas ",".
+# Note: If data_dirs is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
 # For windows platform
 # If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
 # data_dirs=data\\data
@@ -123,7 +152,7 @@ config_nodes=127.0.0.1:22277
 
 
 # mult_dir_strategy
-# The strategy is used to choose a directory from tsfile_dir for the system to store a new tsfile.
+# The strategy is used to choose a directory from data_dirs for the system to store a new tsfile.
 # System provides four strategies to choose from, or user can create his own strategy by extending org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategy.
 # The info of the four strategies are as follows:
 # 1. SequenceStrategy: the system will choose the directory in sequence.
@@ -136,17 +165,18 @@ config_nodes=127.0.0.1:22277
 # multi_dir_strategy=MaxDiskUsableSpaceFirstStrategy
 
 
-# wal dir
+# wal dirs
 # If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/data).
 # If it is absolute, system will save the data in the exact location it points to.
 # If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder.
-# Note: If wal_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
+# If there are more than one directory, please separate them by commas ",".
+# Note: If wal_dirs is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
 # For windows platform
 # If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
-# wal_dir=data\\wal
+# wal_dirs=data\\wal
 # For Linux platform
 # If its prefix is "/", then the path is absolute. Otherwise, it is relative.
-# wal_dir=data/wal
+# wal_dirs=data/wal
 
 # consensus dir
 # If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/data).
@@ -161,7 +191,7 @@ config_nodes=127.0.0.1:22277
 # consensus_dir=data/consensus
 
 
-# TSFile storage file system. Currently, Tsfiles are supported to be stored in LOCAL file system or HDFS.
+# TSFile storage file system. Currently, TsFiles are supported to be stored in LOCAL file system or HDFS.
 # Datatype: FSType
 # tsfile_storage_fs=LOCAL
 
@@ -228,13 +258,6 @@ timestamp_precision=ms
 # Datatype: long
 # default_ttl=36000000
 
-# The size of the log buffer in each log node (in bytes). Due to the double buffer mechanism,
-# if WAL is enabled and the size of the inserted plan is greater than one-half of this parameter,
-# then the insert plan will be rejected by WAL.
-# If it sets a value smaller than 0, use the default value 16777216
-# Datatype: int
-# wal_buffer_size=16777216
-
 # Size of log buffer in each metadata operation plan(in byte).
 # If the size of a metadata operation plan is larger than this parameter, then it will be rejected by SchemaRegion
 # If it sets a value smaller than 0, use the default value 1024*1024
@@ -280,20 +303,6 @@ timestamp_precision=ms
 # Datatype: long
 # unseq_memtable_flush_check_interval_in_ms=600000
 
-# Whether to timed close tsfiles.
-# Datatype: boolean
-# enable_timed_close_tsfile=true
-
-# If a TsfileProcessor's last working memtable flush time is older than current time minus this and its working memtable is null, the TsfileProcessor will be closed.
-# The default close interval is 60 * 60 * 1000. (unit: ms)
-# Datatype: long
-# close_tsfile_interval_after_flushing_in_ms=3600000
-
-# The interval to check whether tsfiles need closing.
-# The default close check interval is 10 * 60 * 1000. (unit: ms)
-# Datatype: long
-# close_tsfile_check_interval_in_ms=600000
-
 # When the average point number of timeseries in memtable exceeds this, the memtable is flushed to disk. The default threshold is 10000.
 # Datatype: int
 # avg_series_point_number_threshold=10000
@@ -348,6 +357,11 @@ timestamp_precision=ms
 # Datatype: int
 # recovery_log_interval_in_ms=5000
 
+# Add a switch to drop ouf-of-order data
+# Out-of-order data will impact the aggregation query a lot. Users may not care about discarding some out-of-order data.
+# Datatype: boolean
+# enable_discard_out_of_order_data=false
+
 ####################
 ### Memory Control Configuration
 ####################
@@ -556,24 +570,6 @@ timestamp_precision=ms
 # Datatype: boolean
 # enable_last_cache=true
 
-####################
-### WAL Direct Buffer Pool Configuration
-####################
-# the interval to trim the wal pool
-# Datatype: long
-# wal_pool_trim_interval_ms=10000
-
-# the max number of wal bytebuffer can be allocated for each time partition, if there is no unseq data you can set it to 4.
-# it should be an even number
-# Datatype: int
-# max_wal_bytebuffer_num_for_each_partition=6
-
-# if OOM occurs when registering bytebuffer, system will sleep awhile and then try again.
-# register_buffer_sleep_interval_in_ms=200
-
-# if total sleep time exceeds this, system will reject this write.
-# register_buffer_reject_threshold_in_ms=10000
-
 ####################
 ### External sort Configuration
 ####################
@@ -923,8 +919,6 @@ timestamp_precision=ms
 # index_buffer_size=134217728
 
 # whether enable data partition. If disabled, all data belongs to partition 0
-# Notice: It's not recommend to open this function. If open, please calculate appropriate concurrent_writing_time_partition and wal_buffer_size, you
-# can calculate wal_buffer_size = MaxDirectMemorySizeInBytes * 0.3 / (storage_group_num * virtual_storage_group_num) / concurrent_writing_time_partition
 # Datatype: boolean
 # enable_partition=false
 
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 54464410d9..ab3e3a0235 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
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.exception.LoadConfigurationException;
 import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
 import org.apache.iotdb.db.service.thrift.impl.InfluxDBServiceImpl;
 import org.apache.iotdb.db.service.thrift.impl.TSServiceImpl;
+import org.apache.iotdb.db.wal.utils.WALMode;
 import org.apache.iotdb.rpc.RpcTransportFactory;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -74,6 +75,8 @@ public class IoTDBConfig {
 
   public static final Pattern NODE_PATTERN = Pattern.compile(NODE_MATCHER);
 
+  private volatile boolean readOnly = false;
+
   /** whether to enable the mqtt service. */
   private boolean enableMQTTService = false;
 
@@ -147,46 +150,51 @@ public class IoTDBConfig {
 
   /** When inserting rejected exceeds this, throw an exception. Unit: millisecond */
   private int maxWaitingTimeWhenInsertBlockedInMs = 10000;
-  /** Is the write ahead log enable. */
-  private boolean enableWal = true;
 
-  private volatile boolean readOnly = false;
+  /** this variable set timestamp precision as millisecond, microsecond or nanosecond */
+  private String timestampPrecision = "ms";
 
-  private boolean enableDiscardOutOfOrderData = false;
+  // region Write Ahead Log Configuration
+  /** Write mode of wal */
+  private WALMode walMode = WALMode.ASYNC;
 
-  /**
-   * When a certain amount of write ahead logs is reached, they will be flushed to the disk. It is
-   * possible to lose at most flush_wal_threshold operations.
-   */
-  private int flushWalThreshold = 10000;
+  /** WAL directories */
+  private String[] walDirs = {DEFAULT_BASE_DIR + File.separator + IoTDBConstant.WAL_FOLDER_NAME};
 
-  /** this variable set timestamp precision as millisecond, microsecond or nanosecond */
-  private String timestampPrecision = "ms";
+  /** Duration a wal flush operation will wait before calling fsync. Unit: millisecond */
+  private long fsyncWalDelayInMs = 10;
 
-  /**
-   * The cycle when write ahead log is periodically forced to be written to disk(in milliseconds) If
-   * set this parameter to 0 it means call channel.force(true) after every each insert. Unit:
-   * millisecond
-   */
-  private long forceWalPeriodInMs = 100;
+  /** Max number of wal nodes, each node corresponds to one wal directory */
+  private int maxWalNodesNum = 0;
 
-  /**
-   * The size of the log buffer in each log node (in bytes). Due to the double buffer mechanism, if
-   * WAL is enabled and the size of the inserted plan is greater than one-half of this parameter,
-   * then the insert plan will be rejected by WAL. Unit: byte
-   */
+  /** Buffer size of each wal node. Unit: byte */
   private int walBufferSize = 16 * 1024 * 1024;
 
-  private int maxWalBytebufferNumForEachPartition = 6;
+  /** Buffer entry size of each wal buffer. Unit: byte */
+  private int walBufferEntrySize = 16 * 1024;
+
+  /** Blocking queue capacity of each wal buffer */
+  private int walBufferQueueCapacity = 10_000;
 
-  /** Unit: millisecond */
-  private long walPoolTrimIntervalInMS = 10_000;
+  /** Size threshold of each wal file. Unit: byte */
+  private long walFileSizeThresholdInByte = 10 * 1024 * 1024;
+
+  /** TTL of wal file. Unit: ms */
+  private long walFileTTLInMs = 24 * 60 * 60 * 1000;
+
+  /**
+   * MemTable size threshold for triggering MemTable snapshot in wal. When a memTable's size exceeds
+   * this, wal can flush this memtable to disk, otherwise wal will snapshot this memtable in wal.
+   * Unit: byte
+   */
+  private long walMemTableSnapshotThreshold = 128 * 1024 * 1024;
 
-  /** if OOM occurs when registering bytebuffer, system will sleep awhile and then try again. */
-  private long registerBufferSleepIntervalInMs = 200;
+  /** MemTable's max snapshot number in wal file */
+  private int maxWalMemTableSnapshotNum = 1;
 
-  /** if total sleep time exceeds this, system will reject this write. */
-  private long registerBufferRejectThresholdInMs = 10_000;
+  /** The period when outdated wal files are periodically deleted. Unit: millisecond */
+  private long deleteWalFilesPeriodInMs = 10 * 60 * 1000;
+  // endregion
 
   /** Unit: byte */
   private int estimatedSeriesSize = 300;
@@ -254,15 +262,12 @@ public class IoTDBConfig {
   private String mqttDir =
       IoTDBConstant.EXT_FOLDER_NAME + File.separator + IoTDBConstant.MQTT_FOLDER_NAME;
 
-  /** Data directory of data. It can be settled as dataDirs = {"data1", "data2", "data3"}; */
-  private String[] dataDirs = {"data" + File.separator + "data"};
+  /** Data directories. It can be settled as dataDirs = {"data1", "data2", "data3"}; */
+  private String[] dataDirs = {DEFAULT_BASE_DIR + File.separator + IoTDBConstant.DATA_FOLDER_NAME};
 
   /** Strategy of multiple directories. */
   private String multiDirStrategyClassName = null;
 
-  /** Wal directory. */
-  private String walDir = DEFAULT_BASE_DIR + File.separator + "wal";
-
   /** Consensus directory. */
   private String consensusDir = DEFAULT_BASE_DIR + File.separator + "consensus";
 
@@ -357,18 +362,6 @@ public class IoTDBConfig {
   /** The interval to check whether unsequence memtables need flushing. Unit: ms */
   private long unseqMemtableFlushCheckInterval = 10 * 60 * 1000L;
 
-  /** Whether to timed close tsfiles. */
-  private boolean enableTimedCloseTsFile = true;
-
-  /**
-   * If a TsfileProcessor's last working memtable flush time is older than current time minus this
-   * and its working memtable is null, the TsfileProcessor will be closed. Unit: ms
-   */
-  private long closeTsFileIntervalAfterFlushing = 60 * 60 * 1000L;
-
-  /** The interval to check whether tsfiles need closing. Unit: ms */
-  private long closeTsFileCheckInterval = 10 * 60 * 1000L;
-
   /** When average series point number reaches this, flush the memtable to disk */
   private int avgSeriesPointNumberThreshold = 10000;
 
@@ -802,6 +795,8 @@ public class IoTDBConfig {
   /** the interval to log recover progress of each vsg when starting iotdb */
   private long recoveryLogIntervalInMs = 5_000L;
 
+  private boolean enableDiscardOutOfOrderData = false;
+
   private String adminName = "root";
 
   private String adminPassword = "root";
@@ -970,13 +965,15 @@ public class IoTDBConfig {
     schemaDir = addHomeDir(schemaDir);
     syncDir = addHomeDir(syncDir);
     tracingDir = addHomeDir(tracingDir);
-    walDir = addHomeDir(walDir);
     consensusDir = addHomeDir(consensusDir);
     indexRootFolder = addHomeDir(indexRootFolder);
     extDir = addHomeDir(extDir);
     udfDir = addHomeDir(udfDir);
     triggerDir = addHomeDir(triggerDir);
     mqttDir = addHomeDir(mqttDir);
+    for (int i = 0; i < walDirs.length; i++) {
+      walDirs[i] = addHomeDir(walDirs[i]);
+    }
 
     if (TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs().equals(FSType.HDFS)) {
       String hdfsDir = getHdfsDir();
@@ -1097,14 +1094,6 @@ public class IoTDBConfig {
     this.timestampPrecision = timestampPrecision;
   }
 
-  public boolean isEnableWal() {
-    return enableWal;
-  }
-
-  public void setEnableWal(boolean enableWal) {
-    this.enableWal = enableWal;
-  }
-
   public boolean isEnableDiscardOutOfOrderData() {
     return enableDiscardOutOfOrderData;
   }
@@ -1113,22 +1102,6 @@ public class IoTDBConfig {
     this.enableDiscardOutOfOrderData = enableDiscardOutOfOrderData;
   }
 
-  public int getFlushWalThreshold() {
-    return flushWalThreshold;
-  }
-
-  public void setFlushWalThreshold(int flushWalThreshold) {
-    this.flushWalThreshold = flushWalThreshold;
-  }
-
-  public long getForceWalPeriodInMs() {
-    return forceWalPeriodInMs;
-  }
-
-  public void setForceWalPeriodInMs(long forceWalPeriodInMs) {
-    this.forceWalPeriodInMs = forceWalPeriodInMs;
-  }
-
   public String getSystemDir() {
     return systemDir;
   }
@@ -1169,14 +1142,6 @@ public class IoTDBConfig {
     this.queryDir = queryDir;
   }
 
-  public String getWalDir() {
-    return walDir;
-  }
-
-  void setWalDir(String walDir) {
-    this.walDir = walDir;
-  }
-
   public String getConsensusDir() {
     return consensusDir;
   }
@@ -1419,6 +1384,38 @@ public class IoTDBConfig {
     this.rpcImplClassName = rpcImplClassName;
   }
 
+  public WALMode getWalMode() {
+    return walMode;
+  }
+
+  public void setWalMode(WALMode walMode) {
+    this.walMode = walMode;
+  }
+
+  public String[] getWalDirs() {
+    return walDirs;
+  }
+
+  public void setWalDirs(String[] walDirs) {
+    this.walDirs = walDirs;
+  }
+
+  public long getFsyncWalDelayInMs() {
+    return fsyncWalDelayInMs;
+  }
+
+  void setFsyncWalDelayInMs(long fsyncWalDelayInMs) {
+    this.fsyncWalDelayInMs = fsyncWalDelayInMs;
+  }
+
+  public int getMaxWalNodesNum() {
+    return maxWalNodesNum;
+  }
+
+  void setMaxWalNodesNum(int maxWalNodesNum) {
+    this.maxWalNodesNum = maxWalNodesNum;
+  }
+
   public int getWalBufferSize() {
     return walBufferSize;
   }
@@ -1427,36 +1424,60 @@ public class IoTDBConfig {
     this.walBufferSize = walBufferSize;
   }
 
-  public int getMaxWalBytebufferNumForEachPartition() {
-    return maxWalBytebufferNumForEachPartition;
+  public int getWalBufferEntrySize() {
+    return walBufferEntrySize;
   }
 
-  public void setMaxWalBytebufferNumForEachPartition(int maxWalBytebufferNumForEachPartition) {
-    this.maxWalBytebufferNumForEachPartition = maxWalBytebufferNumForEachPartition;
+  void setWalBufferEntrySize(int walBufferEntrySize) {
+    this.walBufferEntrySize = walBufferEntrySize;
   }
 
-  public long getWalPoolTrimIntervalInMS() {
-    return walPoolTrimIntervalInMS;
+  public int getWalBufferQueueCapacity() {
+    return walBufferQueueCapacity;
   }
 
-  public void setWalPoolTrimIntervalInMS(long walPoolTrimIntervalInMS) {
-    this.walPoolTrimIntervalInMS = walPoolTrimIntervalInMS;
+  void setWalBufferQueueCapacity(int walBufferQueueCapacity) {
+    this.walBufferQueueCapacity = walBufferQueueCapacity;
   }
 
-  public long getRegisterBufferSleepIntervalInMs() {
-    return registerBufferSleepIntervalInMs;
+  public long getWalFileSizeThresholdInByte() {
+    return walFileSizeThresholdInByte;
   }
 
-  public void setRegisterBufferSleepIntervalInMs(long registerBufferSleepIntervalInMs) {
-    this.registerBufferSleepIntervalInMs = registerBufferSleepIntervalInMs;
+  void setWalFileSizeThresholdInByte(long walFileSizeThresholdInByte) {
+    this.walFileSizeThresholdInByte = walFileSizeThresholdInByte;
   }
 
-  public long getRegisterBufferRejectThresholdInMs() {
-    return registerBufferRejectThresholdInMs;
+  public long getWalFileTTLInMs() {
+    return walFileTTLInMs;
   }
 
-  public void setRegisterBufferRejectThresholdInMs(long registerBufferRejectThresholdInMs) {
-    this.registerBufferRejectThresholdInMs = registerBufferRejectThresholdInMs;
+  void setWalFileTTLInMs(long walFileTTLInMs) {
+    this.walFileTTLInMs = walFileTTLInMs;
+  }
+
+  public long getWalMemTableSnapshotThreshold() {
+    return walMemTableSnapshotThreshold;
+  }
+
+  void setWalMemTableSnapshotThreshold(long walMemTableSnapshotThreshold) {
+    this.walMemTableSnapshotThreshold = walMemTableSnapshotThreshold;
+  }
+
+  public int getMaxWalMemTableSnapshotNum() {
+    return maxWalMemTableSnapshotNum;
+  }
+
+  void setMaxWalMemTableSnapshotNum(int maxWalMemTableSnapshotNum) {
+    this.maxWalMemTableSnapshotNum = maxWalMemTableSnapshotNum;
+  }
+
+  public long getDeleteWalFilesPeriodInMs() {
+    return deleteWalFilesPeriodInMs;
+  }
+
+  void setDeleteWalFilesPeriodInMs(long deleteWalFilesPeriodInMs) {
+    this.deleteWalFilesPeriodInMs = deleteWalFilesPeriodInMs;
   }
 
   public int getEstimatedSeriesSize() {
@@ -1725,30 +1746,6 @@ public class IoTDBConfig {
     this.unseqMemtableFlushCheckInterval = unseqMemtableFlushCheckInterval;
   }
 
-  public boolean isEnableTimedCloseTsFile() {
-    return enableTimedCloseTsFile;
-  }
-
-  public void setEnableTimedCloseTsFile(boolean enableTimedCloseTsFile) {
-    this.enableTimedCloseTsFile = enableTimedCloseTsFile;
-  }
-
-  public long getCloseTsFileIntervalAfterFlushing() {
-    return closeTsFileIntervalAfterFlushing;
-  }
-
-  public void setCloseTsFileIntervalAfterFlushing(long closeTsFileIntervalAfterFlushing) {
-    this.closeTsFileIntervalAfterFlushing = closeTsFileIntervalAfterFlushing;
-  }
-
-  public long getCloseTsFileCheckInterval() {
-    return closeTsFileCheckInterval;
-  }
-
-  public void setCloseTsFileCheckInterval(long closeTsFileCheckInterval) {
-    this.closeTsFileCheckInterval = closeTsFileCheckInterval;
-  }
-
   public int getAvgSeriesPointNumberThreshold() {
     return avgSeriesPointNumberThreshold;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
index fbc1295d41..b098fad4b1 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
@@ -54,7 +54,7 @@ public class IoTDBConfigCheck {
   // If user delete folder "data", system.properties can reset.
   private static final String PROPERTIES_FILE_NAME = "system.properties";
   private static final String SCHEMA_DIR = config.getSchemaDir();
-  private static final String WAL_DIR = config.getWalDir();
+  private static final String[] WAL_DIRS = config.getWalDirs();
 
   private File propertiesFile;
   private File tmpPropertiesFile;
@@ -233,17 +233,19 @@ public class IoTDBConfigCheck {
   }
 
   private void checkWALNotExists() {
-    if (SystemFileFactory.INSTANCE.getFile(WAL_DIR).isDirectory()) {
-      File[] sgWALs = SystemFileFactory.INSTANCE.getFile(WAL_DIR).listFiles();
-      if (sgWALs != null) {
-        for (File sgWAL : sgWALs) {
-          // make sure wal directory of each sg is empty
-          if (sgWAL.isDirectory() && sgWAL.list().length != 0) {
-            logger.error(
-                "WAL detected, please stop insertion and run 'SET SYSTEM TO READONLY', then run 'flush' on IoTDB {} before upgrading to {}.",
-                properties.getProperty(IOTDB_VERSION_STRING),
-                IoTDBConstant.VERSION);
-            System.exit(-1);
+    for (String walDir : WAL_DIRS) {
+      if (SystemFileFactory.INSTANCE.getFile(walDir).isDirectory()) {
+        File[] sgWALs = SystemFileFactory.INSTANCE.getFile(walDir).listFiles();
+        if (sgWALs != null) {
+          for (File sgWAL : sgWALs) {
+            // make sure wal directory of each sg is empty
+            if (sgWAL.isDirectory() && sgWAL.list().length != 0) {
+              logger.error(
+                  "WAL detected, please stop insertion and run 'SET SYSTEM TO READONLY', then run 'flush' on IoTDB {} before upgrading to {}.",
+                  properties.getProperty(IOTDB_VERSION_STRING),
+                  IoTDBConstant.VERSION);
+              System.exit(-1);
+            }
           }
         }
       }
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 ec955aa40a..f2afbc3641 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
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.exception.BadNodeUrlFormatException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.utils.DatetimeUtils;
 import org.apache.iotdb.db.service.metrics.MetricsService;
+import org.apache.iotdb.db.wal.utils.WALMode;
 import org.apache.iotdb.metrics.config.MetricConfigDescriptor;
 import org.apache.iotdb.metrics.config.ReloadLevel;
 import org.apache.iotdb.rpc.RpcTransportFactory;
@@ -245,8 +246,6 @@ public class IoTDBDescriptor {
 
       conf.setDataDirs(properties.getProperty("data_dirs", conf.getDataDirs()[0]).split(","));
 
-      conf.setWalDir(properties.getProperty("wal_dir", conf.getWalDir()));
-
       conf.setConsensusDir(properties.getProperty("consensus_dir", conf.getConsensusDir()));
 
       int mlogBufferSize =
@@ -697,6 +696,12 @@ public class IoTDBDescriptor {
                   "recovery_log_interval_in_ms",
                   String.valueOf(conf.getRecoveryLogIntervalInMs()))));
 
+      conf.setEnableDiscardOutOfOrderData(
+          Boolean.parseBoolean(
+              properties.getProperty(
+                  "enable_discard_out_of_order_data",
+                  Boolean.toString(conf.isEnableDiscardOutOfOrderData()))));
+
       conf.setConcurrentWindowEvaluationThread(
           Integer.parseInt(
               properties.getProperty(
@@ -826,7 +831,7 @@ public class IoTDBDescriptor {
               properties.getProperty("kerberos_principal", conf.getKerberosPrincipal()));
       TSFileDescriptor.getInstance().getConfig().setBatchSize(conf.getBatchSize());
 
-      // timed flush memtable, timed close tsfile
+      // timed flush memtable
       loadTimedService(properties);
 
       // set tsfile-format config
@@ -897,66 +902,92 @@ public class IoTDBDescriptor {
   }
 
   private void loadWALProps(Properties properties) {
-    conf.setEnableWal(
-        Boolean.parseBoolean(
-            properties.getProperty("enable_wal", Boolean.toString(conf.isEnableWal()))));
+    conf.setWalMode(
+        WALMode.valueOf((properties.getProperty("wal_mode", conf.getWalMode().toString()))));
 
-    conf.setFlushWalThreshold(
-        Integer.parseInt(
-            properties.getProperty(
-                "flush_wal_threshold", Integer.toString(conf.getFlushWalThreshold()))));
+    conf.setWalDirs(properties.getProperty("wal_dirs", conf.getWalDirs()[0]).split(","));
 
-    conf.setForceWalPeriodInMs(
+    long fsyncWalDelayInMs =
         Long.parseLong(
             properties.getProperty(
-                "force_wal_period_in_ms", Long.toString(conf.getForceWalPeriodInMs()))));
+                "fsync_wal_delay_in_ms", Long.toString(conf.getFsyncWalDelayInMs())));
+    if (fsyncWalDelayInMs > 0) {
+      conf.setFsyncWalDelayInMs(fsyncWalDelayInMs);
+    }
 
-    conf.setEnableDiscardOutOfOrderData(
-        Boolean.parseBoolean(
+    int maxWalNodesNum =
+        Integer.parseInt(
             properties.getProperty(
-                "enable_discard_out_of_order_data",
-                Boolean.toString(conf.isEnableDiscardOutOfOrderData()))));
+                "max_wal_nodes_num", Integer.toString(conf.getMaxWalNodesNum())));
+    if (maxWalNodesNum > 0) {
+      conf.setMaxWalNodesNum(maxWalNodesNum);
+    }
 
     int walBufferSize =
         Integer.parseInt(
-            properties.getProperty("wal_buffer_size", Integer.toString(conf.getWalBufferSize())));
+            properties.getProperty(
+                "wal_buffer_size_in_byte", Integer.toString(conf.getWalBufferSize())));
     if (walBufferSize > 0) {
       conf.setWalBufferSize(walBufferSize);
     }
 
-    int maxWalBytebufferNumForEachPartition =
+    int walBufferEntrySize =
+        Integer.parseInt(
+            properties.getProperty(
+                "wal_buffer_entry_size_in_byte", Integer.toString(conf.getWalBufferEntrySize())));
+    if (walBufferEntrySize > 0) {
+      conf.setWalBufferEntrySize(walBufferEntrySize);
+    }
+
+    int walBufferQueueCapacity =
         Integer.parseInt(
             properties.getProperty(
-                "max_wal_bytebuffer_num_for_each_partition",
-                Integer.toString(conf.getMaxWalBytebufferNumForEachPartition())));
-    if (maxWalBytebufferNumForEachPartition > 0) {
-      conf.setMaxWalBytebufferNumForEachPartition(maxWalBytebufferNumForEachPartition);
+                "wal_buffer_queue_capacity", Integer.toString(conf.getWalBufferQueueCapacity())));
+    if (walBufferQueueCapacity > 0) {
+      conf.setWalBufferQueueCapacity(walBufferQueueCapacity);
     }
 
-    long poolTrimIntervalInMS =
+    long walFileSizeThreshold =
         Long.parseLong(
             properties.getProperty(
-                "wal_pool_trim_interval_ms", Long.toString(conf.getWalPoolTrimIntervalInMS())));
-    if (poolTrimIntervalInMS > 0) {
-      conf.setWalPoolTrimIntervalInMS(poolTrimIntervalInMS);
+                "wal_file_size_threshold_in_byte",
+                Long.toString(conf.getWalFileSizeThresholdInByte())));
+    if (walFileSizeThreshold > 0) {
+      conf.setWalFileSizeThresholdInByte(walFileSizeThreshold);
     }
 
-    long registerBufferSleepIntervalInMs =
+    long walFileTTL =
         Long.parseLong(
+            properties.getProperty("wal_file_ttl_in_ms", Long.toString(conf.getWalFileTTLInMs())));
+    if (walFileTTL > 0) {
+      conf.setWalFileTTLInMs(walFileTTL);
+    }
+
+    long walMemTableSnapshotThreshold =
+        Long.parseLong(
+            properties.getProperty(
+                "wal_memtable_snapshot_threshold_in_byte",
+                Long.toString(conf.getWalMemTableSnapshotThreshold())));
+    if (walMemTableSnapshotThreshold > 0) {
+      conf.setWalMemTableSnapshotThreshold(walMemTableSnapshotThreshold);
+    }
+
+    int maxWalMemTableSnapshotNum =
+        Integer.parseInt(
             properties.getProperty(
-                "register_buffer_sleep_interval_in_ms",
-                Long.toString(conf.getRegisterBufferSleepIntervalInMs())));
-    if (registerBufferSleepIntervalInMs > 0) {
-      conf.setRegisterBufferSleepIntervalInMs(registerBufferSleepIntervalInMs);
+                "max_wal_memtable_snapshot_num",
+                Integer.toString(conf.getMaxWalMemTableSnapshotNum())));
+    if (maxWalMemTableSnapshotNum > 0) {
+      conf.setMaxWalMemTableSnapshotNum(maxWalMemTableSnapshotNum);
     }
 
-    long registerBufferRejectThresholdInMs =
+    long deleteWalFilesPeriod =
         Long.parseLong(
             properties.getProperty(
-                "register_buffer_reject_threshold_in_ms",
-                Long.toString(conf.getRegisterBufferRejectThresholdInMs())));
-    if (registerBufferRejectThresholdInMs > 0) {
-      conf.setRegisterBufferRejectThresholdInMs(registerBufferRejectThresholdInMs);
+                "delete_wal_files_period_in_ms",
+                Long.toString(conf.getDeleteWalFilesPeriodInMs())));
+    if (deleteWalFilesPeriod > 0) {
+      conf.setDeleteWalFilesPeriodInMs(deleteWalFilesPeriod);
     }
   }
 
@@ -1125,7 +1156,7 @@ public class IoTDBDescriptor {
     }
   }
 
-  // timed flush memtable, timed close tsfile
+  // timed flush memtable
   private void loadTimedService(Properties properties) {
     conf.setEnableTimedFlushSeqMemtable(
         Boolean.parseBoolean(
@@ -1182,33 +1213,6 @@ public class IoTDBDescriptor {
     if (unseqMemTableFlushCheckInterval > 0) {
       conf.setUnseqMemtableFlushCheckInterval(unseqMemTableFlushCheckInterval);
     }
-
-    conf.setEnableTimedCloseTsFile(
-        Boolean.parseBoolean(
-            properties.getProperty(
-                "enable_timed_close_tsfile", Boolean.toString(conf.isEnableTimedCloseTsFile()))));
-
-    long closeTsFileIntervalAfterFlushing =
-        Long.parseLong(
-            properties
-                .getProperty(
-                    "close_tsfile_interval_after_flushing_in_ms",
-                    Long.toString(conf.getCloseTsFileIntervalAfterFlushing()))
-                .trim());
-    if (closeTsFileIntervalAfterFlushing > 0) {
-      conf.setCloseTsFileIntervalAfterFlushing(closeTsFileIntervalAfterFlushing);
-    }
-
-    long closeTsFileCheckInterval =
-        Long.parseLong(
-            properties
-                .getProperty(
-                    "close_tsfile_check_interval_in_ms",
-                    Long.toString(conf.getCloseTsFileCheckInterval()))
-                .trim());
-    if (closeTsFileCheckInterval > 0) {
-      conf.setCloseTsFileCheckInterval(closeTsFileCheckInterval);
-    }
   }
 
   public void loadHotModifiedProps(Properties properties) throws QueryProcessException {
@@ -1227,9 +1231,6 @@ public class IoTDBDescriptor {
         DirectoryManager.getInstance().updateDirectoryStrategy();
       }
 
-      // update WAL conf
-      loadWALProps(properties);
-
       // update timed flush & close conf
       loadTimedService(properties);
       StorageEngine.getInstance().rebootTimedService();
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java b/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
index 82c0b3ee30..a9829ce45f 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
@@ -157,10 +157,6 @@ public class DirectoryManager {
     private static final DirectoryManager INSTANCE = new DirectoryManager();
   }
 
-  public String getWALFolder() {
-    return IoTDBDescriptor.getInstance().getConfig().getWalDir();
-  }
-
   public String getIndexRootFolder() {
     return IoTDBDescriptor.getInstance().getConfig().getIndexRootFolder();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/directories/FolderManager.java b/server/src/main/java/org/apache/iotdb/db/conf/directories/FolderManager.java
new file mode 100644
index 0000000000..bbe912877f
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/conf/directories/FolderManager.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.conf.directories;
+
+import org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategy;
+import org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategyType;
+import org.apache.iotdb.db.conf.directories.strategy.MaxDiskUsableSpaceFirstStrategy;
+import org.apache.iotdb.db.conf.directories.strategy.MinFolderOccupiedSpaceFirstStrategy;
+import org.apache.iotdb.db.conf.directories.strategy.RandomOnDiskUsableSpaceStrategy;
+import org.apache.iotdb.db.conf.directories.strategy.SequenceStrategy;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class FolderManager {
+  private static final Logger logger = LoggerFactory.getLogger(FolderManager.class);
+
+  private final List<String> folders;
+  private final DirectoryStrategy selectStrategy;
+
+  public FolderManager(List<String> folders, DirectoryStrategyType type)
+      throws DiskSpaceInsufficientException {
+    this.folders = folders;
+    switch (type) {
+      case SEQUENCE_STRATEGY:
+        this.selectStrategy = new SequenceStrategy();
+        break;
+      case MAX_DISK_USABLE_SPACE_FIRST_STRATEGY:
+        this.selectStrategy = new MaxDiskUsableSpaceFirstStrategy();
+        break;
+      case MIN_FOLDER_OCCUPIED_SPACE_FIRST_STRATEGY:
+        this.selectStrategy = new MinFolderOccupiedSpaceFirstStrategy();
+        break;
+      case RANDOM_ON_DISK_USABLE_SPACE_STRATEGY:
+        this.selectStrategy = new RandomOnDiskUsableSpaceStrategy();
+        break;
+      default:
+        throw new RuntimeException();
+    }
+    this.selectStrategy.setFolders(folders);
+  }
+
+  public String getNextFolder() throws DiskSpaceInsufficientException {
+    return folders.get(selectStrategy.nextFolderIndex());
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java b/server/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategyType.java
similarity index 58%
copy from server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
copy to server/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategyType.java
index 542eb6448f..7cbe826333 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategyType.java
@@ -16,31 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.conf.directories.strategy;
 
-package org.apache.iotdb.db.engine.flush;
-
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-
-import java.io.IOException;
-
-public interface FlushListener {
-
-  void onFlushStart(IMemTable memTable) throws IOException;
-
-  void onFlushEnd(IMemTable memTable);
-
-  class EmptyListener implements FlushListener {
-
-    public static final EmptyListener INSTANCE = new EmptyListener();
-
-    @Override
-    public void onFlushStart(IMemTable memTable) {
-      // do nothing
-    }
-
-    @Override
-    public void onFlushEnd(IMemTable memTable) {
-      // do nothing
-    }
-  }
+public enum DirectoryStrategyType {
+  SEQUENCE_STRATEGY,
+  MAX_DISK_USABLE_SPACE_FIRST_STRATEGY,
+  MIN_FOLDER_OCCUPIED_SPACE_FIRST_STRATEGY,
+  RANDOM_ON_DISK_USABLE_SPACE_STRATEGY,
 }
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 b0551f5e99..b3f2c5fb7d 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
@@ -64,6 +64,8 @@ import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.ThreadUtils;
 import org.apache.iotdb.db.utils.UpgradeUtils;
+import org.apache.iotdb.db.wal.exception.WALException;
+import org.apache.iotdb.db.wal.recover.WALRecoverManager;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
@@ -90,6 +92,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -130,7 +133,6 @@ public class StorageEngine implements IService {
   private ScheduledExecutorService ttlCheckThread;
   private ScheduledExecutorService seqMemtableTimedFlushCheckThread;
   private ScheduledExecutorService unseqMemtableTimedFlushCheckThread;
-  private ScheduledExecutorService tsFileTimedCloseCheckThread;
 
   private TsFileFlushPolicy fileFlushPolicy = new DirectFlushPolicy();
   private ExecutorService recoveryThreadPool;
@@ -237,18 +239,29 @@ public class StorageEngine implements IService {
         IoTDBThreadPoolFactory.newFixedThreadPool(
             Runtime.getRuntime().availableProcessors(), "Recovery-Thread-Pool");
 
-    // recover all logic storage group processors
     List<IStorageGroupMNode> sgNodes = IoTDB.schemaProcessor.getAllStorageGroupNodes();
+    // init wal recover manager
+    WALRecoverManager.getInstance()
+        .setAllVsgScannedLatch(
+            new CountDownLatch(sgNodes.size() * config.getVirtualStorageGroupNum()));
+    // recover all logic storage groups
     List<Future<Void>> futures = new LinkedList<>();
     for (IStorageGroupMNode storageGroup : sgNodes) {
       StorageGroupManager storageGroupManager =
           processorMap.computeIfAbsent(
               storageGroup.getPartialPath(), id -> new StorageGroupManager(true));
 
-      // recover all virtual storage groups in one logic storage group
+      // recover all virtual storage groups in each logic storage group
       storageGroupManager.asyncRecover(storageGroup, recoveryThreadPool, futures);
     }
 
+    // wait until wal is recovered
+    try {
+      WALRecoverManager.getInstance().recover();
+    } catch (WALException e) {
+      logger.error("Fail to recover wal.", e);
+    }
+
     // operations after all virtual storage groups are recovered
     Thread recoverEndTrigger =
         new Thread(
@@ -335,18 +348,6 @@ public class StorageEngine implements IService {
           TimeUnit.MILLISECONDS);
       logger.info("start unsequence memtable timed flush check thread successfully.");
     }
-    // timed close tsfile
-    if (config.isEnableTimedCloseTsFile()) {
-      tsFileTimedCloseCheckThread =
-          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
-              ThreadName.TIMED_CLOSE_TSFILE.getName());
-      tsFileTimedCloseCheckThread.scheduleAtFixedRate(
-          this::timedCloseTsFileProcessor,
-          config.getCloseTsFileCheckInterval(),
-          config.getCloseTsFileCheckInterval(),
-          TimeUnit.MILLISECONDS);
-      logger.info("start tsfile timed close check thread successfully.");
-    }
   }
 
   private void timedFlushSeqMemTable() {
@@ -369,16 +370,6 @@ public class StorageEngine implements IService {
     }
   }
 
-  private void timedCloseTsFileProcessor() {
-    try {
-      for (StorageGroupManager processor : processorMap.values()) {
-        processor.timedCloseTsFileProcessor();
-      }
-    } catch (Exception e) {
-      logger.error("An error occurred when timed closing tsfiles interval", e);
-    }
-  }
-
   @Override
   public void stop() {
     for (StorageGroupManager storageGroupManager : processorMap.values()) {
@@ -390,11 +381,7 @@ public class StorageEngine implements IService {
         seqMemtableTimedFlushCheckThread, ThreadName.TIMED_FlUSH_SEQ_MEMTABLE);
     ThreadUtils.stopThreadPool(
         unseqMemtableTimedFlushCheckThread, ThreadName.TIMED_FlUSH_UNSEQ_MEMTABLE);
-    ThreadUtils.stopThreadPool(tsFileTimedCloseCheckThread, ThreadName.TIMED_CLOSE_TSFILE);
     recoveryThreadPool.shutdownNow();
-    for (PartialPath storageGroup : IoTDB.schemaProcessor.getAllStorageGroupPaths()) {
-      this.releaseWalDirectByteBufferPoolInOneStorageGroup(storageGroup);
-    }
     processorMap.clear();
   }
 
@@ -411,7 +398,6 @@ public class StorageEngine implements IService {
     shutdownTimedService(ttlCheckThread, "TTlCheckThread");
     shutdownTimedService(seqMemtableTimedFlushCheckThread, "SeqMemtableTimedFlushCheckThread");
     shutdownTimedService(unseqMemtableTimedFlushCheckThread, "UnseqMemtableTimedFlushCheckThread");
-    shutdownTimedService(tsFileTimedCloseCheckThread, "TsFileTimedCloseCheckThread");
     recoveryThreadPool.shutdownNow();
     processorMap.clear();
   }
@@ -428,7 +414,7 @@ public class StorageEngine implements IService {
     }
   }
 
-  /** reboot timed flush sequence/unsequence memetable thread, timed close tsfile thread */
+  /** reboot timed flush sequence/unsequence memetable thread */
   public void rebootTimedService() throws ShutdownException {
     logger.info("Start rebooting all timed service.");
 
@@ -436,7 +422,6 @@ public class StorageEngine implements IService {
     stopTimedServiceAndThrow(seqMemtableTimedFlushCheckThread, "SeqMemtableTimedFlushCheckThread");
     stopTimedServiceAndThrow(
         unseqMemtableTimedFlushCheckThread, "UnseqMemtableTimedFlushCheckThread");
-    stopTimedServiceAndThrow(tsFileTimedCloseCheckThread, "TsFileTimedCloseCheckThread");
 
     logger.info("Stop all timed service successfully, and now restart them.");
 
@@ -497,6 +482,16 @@ public class StorageEngine implements IService {
     }
   }
 
+  public VirtualStorageGroupProcessor getProcessorByVSGId(PartialPath path, int vsgId)
+      throws StorageEngineException {
+    try {
+      IStorageGroupMNode storageGroupMNode = IoTDB.schemaProcessor.getStorageGroupNodeByPath(path);
+      return getStorageGroupManager(storageGroupMNode).getProcessor(storageGroupMNode, vsgId);
+    } catch (StorageGroupProcessorException | MetadataException e) {
+      throw new StorageEngineException(e);
+    }
+  }
+
   /**
    * get lock holder for each sg
    *
@@ -526,11 +521,22 @@ public class StorageEngine implements IService {
    *     modification in mtree
    * @return found or new storage group processor
    */
-  @SuppressWarnings("java:S2445")
-  // actually storageGroupMNode is a unique object on the mtree, synchronize it is reasonable
   private VirtualStorageGroupProcessor getStorageGroupProcessorByPath(
       PartialPath devicePath, IStorageGroupMNode storageGroupMNode)
       throws StorageGroupProcessorException, StorageEngineException {
+    return getStorageGroupManager(storageGroupMNode).getProcessor(devicePath, storageGroupMNode);
+  }
+
+  /**
+   * get storage group manager by storage group mnode
+   *
+   * @param storageGroupMNode mnode of the storage group, we need synchronize this to avoid
+   *     modification in mtree
+   * @return found or new storage group manager
+   */
+  @SuppressWarnings("java:S2445")
+  // actually storageGroupMNode is a unique object on the mtree, synchronize it is reasonable
+  private StorageGroupManager getStorageGroupManager(IStorageGroupMNode storageGroupMNode) {
     StorageGroupManager storageGroupManager = processorMap.get(storageGroupMNode.getPartialPath());
     if (storageGroupManager == null) {
       synchronized (this) {
@@ -541,7 +547,7 @@ public class StorageEngine implements IService {
         }
       }
     }
-    return storageGroupManager.getProcessor(devicePath, storageGroupMNode);
+    return storageGroupManager;
   }
 
   /**
@@ -841,13 +847,6 @@ public class StorageEngine implements IService {
     processorMap.get(storageGroupPath).syncDeleteDataFiles();
   }
 
-  /** release all the allocated non-heap */
-  public void releaseWalDirectByteBufferPoolInOneStorageGroup(PartialPath storageGroupPath) {
-    if (processorMap.containsKey(storageGroupPath)) {
-      processorMap.get(storageGroupPath).releaseWalDirectByteBufferPool();
-    }
-  }
-
   /** delete all data of storage groups' timeseries. */
   public synchronized boolean deleteAll() {
     logger.info("Start deleting all storage groups' timeseries");
@@ -873,7 +872,6 @@ public class StorageEngine implements IService {
     }
     abortCompactionTaskForStorageGroup(storageGroupPath);
     deleteAllDataFilesInOneStorageGroup(storageGroupPath);
-    releaseWalDirectByteBufferPoolInOneStorageGroup(storageGroupPath);
     StorageGroupManager storageGroupManager = processorMap.remove(storageGroupPath);
     storageGroupManager.deleteStorageGroupSystemFolder(
         systemDir + File.pathSeparator + storageGroupPath);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
index 538ae1ef67..d03496e152 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
@@ -103,7 +103,6 @@ public class StorageEngineV2 implements IService {
   private ScheduledExecutorService ttlCheckThread;
   private ScheduledExecutorService seqMemtableTimedFlushCheckThread;
   private ScheduledExecutorService unseqMemtableTimedFlushCheckThread;
-  private ScheduledExecutorService tsFileTimedCloseCheckThread;
 
   private TsFileFlushPolicy fileFlushPolicy = new DirectFlushPolicy();
   private ExecutorService recoveryThreadPool;
@@ -326,18 +325,6 @@ public class StorageEngineV2 implements IService {
           TimeUnit.MILLISECONDS);
       logger.info("start unsequence memtable timed flush check thread successfully.");
     }
-    // timed close tsfile
-    if (config.isEnableTimedCloseTsFile()) {
-      tsFileTimedCloseCheckThread =
-          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
-              ThreadName.TIMED_CLOSE_TSFILE.getName());
-      tsFileTimedCloseCheckThread.scheduleAtFixedRate(
-          this::timedCloseTsFileProcessor,
-          config.getCloseTsFileCheckInterval(),
-          config.getCloseTsFileCheckInterval(),
-          TimeUnit.MILLISECONDS);
-      logger.info("start tsfile timed close check thread successfully.");
-    }
   }
 
   private void timedFlushSeqMemTable() {
@@ -364,25 +351,12 @@ public class StorageEngineV2 implements IService {
     }
   }
 
-  private void timedCloseTsFileProcessor() {
-    try {
-      for (VirtualStorageGroupProcessor dataRegion : dataRegionMap.values()) {
-        if (dataRegion != null) {
-          dataRegion.timedCloseTsFileProcessor();
-        }
-      }
-    } catch (Exception e) {
-      logger.error("An error occurred when timed closing tsfiles interval", e);
-    }
-  }
-
   @Override
   public void stop() {
     for (VirtualStorageGroupProcessor vsg : dataRegionMap.values()) {
       if (vsg != null) {
         ThreadUtils.stopThreadPool(
             vsg.getTimedCompactionScheduleTask(), ThreadName.COMPACTION_SCHEDULE);
-        ThreadUtils.stopThreadPool(vsg.getWALTrimScheduleTask(), ThreadName.WAL_TRIM);
       }
     }
     syncCloseAllProcessor();
@@ -391,12 +365,7 @@ public class StorageEngineV2 implements IService {
         seqMemtableTimedFlushCheckThread, ThreadName.TIMED_FlUSH_SEQ_MEMTABLE);
     ThreadUtils.stopThreadPool(
         unseqMemtableTimedFlushCheckThread, ThreadName.TIMED_FlUSH_UNSEQ_MEMTABLE);
-    ThreadUtils.stopThreadPool(tsFileTimedCloseCheckThread, ThreadName.TIMED_CLOSE_TSFILE);
     recoveryThreadPool.shutdownNow();
-    // TODO(Removed from new wal)
-    //    for (PartialPath storageGroup : IoTDB.schemaEngine.getAllStorageGroupPaths()) {
-    //      this.releaseWalDirectByteBufferPoolInOneStorageGroup(storageGroup);
-    //    }
     dataRegionMap.clear();
   }
 
@@ -407,8 +376,6 @@ public class StorageEngineV2 implements IService {
         ThreadUtils.stopThreadPool(
             virtualStorageGroupProcessor.getTimedCompactionScheduleTask(),
             ThreadName.COMPACTION_SCHEDULE);
-        ThreadUtils.stopThreadPool(
-            virtualStorageGroupProcessor.getWALTrimScheduleTask(), ThreadName.WAL_TRIM);
       }
       forceCloseAllProcessor();
     } catch (TsFileProcessorException e) {
@@ -417,7 +384,6 @@ public class StorageEngineV2 implements IService {
     shutdownTimedService(ttlCheckThread, "TTlCheckThread");
     shutdownTimedService(seqMemtableTimedFlushCheckThread, "SeqMemtableTimedFlushCheckThread");
     shutdownTimedService(unseqMemtableTimedFlushCheckThread, "UnseqMemtableTimedFlushCheckThread");
-    shutdownTimedService(tsFileTimedCloseCheckThread, "TsFileTimedCloseCheckThread");
     recoveryThreadPool.shutdownNow();
     dataRegionMap.clear();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java b/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
index 6dcb92f64c..fa14b0b653 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
@@ -19,7 +19,7 @@
 package org.apache.iotdb.db.engine.cq;
 
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.io.SingleFileLogReader;
+import org.apache.iotdb.db.utils.writelog.SingleFileLogReader;
 
 import java.io.File;
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogWriter.java
index 7333220f92..35738179e6 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogWriter.java
@@ -22,7 +22,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.qp.physical.sys.CreateContinuousQueryPlan;
 import org.apache.iotdb.db.qp.physical.sys.DropContinuousQueryPlan;
-import org.apache.iotdb.db.writelog.io.LogWriter;
+import org.apache.iotdb.db.utils.writelog.LogWriter;
 
 import java.io.File;
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java b/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
index 542eb6448f..d87c7a1c07 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
@@ -24,23 +24,22 @@ import org.apache.iotdb.db.engine.memtable.IMemTable;
 import java.io.IOException;
 
 public interface FlushListener {
+  void onMemTableFlushStarted(IMemTable memTable) throws IOException;
 
-  void onFlushStart(IMemTable memTable) throws IOException;
+  void onMemTableFlushed(IMemTable memTable);
 
-  void onFlushEnd(IMemTable memTable);
+  class DefaultMemTableFLushListener implements FlushListener {
 
-  class EmptyListener implements FlushListener {
-
-    public static final EmptyListener INSTANCE = new EmptyListener();
+    public static final DefaultMemTableFLushListener INSTANCE = new DefaultMemTableFLushListener();
 
     @Override
-    public void onFlushStart(IMemTable memTable) {
-      // do nothing
+    public void onMemTableFlushStarted(IMemTable memTable) {
+      memTable.setFlushStatus(FlushStatus.FLUSHING);
     }
 
     @Override
-    public void onFlushEnd(IMemTable memTable) {
-      // do nothing
+    public void onMemTableFlushed(IMemTable memTable) {
+      memTable.setFlushStatus(FlushStatus.FLUSHED);
     }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java b/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushStatus.java
similarity index 61%
copy from server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
copy to server/src/main/java/org/apache/iotdb/db/engine/flush/FlushStatus.java
index 542eb6448f..06d90b4afa 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushStatus.java
@@ -16,31 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.iotdb.db.engine.flush;
 
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-
-import java.io.IOException;
-
-public interface FlushListener {
-
-  void onFlushStart(IMemTable memTable) throws IOException;
-
-  void onFlushEnd(IMemTable memTable);
-
-  class EmptyListener implements FlushListener {
-
-    public static final EmptyListener INSTANCE = new EmptyListener();
-
-    @Override
-    public void onFlushStart(IMemTable memTable) {
-      // do nothing
-    }
-
-    @Override
-    public void onFlushEnd(IMemTable memTable) {
-      // do nothing
-    }
-  }
+public enum FlushStatus {
+  /** MemTable is still working */
+  WORKING,
+  /** MemTable flush is flushing */
+  FLUSHING,
+  /** MemTable flush has been flushed */
+  FLUSHED
 }
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 aa5c5cc09b..ad2d99616b 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
@@ -19,6 +19,8 @@
 package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.flush.FlushStatus;
+import org.apache.iotdb.db.engine.flush.NotifyFlushMemTable;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.exception.WriteProcessException;
@@ -34,23 +36,34 @@ import org.apache.iotdb.db.service.metrics.Metric;
 import org.apache.iotdb.db.service.metrics.MetricsService;
 import org.apache.iotdb.db.service.metrics.Tag;
 import org.apache.iotdb.db.utils.MemUtils;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 import org.apache.iotdb.metrics.config.MetricConfigDescriptor;
 import org.apache.iotdb.metrics.utils.MetricLevel;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicInteger;
 
 public abstract class AbstractMemTable implements IMemTable {
+  private static final Logger logger = LoggerFactory.getLogger(AbstractMemTable.class);
+  private static final int FIXED_SERIALIZED_SIZE = Byte.BYTES + 2 * Integer.BYTES + 6 * Long.BYTES;
+  /** each memTable node has a unique int value identifier */
+  private static final AtomicInteger memTableIdCounter = new AtomicInteger();
+
+  private static final DeviceIDFactory deviceIDFactory = DeviceIDFactory.getInstance();
 
   /** DeviceId -> chunkGroup(MeasurementId -> chunk) */
   private final Map<IDeviceID, IWritableMemChunkGroup> memTableMap;
@@ -60,9 +73,8 @@ public abstract class AbstractMemTable implements IMemTable {
    */
   protected boolean disableMemControl = true;
 
-  private static final Logger logger = LoggerFactory.getLogger(AbstractMemTable.class);
-
   private boolean shouldFlush = false;
+  private volatile FlushStatus flushStatus = FlushStatus.WORKING;
   private final int avgSeriesPointNumThreshold =
       IoTDBDescriptor.getInstance().getConfig().getAvgSeriesPointNumberThreshold();
   /** memory size of data points, including TEXT values */
@@ -83,7 +95,9 @@ public abstract class AbstractMemTable implements IMemTable {
 
   private long minPlanIndex = Long.MAX_VALUE;
 
-  private long createdTime = System.currentTimeMillis();
+  private final int memTableId = memTableIdCounter.getAndIncrement();
+
+  private final long createdTime = System.currentTimeMillis();
 
   private static final String METRIC_POINT_IN = "pointsIn";
 
@@ -144,8 +158,7 @@ public abstract class AbstractMemTable implements IMemTable {
     // if this insert plan isn't from storage engine (mainly from test), we should set a temp device
     // id for it
     if (insertRowPlan.getDeviceID() == null) {
-      insertRowPlan.setDeviceID(
-          DeviceIDFactory.getInstance().getDeviceID(insertRowPlan.getDevicePath()));
+      insertRowPlan.setDeviceID(deviceIDFactory.getDeviceID(insertRowPlan.getDevicePath()));
     }
 
     updatePlanIndexes(insertRowPlan.getIndex());
@@ -229,8 +242,7 @@ public abstract class AbstractMemTable implements IMemTable {
   public void insertAlignedRow(InsertRowPlan insertRowPlan) {
     // if this insert plan isn't from storage engine, we should set a temp device id for it
     if (insertRowPlan.getDeviceID() == null) {
-      insertRowPlan.setDeviceID(
-          DeviceIDFactory.getInstance().getDeviceID(insertRowPlan.getDevicePath()));
+      insertRowPlan.setDeviceID(deviceIDFactory.getDeviceID(insertRowPlan.getDevicePath()));
     }
 
     updatePlanIndexes(insertRowPlan.getIndex());
@@ -448,8 +460,7 @@ public abstract class AbstractMemTable implements IMemTable {
   public void write(InsertTabletPlan insertTabletPlan, int start, int end) {
     // if this insert plan isn't from storage engine, we should set a temp device id for it
     if (insertTabletPlan.getDeviceID() == null) {
-      insertTabletPlan.setDeviceID(
-          DeviceIDFactory.getInstance().getDeviceID(insertTabletPlan.getDevicePath()));
+      insertTabletPlan.setDeviceID(deviceIDFactory.getDeviceID(insertTabletPlan.getDevicePath()));
     }
 
     List<IMeasurementSchema> schemaList = new ArrayList<>();
@@ -501,8 +512,7 @@ public abstract class AbstractMemTable implements IMemTable {
   public void writeAlignedTablet(InsertTabletPlan insertTabletPlan, int start, int end) {
     // if this insert plan isn't from storage engine, we should set a temp device id for it
     if (insertTabletPlan.getDeviceID() == null) {
-      insertTabletPlan.setDeviceID(
-          DeviceIDFactory.getInstance().getDeviceID(insertTabletPlan.getDevicePath()));
+      insertTabletPlan.setDeviceID(deviceIDFactory.getDeviceID(insertTabletPlan.getDevicePath()));
     }
 
     List<IMeasurementSchema> schemaList = new ArrayList<>();
@@ -612,6 +622,7 @@ public abstract class AbstractMemTable implements IMemTable {
     totalPointsNumThreshold = 0;
     tvListRamCost = 0;
     maxPlanIndex = 0;
+    minPlanIndex = 0;
   }
 
   @Override
@@ -694,12 +705,108 @@ public abstract class AbstractMemTable implements IMemTable {
     minPlanIndex = Math.min(index, minPlanIndex);
   }
 
+  @Override
+  public int getMemTableId() {
+    return memTableId;
+  }
+
   @Override
   public long getCreatedTime() {
     return createdTime;
   }
 
+  @Override
+  public FlushStatus getFlushStatus() {
+    return flushStatus;
+  }
+
+  @Override
+  public void setFlushStatus(FlushStatus flushStatus) {
+    this.flushStatus = flushStatus;
+  }
+
   private IDeviceID getDeviceID(PartialPath deviceId) {
-    return DeviceIDFactory.getInstance().getDeviceID(deviceId);
+    return deviceIDFactory.getDeviceID(deviceId);
+  }
+
+  /** Notice: this method is concurrent unsafe */
+  @Override
+  public int serializedSize() {
+    if (isSignalMemTable()) {
+      return Byte.BYTES;
+    }
+    int size = FIXED_SERIALIZED_SIZE;
+    for (Map.Entry<IDeviceID, IWritableMemChunkGroup> entry : memTableMap.entrySet()) {
+      size += ReadWriteIOUtils.sizeToWrite(entry.getKey().toStringID());
+      size += Byte.BYTES;
+      size += entry.getValue().serializedSize();
+    }
+    return size;
+  }
+
+  /** Notice: this method is concurrent unsafe */
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(isSignalMemTable(), buffer);
+    if (isSignalMemTable()) {
+      return;
+    }
+    buffer.putInt(seriesNumber);
+    buffer.putLong(memSize);
+    buffer.putLong(tvListRamCost);
+    buffer.putLong(totalPointsNum);
+    buffer.putLong(totalPointsNumThreshold);
+    buffer.putLong(maxPlanIndex);
+    buffer.putLong(minPlanIndex);
+
+    buffer.putInt(memTableMap.size());
+    for (Map.Entry<IDeviceID, IWritableMemChunkGroup> entry : memTableMap.entrySet()) {
+      WALWriteUtils.write(entry.getKey().toStringID(), buffer);
+
+      IWritableMemChunkGroup memChunkGroup = entry.getValue();
+      WALWriteUtils.write(memChunkGroup instanceof AlignedWritableMemChunkGroup, buffer);
+      memChunkGroup.serializeToWAL(buffer);
+    }
+  }
+
+  public void deserialize(DataInputStream stream) throws IOException {
+    seriesNumber = stream.readInt();
+    memSize = stream.readLong();
+    tvListRamCost = stream.readLong();
+    totalPointsNum = stream.readLong();
+    totalPointsNumThreshold = stream.readLong();
+    maxPlanIndex = stream.readLong();
+    minPlanIndex = stream.readLong();
+
+    int memTableMapSize = stream.readInt();
+    for (int i = 0; i < memTableMapSize; ++i) {
+      IDeviceID deviceID = deviceIDFactory.getDeviceID(ReadWriteIOUtils.readString(stream));
+
+      boolean isAligned = ReadWriteIOUtils.readBool(stream);
+      IWritableMemChunkGroup memChunkGroup;
+      if (isAligned) {
+        memChunkGroup = AlignedWritableMemChunkGroup.deserialize(stream);
+      } else {
+        memChunkGroup = WritableMemChunkGroup.deserialize(stream);
+      }
+      memTableMap.put(deviceID, memChunkGroup);
+    }
+  }
+
+  public static class Factory {
+    private Factory() {}
+
+    public static IMemTable create(DataInputStream stream) throws IOException {
+      boolean isSignal = ReadWriteIOUtils.readBool(stream);
+      IMemTable memTable;
+      if (isSignal) {
+        memTable = new NotifyFlushMemTable();
+      } else {
+        PrimitiveMemTable primitiveMemTable = new PrimitiveMemTable();
+        primitiveMemTable.deserialize(stream);
+        memTable = primitiveMemTable;
+      }
+      return memTable;
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunk.java
index c7ec62f1f1..92f6adc314 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunk.java
@@ -20,6 +20,8 @@ package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.db.utils.datastructure.AlignedTVList;
 import org.apache.iotdb.db.utils.datastructure.TVList;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
@@ -28,10 +30,14 @@ import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl;
 import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
@@ -58,6 +64,15 @@ public class AlignedWritableMemChunk implements IWritableMemChunk {
     this.list = AlignedTVList.newAlignedList(dataTypeList);
   }
 
+  private AlignedWritableMemChunk(List<IMeasurementSchema> schemaList, AlignedTVList list) {
+    this.measurementIndexMap = new LinkedHashMap<>();
+    this.schemaList = schemaList;
+    for (int i = 0; i < schemaList.size(); i++) {
+      measurementIndexMap.put(schemaList.get(i).getMeasurementId(), i);
+    }
+    this.list = list;
+  }
+
   public Set<String> getAllMeasurements() {
     return measurementIndexMap.keySet();
   }
@@ -356,4 +371,40 @@ public class AlignedWritableMemChunk implements IWritableMemChunk {
         .getTimeValuePair(getSortedTvListForQuery().rowCount() - 1)
         .getTimestamp();
   }
+
+  @Override
+  public int serializedSize() {
+    int size = 0;
+    size += Integer.BYTES;
+    for (IMeasurementSchema schema : schemaList) {
+      size += schema.serializedSize();
+    }
+
+    size += list.serializedSize();
+    return size;
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(schemaList.size(), buffer);
+    for (IMeasurementSchema schema : schemaList) {
+      byte[] bytes = new byte[schema.serializedSize()];
+      schema.serializeTo(ByteBuffer.wrap(bytes));
+      buffer.put(bytes);
+    }
+
+    list.serializeToWAL(buffer);
+  }
+
+  public static AlignedWritableMemChunk deserialize(DataInputStream stream) throws IOException {
+    int schemaListSize = stream.readInt();
+    List<IMeasurementSchema> schemaList = new ArrayList<>(schemaListSize);
+    for (int i = 0; i < schemaListSize; i++) {
+      IMeasurementSchema schema = MeasurementSchema.deserializeFrom(stream);
+      schemaList.add(schema);
+    }
+
+    AlignedTVList list = (AlignedTVList) TVList.deserialize(stream);
+    return new AlignedWritableMemChunk(schemaList, list);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunkGroup.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunkGroup.java
index 56fc7e9501..9675db9ef8 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunkGroup.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunkGroup.java
@@ -21,10 +21,13 @@ package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -39,6 +42,8 @@ public class AlignedWritableMemChunkGroup implements IWritableMemChunkGroup {
     memChunk = new AlignedWritableMemChunk(schemaList);
   }
 
+  private AlignedWritableMemChunkGroup() {}
+
   @Override
   public void writeValues(
       long[] times,
@@ -117,4 +122,21 @@ public class AlignedWritableMemChunkGroup implements IWritableMemChunkGroup {
   public AlignedWritableMemChunk getAlignedMemChunk() {
     return memChunk;
   }
+
+  @Override
+  public int serializedSize() {
+    return memChunk.serializedSize();
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    memChunk.serializeToWAL(buffer);
+  }
+
+  public static AlignedWritableMemChunkGroup deserialize(DataInputStream stream)
+      throws IOException {
+    AlignedWritableMemChunkGroup memChunkGroup = new AlignedWritableMemChunkGroup();
+    memChunkGroup.memChunk = AlignedWritableMemChunk.deserialize(stream);
+    return memChunkGroup;
+  }
 }
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 9b5a2b3580..4dae9cbef3 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
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.engine.memtable;
 
+import org.apache.iotdb.db.engine.flush.FlushStatus;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.exception.WriteProcessException;
@@ -29,6 +30,7 @@ import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertTabletNode;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
@@ -44,7 +46,7 @@ import java.util.Map;
  * i.e., Writing and querying operations must already have gotten writeLock and readLock
  * respectively.<br>
  */
-public interface IMemTable {
+public interface IMemTable extends WALEntryValue {
 
   Map<IDeviceID, IWritableMemChunkGroup> getMemTableMap();
 
@@ -179,5 +181,11 @@ public interface IMemTable {
 
   long getMinPlanIndex();
 
+  int getMemTableId();
+
   long getCreatedTime();
+
+  FlushStatus getFlushStatus();
+
+  void setFlushStatus(FlushStatus flushStatus);
 }
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 c46714fbba..7de03ccc80 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
@@ -19,6 +19,7 @@
 package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.db.utils.datastructure.TVList;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.BitMap;
@@ -27,7 +28,7 @@ import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
 import java.util.List;
 
-public interface IWritableMemChunk {
+public interface IWritableMemChunk extends WALEntryValue {
 
   void putLong(long t, long v);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunkGroup.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunkGroup.java
index 0ee70755b5..a1ad3715ba 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunkGroup.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunkGroup.java
@@ -20,13 +20,14 @@
 package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
 import java.util.List;
 import java.util.Map;
 
-public interface IWritableMemChunkGroup {
+public interface IWritableMemChunkGroup extends WALEntryValue {
 
   void writeValues(
       long[] times,
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 5f99bb6d1a..1951e472c8 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
@@ -19,6 +19,7 @@
 package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.db.utils.datastructure.TVList;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
@@ -26,10 +27,14 @@ import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
 import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.List;
 
 public class WritableMemChunk implements IWritableMemChunk {
@@ -44,6 +49,8 @@ public class WritableMemChunk implements IWritableMemChunk {
     this.list = TVList.newList(schema.getType());
   }
 
+  private WritableMemChunk() {}
+
   @Override
   public void write(long insertTime, Object objectValue) {
     switch (schema.getType()) {
@@ -350,4 +357,25 @@ public class WritableMemChunk implements IWritableMemChunk {
       list.clear();
     }
   }
+
+  @Override
+  public int serializedSize() {
+    return schema.serializedSize() + list.serializedSize();
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    byte[] bytes = new byte[schema.serializedSize()];
+    schema.serializeTo(ByteBuffer.wrap(bytes));
+    buffer.put(bytes);
+
+    list.serializeToWAL(buffer);
+  }
+
+  public static WritableMemChunk deserialize(DataInputStream stream) throws IOException {
+    WritableMemChunk memChunk = new WritableMemChunk();
+    memChunk.schema = MeasurementSchema.deserializeFrom(stream);
+    memChunk.list = TVList.deserialize(stream);
+    return memChunk;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkGroup.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkGroup.java
index 820fa41459..9da8777e22 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkGroup.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkGroup.java
@@ -20,9 +20,14 @@
 package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -134,4 +139,36 @@ public class WritableMemChunkGroup implements IWritableMemChunkGroup {
   public long getCurrentTVListSize(String measurement) {
     return memChunkMap.get(measurement).getTVList().rowCount();
   }
+
+  @Override
+  public int serializedSize() {
+    int size = 0;
+    size += Integer.BYTES;
+    for (Map.Entry<String, IWritableMemChunk> entry : memChunkMap.entrySet()) {
+      size += ReadWriteIOUtils.sizeToWrite(entry.getKey());
+      size += entry.getValue().serializedSize();
+    }
+    return size;
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    buffer.putInt(memChunkMap.size());
+    for (Map.Entry<String, IWritableMemChunk> entry : memChunkMap.entrySet()) {
+      WALWriteUtils.write(entry.getKey(), buffer);
+      IWritableMemChunk memChunk = entry.getValue();
+      memChunk.serializeToWAL(buffer);
+    }
+  }
+
+  public static WritableMemChunkGroup deserialize(DataInputStream stream) throws IOException {
+    WritableMemChunkGroup memChunkGroup = new WritableMemChunkGroup();
+    int memChunkMapSize = stream.readInt();
+    for (int i = 0; i < memChunkMapSize; ++i) {
+      String measurement = ReadWriteIOUtils.readString(stream);
+      IWritableMemChunk memChunk = WritableMemChunk.deserialize(stream);
+      memChunkGroup.memChunkMap.put(measurement, memChunk);
+    }
+    return memChunkGroup;
+  }
 }
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
index a8ba599515..77cbfe9680 100644
--- 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
@@ -18,18 +18,12 @@
  */
 package org.apache.iotdb.db.engine.storagegroup;
 
-import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.rescon.SystemInfo;
-import org.apache.iotdb.db.utils.MmapUtil;
 
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
 import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
 
 /** The storageGroupInfo records the total memory cost of the Storage Group. */
 public class StorageGroupInfo {
@@ -99,37 +93,4 @@ public class StorageGroupInfo {
     reportedTsps.remove(tsFileProcessor);
     SystemInfo.getInstance().resetStorageGroupStatus(this);
   }
-
-  public Supplier<ByteBuffer[]> getWalSupplier() {
-    if (virtualStorageGroupProcessor != null) {
-      return virtualStorageGroupProcessor::getWalDirectByteBuffer;
-    } else { // only happens in test
-      return this::walSupplier;
-    }
-  }
-
-  @TestOnly
-  private ByteBuffer[] walSupplier() {
-    ByteBuffer[] buffers = new ByteBuffer[2];
-    buffers[0] =
-        ByteBuffer.allocateDirect(IoTDBDescriptor.getInstance().getConfig().getWalBufferSize() / 2);
-    buffers[1] =
-        ByteBuffer.allocateDirect(IoTDBDescriptor.getInstance().getConfig().getWalBufferSize() / 2);
-    return buffers;
-  }
-
-  public Consumer<ByteBuffer[]> getWalConsumer() {
-    if (virtualStorageGroupProcessor != null) {
-      return virtualStorageGroupProcessor::releaseWalBuffer;
-    } else { // only happens in test
-      return this::walConsumer;
-    }
-  }
-
-  @TestOnly
-  private void walConsumer(ByteBuffer[] buffers) {
-    for (ByteBuffer byteBuffer : buffers) {
-      MmapUtil.clean((MappedByteBuffer) byteBuffer);
-    }
-  }
 }
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 5fd5aad7f5..1f1363edda 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
@@ -31,7 +31,6 @@ import org.apache.iotdb.db.engine.flush.NotifyFlushMemTable;
 import org.apache.iotdb.db.engine.memtable.AlignedWritableMemChunk;
 import org.apache.iotdb.db.engine.memtable.AlignedWritableMemChunkGroup;
 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.querycontext.ReadOnlyMemChunk;
@@ -48,6 +47,7 @@ import org.apache.iotdb.db.metadata.path.AlignedPath;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertTabletNode;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 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;
@@ -57,9 +57,9 @@ import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.utils.MemUtils;
 import org.apache.iotdb.db.utils.datastructure.AlignedTVList;
 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;
+import org.apache.iotdb.db.wal.WALManager;
+import org.apache.iotdb.db.wal.node.IWALNode;
+import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
@@ -142,8 +142,8 @@ public class TsFileProcessor {
   /** this callback is called before the workMemtable is added into the flushingMemTables. */
   private final UpdateEndTimeCallBack updateLatestFlushTimeCallback;
 
-  /** Wal log node */
-  private WriteLogNode logNode;
+  /** wal node */
+  private final IWALNode walNode;
 
   /** whether it's a sequence file or not */
   private final boolean sequence;
@@ -176,9 +176,11 @@ public class TsFileProcessor {
     this.writer = new RestorableTsFileIOWriter(tsfile);
     this.updateLatestFlushTimeCallback = updateLatestFlushTimeCallback;
     this.sequence = sequence;
-    logger.info("create a new tsfile processor {}", tsfile.getAbsolutePath());
-    flushListeners.add(new WALFlushListener(this));
+    this.walNode = WALManager.getInstance().applyForWALNode();
+    flushListeners.add(FlushListener.DefaultMemTableFLushListener.INSTANCE);
+    flushListeners.add(this.walNode);
     closeFileListeners.add(closeTsFileCallback);
+    logger.info("create a new tsfile processor {}", tsfile.getAbsolutePath());
   }
 
   @SuppressWarnings("java:S107") // ignore number of arguments
@@ -196,9 +198,11 @@ public class TsFileProcessor {
     this.writer = writer;
     this.updateLatestFlushTimeCallback = updateLatestFlushTimeCallback;
     this.sequence = sequence;
-    logger.info("reopen a tsfile processor {}", tsFileResource.getTsFile());
-    flushListeners.add(new WALFlushListener(this));
+    this.walNode = WALManager.getInstance().applyForWALNode();
+    flushListeners.add(FlushListener.DefaultMemTableFLushListener.INSTANCE);
+    flushListeners.add(this.walNode);
     closeFileListeners.add(closeUnsealedTsFileProcessor);
+    logger.info("reopen a tsfile processor {}", tsFileResource.getTsFile());
   }
 
   /**
@@ -209,12 +213,7 @@ public class TsFileProcessor {
   public void insert(InsertRowPlan insertRowPlan) throws WriteProcessException {
 
     if (workMemTable == null) {
-      if (enableMemControl) {
-        workMemTable = new PrimitiveMemTable(enableMemControl);
-        MemTableManager.getInstance().addMemtableNumber();
-      } else {
-        workMemTable = MemTableManager.getInstance().getAvailableMemTable(storageGroupName);
-      }
+      createNewWorkingMemTable();
     }
 
     long[] memIncrements = null;
@@ -226,19 +225,20 @@ public class TsFileProcessor {
       }
     }
 
-    if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-      try {
-        getLogNode().write(insertRowPlan);
-      } catch (Exception e) {
-        if (enableMemControl && memIncrements != null) {
-          rollbackMemoryInfo(memIncrements);
-        }
-        throw new WriteProcessException(
-            String.format(
-                "%s: %s write WAL failed",
-                storageGroupName, tsFileResource.getTsFile().getAbsolutePath()),
-            e);
+    try {
+      WALFlushListener walFlushListener = walNode.log(workMemTable.getMemTableId(), insertRowPlan);
+      if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
+        throw walFlushListener.getCause();
       }
+    } catch (Exception e) {
+      if (enableMemControl && memIncrements != null) {
+        rollbackMemoryInfo(memIncrements);
+      }
+      throw new WriteProcessException(
+          String.format(
+              "%s: %s write WAL failed",
+              storageGroupName, tsFileResource.getTsFile().getAbsolutePath()),
+          e);
     }
 
     if (insertRowPlan.isAligned()) {
@@ -267,12 +267,7 @@ public class TsFileProcessor {
   public void insert(InsertRowNode insertRowNode) throws WriteProcessException {
 
     if (workMemTable == null) {
-      if (enableMemControl) {
-        workMemTable = new PrimitiveMemTable(enableMemControl);
-        MemTableManager.getInstance().addMemtableNumber();
-      } else {
-        workMemTable = MemTableManager.getInstance().getAvailableMemTable(storageGroupName);
-      }
+      createNewWorkingMemTable();
     }
 
     long[] memIncrements = null;
@@ -284,19 +279,20 @@ public class TsFileProcessor {
       }
     }
 
-    if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-      try {
-        getLogNode().write(insertRowNode);
-      } catch (Exception e) {
-        if (enableMemControl && memIncrements != null) {
-          rollbackMemoryInfo(memIncrements);
-        }
-        throw new WriteProcessException(
-            String.format(
-                "%s: %s write WAL failed",
-                storageGroupName, tsFileResource.getTsFile().getAbsolutePath()),
-            e);
+    try {
+      WALFlushListener walFlushListener = walNode.log(workMemTable.getMemTableId(), insertRowNode);
+      if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
+        throw walFlushListener.getCause();
+      }
+    } catch (Exception e) {
+      if (enableMemControl && memIncrements != null) {
+        rollbackMemoryInfo(memIncrements);
       }
+      throw new WriteProcessException(
+          String.format(
+              "%s: %s write WAL failed",
+              storageGroupName, tsFileResource.getTsFile().getAbsolutePath()),
+          e);
     }
 
     if (insertRowNode.isAligned()) {
@@ -332,12 +328,7 @@ public class TsFileProcessor {
       throws WriteProcessException {
 
     if (workMemTable == null) {
-      if (enableMemControl) {
-        workMemTable = new PrimitiveMemTable(enableMemControl);
-        MemTableManager.getInstance().addMemtableNumber();
-      } else {
-        workMemTable = MemTableManager.getInstance().getAvailableMemTable(storageGroupName);
-      }
+      createNewWorkingMemTable();
     }
 
     long[] memIncrements = null;
@@ -357,10 +348,10 @@ public class TsFileProcessor {
     }
 
     try {
-      if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-        insertTabletPlan.setStart(start);
-        insertTabletPlan.setEnd(end);
-        getLogNode().write(insertTabletPlan);
+      WALFlushListener walFlushListener =
+          walNode.log(workMemTable.getMemTableId(), insertTabletPlan, start, end);
+      if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
+        throw walFlushListener.getCause();
       }
     } catch (Exception e) {
       for (int i = start; i < end; i++) {
@@ -400,6 +391,11 @@ public class TsFileProcessor {
     tsFileResource.updatePlanIndexes(insertTabletPlan.getIndex());
   }
 
+  private void createNewWorkingMemTable() throws WriteProcessException {
+    workMemTable = MemTableManager.getInstance().getAvailableMemTable(storageGroupName);
+    walNode.onMemTableCreated(workMemTable, tsFileResource.getTsFilePath());
+  }
+
   /**
    * insert batch data of insertTabletPlan into the workingMemtable. The rows to be inserted are in
    * the range [start, end). Null value in each column values will be replaced by the subsequent
@@ -415,12 +411,7 @@ public class TsFileProcessor {
       throws WriteProcessException {
 
     if (workMemTable == null) {
-      if (enableMemControl) {
-        workMemTable = new PrimitiveMemTable(enableMemControl);
-        MemTableManager.getInstance().addMemtableNumber();
-      } else {
-        workMemTable = MemTableManager.getInstance().getAvailableMemTable(storageGroupName);
-      }
+      createNewWorkingMemTable();
     }
 
     long[] memIncrements = null;
@@ -440,12 +431,10 @@ public class TsFileProcessor {
     }
 
     try {
-      if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-        // TODO(WAL)
-        // Start and end should be removed from new WAL
-        // insertTabletNode.setStart(start);
-        // insertTabletNode.setEnd(end);
-        getLogNode().write(insertTabletNode);
+      WALFlushListener walFlushListener =
+          walNode.log(workMemTable.getMemTableId(), insertTabletNode, start, end);
+      if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
+        throw walFlushListener.getCause();
       }
     } catch (Exception e) {
       for (int i = start; i < end; i++) {
@@ -873,6 +862,10 @@ public class TsFileProcessor {
     }
   }
 
+  WALFlushListener logDeleteInWAL(DeletePlan deletePlan) {
+    return walNode.log(workMemTable.getMemTableId(), deletePlan);
+  }
+
   public TsFileResource getTsFileResource() {
     return tsFileResource;
   }
@@ -1126,7 +1119,7 @@ public class TsFileProcessor {
     }
 
     for (FlushListener flushListener : flushListeners) {
-      flushListener.onFlushStart(tobeFlushed);
+      flushListener.onMemTableFlushStarted(tobeFlushed);
     }
 
     if (enableMemControl) {
@@ -1260,7 +1253,7 @@ public class TsFileProcessor {
     }
 
     for (FlushListener flushListener : flushListeners) {
-      flushListener.onFlushEnd(memTableToFlush);
+      flushListener.onMemTableFlushed(memTableToFlush);
     }
 
     try {
@@ -1428,31 +1421,11 @@ public class TsFileProcessor {
     this.managedByFlushManager = managedByFlushManager;
   }
 
-  /**
-   * get WAL log node
-   *
-   * @return WAL log node
-   */
-  public WriteLogNode getLogNode() {
-    if (logNode == null) {
-      logNode =
-          MultiFileLogNodeManager.getInstance()
-              .getNode(
-                  storageGroupName + "-" + tsFileResource.getTsFile().getName(),
-                  storageGroupInfo.getWalSupplier());
-    }
-    return logNode;
-  }
-
   /** close this tsfile */
   public void close() throws TsFileProcessorException {
     try {
       // when closing resource file, its corresponding mod file is also closed.
       tsFileResource.close();
-      MultiFileLogNodeManager.getInstance()
-          .deleteNode(
-              storageGroupName + "-" + tsFileResource.getTsFile().getName(),
-              storageGroupInfo.getWalConsumer());
     } catch (IOException e) {
       throw new TsFileProcessorException(e);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
index 3b95fad455..3d07a6cdfc 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
@@ -78,9 +78,13 @@ import org.apache.iotdb.db.service.metrics.MetricsService;
 import org.apache.iotdb.db.service.metrics.Tag;
 import org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool;
 import org.apache.iotdb.db.utils.CopyOnReadLinkedList;
-import org.apache.iotdb.db.utils.MmapUtil;
 import org.apache.iotdb.db.utils.UpgradeUtils;
-import org.apache.iotdb.db.writelog.recover.TsFileRecoverPerformer;
+import org.apache.iotdb.db.wal.recover.WALRecoverManager;
+import org.apache.iotdb.db.wal.recover.file.SealedTsFileRecoverPerformer;
+import org.apache.iotdb.db.wal.recover.file.UnsealedTsFileRecoverPerformer;
+import org.apache.iotdb.db.wal.utils.WALMode;
+import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
+import org.apache.iotdb.db.wal.utils.listener.WALRecoverListener;
 import org.apache.iotdb.metrics.config.MetricConfigDescriptor;
 import org.apache.iotdb.metrics.utils.MetricLevel;
 import org.apache.iotdb.rpc.RpcUtils;
@@ -99,15 +103,12 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
 import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
-import java.util.Deque;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -163,8 +164,6 @@ public class VirtualStorageGroupProcessor {
   /** indicating the file to be loaded overlap with some files. */
   private static final int POS_OVERLAP = -3;
 
-  private static final int WAL_BUFFER_SIZE =
-      IoTDBDescriptor.getInstance().getConfig().getWalBufferSize() / 2;
   private final boolean enableMemControl = config.isEnableMemControl();
   /**
    * a read write lock for guaranteeing concurrent safety when accessing all fields in this class
@@ -184,8 +183,6 @@ public class VirtualStorageGroupProcessor {
   /** time partition id in the storage group -> tsFileProcessor for this time partition */
   private final TreeMap<Long, TsFileProcessor> workUnsequenceTsFileProcessors = new TreeMap<>();
 
-  private final Deque<ByteBuffer> walByteBufferPool = new LinkedList<>();
-
   // upgrading sequence TsFile resource list
   private List<TsFileResource> upgradeSeqFileList = new LinkedList<>();
   /** sequence tsfile processors which are closing */
@@ -245,18 +242,6 @@ public class VirtualStorageGroupProcessor {
   /** flush listeners */
   private List<FlushListener> customFlushListeners = Collections.emptyList();
 
-  private int currentWalPoolSize = 0;
-
-  // this field is used to avoid when one writer release bytebuffer back to pool,
-  // and the next writer has already arrived, but the check thread get the lock first, it find the
-  // pool
-  // is not empty, so it free the memory. When the next writer get the lock, it will apply the
-  // memory again.
-  // So our free memory strategy is only when the expected size less than the current pool size
-  // and the pool is not empty and the time interval since the pool is not empty is larger than
-  // DEFAULT_POOL_TRIM_INTERVAL_MILLIS
-  private long timeWhenPoolNotEmpty = Long.MAX_VALUE;
-
   private ILastFlushTimeManager lastFlushTimeManager;
 
   /**
@@ -266,105 +251,11 @@ public class VirtualStorageGroupProcessor {
   private String insertWriteLockHolder = "";
 
   private ScheduledExecutorService timedCompactionScheduleTask;
-  private ScheduledExecutorService walTrimScheduleTask;
 
   public static final long COMPACTION_TASK_SUBMIT_DELAY = 20L * 1000L;
 
   private IDTable idTable;
 
-  /**
-   * get the direct byte buffer from pool, each fetch contains two ByteBuffer, return null if fetch
-   * fails
-   */
-  public ByteBuffer[] getWalDirectByteBuffer() {
-    ByteBuffer[] res = new ByteBuffer[2];
-    synchronized (walByteBufferPool) {
-      long startTime = System.nanoTime();
-      int MAX_WAL_BYTEBUFFER_NUM =
-          config.getConcurrentWritingTimePartition()
-              * config.getMaxWalBytebufferNumForEachPartition();
-      while (walByteBufferPool.isEmpty() && currentWalPoolSize + 2 > MAX_WAL_BYTEBUFFER_NUM) {
-        try {
-          walByteBufferPool.wait();
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          logger.error(
-              "getDirectByteBuffer occurs error while waiting for DirectByteBuffer" + "group {}-{}",
-              logicalStorageGroupName,
-              virtualStorageGroupId,
-              e);
-        }
-        logger.info(
-            "Waiting {} ms for wal direct byte buffer.",
-            (System.nanoTime() - startTime) / 1_000_000);
-      }
-      // If the queue is not empty, it must have at least two.
-      if (!walByteBufferPool.isEmpty()) {
-        res[0] = walByteBufferPool.pollFirst();
-        res[1] = walByteBufferPool.pollFirst();
-      } else {
-        // if the queue is empty and current size is less than MAX_BYTEBUFFER_NUM
-        // we can construct another two more new byte buffer
-        try {
-          res[0] = ByteBuffer.allocateDirect(WAL_BUFFER_SIZE);
-          res[1] = ByteBuffer.allocateDirect(WAL_BUFFER_SIZE);
-          currentWalPoolSize += 2;
-        } catch (OutOfMemoryError e) {
-          logger.error("Allocate ByteBuffers error", e);
-          if (res[0] != null) {
-            MmapUtil.clean((MappedByteBuffer) res[0]);
-          }
-          if (res[1] != null) {
-            MmapUtil.clean((MappedByteBuffer) res[1]);
-          }
-          return null;
-        }
-      }
-      // if the pool is empty, set the time back to MAX_VALUE
-      if (walByteBufferPool.isEmpty()) {
-        timeWhenPoolNotEmpty = Long.MAX_VALUE;
-      }
-    }
-    return res;
-  }
-
-  /** put the byteBuffer back to pool */
-  public void releaseWalBuffer(ByteBuffer[] byteBuffers) {
-    for (ByteBuffer byteBuffer : byteBuffers) {
-      byteBuffer.clear();
-    }
-    synchronized (walByteBufferPool) {
-      // if the pool is empty before, update the time
-      if (walByteBufferPool.isEmpty()) {
-        timeWhenPoolNotEmpty = System.nanoTime();
-      }
-      walByteBufferPool.addLast(byteBuffers[0]);
-      walByteBufferPool.addLast(byteBuffers[1]);
-      walByteBufferPool.notifyAll();
-    }
-  }
-
-  /** trim the size of the pool and release the memory of needless direct byte buffer */
-  private void trimTask() {
-    synchronized (walByteBufferPool) {
-      int expectedSize =
-          (workSequenceTsFileProcessors.size() + workUnsequenceTsFileProcessors.size()) * 2;
-      // the unit is ms
-      long poolNotEmptyIntervalInMS = (System.nanoTime() - timeWhenPoolNotEmpty) / 1_000_000;
-      // only when the expected size less than the current pool size
-      // and the pool is not empty and the time interval since the pool is not empty is larger than
-      // 10s
-      // we will trim the size to expectedSize until the pool is empty
-      while (expectedSize < currentWalPoolSize
-          && !walByteBufferPool.isEmpty()
-          && poolNotEmptyIntervalInMS >= config.getWalPoolTrimIntervalInMS()) {
-        MmapUtil.clean((MappedByteBuffer) walByteBufferPool.removeLast());
-        MmapUtil.clean((MappedByteBuffer) walByteBufferPool.removeLast());
-        currentWalPoolSize -= 2;
-      }
-    }
-  }
-
   /**
    * constrcut a storage group processor
    *
@@ -420,20 +311,6 @@ public class VirtualStorageGroupProcessor {
               Tag.NAME.toString(),
               "storageGroup_" + getLogicalStorageGroupName());
     }
-
-    // start trim task at last
-    walTrimScheduleTask =
-        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
-            ThreadName.WAL_TRIM.getName()
-                + "-"
-                + logicalStorageGroupName
-                + "-"
-                + virtualStorageGroupId);
-    walTrimScheduleTask.scheduleWithFixedDelay(
-        this::trimTask,
-        config.getWalPoolTrimIntervalInMS(),
-        config.getWalPoolTrimIntervalInMS(),
-        TimeUnit.MILLISECONDS);
   }
 
   public String getLogicalStorageGroupName() {
@@ -466,9 +343,9 @@ public class VirtualStorageGroupProcessor {
   }
 
   /** this class is used to store recovering context */
-  private class RecoveryContext {
+  private class VSGRecoveryContext {
     /** number of files to be recovered */
-    private final long filesToRecoverNum;
+    private final long numOfFilesToRecover;
     /** when the change of recoveredFilesNum exceeds this, log check will be triggered */
     private final long filesNumLogCheckTrigger;
     /** number of already recovered files */
@@ -478,10 +355,10 @@ public class VirtualStorageGroupProcessor {
     /** last recovery log files num */
     private long lastLogCheckFilesNum;
 
-    public RecoveryContext(long filesToRecoverNum, long recoveredFilesNum) {
-      this.filesToRecoverNum = filesToRecoverNum;
-      this.recoveredFilesNum = recoveredFilesNum;
-      this.filesNumLogCheckTrigger = this.filesToRecoverNum / 100;
+    public VSGRecoveryContext(long numOfFilesToRecover) {
+      this.numOfFilesToRecover = numOfFilesToRecover;
+      this.recoveredFilesNum = 0;
+      this.filesNumLogCheckTrigger = this.numOfFilesToRecover / 100;
       this.lastLogTime = System.currentTimeMillis();
       this.lastLogCheckFilesNum = 0;
     }
@@ -497,7 +374,7 @@ public class VirtualStorageGroupProcessor {
               "The virtual storage group {}[{}] has recovered {}%, please wait a moment.",
               logicalStorageGroupName,
               virtualStorageGroupId,
-              recoveredFilesNum * 1.0 / filesToRecoverNum);
+              recoveredFilesNum * 1.0 / numOfFilesToRecover);
           lastLogTime = System.currentTimeMillis();
         }
       }
@@ -531,17 +408,64 @@ public class VirtualStorageGroupProcessor {
 
       // split by partition so that we can find the last file of each partition and decide to
       // close it or not
-      RecoveryContext recoveryContext =
-          new RecoveryContext(tmpSeqTsFiles.size() + tmpUnseqTsFiles.size(), 0);
+      VSGRecoveryContext VSGRecoveryContext =
+          new VSGRecoveryContext(tmpSeqTsFiles.size() + tmpUnseqTsFiles.size());
       Map<Long, List<TsFileResource>> partitionTmpSeqTsFiles =
           splitResourcesByPartition(tmpSeqTsFiles);
       Map<Long, List<TsFileResource>> partitionTmpUnseqTsFiles =
           splitResourcesByPartition(tmpUnseqTsFiles);
+      // recover unsealed TsFiles
+      List<WALRecoverListener> recoverListeners = new ArrayList<>();
+      for (List<TsFileResource> value : partitionTmpSeqTsFiles.values()) {
+        // tsFiles without resource file are unsealed
+        while (!value.isEmpty()) {
+          TsFileResource tsFileResource = value.get(value.size() - 1);
+          if (tsFileResource.resourceFileExists()) {
+            break;
+          } else {
+            value.remove(value.size() - 1);
+            WALRecoverListener recoverListener =
+                recoverUnsealedTsFile(tsFileResource, VSGRecoveryContext, true);
+            recoverListeners.add(recoverListener);
+          }
+        }
+      }
+      for (List<TsFileResource> value : partitionTmpUnseqTsFiles.values()) {
+        // tsFiles without resource file are unsealed
+        while (!value.isEmpty()) {
+          TsFileResource tsFileResource = value.get(value.size() - 1);
+          if (tsFileResource.resourceFileExists()) {
+            break;
+          } else {
+            value.remove(value.size() - 1);
+            WALRecoverListener recoverListener =
+                recoverUnsealedTsFile(tsFileResource, VSGRecoveryContext, false);
+            recoverListeners.add(recoverListener);
+          }
+        }
+      }
+      WALRecoverManager.getInstance().getAllVsgScannedLatch().countDown();
+      // recover sealed TsFiles
       for (List<TsFileResource> value : partitionTmpSeqTsFiles.values()) {
-        recoverTsFiles(value, recoveryContext, true);
+        for (TsFileResource tsFileResource : value) {
+          recoverSealedTsFiles(tsFileResource, VSGRecoveryContext, true);
+        }
       }
       for (List<TsFileResource> value : partitionTmpUnseqTsFiles.values()) {
-        recoverTsFiles(value, recoveryContext, false);
+        for (TsFileResource tsFileResource : value) {
+          recoverSealedTsFiles(tsFileResource, VSGRecoveryContext, false);
+        }
+      }
+      // wait until all unsealed TsFiles have been recovered
+      for (WALRecoverListener recoverListener : recoverListeners) {
+        if (recoverListener.waitForResult() == WALRecoverListener.Status.FAILURE) {
+          logger.error(
+              "Fail to recover unsealed TsFile {}, skip it.",
+              recoverListener.getFilePath(),
+              recoverListener.getCause());
+        }
+        // update VSGRecoveryContext
+        VSGRecoveryContext.incrementRecoveredFilesNum();
       }
       for (TsFileResource resource : tsFileManager.getTsFileList(true)) {
         long partitionNum = resource.getTimePartition();
@@ -716,7 +640,6 @@ public class VirtualStorageGroupProcessor {
     for (File f : upgradeFiles) {
       TsFileResource fileResource = new TsFileResource(f);
       fileResource.setStatus(TsFileResourceStatus.CLOSED);
-      ;
       // make sure the flush command is called before IoTDB is down.
       fileResource.deserializeFromOldFile();
       upgradeRet.add(fileResource);
@@ -756,113 +679,94 @@ public class VirtualStorageGroupProcessor {
     }
   }
 
-  private void recoverTsFiles(List<TsFileResource> tsFiles, RecoveryContext context, boolean isSeq)
-      throws IOException {
-    for (int i = 0; i < tsFiles.size(); i++) {
-      // update recovery context
-      context.incrementRecoveredFilesNum();
+  /** submit unsealed TsFile to WALRecoverManager */
+  private WALRecoverListener recoverUnsealedTsFile(
+      TsFileResource unsealedTsFile, VSGRecoveryContext context, boolean isSeq) {
+    UnsealedTsFileRecoverPerformer recoverPerformer =
+        new UnsealedTsFileRecoverPerformer(
+            unsealedTsFile, isSeq, idTable, this::callbackAfterUnsealedTsFileRecovered);
+    // remember to close UnsealedTsFileRecoverPerformer
+    return WALRecoverManager.getInstance().addRecoverPerformer(recoverPerformer);
+  }
 
-      TsFileResource tsFileResource = tsFiles.get(i);
+  private void callbackAfterUnsealedTsFileRecovered(
+      UnsealedTsFileRecoverPerformer recoverPerformer) {
+    TsFileResource tsFileResource = recoverPerformer.getTsFileResource();
+    if (!recoverPerformer.canWrite()) {
+      // cannot write, just close it
+      try {
+        tsFileResource.close();
+      } catch (IOException e) {
+        logger.error("Fail to close TsFile {} when recovering", tsFileResource.getTsFile(), e);
+      }
+      tsFileResourceManager.registerSealedTsFileResource(tsFileResource);
+    } else {
+      // the last file is not closed, continue writing to it
+      RestorableTsFileIOWriter writer = recoverPerformer.getWriter();
       long timePartitionId = tsFileResource.getTimePartition();
-
-      TsFileRecoverPerformer recoverPerformer =
-          new TsFileRecoverPerformer(
-              logicalStorageGroupName
-                  + File.separator
-                  + virtualStorageGroupId
-                  + FILE_NAME_SEPARATOR,
+      boolean isSeq = recoverPerformer.isSequence();
+      TsFileProcessor tsFileProcessor =
+          new TsFileProcessor(
+              virtualStorageGroupId,
+              storageGroupInfo,
               tsFileResource,
+              this::closeUnsealedTsFileProcessorCallBack,
+              isSeq ? this::updateLatestFlushTimeCallback : this::unsequenceFlushCallback,
               isSeq,
-              i == tsFiles.size() - 1,
-              this);
-
-      RestorableTsFileIOWriter writer = null;
-      try {
-        // this tsfile is not zero level, no need to perform redo wal
-        if (TsFileResource.getInnerCompactionCount(tsFileResource.getTsFile().getName()) > 0) {
-          writer =
-              recoverPerformer.recover(false, this::getWalDirectByteBuffer, this::releaseWalBuffer);
-          if (writer != null && writer.hasCrashed()) {
-            tsFileManager.addForRecover(tsFileResource, isSeq);
-          } else {
-            tsFileResource.setStatus(TsFileResourceStatus.CLOSED);
-            tsFileManager.add(tsFileResource, isSeq);
-            tsFileResourceManager.registerSealedTsFileResource(tsFileResource);
-          }
-          continue;
-        } else {
-          writer =
-              recoverPerformer.recover(true, this::getWalDirectByteBuffer, this::releaseWalBuffer);
-        }
-
-        if (i != tsFiles.size() - 1 || writer == null || !writer.canWrite()) {
-          // not the last file or cannot write, just close it
-          tsFileResource.close();
-          tsFileResourceManager.registerSealedTsFileResource(tsFileResource);
-        } else if (writer.canWrite()) {
-          // the last file is not closed, continue writing to in
-          TsFileProcessor tsFileProcessor;
-          if (isSeq) {
-            tsFileProcessor =
-                new TsFileProcessor(
-                    virtualStorageGroupId,
-                    storageGroupInfo,
-                    tsFileResource,
-                    this::closeUnsealedTsFileProcessorCallBack,
-                    this::updateLatestFlushTimeCallback,
-                    true,
-                    writer);
-            if (enableMemControl) {
-              TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(storageGroupInfo);
-              tsFileProcessor.setTsFileProcessorInfo(tsFileProcessorInfo);
-              this.storageGroupInfo.initTsFileProcessorInfo(tsFileProcessor);
-            }
-            workSequenceTsFileProcessors.put(timePartitionId, tsFileProcessor);
-          } else {
-            tsFileProcessor =
-                new TsFileProcessor(
-                    virtualStorageGroupId,
-                    storageGroupInfo,
-                    tsFileResource,
-                    this::closeUnsealedTsFileProcessorCallBack,
-                    this::unsequenceFlushCallback,
-                    false,
-                    writer);
-            if (enableMemControl) {
-              TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(storageGroupInfo);
-              tsFileProcessor.setTsFileProcessorInfo(tsFileProcessorInfo);
-              this.storageGroupInfo.initTsFileProcessorInfo(tsFileProcessor);
-            }
-            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();
-                }
-              }
+              writer);
+      if (isSeq) {
+        workSequenceTsFileProcessors.put(timePartitionId, tsFileProcessor);
+      } else {
+        workUnsequenceTsFileProcessors.put(timePartitionId, tsFileProcessor);
+      }
+      tsFileResource.setProcessor(tsFileProcessor);
+      tsFileResource.removeResourceFile();
+      tsFileProcessor.setTimeRangeId(timePartitionId);
+      writer.makeMetadataVisible();
+      if (enableMemControl) {
+        TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(storageGroupInfo);
+        tsFileProcessor.setTsFileProcessorInfo(tsFileProcessorInfo);
+        this.storageGroupInfo.initTsFileProcessorInfo(tsFileProcessor);
+        // 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);
           }
         }
-        tsFileManager.add(tsFileResource, isSeq);
-      } catch (StorageGroupProcessorException | IOException e) {
-        logger.warn(
-            "Skip TsFile: {} because of error in recover: ", tsFileResource.getTsFilePath(), e);
-        continue;
-      } finally {
-        if (writer != null) {
-          writer.close();
+        tsFileProcessorInfo.addTSPMemCost(chunkMetadataSize);
+      }
+    }
+    tsFileManager.add(tsFileResource, recoverPerformer.isSequence());
+  }
+
+  /** recover sealed TsFile */
+  private void recoverSealedTsFiles(
+      TsFileResource sealedTsFile, VSGRecoveryContext context, boolean isSeq) {
+    try (SealedTsFileRecoverPerformer recoverPerformer =
+        new SealedTsFileRecoverPerformer(sealedTsFile)) {
+      recoverPerformer.recover();
+      // pick up crashed compaction target files
+      if (recoverPerformer.hasCrashed()) {
+        if (TsFileResource.getInnerCompactionCount(sealedTsFile.getTsFile().getName()) > 0) {
+          tsFileManager.addForRecover(sealedTsFile, isSeq);
+          return;
+        } else {
+          logger.warn(
+              "Sealed TsFile {} has crashed at zero level, truncate and recover it.",
+              sealedTsFile.getTsFilePath());
         }
       }
+      sealedTsFile.close();
+      tsFileManager.add(sealedTsFile, isSeq);
+      tsFileResourceManager.registerSealedTsFileResource(sealedTsFile);
+    } catch (StorageGroupProcessorException | IOException e) {
+      logger.error("Fail to recover sealed TsFile {}, skip it.", sealedTsFile.getTsFilePath(), e);
+    } finally {
+      // update recovery context
+      context.incrementRecoveredFilesNum();
     }
   }
 
@@ -1405,9 +1309,32 @@ public class VirtualStorageGroupProcessor {
   }
 
   /**
-   * mem control module use this method to flush memtable
+   * WAL module uses this method to flush memTable
+   *
+   * @return True if flush task is submitted successfully
+   */
+  public boolean submitAFlushTask(long timeRangeId, boolean sequence) {
+    writeLock("submitAFlushTask");
+    try {
+      TsFileProcessor tsFileProcessor;
+      if (sequence) {
+        tsFileProcessor = workSequenceTsFileProcessors.get(timeRangeId);
+      } else {
+        tsFileProcessor = workUnsequenceTsFileProcessors.get(timeRangeId);
+      }
+      if (tsFileProcessor != null) {
+        fileFlushPolicy.apply(this, tsFileProcessor, tsFileProcessor.isSequence());
+      }
+      return tsFileProcessor != null;
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  /**
+   * mem control module uses this method to flush memTable
    *
-   * @param tsFileProcessor tsfile processor in which memtable to be flushed
+   * @param tsFileProcessor tsfile processor in which memTable to be flushed
    */
   public void submitAFlushTaskWhenShouldFlush(TsFileProcessor tsFileProcessor) {
     writeLock("submitAFlushTaskWhenShouldFlush");
@@ -1653,16 +1580,6 @@ public class VirtualStorageGroupProcessor {
     }
   }
 
-  /** release wal buffer */
-  public void releaseWalDirectByteBufferPool() {
-    synchronized (walByteBufferPool) {
-      while (!walByteBufferPool.isEmpty()) {
-        MmapUtil.clean((MappedByteBuffer) walByteBufferPool.removeFirst());
-        currentWalPoolSize--;
-      }
-    }
-  }
-
   /** delete tsfile */
   public void syncDeleteDataFiles() {
     logger.info(
@@ -1798,46 +1715,6 @@ public class VirtualStorageGroupProcessor {
     }
   }
 
-  public void timedCloseTsFileProcessor() {
-    writeLock("timedCloseTsFileProcessor");
-    try {
-      List<TsFileProcessor> seqTsFileProcessors =
-          new ArrayList<>(workSequenceTsFileProcessors.values());
-      long timeLowerBound =
-          System.currentTimeMillis() - config.getCloseTsFileIntervalAfterFlushing();
-      for (TsFileProcessor tsFileProcessor : seqTsFileProcessors) {
-        // working memtable is null(no more write ops) and last flush time exceeds close interval
-        if (tsFileProcessor.getWorkMemTableCreatedTime() == Long.MAX_VALUE
-            && tsFileProcessor.getLastWorkMemtableFlushTime() < timeLowerBound) {
-          logger.info(
-              "Exceed tsfile close interval, so close TsFileProcessor of time partition {} in storage group {}[{}]",
-              tsFileProcessor.getTimeRangeId(),
-              logicalStorageGroupName,
-              virtualStorageGroupId);
-          asyncCloseOneTsFileProcessor(true, tsFileProcessor);
-        }
-      }
-
-      List<TsFileProcessor> unSeqTsFileProcessors =
-          new ArrayList<>(workUnsequenceTsFileProcessors.values());
-      timeLowerBound = System.currentTimeMillis() - config.getCloseTsFileIntervalAfterFlushing();
-      for (TsFileProcessor tsFileProcessor : unSeqTsFileProcessors) {
-        // working memtable is null(no more write ops) and last flush time exceeds close interval
-        if (tsFileProcessor.getWorkMemTableCreatedTime() == Long.MAX_VALUE
-            && tsFileProcessor.getLastWorkMemtableFlushTime() < timeLowerBound) {
-          logger.info(
-              "Exceed tsfile close interval, so close TsFileProcessor of time partition {} in storage group {}[{}]",
-              tsFileProcessor.getTimeRangeId(),
-              logicalStorageGroupName,
-              virtualStorageGroupId);
-          asyncCloseOneTsFileProcessor(false, tsFileProcessor);
-        }
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
   /** This method will be blocked until all tsfile processors are closed. */
   public void syncCloseAllWorkingTsFileProcessors() {
     synchronized (closeStorageGroupCondition) {
@@ -2124,7 +2001,15 @@ public class VirtualStorageGroupProcessor {
       }
 
       // write log to impacted working TsFileProcessors
-      logDeletion(startTime, endTime, path, timePartitionFilter);
+      List<WALFlushListener> walListeners =
+          logDeleteInWAL(startTime, endTime, path, timePartitionFilter);
+
+      for (WALFlushListener walFlushListener : walListeners) {
+        if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
+          logger.error("Fail to log delete to wal.", walFlushListener.getCause());
+          throw walFlushListener.getCause();
+        }
+      }
 
       Deletion deletion = new Deletion(path, MERGE_MOD_START_VERSION_NUM, startTime, endTime);
 
@@ -2156,31 +2041,34 @@ public class VirtualStorageGroupProcessor {
     }
   }
 
-  private void logDeletion(
-      long startTime, long endTime, PartialPath path, TimePartitionFilter timePartitionFilter)
-      throws IOException {
+  private List<WALFlushListener> logDeleteInWAL(
+      long startTime, long endTime, PartialPath path, TimePartitionFilter timePartitionFilter) {
     long timePartitionStartId = StorageEngine.getTimePartition(startTime);
     long timePartitionEndId = StorageEngine.getTimePartition(endTime);
-    if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
-      DeletePlan deletionPlan = new DeletePlan(startTime, endTime, path);
-      for (Map.Entry<Long, TsFileProcessor> entry : workSequenceTsFileProcessors.entrySet()) {
-        if (timePartitionStartId <= entry.getKey()
-            && entry.getKey() <= timePartitionEndId
-            && (timePartitionFilter == null
-                || timePartitionFilter.satisfy(logicalStorageGroupName, entry.getKey()))) {
-          entry.getValue().getLogNode().write(deletionPlan);
-        }
+    List<WALFlushListener> walFlushListeners = new ArrayList<>();
+    if (config.getWalMode() == WALMode.DISABLE) {
+      return walFlushListeners;
+    }
+    DeletePlan deletionPlan = new DeletePlan(startTime, endTime, path);
+    for (Map.Entry<Long, TsFileProcessor> entry : workSequenceTsFileProcessors.entrySet()) {
+      if (timePartitionStartId <= entry.getKey()
+          && entry.getKey() <= timePartitionEndId
+          && (timePartitionFilter == null
+              || timePartitionFilter.satisfy(logicalStorageGroupName, entry.getKey()))) {
+        WALFlushListener walFlushListener = entry.getValue().logDeleteInWAL(deletionPlan);
+        walFlushListeners.add(walFlushListener);
       }
-
-      for (Map.Entry<Long, TsFileProcessor> entry : workUnsequenceTsFileProcessors.entrySet()) {
-        if (timePartitionStartId <= entry.getKey()
-            && entry.getKey() <= timePartitionEndId
-            && (timePartitionFilter == null
-                || timePartitionFilter.satisfy(logicalStorageGroupName, entry.getKey()))) {
-          entry.getValue().getLogNode().write(deletionPlan);
-        }
+    }
+    for (Map.Entry<Long, TsFileProcessor> entry : workUnsequenceTsFileProcessors.entrySet()) {
+      if (timePartitionStartId <= entry.getKey()
+          && entry.getKey() <= timePartitionEndId
+          && (timePartitionFilter == null
+              || timePartitionFilter.satisfy(logicalStorageGroupName, entry.getKey()))) {
+        WALFlushListener walFlushListener = entry.getValue().logDeleteInWAL(deletionPlan);
+        walFlushListeners.add(walFlushListener);
       }
     }
+    return walFlushListeners;
   }
 
   private boolean canSkipDelete(
@@ -3483,10 +3371,6 @@ public class VirtualStorageGroupProcessor {
     return timedCompactionScheduleTask;
   }
 
-  public ScheduledExecutorService getWALTrimScheduleTask() {
-    return walTrimScheduleTask;
-  }
-
   public IDTable getIdTable() {
     return idTable;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/virtualSg/StorageGroupManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/virtualSg/StorageGroupManager.java
index 3f895d07d8..0f7e70520a 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/virtualSg/StorageGroupManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/virtualSg/StorageGroupManager.java
@@ -139,41 +139,37 @@ public class StorageGroupManager {
     }
   }
 
-  /** push check TsFileProcessor close interval down to all sg */
-  public void timedCloseTsFileProcessor() {
-    for (VirtualStorageGroupProcessor virtualStorageGroupProcessor :
-        this.virtualStorageGroupProcessor) {
-      if (virtualStorageGroupProcessor != null) {
-        virtualStorageGroupProcessor.timedCloseTsFileProcessor();
-      }
-    }
-  }
-
   /**
    * get processor from device id
    *
    * @param partialPath device path
    * @return virtual storage group processor
    */
-  @SuppressWarnings("java:S2445")
-  // actually storageGroupMNode is a unique object on the mtree, synchronize it is reasonable
   public VirtualStorageGroupProcessor getProcessor(
       PartialPath partialPath, IStorageGroupMNode storageGroupMNode)
       throws StorageGroupProcessorException, StorageEngineException {
-    int loc = partitioner.deviceToVirtualStorageGroupId(partialPath);
+    int vsgId = partitioner.deviceToVirtualStorageGroupId(partialPath);
+    return getProcessor(storageGroupMNode, vsgId);
+  }
 
-    VirtualStorageGroupProcessor processor = virtualStorageGroupProcessor[loc];
+  @SuppressWarnings("java:S2445")
+  // actually storageGroupMNode is a unique object on the mtree, synchronize it is reasonable
+  public VirtualStorageGroupProcessor getProcessor(IStorageGroupMNode storageGroupMNode, int vsgId)
+      throws StorageGroupProcessorException, StorageEngineException {
+    VirtualStorageGroupProcessor processor = virtualStorageGroupProcessor[vsgId];
     if (processor == null) {
       // if finish recover
-      if (isVsgReady[loc].get()) {
+      if (isVsgReady[vsgId].get()) {
         synchronized (storageGroupMNode) {
-          processor = virtualStorageGroupProcessor[loc];
+          processor = virtualStorageGroupProcessor[vsgId];
           if (processor == null) {
             processor =
                 StorageEngine.getInstance()
                     .buildNewStorageGroupProcessor(
-                        storageGroupMNode.getPartialPath(), storageGroupMNode, String.valueOf(loc));
-            virtualStorageGroupProcessor[loc] = processor;
+                        storageGroupMNode.getPartialPath(),
+                        storageGroupMNode,
+                        String.valueOf(vsgId));
+            virtualStorageGroupProcessor[vsgId] = processor;
           }
         }
       } else {
@@ -478,16 +474,6 @@ public class StorageGroupManager {
     }
   }
 
-  /** release resource of direct wal buffer */
-  public void releaseWalDirectByteBufferPool() {
-    for (VirtualStorageGroupProcessor virtualStorageGroupProcessor :
-        this.virtualStorageGroupProcessor) {
-      if (virtualStorageGroupProcessor != null) {
-        virtualStorageGroupProcessor.releaseWalDirectByteBufferPool();
-      }
-    }
-  }
-
   /** only for test */
   public void reset() {
     Arrays.fill(virtualStorageGroupProcessor, null);
@@ -498,7 +484,6 @@ public class StorageGroupManager {
       if (vsg != null) {
         ThreadUtils.stopThreadPool(
             vsg.getTimedCompactionScheduleTask(), ThreadName.COMPACTION_SCHEDULE);
-        ThreadUtils.stopThreadPool(vsg.getWALTrimScheduleTask(), ThreadName.WAL_TRIM);
       }
     }
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerLogReader.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerLogReader.java
index 4bad1f5a6f..db021344df 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerLogReader.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.engine.trigger.service;
 
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.io.SingleFileLogReader;
+import org.apache.iotdb.db.utils.writelog.SingleFileLogReader;
 
 import java.io.File;
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerLogWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerLogWriter.java
index 098bb035aa..dfb51162ab 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerLogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerLogWriter.java
@@ -22,8 +22,8 @@ package org.apache.iotdb.db.engine.trigger.service;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.io.ILogWriter;
-import org.apache.iotdb.db.writelog.io.LogWriter;
+import org.apache.iotdb.db.utils.writelog.ILogWriter;
+import org.apache.iotdb.db.utils.writelog.LogWriter;
 
 import org.apache.commons.io.FileUtils;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogReader.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogReader.java
index 9c7ab6598f..a3f2a47c61 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogReader.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.metadata.logfile;
 
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.io.SingleFileLogReader;
+import org.apache.iotdb.db.utils.writelog.SingleFileLogReader;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
index 789be2f8c2..add8722f4b 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
@@ -45,7 +45,7 @@ import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
-import org.apache.iotdb.db.writelog.io.LogWriter;
+import org.apache.iotdb.db.utils.writelog.LogWriter;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegion.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegion.java
index 4620871b47..7cce3be443 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegion.java
@@ -671,8 +671,6 @@ public class SchemaRegion {
       if (!isRecovering) {
         if (emptyStorageGroup != null) {
           StorageEngine.getInstance().deleteAllDataFilesInOneStorageGroup(emptyStorageGroup);
-          StorageEngine.getInstance()
-              .releaseWalDirectByteBufferPoolInOneStorageGroup(emptyStorageGroup);
         }
         deleteTimeSeriesPlan.setDeletePathList(Collections.singletonList(p));
         logWriter.deleteTimeseries(deleteTimeSeriesPlan);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/parser/StatementGenerator.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/parser/StatementGenerator.java
index 33d24c1974..c069b41fc4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/parser/StatementGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/parser/StatementGenerator.java
@@ -180,7 +180,7 @@ public class StatementGenerator {
     insertStatement.setTimes(
         QueryDataSetUtils.readTimesFromBuffer(insertTabletReq.timestamps, insertTabletReq.size));
     insertStatement.setColumns(
-        QueryDataSetUtils.readValuesFromBuffer(
+        QueryDataSetUtils.readTabletValuesFromBuffer(
             insertTabletReq.values,
             insertTabletReq.types,
             insertTabletReq.types.size(),
@@ -209,7 +209,7 @@ public class StatementGenerator {
       insertTabletStatement.setTimes(
           QueryDataSetUtils.readTimesFromBuffer(req.timestampsList.get(i), req.sizeList.get(i)));
       insertTabletStatement.setColumns(
-          QueryDataSetUtils.readValuesFromBuffer(
+          QueryDataSetUtils.readTabletValuesFromBuffer(
               req.valuesList.get(i),
               req.typesList.get(i),
               req.measurementsList.get(i).size(),
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
index 72fe657491..a1bcb7e5fa 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
@@ -24,6 +24,8 @@ import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
 import org.apache.iotdb.tsfile.exception.NotImplementedException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
@@ -32,7 +34,7 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 
-public class InsertRowNode extends InsertNode {
+public class InsertRowNode extends InsertNode implements WALEntryValue {
 
   private long time;
   private Object[] values;
@@ -91,9 +93,17 @@ public class InsertRowNode extends InsertNode {
     return null;
   }
 
+  @Override
+  public int serializedSize() {
+    return 0;
+  }
+
   @Override
   public void serialize(ByteBuffer byteBuffer) {}
 
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {}
+
   public Object[] getValues() {
     return values;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
index 700fdbab93..6e5d4d1e3a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
@@ -25,6 +25,8 @@ import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
 import org.apache.iotdb.tsfile.exception.NotImplementedException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
@@ -34,7 +36,7 @@ import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-public class InsertTabletNode extends InsertNode {
+public class InsertTabletNode extends InsertNode implements WALEntryValue {
 
   private long[] times; // times should be sorted. It is done in the session API.
 
@@ -142,9 +144,19 @@ public class InsertTabletNode extends InsertNode {
     return null;
   }
 
+  @Override
+  public int serializedSize() {
+    return 0;
+  }
+
   @Override
   public void serialize(ByteBuffer byteBuffer) {}
 
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {}
+
+  public void serializeToWAL(IWALByteBufferView buffer, int start, int end) {}
+
   @Override
   public List<InsertNode> splitByPartition(Analysis analysis) {
     // only single device in single storage group
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 eb6a47f4f0..52156b8dd8 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
@@ -128,6 +128,7 @@ import org.apache.iotdb.db.utils.AuthUtils;
 import org.apache.iotdb.db.utils.FileLoaderUtils;
 import org.apache.iotdb.db.utils.TypeInferenceUtils;
 import org.apache.iotdb.db.utils.UpgradeUtils;
+import org.apache.iotdb.db.wal.WALManager;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
@@ -520,6 +521,7 @@ public class PlanExecutor implements IPlanExecutor {
   private void operateFlush(FlushPlan plan) throws StorageGroupNotSetException {
     if (plan.getPaths().isEmpty()) {
       StorageEngine.getInstance().syncCloseAllProcessor();
+      WALManager.getInstance().deleteOutdatedWALFiles();
     } else {
       flushSpecifiedStorageGroups(plan);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
index c825aa35fe..cc8f14bfec 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
@@ -67,12 +67,12 @@ import org.apache.iotdb.db.qp.physical.sys.StartTriggerPlan;
 import org.apache.iotdb.db.qp.physical.sys.StopTriggerPlan;
 import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
-import org.apache.iotdb.db.qp.utils.EmptyOutputStream;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.BufferOverflowException;
@@ -90,7 +90,6 @@ public abstract class PhysicalPlan implements IConsensusRequest {
   private boolean isQuery = false;
 
   private Operator.OperatorType operatorType;
-  private static final int NULL_VALUE_LEN = -1;
 
   // for cluster mode, whether the plan may be splitted into several sub plans
   protected boolean canBeSplit = true;
@@ -160,22 +159,6 @@ public abstract class PhysicalPlan implements IConsensusRequest {
     this.debug = debug;
   }
 
-  /**
-   * Calculate size after serialization.
-   *
-   * @return size
-   * @throws IOException
-   */
-  public int getSerializedSize() throws IOException {
-    try {
-      DataOutputStream dataOutputStream = new DataOutputStream(new EmptyOutputStream());
-      serialize(dataOutputStream);
-      return dataOutputStream.size();
-    } catch (UnsupportedOperationException e) {
-      throw e;
-    }
-  }
-
   /**
    * Serialize the plan into the given buffer. All necessary fields will be serialized.
    *
@@ -191,6 +174,10 @@ public abstract class PhysicalPlan implements IConsensusRequest {
     serialize(buffer);
   }
 
+  public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
+    throw new UnsupportedOperationException(SERIALIZATION_UNIMPLEMENTED);
+  }
+
   /**
    * Serialize the plan into the given buffer. This is provided for WAL, so fields that can be
    * recovered will not be serialized. If error occurs when serializing this plan, the buffer will
@@ -221,8 +208,7 @@ public abstract class PhysicalPlan implements IConsensusRequest {
   }
 
   /**
-   * Deserialize the plan from the given buffer. This is provided for WAL, and must be used with
-   * serializeToWAL.
+   * Deserialize the plan from the given buffer.
    *
    * @param buffer
    */
@@ -231,11 +217,7 @@ public abstract class PhysicalPlan implements IConsensusRequest {
   }
 
   protected void putString(ByteBuffer buffer, String value) {
-    if (value == null) {
-      buffer.putInt(NULL_VALUE_LEN);
-    } else {
-      ReadWriteIOUtils.write(value, buffer);
-    }
+    ReadWriteIOUtils.write(value, buffer);
   }
 
   protected void putStrings(ByteBuffer buffer, List<String> values) {
@@ -245,11 +227,7 @@ public abstract class PhysicalPlan implements IConsensusRequest {
   }
 
   protected void putString(DataOutputStream stream, String value) throws IOException {
-    if (value == null) {
-      stream.writeInt(NULL_VALUE_LEN);
-    } else {
-      ReadWriteIOUtils.write(value, stream);
-    }
+    ReadWriteIOUtils.write(value, stream);
   }
 
   protected void putStrings(DataOutputStream stream, List<String> values) throws IOException {
@@ -259,11 +237,7 @@ public abstract class PhysicalPlan implements IConsensusRequest {
   }
 
   protected String readString(ByteBuffer buffer) {
-    int valueLen = buffer.getInt();
-    if (valueLen == NULL_VALUE_LEN) {
-      return null;
-    }
-    return ReadWriteIOUtils.readStringWithLength(buffer, valueLen);
+    return ReadWriteIOUtils.readString(buffer);
   }
 
   protected List<String> readStrings(ByteBuffer buffer, int totalSize) {
@@ -301,7 +275,21 @@ public abstract class PhysicalPlan implements IConsensusRequest {
 
     public static PhysicalPlan create(ByteBuffer buffer) throws IOException, IllegalPathException {
       int typeNum = buffer.get();
-      if (typeNum >= PhysicalPlanType.values().length) {
+      PhysicalPlan plan = createByTypeNum(typeNum);
+      plan.deserialize(buffer);
+      return plan;
+    }
+
+    public static PhysicalPlan create(DataInputStream stream)
+        throws IOException, IllegalPathException {
+      int typeNum = stream.readByte();
+      PhysicalPlan plan = createByTypeNum(typeNum);
+      plan.deserialize(stream);
+      return plan;
+    }
+
+    private static PhysicalPlan createByTypeNum(int typeNum) throws IOException {
+      if (typeNum < 0 || typeNum >= PhysicalPlanType.values().length) {
         throw new IOException("unrecognized log type " + typeNum);
       }
       PhysicalPlanType type = PhysicalPlanType.values()[typeNum];
@@ -488,7 +476,6 @@ public abstract class PhysicalPlan implements IConsensusRequest {
         default:
           throw new IOException("unrecognized log type " + type);
       }
-      plan.deserialize(buffer);
       return plan;
     }
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
index 0a6cae9b6b..d2c1a3eb7f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
@@ -23,7 +23,12 @@ import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
+import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -31,7 +36,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 
-public class DeletePlan extends PhysicalPlan {
+public class DeletePlan extends PhysicalPlan implements WALEntryValue {
+  /** byte: type, integer: paths.size(), long: deleteStartTime, deleteEndTime, index */
+  private static final int FIXED_SERIALIZED_SIZE = Byte.BYTES + Integer.BYTES + Long.BYTES * 3;
 
   private long deleteStartTime;
   private long deleteEndTime;
@@ -136,6 +143,15 @@ public class DeletePlan extends PhysicalPlan {
         && Objects.equals(paths, that.paths);
   }
 
+  @Override
+  public int serializedSize() {
+    int size = FIXED_SERIALIZED_SIZE;
+    for (PartialPath path : paths) {
+      size += ReadWriteIOUtils.sizeToWrite(path.getFullPath());
+    }
+    return size;
+  }
+
   @Override
   public void serialize(DataOutputStream stream) throws IOException {
     int type = PhysicalPlanType.DELETE.ordinal();
@@ -164,6 +180,32 @@ public class DeletePlan extends PhysicalPlan {
     buffer.putLong(index);
   }
 
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    int type = PhysicalPlanType.DELETE.ordinal();
+    buffer.put((byte) type);
+    buffer.putLong(deleteStartTime);
+    buffer.putLong(deleteEndTime);
+    buffer.putInt(paths.size());
+    for (PartialPath path : paths) {
+      WALWriteUtils.write(path.getFullPath(), buffer);
+    }
+    buffer.putLong(index);
+  }
+
+  @Override
+  public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
+    this.deleteStartTime = stream.readLong();
+    this.deleteEndTime = stream.readLong();
+    int pathSize = stream.readInt();
+    this.paths = new ArrayList<>();
+    for (int i = 0; i < pathSize; i++) {
+      paths.add(new PartialPath(ReadWriteIOUtils.readString(stream)));
+    }
+
+    this.index = stream.readLong();
+  }
+
   @Override
   public void deserialize(ByteBuffer buffer) throws IllegalPathException {
     this.deleteStartTime = buffer.getLong();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java
index 7d0e431338..fcf884597d 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java
@@ -271,7 +271,7 @@ public class InsertMultiTabletsPlan extends InsertPlan implements BatchPlan {
     buffer.put((byte) type);
     buffer.putInt(insertTabletPlanList.size());
     for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      insertTabletPlan.subSerialize(buffer);
+      insertTabletPlan.subSerialize(buffer, 0, insertTabletPlan.getRowCount());
     }
 
     buffer.putInt(parentInsertTabletPlanIndexList.size());
@@ -286,7 +286,7 @@ public class InsertMultiTabletsPlan extends InsertPlan implements BatchPlan {
     stream.writeByte((byte) type);
     stream.writeInt(insertTabletPlanList.size());
     for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      insertTabletPlan.subSerialize(stream);
+      insertTabletPlan.subSerialize(stream, 0, insertTabletPlan.getRowCount());
     }
 
     stream.writeInt(parentInsertTabletPlanIndexList.size());
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
index 97df0b38f7..996ebd84f2 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
@@ -30,6 +30,9 @@ import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
 import org.apache.iotdb.db.utils.CommonUtils;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 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;
@@ -42,6 +45,7 @@ import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -50,8 +54,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Objects;
 
-public class InsertRowPlan extends InsertPlan {
-
+public class InsertRowPlan extends InsertPlan implements WALEntryValue {
   private static final Logger logger = LoggerFactory.getLogger(InsertRowPlan.class);
   private static final byte TYPE_RAW_STRING = -1;
   private static final byte TYPE_NULL = -2;
@@ -328,6 +331,67 @@ public class InsertRowPlan extends InsertPlan {
     return Objects.hash(devicePath, time);
   }
 
+  @Override
+  public int serializedSize() {
+    int size = 0;
+    size += Byte.BYTES;
+    return size + subSerializeSize();
+  }
+
+  int subSerializeSize() {
+    int size = 0;
+    size += Long.BYTES;
+    size += ReadWriteIOUtils.sizeToWrite(devicePath.getFullPath());
+    return size + serializeMeasurementsAndValuesSize();
+  }
+
+  int serializeMeasurementsAndValuesSize() {
+    int size = 0;
+    size += Integer.BYTES;
+    for (String m : measurements) {
+      if (m != null) {
+        size += ReadWriteIOUtils.sizeToWrite(m);
+      }
+    }
+    // putValues
+    size += Integer.BYTES;
+    for (int i = 0; i < values.length; i++) {
+      if (values[i] == null) {
+        size += Byte.BYTES;
+        continue;
+      }
+      if (dataTypes == null || dataTypes[i] == null) {
+        size += Byte.BYTES + ReadWriteIOUtils.sizeToWrite(values[i].toString());
+      } else {
+        size += Byte.BYTES;
+        switch (dataTypes[i]) {
+          case BOOLEAN:
+            size += Byte.BYTES;
+            break;
+          case INT32:
+            size += Integer.BYTES;
+            break;
+          case INT64:
+            size += Long.BYTES;
+            break;
+          case FLOAT:
+            size += Float.BYTES;
+            break;
+          case DOUBLE:
+            size += Double.BYTES;
+            break;
+          case TEXT:
+            size += ReadWriteIOUtils.sizeToWrite((Binary) values[i]);
+            break;
+        }
+      }
+    }
+    size += Byte.BYTES;
+    size += Long.BYTES;
+    size += Byte.BYTES;
+    return size;
+  }
+
   @Override
   public void serialize(DataOutputStream stream) throws IOException {
     int type = PhysicalPlanType.INSERT.ordinal();
@@ -404,41 +468,37 @@ public class InsertRowPlan extends InsertPlan {
     }
   }
 
-  private void putValues(ByteBuffer buffer) throws QueryProcessException {
-    for (int i = 0; i < values.length; i++) {
-      if (values[i] == null) {
-        ReadWriteIOUtils.write(TYPE_NULL, buffer);
-        continue;
-      }
+  public void fillValues(DataInputStream stream) throws IOException, QueryProcessException {
+    for (int i = 0; i < dataTypes.length; i++) {
       // types are not determined, the situation mainly occurs when the plan uses string values
       // and is forwarded to other nodes
-      if (dataTypes == null || dataTypes[i] == null) {
-        ReadWriteIOUtils.write(TYPE_RAW_STRING, buffer);
-        ReadWriteIOUtils.write(values[i].toString(), buffer);
-      } else {
-        ReadWriteIOUtils.write(dataTypes[i], buffer);
-        switch (dataTypes[i]) {
-          case BOOLEAN:
-            ReadWriteIOUtils.write((Boolean) values[i], buffer);
-            break;
-          case INT32:
-            ReadWriteIOUtils.write((Integer) values[i], buffer);
-            break;
-          case INT64:
-            ReadWriteIOUtils.write((Long) values[i], buffer);
-            break;
-          case FLOAT:
-            ReadWriteIOUtils.write((Float) values[i], buffer);
-            break;
-          case DOUBLE:
-            ReadWriteIOUtils.write((Double) values[i], buffer);
-            break;
-          case TEXT:
-            ReadWriteIOUtils.write((Binary) values[i], buffer);
-            break;
-          default:
-            throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
-        }
+      byte typeNum = stream.readByte();
+      if (typeNum == TYPE_RAW_STRING || typeNum == TYPE_NULL) {
+        values[i] = typeNum == TYPE_RAW_STRING ? ReadWriteIOUtils.readString(stream) : null;
+        continue;
+      }
+      dataTypes[i] = TSDataType.values()[typeNum];
+      switch (dataTypes[i]) {
+        case BOOLEAN:
+          values[i] = ReadWriteIOUtils.readBool(stream);
+          break;
+        case INT32:
+          values[i] = ReadWriteIOUtils.readInt(stream);
+          break;
+        case INT64:
+          values[i] = ReadWriteIOUtils.readLong(stream);
+          break;
+        case FLOAT:
+          values[i] = ReadWriteIOUtils.readFloat(stream);
+          break;
+        case DOUBLE:
+          values[i] = ReadWriteIOUtils.readDouble(stream);
+          break;
+        case TEXT:
+          values[i] = ReadWriteIOUtils.readBinary(stream);
+          break;
+        default:
+          throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
       }
     }
   }
@@ -486,7 +546,7 @@ public class InsertRowPlan extends InsertPlan {
     subSerialize(buffer);
   }
 
-  public void subSerialize(ByteBuffer buffer) {
+  void subSerialize(ByteBuffer buffer) {
     buffer.putLong(time);
     putString(buffer, devicePath.getFullPath());
     serializeMeasurementsAndValues(buffer);
@@ -515,6 +575,149 @@ public class InsertRowPlan extends InsertPlan {
     buffer.put((byte) (isAligned ? 1 : 0));
   }
 
+  private void putValues(ByteBuffer buffer) throws QueryProcessException {
+    for (int i = 0; i < values.length; i++) {
+      if (values[i] == null) {
+        ReadWriteIOUtils.write(TYPE_NULL, buffer);
+        continue;
+      }
+      // types are not determined, the situation mainly occurs when the plan uses string values
+      // and is forwarded to other nodes
+      if (dataTypes == null || dataTypes[i] == null) {
+        ReadWriteIOUtils.write(TYPE_RAW_STRING, buffer);
+        ReadWriteIOUtils.write(values[i].toString(), buffer);
+      } else {
+        ReadWriteIOUtils.write(dataTypes[i], buffer);
+        switch (dataTypes[i]) {
+          case BOOLEAN:
+            ReadWriteIOUtils.write((Boolean) values[i], buffer);
+            break;
+          case INT32:
+            ReadWriteIOUtils.write((Integer) values[i], buffer);
+            break;
+          case INT64:
+            ReadWriteIOUtils.write((Long) values[i], buffer);
+            break;
+          case FLOAT:
+            ReadWriteIOUtils.write((Float) values[i], buffer);
+            break;
+          case DOUBLE:
+            ReadWriteIOUtils.write((Double) values[i], buffer);
+            break;
+          case TEXT:
+            ReadWriteIOUtils.write((Binary) values[i], buffer);
+            break;
+          default:
+            throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    int type = PhysicalPlanType.INSERT.ordinal();
+    buffer.put((byte) type);
+    subSerialize(buffer);
+  }
+
+  void subSerialize(IWALByteBufferView buffer) {
+    buffer.putLong(time);
+    WALWriteUtils.write(devicePath.getFullPath(), buffer);
+    serializeMeasurementsAndValues(buffer);
+  }
+
+  void serializeMeasurementsAndValues(IWALByteBufferView buffer) {
+    buffer.putInt(
+        measurements.length - (failedMeasurements == null ? 0 : failedMeasurements.size()));
+
+    for (String measurement : measurements) {
+      if (measurement != null) {
+        WALWriteUtils.write(measurement, buffer);
+      }
+    }
+    try {
+      buffer.putInt(dataTypes.length);
+      putValues(buffer);
+    } catch (QueryProcessException e) {
+      logger.error("Failed to serialize values for {}", this, e);
+    }
+
+    // the types are not inferred before the plan is serialized
+    buffer.put((byte) (isNeedInferType ? 1 : 0));
+    buffer.putLong(index);
+
+    buffer.put((byte) (isAligned ? 1 : 0));
+  }
+
+  private void putValues(IWALByteBufferView buffer) throws QueryProcessException {
+    for (int i = 0; i < values.length; i++) {
+      if (values[i] == null) {
+        WALWriteUtils.write(TYPE_NULL, buffer);
+        continue;
+      }
+      // types are not determined, the situation mainly occurs when the plan uses string values
+      // and is forwarded to other nodes
+      if (dataTypes == null || dataTypes[i] == null) {
+        WALWriteUtils.write(TYPE_RAW_STRING, buffer);
+        WALWriteUtils.write(values[i].toString(), buffer);
+      } else {
+        WALWriteUtils.write(dataTypes[i], buffer);
+        switch (dataTypes[i]) {
+          case BOOLEAN:
+            WALWriteUtils.write((Boolean) values[i], buffer);
+            break;
+          case INT32:
+            WALWriteUtils.write((Integer) values[i], buffer);
+            break;
+          case INT64:
+            WALWriteUtils.write((Long) values[i], buffer);
+            break;
+          case FLOAT:
+            WALWriteUtils.write((Float) values[i], buffer);
+            break;
+          case DOUBLE:
+            WALWriteUtils.write((Double) values[i], buffer);
+            break;
+          case TEXT:
+            WALWriteUtils.write((Binary) values[i], buffer);
+            break;
+          default:
+            throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
+    this.time = stream.readLong();
+    this.devicePath = new PartialPath(ReadWriteIOUtils.readString(stream));
+    deserializeMeasurementsAndValues(stream);
+  }
+
+  void deserializeMeasurementsAndValues(DataInputStream stream) throws IOException {
+    int measurementSize = stream.readInt();
+
+    this.measurements = new String[measurementSize];
+    for (int i = 0; i < measurementSize; i++) {
+      measurements[i] = ReadWriteIOUtils.readString(stream);
+    }
+
+    int dataTypeSize = stream.readInt();
+    this.dataTypes = new TSDataType[dataTypeSize];
+    this.values = new Object[dataTypeSize];
+    try {
+      fillValues(stream);
+    } catch (QueryProcessException e) {
+      e.printStackTrace();
+    }
+
+    isNeedInferType = stream.readByte() == 1;
+    this.index = stream.readLong();
+    isAligned = stream.readByte() == 1;
+  }
+
   @Override
   public void deserialize(ByteBuffer buffer) throws IllegalPathException {
     this.time = buffer.getLong();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
index d761506195..8c6688a37f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
@@ -23,12 +23,16 @@ import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
 import org.apache.iotdb.db.utils.QueryDataSetUtils;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBinary;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBoolean;
@@ -37,6 +41,7 @@ import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsInt;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsLong;
 
+import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -46,23 +51,15 @@ import java.util.List;
 import java.util.Objects;
 
 @SuppressWarnings("java:S1135") // ignore todos
-public class InsertTabletPlan extends InsertPlan {
-
+public class InsertTabletPlan extends InsertPlan implements WALEntryValue {
   private static final String DATATYPE_UNSUPPORTED = "Data type %s is not supported.";
 
   private long[] times; // times should be sorted. It is done in the session API.
-  private ByteBuffer timeBuffer;
 
   private BitMap[] bitMaps;
   private Object[] columns;
-  private ByteBuffer valueBuffer;
   private int rowCount = 0;
-  // indicate whether this plan has been set 'start' or 'end' in order to support plan transmission
-  // without data loss in cluster version
-  boolean isExecuting = false;
   private List<PartialPath> paths;
-  private int start;
-  private int end;
   // when this plan is sub-plan split from another InsertTabletPlan, this indicates the original
   // positions of values in
   // this plan. For example, if the plan contains 5 timestamps, and range = [1,4,10,12], then it
@@ -112,24 +109,6 @@ public class InsertTabletPlan extends InsertPlan {
     this.isAligned = isAligned;
   }
 
-  public int getStart() {
-    return start;
-  }
-
-  public void setStart(int start) {
-    this.isExecuting = true;
-    this.start = start;
-  }
-
-  public int getEnd() {
-    return end;
-  }
-
-  public void setEnd(int end) {
-    this.isExecuting = true;
-    this.end = end;
-  }
-
   public List<Integer> getRange() {
     return range;
   }
@@ -152,20 +131,103 @@ public class InsertTabletPlan extends InsertPlan {
     return ret;
   }
 
+  @Override
+  public int serializedSize() {
+    int size = Byte.BYTES;
+    return size + subSerializeSize(0, rowCount);
+  }
+
+  public int serializedSize(int start, int end) {
+    int size = Byte.BYTES;
+    return size + subSerializeSize(start, end);
+  }
+
+  int subSerializeSize(int start, int end) {
+    int size = 0;
+    size += ReadWriteIOUtils.sizeToWrite(devicePath.getFullPath());
+    // measurements size
+    size += Integer.BYTES;
+    for (String m : measurements) {
+      if (m != null) {
+        size += ReadWriteIOUtils.sizeToWrite(m);
+      }
+    }
+    // data types size
+    size += Integer.BYTES;
+    for (int i = 0; i < dataTypes.length; i++) {
+      if (columns[i] != null) {
+        size += Byte.BYTES;
+      }
+    }
+    // times size
+    size += Integer.BYTES;
+    size += Long.BYTES * (end - start);
+    // bitmaps size
+    size += Byte.BYTES;
+    if (bitMaps != null) {
+      for (BitMap bitMap : bitMaps) {
+        size += Byte.BYTES;
+        if (bitMap != null) {
+          int len = end - start;
+          BitMap partBitMap = new BitMap(len);
+          BitMap.copyOfRange(bitMap, start, partBitMap, 0, len);
+          size += partBitMap.getByteArray().length;
+        }
+      }
+    }
+    // values size
+    for (int i = 0; i < dataTypes.length; i++) {
+      if (columns[i] != null) {
+        size += getColumnSize(dataTypes[i], columns[i], start, end);
+      }
+    }
+    size += Long.BYTES;
+    size += Byte.BYTES;
+    return size;
+  }
+
+  private int getColumnSize(TSDataType dataType, Object column, int start, int end) {
+    int size = 0;
+    switch (dataType) {
+      case INT32:
+        size += Integer.BYTES * (end - start);
+        break;
+      case INT64:
+        size += Long.BYTES * (end - start);
+        break;
+      case FLOAT:
+        size += Float.BYTES * (end - start);
+        break;
+      case DOUBLE:
+        size += Double.BYTES * (end - start);
+        break;
+      case BOOLEAN:
+        size += Byte.BYTES * (end - start);
+        break;
+      case TEXT:
+        Binary[] binaryValues = (Binary[]) column;
+        for (int j = start; j < end; j++) {
+          size += ReadWriteIOUtils.sizeToWrite(binaryValues[j]);
+        }
+        break;
+    }
+    return size;
+  }
+
   @Override
   public void serialize(DataOutputStream stream) throws IOException {
     int type = PhysicalPlanType.BATCHINSERT.ordinal();
     stream.writeByte((byte) type);
-    subSerialize(stream);
+    subSerialize(stream, 0, rowCount);
   }
 
-  public void subSerialize(DataOutputStream stream) throws IOException {
+  void subSerialize(DataOutputStream stream, int start, int end) throws IOException {
     putString(stream, devicePath.getFullPath());
     writeMeasurements(stream);
     writeDataTypes(stream);
-    writeTimes(stream);
-    writeBitMaps(stream);
-    writeValues(stream);
+    writeTimes(stream, start, end);
+    writeBitMaps(stream, start, end);
+    writeValues(stream, start, end);
     stream.write((byte) (isAligned ? 1 : 0));
   }
 
@@ -191,30 +253,14 @@ public class InsertTabletPlan extends InsertPlan {
     }
   }
 
-  private void writeTimes(DataOutputStream stream) throws IOException {
-    if (isExecuting) {
-      stream.writeInt(end - start);
-    } else {
-      stream.writeInt(rowCount);
-    }
-
-    if (timeBuffer == null) {
-      if (isExecuting) {
-        for (int i = start; i < end; i++) {
-          stream.writeLong(times[i]);
-        }
-      } else {
-        for (long time : times) {
-          stream.writeLong(time);
-        }
-      }
-    } else {
-      stream.write(timeBuffer.array());
-      timeBuffer = null;
+  private void writeTimes(DataOutputStream stream, int start, int end) throws IOException {
+    stream.writeInt(end - start);
+    for (int i = start; i < end; i++) {
+      stream.writeLong(times[i]);
     }
   }
 
-  private void writeBitMaps(DataOutputStream stream) throws IOException {
+  private void writeBitMaps(DataOutputStream stream, int start, int end) throws IOException {
     stream.writeBoolean(bitMaps != null);
     if (bitMaps != null) {
       for (BitMap bitMap : bitMaps) {
@@ -222,44 +268,90 @@ public class InsertTabletPlan extends InsertPlan {
           stream.writeBoolean(false);
         } else {
           stream.writeBoolean(true);
-          if (isExecuting) {
-            int len = end - start;
-            BitMap partBitMap = new BitMap(len);
-            BitMap.copyOfRange(bitMap, start, partBitMap, 0, len);
-            stream.write(partBitMap.getByteArray());
-          } else {
-            stream.write(bitMap.getByteArray());
-          }
+          int len = end - start;
+          BitMap partBitMap = new BitMap(len);
+          BitMap.copyOfRange(bitMap, start, partBitMap, 0, len);
+          stream.write(partBitMap.getByteArray());
         }
       }
     }
   }
 
-  private void writeValues(DataOutputStream stream) throws IOException {
-    if (valueBuffer == null) {
-      serializeValues(stream);
-    } else {
-      stream.write(valueBuffer.array());
-      valueBuffer = null;
+  private void writeValues(DataOutputStream stream, int start, int end) throws IOException {
+    serializeValues(stream, start, end);
+    stream.writeLong(index);
+  }
+
+  private void serializeValues(DataOutputStream outputStream, int start, int end)
+      throws IOException {
+    for (int i = 0; i < dataTypes.length; i++) {
+      if (columns[i] == null) {
+        continue;
+      }
+      serializeColumn(dataTypes[i], columns[i], outputStream, start, end);
     }
+  }
 
-    stream.writeLong(index);
+  private void serializeColumn(
+      TSDataType dataType, Object column, DataOutputStream outputStream, int start, int end)
+      throws IOException {
+    switch (dataType) {
+      case INT32:
+        int[] intValues = (int[]) column;
+        for (int j = start; j < end; j++) {
+          outputStream.writeInt(intValues[j]);
+        }
+        break;
+      case INT64:
+        long[] longValues = (long[]) column;
+        for (int j = start; j < end; j++) {
+          outputStream.writeLong(longValues[j]);
+        }
+        break;
+      case FLOAT:
+        float[] floatValues = (float[]) column;
+        for (int j = start; j < end; j++) {
+          outputStream.writeFloat(floatValues[j]);
+        }
+        break;
+      case DOUBLE:
+        double[] doubleValues = (double[]) column;
+        for (int j = start; j < end; j++) {
+          outputStream.writeDouble(doubleValues[j]);
+        }
+        break;
+      case BOOLEAN:
+        boolean[] boolValues = (boolean[]) column;
+        for (int j = start; j < end; j++) {
+          outputStream.writeByte(BytesUtils.boolToByte(boolValues[j]));
+        }
+        break;
+      case TEXT:
+        Binary[] binaryValues = (Binary[]) column;
+        for (int j = start; j < end; j++) {
+          outputStream.writeInt(binaryValues[j].getLength());
+          outputStream.write(binaryValues[j].getValues());
+        }
+        break;
+      default:
+        throw new UnSupportedDataTypeException(String.format(DATATYPE_UNSUPPORTED, dataType));
+    }
   }
 
   @Override
   public void serializeImpl(ByteBuffer buffer) {
     int type = PhysicalPlanType.BATCHINSERT.ordinal();
     buffer.put((byte) type);
-    subSerialize(buffer);
+    subSerialize(buffer, 0, rowCount);
   }
 
-  public void subSerialize(ByteBuffer buffer) {
+  void subSerialize(ByteBuffer buffer, int start, int end) {
     putString(buffer, devicePath.getFullPath());
     writeMeasurements(buffer);
     writeDataTypes(buffer);
-    writeTimes(buffer);
-    writeBitMaps(buffer);
-    writeValues(buffer);
+    writeTimes(buffer, 0, rowCount);
+    writeBitMaps(buffer, 0, rowCount);
+    writeValues(buffer, 0, rowCount);
     buffer.put((byte) (isAligned ? 1 : 0));
   }
 
@@ -284,30 +376,14 @@ public class InsertTabletPlan extends InsertPlan {
     }
   }
 
-  private void writeTimes(ByteBuffer buffer) {
-    if (isExecuting) {
-      buffer.putInt(end - start);
-    } else {
-      buffer.putInt(rowCount);
-    }
-
-    if (timeBuffer == null) {
-      if (isExecuting) {
-        for (int i = start; i < end; i++) {
-          buffer.putLong(times[i]);
-        }
-      } else {
-        for (long time : times) {
-          buffer.putLong(time);
-        }
-      }
-    } else {
-      buffer.put(timeBuffer.array());
-      timeBuffer = null;
+  private void writeTimes(ByteBuffer buffer, int start, int end) {
+    buffer.putInt(rowCount);
+    for (int i = start; i < end; i++) {
+      buffer.putLong(times[i]);
     }
   }
 
-  private void writeBitMaps(ByteBuffer buffer) {
+  private void writeBitMaps(ByteBuffer buffer, int start, int end) {
     buffer.put(BytesUtils.boolToByte(bitMaps != null));
     if (bitMaps != null) {
       for (BitMap bitMap : bitMaps) {
@@ -315,40 +391,21 @@ public class InsertTabletPlan extends InsertPlan {
           buffer.put(BytesUtils.boolToByte(false));
         } else {
           buffer.put(BytesUtils.boolToByte(true));
-          if (isExecuting) {
-            int len = end - start;
-            BitMap partBitMap = new BitMap(len);
-            BitMap.copyOfRange(bitMap, start, partBitMap, 0, len);
-            buffer.put(partBitMap.getByteArray());
-          } else {
-            buffer.put(bitMap.getByteArray());
-          }
+          int len = end - start;
+          BitMap partBitMap = new BitMap(len);
+          BitMap.copyOfRange(bitMap, start, partBitMap, 0, len);
+          buffer.put(partBitMap.getByteArray());
         }
       }
     }
   }
 
-  private void writeValues(ByteBuffer buffer) {
-    if (valueBuffer == null) {
-      serializeValues(buffer);
-    } else {
-      buffer.put(valueBuffer.array());
-      valueBuffer = null;
-    }
-
+  private void writeValues(ByteBuffer buffer, int start, int end) {
+    serializeValues(buffer, start, end);
     buffer.putLong(index);
   }
 
-  private void serializeValues(DataOutputStream outputStream) throws IOException {
-    for (int i = 0; i < dataTypes.length; i++) {
-      if (columns[i] == null) {
-        continue;
-      }
-      serializeColumn(dataTypes[i], columns[i], outputStream, start, end);
-    }
-  }
-
-  private void serializeValues(ByteBuffer buffer) {
+  private void serializeValues(ByteBuffer buffer, int start, int end) {
     for (int i = 0; i < dataTypes.length; i++) {
       if (columns[i] == null) {
         continue;
@@ -359,42 +416,40 @@ public class InsertTabletPlan extends InsertPlan {
 
   private void serializeColumn(
       TSDataType dataType, Object column, ByteBuffer buffer, int start, int end) {
-    int curStart = isExecuting ? start : 0;
-    int curEnd = isExecuting ? end : rowCount;
     switch (dataType) {
       case INT32:
         int[] intValues = (int[]) column;
-        for (int j = curStart; j < curEnd; j++) {
+        for (int j = start; j < end; j++) {
           buffer.putInt(intValues[j]);
         }
         break;
       case INT64:
         long[] longValues = (long[]) column;
-        for (int j = curStart; j < curEnd; j++) {
+        for (int j = start; j < end; j++) {
           buffer.putLong(longValues[j]);
         }
         break;
       case FLOAT:
         float[] floatValues = (float[]) column;
-        for (int j = curStart; j < curEnd; j++) {
+        for (int j = start; j < end; j++) {
           buffer.putFloat(floatValues[j]);
         }
         break;
       case DOUBLE:
         double[] doubleValues = (double[]) column;
-        for (int j = curStart; j < curEnd; j++) {
+        for (int j = start; j < end; j++) {
           buffer.putDouble(doubleValues[j]);
         }
         break;
       case BOOLEAN:
         boolean[] boolValues = (boolean[]) column;
-        for (int j = curStart; j < curEnd; j++) {
+        for (int j = start; j < end; j++) {
           buffer.put(BytesUtils.boolToByte(boolValues[j]));
         }
         break;
       case TEXT:
         Binary[] binaryValues = (Binary[]) column;
-        for (int j = curStart; j < curEnd; j++) {
+        for (int j = start; j < end; j++) {
           buffer.putInt(binaryValues[j].getLength());
           buffer.put(binaryValues[j].getValues());
         }
@@ -404,47 +459,124 @@ public class InsertTabletPlan extends InsertPlan {
     }
   }
 
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    serializeToWAL(buffer, 0, rowCount);
+  }
+
+  public void serializeToWAL(IWALByteBufferView buffer, int start, int end) {
+    int type = PhysicalPlanType.BATCHINSERT.ordinal();
+    buffer.put((byte) type);
+    subSerialize(buffer, start, end);
+  }
+
+  void subSerialize(IWALByteBufferView buffer, int start, int end) {
+    WALWriteUtils.write(devicePath.getFullPath(), buffer);
+    writeMeasurements(buffer);
+    writeDataTypes(buffer);
+    writeTimes(buffer, start, end);
+    writeBitMaps(buffer, start, end);
+    writeValues(buffer, start, end);
+    buffer.put((byte) (isAligned ? 1 : 0));
+  }
+
+  private void writeMeasurements(IWALByteBufferView buffer) {
+    buffer.putInt(
+        measurements.length - (failedMeasurements == null ? 0 : failedMeasurements.size()));
+    for (String m : measurements) {
+      if (m != null) {
+        WALWriteUtils.write(m, buffer);
+      }
+    }
+  }
+
+  private void writeDataTypes(IWALByteBufferView buffer) {
+    buffer.putInt(dataTypes.length - (failedMeasurements == null ? 0 : failedMeasurements.size()));
+    for (int i = 0, dataTypesLength = dataTypes.length; i < dataTypesLength; i++) {
+      TSDataType dataType = dataTypes[i];
+      if (columns[i] == null) {
+        continue;
+      }
+      WALWriteUtils.write(dataType, buffer);
+    }
+  }
+
+  private void writeTimes(IWALByteBufferView buffer, int start, int end) {
+    buffer.putInt(end - start);
+    for (int i = start; i < end; i++) {
+      buffer.putLong(times[i]);
+    }
+  }
+
+  private void writeBitMaps(IWALByteBufferView buffer, int start, int end) {
+    buffer.put(BytesUtils.boolToByte(bitMaps != null));
+    if (bitMaps != null) {
+      for (BitMap bitMap : bitMaps) {
+        if (bitMap == null) {
+          buffer.put(BytesUtils.boolToByte(false));
+        } else {
+          buffer.put(BytesUtils.boolToByte(true));
+          int len = end - start;
+          BitMap partBitMap = new BitMap(len);
+          BitMap.copyOfRange(bitMap, start, partBitMap, 0, len);
+          buffer.put(partBitMap.getByteArray());
+        }
+      }
+    }
+  }
+
+  private void writeValues(IWALByteBufferView buffer, int start, int end) {
+    serializeValues(buffer, start, end);
+    buffer.putLong(index);
+  }
+
+  private void serializeValues(IWALByteBufferView buffer, int start, int end) {
+    for (int i = 0; i < dataTypes.length; i++) {
+      if (columns[i] == null) {
+        continue;
+      }
+      serializeColumn(dataTypes[i], columns[i], buffer, start, end);
+    }
+  }
+
   private void serializeColumn(
-      TSDataType dataType, Object column, DataOutputStream outputStream, int start, int end)
-      throws IOException {
-    int curStart = isExecuting ? start : 0;
-    int curEnd = isExecuting ? end : rowCount;
+      TSDataType dataType, Object column, IWALByteBufferView buffer, int start, int end) {
     switch (dataType) {
       case INT32:
         int[] intValues = (int[]) column;
-        for (int j = curStart; j < curEnd; j++) {
-          outputStream.writeInt(intValues[j]);
+        for (int j = start; j < end; j++) {
+          buffer.putInt(intValues[j]);
         }
         break;
       case INT64:
         long[] longValues = (long[]) column;
-        for (int j = curStart; j < curEnd; j++) {
-          outputStream.writeLong(longValues[j]);
+        for (int j = start; j < end; j++) {
+          buffer.putLong(longValues[j]);
         }
         break;
       case FLOAT:
         float[] floatValues = (float[]) column;
-        for (int j = curStart; j < curEnd; j++) {
-          outputStream.writeFloat(floatValues[j]);
+        for (int j = start; j < end; j++) {
+          buffer.putFloat(floatValues[j]);
         }
         break;
       case DOUBLE:
         double[] doubleValues = (double[]) column;
-        for (int j = curStart; j < curEnd; j++) {
-          outputStream.writeDouble(doubleValues[j]);
+        for (int j = start; j < end; j++) {
+          buffer.putDouble(doubleValues[j]);
         }
         break;
       case BOOLEAN:
         boolean[] boolValues = (boolean[]) column;
-        for (int j = curStart; j < curEnd; j++) {
-          outputStream.writeByte(BytesUtils.boolToByte(boolValues[j]));
+        for (int j = start; j < end; j++) {
+          buffer.put(BytesUtils.boolToByte(boolValues[j]));
         }
         break;
       case TEXT:
         Binary[] binaryValues = (Binary[]) column;
-        for (int j = curStart; j < curEnd; j++) {
-          outputStream.writeInt(binaryValues[j].getLength());
-          outputStream.write(binaryValues[j].getValues());
+        for (int j = start; j < end; j++) {
+          buffer.putInt(binaryValues[j].getLength());
+          buffer.put(binaryValues[j].getValues());
         }
         break;
       default:
@@ -452,14 +584,34 @@ public class InsertTabletPlan extends InsertPlan {
     }
   }
 
-  public void setTimeBuffer(ByteBuffer timeBuffer) {
-    this.timeBuffer = timeBuffer;
-    this.timeBuffer.position(0);
-  }
+  @Override
+  public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
+    this.devicePath = new PartialPath(ReadWriteIOUtils.readString(stream));
 
-  public void setValueBuffer(ByteBuffer valueBuffer) {
-    this.valueBuffer = valueBuffer;
-    this.timeBuffer.position(0);
+    int measurementSize = stream.readInt();
+    this.measurements = new String[measurementSize];
+    for (int i = 0; i < measurementSize; i++) {
+      measurements[i] = ReadWriteIOUtils.readString(stream);
+    }
+
+    int dataTypeSize = stream.readInt();
+    this.dataTypes = new TSDataType[dataTypeSize];
+    for (int i = 0; i < dataTypeSize; i++) {
+      dataTypes[i] = TSDataType.deserialize(stream.readByte());
+    }
+
+    int rows = stream.readInt();
+    rowCount = rows;
+    this.times = new long[rows];
+    times = QueryDataSetUtils.readTimesFromStream(stream, rows);
+
+    boolean hasBitMaps = BytesUtils.byteToBool(stream.readByte());
+    if (hasBitMaps) {
+      bitMaps = QueryDataSetUtils.readBitMapsFromStream(stream, dataTypeSize, rows);
+    }
+    columns = QueryDataSetUtils.readTabletValuesFromStream(stream, dataTypes, dataTypeSize, rows);
+    this.index = stream.readLong();
+    this.isAligned = stream.readByte() == 1;
   }
 
   @Override
@@ -487,7 +639,7 @@ public class InsertTabletPlan extends InsertPlan {
     if (hasBitMaps) {
       bitMaps = QueryDataSetUtils.readBitMapsFromBuffer(buffer, dataTypeSize, rows);
     }
-    columns = QueryDataSetUtils.readValuesFromBuffer(buffer, dataTypes, dataTypeSize, rows);
+    columns = QueryDataSetUtils.readTabletValuesFromBuffer(buffer, dataTypes, dataTypeSize, rows);
     this.index = buffer.getLong();
     this.isAligned = buffer.get() == 1;
   }
@@ -649,8 +801,6 @@ public class InsertTabletPlan extends InsertPlan {
     return rowCount == that.rowCount
         && Objects.equals(devicePath, that.devicePath)
         && Arrays.equals(times, that.times)
-        && Objects.equals(timeBuffer, that.timeBuffer)
-        && Objects.equals(valueBuffer, that.valueBuffer)
         && Objects.equals(paths, that.paths)
         && Objects.equals(range, that.range)
         && Objects.equals(isAligned, that.isAligned);
@@ -658,7 +808,7 @@ public class InsertTabletPlan extends InsertPlan {
 
   @Override
   public int hashCode() {
-    int result = Objects.hash(timeBuffer, valueBuffer, rowCount, paths, range);
+    int result = Objects.hash(rowCount, paths, range);
     result = 31 * result + Arrays.hashCode(times);
     return result;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/MemTableManager.java b/server/src/main/java/org/apache/iotdb/db/rescon/MemTableManager.java
index 461b64b262..03b32e71fc 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/MemTableManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/MemTableManager.java
@@ -43,13 +43,13 @@ public class MemTableManager {
     return InstanceHolder.INSTANCE;
   }
 
-  /**
-   * Called when memory control is disabled
-   *
-   * @throws WriteProcessException
-   */
   public synchronized IMemTable getAvailableMemTable(String storageGroup)
       throws WriteProcessException {
+    if (CONFIG.isEnableMemControl()) {
+      currentMemtableNumber++;
+      return new PrimitiveMemTable(CONFIG.isEnableMemControl());
+    }
+
     if (!reachMaxMemtableNumber()) {
       currentMemtableNumber++;
       return new PrimitiveMemTable();
@@ -79,10 +79,6 @@ public class MemTableManager {
     return currentMemtableNumber;
   }
 
-  public synchronized void addMemtableNumber() {
-    currentMemtableNumber++;
-  }
-
   public synchronized void decreaseMemtableNumber() {
     currentMemtableNumber--;
     notifyAll();
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 d32049189c..c5ac7f86a1 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
@@ -49,7 +49,7 @@ import org.apache.iotdb.db.service.basic.ServiceProvider;
 import org.apache.iotdb.db.service.basic.StandaloneServiceProvider;
 import org.apache.iotdb.db.service.metrics.MetricsService;
 import org.apache.iotdb.db.sync.receiver.SyncServerManager;
-import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
+import org.apache.iotdb.db.wal.WALManager;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -131,10 +131,10 @@ public class IoTDB implements IoTDBMBean {
     initConfigManager();
     registerManager.register(JMXService.getInstance());
     registerManager.register(FlushManager.getInstance());
-    registerManager.register(MultiFileLogNodeManager.getInstance());
     registerManager.register(CacheHitRatioMonitor.getInstance());
     registerManager.register(CompactionTaskManager.getInstance());
     JMXService.registerMBean(getInstance(), mbeanName);
+    registerManager.register(WALManager.getInstance());
     registerManager.register(StorageEngine.getInstance());
     registerManager.register(TemporaryQueryDataFileService.getInstance());
     registerManager.register(UDFClassLoaderManager.getInstance());
diff --git a/server/src/main/java/org/apache/iotdb/db/service/metrics/MetricsService.java b/server/src/main/java/org/apache/iotdb/db/service/metrics/MetricsService.java
index 9127c34da9..46f5491cd3 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/metrics/MetricsService.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/metrics/MetricsService.java
@@ -24,8 +24,8 @@ import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.JMXService;
 import org.apache.iotdb.commons.service.ServiceType;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.utils.FileUtils;
+import org.apache.iotdb.db.wal.node.WALNode;
 import org.apache.iotdb.metrics.MetricService;
 import org.apache.iotdb.metrics.config.MetricConfig;
 import org.apache.iotdb.metrics.config.MetricConfigDescriptor;
@@ -81,12 +81,12 @@ public class MetricsService extends MetricService implements MetricsServiceMBean
   @Override
   public void collectFileSystemInfo() {
     logger.info("start collecting fileSize and fileCount of wal/seq/unseq");
-    String walDir = DirectoryManager.getInstance().getWALFolder();
+    String[] walDirs = IoTDBDescriptor.getInstance().getConfig().getWalDirs();
     metricManager.getOrCreateAutoGauge(
         Metric.FILE_SIZE.toString(),
         MetricLevel.IMPORTANT,
-        walDir,
-        FileUtils::getDirSize,
+        walDirs,
+        value -> Stream.of(value).mapToLong(dir -> FileUtils.getDirSize(dir)).sum(),
         Tag.NAME.toString(),
         "wal");
 
@@ -122,14 +122,23 @@ public class MetricsService extends MetricService implements MetricsServiceMBean
     metricManager.getOrCreateAutoGauge(
         Metric.FILE_COUNT.toString(),
         MetricLevel.IMPORTANT,
-        walDir,
-        value -> {
-          File walFolder = new File(value);
-          if (walFolder.exists() && walFolder.isDirectory()) {
-            return org.apache.commons.io.FileUtils.listFiles(new File(value), null, true).size();
-          }
-          return 0L;
-        },
+        walDirs,
+        value ->
+            Stream.of(value)
+                .mapToLong(
+                    dir -> {
+                      File walFolder = new File(dir);
+                      File[] walNodeFolders = walFolder.listFiles(WALNode::walNodeFolderNameFilter);
+                      for (File walNodeFolder : walNodeFolders) {
+                        if (walNodeFolder.exists() && walNodeFolder.isDirectory()) {
+                          return org.apache.commons.io.FileUtils.listFiles(
+                                  walNodeFolder, null, true)
+                              .size();
+                        }
+                      }
+                      return 0L;
+                    })
+                .sum(),
         Tag.NAME.toString(),
         "wal");
     metricManager.getOrCreateAutoGauge(
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
index fdd6b1af87..e460a61aee 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
@@ -1799,7 +1799,7 @@ public class TSServiceImpl implements TSIService.Iface {
           new InsertTabletPlan(new PartialPath(req.getPrefixPath()), req.measurements);
       insertTabletPlan.setTimes(QueryDataSetUtils.readTimesFromBuffer(req.timestamps, req.size));
       insertTabletPlan.setColumns(
-          QueryDataSetUtils.readValuesFromBuffer(
+          QueryDataSetUtils.readTabletValuesFromBuffer(
               req.values, req.types, req.types.size(), req.size));
       insertTabletPlan.setBitMaps(
           QueryDataSetUtils.readBitMapsFromBuffer(req.values, req.types.size(), req.size));
@@ -1913,7 +1913,7 @@ public class TSServiceImpl implements TSIService.Iface {
     insertTabletPlan.setTimes(
         QueryDataSetUtils.readTimesFromBuffer(req.timestampsList.get(i), req.sizeList.get(i)));
     insertTabletPlan.setColumns(
-        QueryDataSetUtils.readValuesFromBuffer(
+        QueryDataSetUtils.readTabletValuesFromBuffer(
             req.valuesList.get(i),
             req.typesList.get(i),
             req.measurementsList.get(i).size(),
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/WalChecker.java b/server/src/main/java/org/apache/iotdb/db/tools/WalChecker.java
index 96d6f84f70..98fbd20ad5 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/WalChecker.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/WalChecker.java
@@ -20,19 +20,25 @@ package org.apache.iotdb.db.tools;
 
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.exception.SystemCheckException;
-import org.apache.iotdb.db.writelog.io.SingleFileLogReader;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.wal.buffer.WALEntry;
+import org.apache.iotdb.db.wal.io.WALWriter;
+import org.apache.iotdb.db.wal.node.WALNode;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import static org.apache.iotdb.db.writelog.node.ExclusiveWriteLogNode.WAL_FILE_NAME;
-
 /** WalChecker verifies that whether all insert ahead logs in the WAL folder are recognizable. */
 public class WalChecker {
 
@@ -58,55 +64,56 @@ public class WalChecker {
       throw new SystemCheckException(walFolder);
     }
 
-    File[] storageWalFolders = walFolderFile.listFiles();
-    if (storageWalFolders == null || storageWalFolders.length == 0) {
+    File[] walNodeFolders =
+        walFolderFile.listFiles(
+            (dir, name) -> {
+              File walNodeFolder = SystemFileFactory.INSTANCE.getFile(dir, name);
+              return walNodeFolder.isDirectory()
+                  && WALNode.WAL_NODE_FOLDER_PATTERN.matcher(name).find();
+            });
+    if (walNodeFolders == null || walNodeFolders.length == 0) {
       logger.info("No sub-directories under the given directory, check ends");
       return Collections.emptyList();
     }
 
     List<File> failedFiles = new ArrayList<>();
-    for (int dirIndex = 0; dirIndex < storageWalFolders.length; dirIndex++) {
-      File storageWalFolder = storageWalFolders[dirIndex];
-      logger.info("Checking the No.{} directory {}", dirIndex, storageWalFolder.getName());
-      File walFile = SystemFileFactory.INSTANCE.getFile(storageWalFolder, WAL_FILE_NAME);
-      if (!checkFile(walFile)) {
-        failedFiles.add(walFile);
+    for (int dirIndex = 0; dirIndex < walNodeFolders.length; dirIndex++) {
+      File walNodeFolder = walNodeFolders[dirIndex];
+      logger.info("Checking the No.{} directory {}", dirIndex, walNodeFolder.getName());
+      File[] walFiles = walNodeFolder.listFiles(WALWriter::walFilenameFilter);
+      if (walFiles == null) {
+        continue;
+      }
+      for (File walFile : walFiles) {
+        if (!checkFile(walFile)) {
+          failedFiles.add(walFile);
+        }
       }
     }
     return failedFiles;
   }
 
   private boolean checkFile(File walFile) {
-    if (!walFile.exists()) {
-      logger.debug("No wal file in this dir, skipping");
-      return true;
-    }
-
-    if (walFile.length() > 0 && walFile.length() < SingleFileLogReader.LEAST_LOG_SIZE) {
-      // contains only one damaged log
-      logger.error(
-          "{} fails the check because it is non-empty but does not contain enough bytes "
-              + "even for one log.",
-          walFile.getAbsoluteFile());
-      return false;
-    }
-
-    SingleFileLogReader logReader = null;
-    try {
-      logReader = new SingleFileLogReader(walFile);
-      while (logReader.hasNext()) {
-        logReader.next();
+    int totalSize = 0;
+    try (DataInputStream logStream =
+        new DataInputStream(new BufferedInputStream(new FileInputStream(walFile)))) {
+      while (logStream.available() > 0) {
+        WALEntry walEntry = WALEntry.deserialize(logStream);
+        totalSize += walEntry.serializedSize();
       }
-      if (logReader.isFileCorrupted()) {
+    } catch (EOFException e) {
+      if (totalSize == walFile.length()) {
+        return true;
+      } else {
+        logger.error("{} fails the check because", walFile, e);
         return false;
       }
-    } catch (IOException e) {
-      logger.error("{} fails the check because", walFile.getAbsoluteFile(), e);
+    } catch (FileNotFoundException e) {
+      logger.debug("Wal file doesn't exist, skipping");
+      return true;
+    } catch (IOException | IllegalPathException e) {
+      logger.error("{} fails the check because", walFile, e);
       return false;
-    } finally {
-      if (logReader != null) {
-        logReader.close();
-      }
     }
     return true;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
index 464b5c8d95..242bf9b946 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
@@ -48,6 +48,8 @@ import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.rescon.TsFileResourceManager;
 import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.wal.WALManager;
+import org.apache.iotdb.db.wal.utils.WALMode;
 import org.apache.iotdb.metrics.config.MetricConfigDescriptor;
 import org.apache.iotdb.rpc.TConfigurationConst;
 import org.apache.iotdb.rpc.TSocketWrapper;
@@ -135,6 +137,9 @@ public class EnvironmentUtils {
       }
     }
 
+    // clean wal manager
+    WALManager.getInstance().clear();
+
     // clean storage group manager
     if (!StorageEngine.getInstance().deleteAll()) {
       logger.error("Can't close the storage group manager in EnvironmentUtils");
@@ -239,8 +244,6 @@ public class EnvironmentUtils {
     }
     // delete system info
     cleanDir(config.getSystemDir());
-    // delete wal
-    cleanDir(config.getWalDir());
     // delete query
     cleanDir(config.getQueryDir());
     // delete tracing
@@ -251,6 +254,10 @@ public class EnvironmentUtils {
     cleanDir(config.getTriggerDir());
     // delete mqtt dir
     cleanDir(config.getMqttDir());
+    // delete wal
+    for (String walDir : config.getWalDirs()) {
+      cleanDir(walDir);
+    }
     // delete data files
     for (String dataDir : config.getDataDirs()) {
       cleanDir(dataDir);
@@ -264,12 +271,13 @@ public class EnvironmentUtils {
   /** disable memory control</br> this function should be called before all code in the setup */
   public static void envSetUp() {
     logger.debug("EnvironmentUtil setup...");
-    IoTDBDescriptor.getInstance().getConfig().setThriftServerAwaitTimeForStopService(60);
+    config.setThriftServerAwaitTimeForStopService(60);
     // we do not start 9091 port in test.
     MetricConfigDescriptor.getInstance().getMetricConfig().setEnableMetric(false);
-    IoTDBDescriptor.getInstance().getConfig().setAvgSeriesPointNumberThreshold(Integer.MAX_VALUE);
-    // the default wal buffer size may cause ci failed
-    IoTDBDescriptor.getInstance().getConfig().setWalBufferSize(4 * 1024 * 1024);
+    config.setAvgSeriesPointNumberThreshold(Integer.MAX_VALUE);
+    // use async wal mode in test
+    config.setWalMode(WALMode.ASYNC);
+    config.setAvgSeriesPointNumberThreshold(Integer.MAX_VALUE);
     if (daemon == null) {
       daemon = new IoTDB();
     }
@@ -321,6 +329,7 @@ public class EnvironmentUtils {
     IoTDB.configManager.clear();
     IDTableManager.getInstance().clear();
     TsFileResourceManager.getInstance().clear();
+    WALManager.getInstance().clear();
     reactiveDaemon();
   }
 
@@ -338,11 +347,13 @@ public class EnvironmentUtils {
     // create sg dir
     String sgDir = FilePathUtils.regularizePath(config.getSystemDir()) + "storage_groups";
     createDir(sgDir);
-    // create wal
-    createDir(config.getWalDir());
     // create query
     createDir(config.getQueryDir());
     createDir(TestConstant.OUTPUT_DATA_DIR);
+    // create wal
+    for (String walDir : config.getWalDirs()) {
+      createDir(walDir);
+    }
     // create data
     for (String dataDir : config.getDataDirs()) {
       createDir(dataDir);
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java b/server/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java
index 0563aa111f..ee909a5c4f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java
@@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory;
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumMap;
 import java.util.List;
@@ -264,7 +265,7 @@ public class OpenFileNumUtil {
     TOTAL_OPEN_FILE_NUM(null),
     SEQUENCE_FILE_OPEN_NUM(directoryManager.getAllSequenceFileFolders()),
     UNSEQUENCE_FILE_OPEN_NUM(directoryManager.getAllUnSequenceFileFolders()),
-    WAL_OPEN_FILE_NUM(Collections.singletonList(config.getWalDir())),
+    WAL_OPEN_FILE_NUM(Arrays.asList(config.getWalDirs())),
     DIGEST_OPEN_FILE_NUM(Collections.singletonList(config.getSystemDir())),
     SOCKET_OPEN_FILE_NUM(null);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
index c33c5d7028..075e5a2ba7 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
@@ -30,6 +30,7 @@ import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.BytesUtils;
 
 import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -180,6 +181,14 @@ public class QueryDataSetUtils {
     return times;
   }
 
+  public static long[] readTimesFromStream(DataInputStream stream, int size) throws IOException {
+    long[] times = new long[size];
+    for (int i = 0; i < size; i++) {
+      times[i] = stream.readLong();
+    }
+    return times;
+  }
+
   public static BitMap[] readBitMapsFromBuffer(ByteBuffer buffer, int columns, int size) {
     if (!buffer.hasRemaining()) {
       return null;
@@ -198,13 +207,41 @@ public class QueryDataSetUtils {
     return bitMaps;
   }
 
-  public static Object[] readValuesFromBuffer(
+  public static BitMap[] readBitMapsFromStream(DataInputStream stream, int columns, int size)
+      throws IOException {
+    if (stream.available() <= 0) {
+      return null;
+    }
+    BitMap[] bitMaps = new BitMap[columns];
+    for (int i = 0; i < columns; i++) {
+      boolean hasBitMap = BytesUtils.byteToBool(stream.readByte());
+      if (hasBitMap) {
+        byte[] bytes = new byte[size / Byte.SIZE + 1];
+        for (int j = 0; j < bytes.length; j++) {
+          bytes[j] = stream.readByte();
+        }
+        bitMaps[i] = new BitMap(size, bytes);
+      }
+    }
+    return bitMaps;
+  }
+
+  public static Object[] readTabletValuesFromBuffer(
       ByteBuffer buffer, List<Integer> types, int columns, int size) {
     TSDataType[] dataTypes = new TSDataType[types.size()];
     for (int i = 0; i < dataTypes.length; i++) {
       dataTypes[i] = TSDataType.values()[types.get(i)];
     }
-    return readValuesFromBuffer(buffer, dataTypes, columns, size);
+    return readTabletValuesFromBuffer(buffer, dataTypes, columns, size);
+  }
+
+  public static Object[] readTabletValuesFromStream(
+      DataInputStream stream, List<Integer> types, int columns, int size) throws IOException {
+    TSDataType[] dataTypes = new TSDataType[types.size()];
+    for (int i = 0; i < dataTypes.length; i++) {
+      dataTypes[i] = TSDataType.values()[types.get(i)];
+    }
+    return readTabletValuesFromStream(stream, dataTypes, columns, size);
   }
 
   /**
@@ -213,7 +250,7 @@ public class QueryDataSetUtils {
    * @param size value count in each column
    */
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public static Object[] readValuesFromBuffer(
+  public static Object[] readTabletValuesFromBuffer(
       ByteBuffer buffer, TSDataType[] types, int columns, int size) {
     Object[] values = new Object[columns];
     for (int i = 0; i < columns; i++) {
@@ -270,4 +307,62 @@ public class QueryDataSetUtils {
     }
     return values;
   }
+
+  public static Object[] readTabletValuesFromStream(
+      DataInputStream stream, TSDataType[] types, int columns, int size) throws IOException {
+    Object[] values = new Object[columns];
+    for (int i = 0; i < columns; i++) {
+      switch (types[i]) {
+        case BOOLEAN:
+          boolean[] boolValues = new boolean[size];
+          for (int index = 0; index < size; index++) {
+            boolValues[index] = BytesUtils.byteToBool(stream.readByte());
+          }
+          values[i] = boolValues;
+          break;
+        case INT32:
+          int[] intValues = new int[size];
+          for (int index = 0; index < size; index++) {
+            intValues[index] = stream.readInt();
+          }
+          values[i] = intValues;
+          break;
+        case INT64:
+          long[] longValues = new long[size];
+          for (int index = 0; index < size; index++) {
+            longValues[index] = stream.readLong();
+          }
+          values[i] = longValues;
+          break;
+        case FLOAT:
+          float[] floatValues = new float[size];
+          for (int index = 0; index < size; index++) {
+            floatValues[index] = stream.readFloat();
+          }
+          values[i] = floatValues;
+          break;
+        case DOUBLE:
+          double[] doubleValues = new double[size];
+          for (int index = 0; index < size; index++) {
+            doubleValues[index] = stream.readDouble();
+          }
+          values[i] = doubleValues;
+          break;
+        case TEXT:
+          Binary[] binaryValues = new Binary[size];
+          for (int index = 0; index < size; index++) {
+            int binarySize = stream.readInt();
+            byte[] binaryValue = new byte[binarySize];
+            stream.read(binaryValue);
+            binaryValues[index] = new Binary(binaryValue);
+          }
+          values[i] = binaryValues;
+          break;
+        default:
+          throw new UnSupportedDataTypeException(
+              String.format("data type %s is not supported when convert data at client", types[i]));
+      }
+    }
+    return values;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java b/server/src/main/java/org/apache/iotdb/db/utils/SerializedSize.java
similarity index 58%
copy from server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
copy to server/src/main/java/org/apache/iotdb/db/utils/SerializedSize.java
index 542eb6448f..dce3d9bfe5 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SerializedSize.java
@@ -16,31 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-package org.apache.iotdb.db.engine.flush;
-
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-
-import java.io.IOException;
-
-public interface FlushListener {
-
-  void onFlushStart(IMemTable memTable) throws IOException;
-
-  void onFlushEnd(IMemTable memTable);
-
-  class EmptyListener implements FlushListener {
-
-    public static final EmptyListener INSTANCE = new EmptyListener();
-
-    @Override
-    public void onFlushStart(IMemTable memTable) {
-      // do nothing
-    }
-
-    @Override
-    public void onFlushEnd(IMemTable memTable) {
-      // do nothing
-    }
-  }
+package org.apache.iotdb.db.utils;
+
+/** Implementations should calculate their accurate serialized size in bytes. */
+public interface SerializedSize {
+  /**
+   * Calculate size after serialization.
+   *
+   * @return bytes number
+   */
+  int serializedSize();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java
index 0594002c36..e8d4af943e 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java
@@ -22,6 +22,8 @@ package org.apache.iotdb.db.utils.datastructure;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.utils.MathUtils;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 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;
@@ -30,8 +32,11 @@ import org.apache.iotdb.tsfile.read.reader.IPointReader;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -40,6 +45,7 @@ import static org.apache.iotdb.tsfile.utils.RamUsageEstimator.NUM_BYTES_ARRAY_HE
 import static org.apache.iotdb.tsfile.utils.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
 
 public class AlignedTVList extends TVList {
+  private static final int NULL_FLAG = -1;
 
   // data types of this aligned tvlist
   private List<TSDataType> dataTypes;
@@ -1003,4 +1009,188 @@ public class AlignedTVList extends TVList {
       return deletedAll;
     }
   }
+
+  @Override
+  public int serializedSize() {
+    int size = (1 + dataTypes.size()) * Byte.BYTES + 2 * Integer.BYTES;
+    // time
+    size += rowCount * Long.BYTES;
+    // value
+    for (int columnIndex = 0; columnIndex < values.size(); ++columnIndex) {
+      switch (dataTypes.get(columnIndex)) {
+        case TEXT:
+          for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+            size += ReadWriteIOUtils.sizeToWrite(getBinaryByValueIndex(rowIdx, columnIndex));
+          }
+          break;
+        case FLOAT:
+          size += rowCount * Float.BYTES;
+          break;
+        case INT32:
+          size += rowCount * Integer.BYTES;
+          break;
+        case INT64:
+          size += rowCount * Long.BYTES;
+          break;
+        case DOUBLE:
+          size += rowCount * Double.BYTES;
+          break;
+        case BOOLEAN:
+          size += rowCount * Byte.BYTES;
+          break;
+        default:
+          throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+      }
+    }
+    // bitmap
+    size += rowCount * dataTypes.size() * Byte.BYTES;
+    return size;
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(TSDataType.VECTOR, buffer);
+    buffer.putInt(dataTypes.size());
+    for (TSDataType dataType : dataTypes) {
+      buffer.put(dataType.serialize());
+    }
+    buffer.putInt(rowCount);
+    // time
+    for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+      buffer.putLong(getTime(rowIndex));
+    }
+    // serialize value and bitmap by column
+    for (int columnIndex = 0; columnIndex < values.size(); columnIndex++) {
+      List<Object> columnValues = values.get(columnIndex);
+      for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+        int arrayIndex = rowIndex / ARRAY_SIZE;
+        int elementIndex = rowIndex % ARRAY_SIZE;
+        // value
+        switch (dataTypes.get(columnIndex)) {
+          case TEXT:
+            Binary valueT = ((Binary[]) columnValues.get(arrayIndex))[elementIndex];
+            WALWriteUtils.write(valueT, buffer);
+            break;
+          case FLOAT:
+            float valueF = ((float[]) columnValues.get(arrayIndex))[elementIndex];
+            buffer.putFloat(valueF);
+            break;
+          case INT32:
+            int valueI = ((int[]) columnValues.get(arrayIndex))[elementIndex];
+            buffer.putInt(valueI);
+            break;
+          case INT64:
+            long valueL = ((long[]) columnValues.get(arrayIndex))[elementIndex];
+            buffer.putLong(valueL);
+            break;
+          case DOUBLE:
+            double valueD = ((double[]) columnValues.get(arrayIndex))[elementIndex];
+            buffer.putDouble(valueD);
+            break;
+          case BOOLEAN:
+            boolean valueB = ((boolean[]) columnValues.get(arrayIndex))[elementIndex];
+            WALWriteUtils.write(valueB, buffer);
+            break;
+          default:
+            throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+        }
+        // bitmap
+        WALWriteUtils.write(isValueMarked(rowIndex, columnIndex), buffer);
+      }
+    }
+  }
+
+  public static AlignedTVList deserialize(DataInputStream stream) throws IOException {
+    int dataTypeNum = stream.readInt();
+    List<TSDataType> dataTypes = new ArrayList<>(dataTypeNum);
+    int[] columnIndexArray = new int[dataTypeNum];
+    for (int columnIndex = 0; columnIndex < dataTypeNum; ++columnIndex) {
+      dataTypes.add(ReadWriteIOUtils.readDataType(stream));
+      columnIndexArray[columnIndex] = columnIndex;
+    }
+
+    int rowCount = stream.readInt();
+    // time
+    long[] times = new long[rowCount];
+    for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+      times[rowIndex] = stream.readLong();
+    }
+    // read value and bitmap by column
+    Object[] values = new Object[dataTypeNum];
+    BitMap[] bitMaps = new BitMap[dataTypeNum];
+    for (int columnIndex = 0; columnIndex < dataTypeNum; ++columnIndex) {
+      BitMap bitMap = new BitMap(rowCount);
+      Object valuesOfOneColumn;
+      switch (dataTypes.get(columnIndex)) {
+        case TEXT:
+          Binary[] binaryValues = new Binary[rowCount];
+          for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+            binaryValues[rowIndex] = ReadWriteIOUtils.readBinary(stream);
+            if (ReadWriteIOUtils.readBool(stream)) {
+              bitMap.mark(rowIndex);
+            }
+          }
+          valuesOfOneColumn = binaryValues;
+          break;
+        case FLOAT:
+          float[] floatValues = new float[rowCount];
+          for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+            floatValues[rowIndex] = stream.readFloat();
+            if (ReadWriteIOUtils.readBool(stream)) {
+              bitMap.mark(rowIndex);
+            }
+          }
+          valuesOfOneColumn = floatValues;
+          break;
+        case INT32:
+          int[] intValues = new int[rowCount];
+          for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+            intValues[rowIndex] = stream.readInt();
+            if (ReadWriteIOUtils.readBool(stream)) {
+              bitMap.mark(rowIndex);
+            }
+          }
+          valuesOfOneColumn = intValues;
+          break;
+        case INT64:
+          long[] longValues = new long[rowCount];
+          for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+            longValues[rowIndex] = stream.readLong();
+            if (ReadWriteIOUtils.readBool(stream)) {
+              bitMap.mark(rowIndex);
+            }
+          }
+          valuesOfOneColumn = longValues;
+          break;
+        case DOUBLE:
+          double[] doubleValues = new double[rowCount];
+          for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+            doubleValues[rowIndex] = stream.readDouble();
+            if (ReadWriteIOUtils.readBool(stream)) {
+              bitMap.mark(rowIndex);
+            }
+          }
+          valuesOfOneColumn = doubleValues;
+          break;
+        case BOOLEAN:
+          boolean[] booleanValues = new boolean[rowCount];
+          for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) {
+            booleanValues[rowIndex] = ReadWriteIOUtils.readBool(stream);
+            if (ReadWriteIOUtils.readBool(stream)) {
+              bitMap.mark(rowIndex);
+            }
+          }
+          valuesOfOneColumn = booleanValues;
+          break;
+        default:
+          throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+      }
+      values[columnIndex] = valuesOfOneColumn;
+      bitMaps[columnIndex] = bitMap;
+    }
+
+    AlignedTVList tvList = new AlignedTVList(dataTypes);
+    tvList.putAlignedValues(times, values, bitMaps, columnIndexArray, 0, rowCount);
+    return tvList;
+  }
 }
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 a3658623b2..4a711a7b0f 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
@@ -19,13 +19,18 @@
 package org.apache.iotdb.db.utils.datastructure;
 
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 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;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -276,4 +281,36 @@ public class BinaryTVList extends TVList {
   public TSDataType getDataType() {
     return TSDataType.TEXT;
   }
+
+  @Override
+  public int serializedSize() {
+    int size = Byte.BYTES + Integer.BYTES + rowCount * Long.BYTES;
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      size += ReadWriteIOUtils.sizeToWrite(getBinary(rowIdx));
+    }
+    return size;
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(TSDataType.TEXT, buffer);
+    buffer.putInt(rowCount);
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      buffer.putLong(getTime(rowIdx));
+      WALWriteUtils.write(getBinary(rowIdx), buffer);
+    }
+  }
+
+  public static BinaryTVList deserialize(DataInputStream stream) throws IOException {
+    BinaryTVList tvList = new BinaryTVList();
+    int rowCount = stream.readInt();
+    long[] times = new long[rowCount];
+    Binary[] values = new Binary[rowCount];
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      times[rowIdx] = stream.readLong();
+      values[rowIdx] = ReadWriteIOUtils.readBinary(stream);
+    }
+    tvList.putBinaries(times, values, null, 0, rowCount);
+    return tvList;
+  }
 }
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 390c9b69b0..2b15a4e286 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
@@ -19,12 +19,17 @@
 package org.apache.iotdb.db.utils.datastructure;
 
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 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;
 import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -276,4 +281,32 @@ public class BooleanTVList extends TVList {
   public TSDataType getDataType() {
     return TSDataType.BOOLEAN;
   }
+
+  @Override
+  public int serializedSize() {
+    return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Byte.BYTES);
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(TSDataType.BOOLEAN, buffer);
+    buffer.putInt(rowCount);
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      buffer.putLong(getTime(rowIdx));
+      WALWriteUtils.write(getBoolean(rowIdx), buffer);
+    }
+  }
+
+  public static BooleanTVList deserialize(DataInputStream stream) throws IOException {
+    BooleanTVList tvList = new BooleanTVList();
+    int rowCount = stream.readInt();
+    long[] times = new long[rowCount];
+    boolean[] values = new boolean[rowCount];
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      times[rowIdx] = stream.readLong();
+      values[rowIdx] = ReadWriteIOUtils.readBool(stream);
+    }
+    tvList.putBooleans(times, values, null, 0, rowCount);
+    return tvList;
+  }
 }
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 a7b6718be9..3fe9f016d6 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
@@ -20,12 +20,16 @@ package org.apache.iotdb.db.utils.datastructure;
 
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.utils.MathUtils;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 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;
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -280,4 +284,32 @@ public class DoubleTVList extends TVList {
   public TSDataType getDataType() {
     return TSDataType.DOUBLE;
   }
+
+  @Override
+  public int serializedSize() {
+    return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Double.BYTES);
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(TSDataType.DOUBLE, buffer);
+    buffer.putInt(rowCount);
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      buffer.putLong(getTime(rowIdx));
+      buffer.putDouble(getDouble(rowIdx));
+    }
+  }
+
+  public static DoubleTVList deserialize(DataInputStream stream) throws IOException {
+    DoubleTVList tvList = new DoubleTVList();
+    int rowCount = stream.readInt();
+    long[] times = new long[rowCount];
+    double[] values = new double[rowCount];
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      times[rowIdx] = stream.readLong();
+      values[rowIdx] = stream.readDouble();
+    }
+    tvList.putDoubles(times, values, null, 0, rowCount);
+    return tvList;
+  }
 }
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 8594067c9d..b11d26360d 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
@@ -20,12 +20,16 @@ package org.apache.iotdb.db.utils.datastructure;
 
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.utils.MathUtils;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 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;
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -280,4 +284,32 @@ public class FloatTVList extends TVList {
   public TSDataType getDataType() {
     return TSDataType.FLOAT;
   }
+
+  @Override
+  public int serializedSize() {
+    return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Float.BYTES);
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(TSDataType.FLOAT, buffer);
+    buffer.putInt(rowCount);
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      buffer.putLong(getTime(rowIdx));
+      buffer.putFloat(getFloat(rowIdx));
+    }
+  }
+
+  public static FloatTVList deserialize(DataInputStream stream) throws IOException {
+    FloatTVList tvList = new FloatTVList();
+    int rowCount = stream.readInt();
+    long[] times = new long[rowCount];
+    float[] values = new float[rowCount];
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      times[rowIdx] = stream.readLong();
+      values[rowIdx] = stream.readFloat();
+    }
+    tvList.putFloats(times, values, null, 0, rowCount);
+    return tvList;
+  }
 }
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 8f06aa0da3..c6779acf17 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
@@ -19,12 +19,16 @@
 package org.apache.iotdb.db.utils.datastructure;
 
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 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;
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -275,4 +279,32 @@ public class IntTVList extends TVList {
   public TSDataType getDataType() {
     return TSDataType.INT32;
   }
+
+  @Override
+  public int serializedSize() {
+    return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Integer.BYTES);
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(TSDataType.INT32, buffer);
+    buffer.putInt(rowCount);
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      buffer.putLong(getTime(rowIdx));
+      buffer.putInt(getInt(rowIdx));
+    }
+  }
+
+  public static IntTVList deserialize(DataInputStream stream) throws IOException {
+    IntTVList tvList = new IntTVList();
+    int rowCount = stream.readInt();
+    long[] times = new long[rowCount];
+    int[] values = new int[rowCount];
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      times[rowIdx] = stream.readLong();
+      values[rowIdx] = stream.readInt();
+    }
+    tvList.putInts(times, values, null, 0, rowCount);
+    return tvList;
+  }
 }
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 1cb1e60623..cd7e1e5124 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
@@ -19,12 +19,16 @@
 package org.apache.iotdb.db.utils.datastructure;
 
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
+import org.apache.iotdb.db.wal.utils.WALWriteUtils;
 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;
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -275,4 +279,32 @@ public class LongTVList extends TVList {
   public TSDataType getDataType() {
     return TSDataType.INT64;
   }
+
+  @Override
+  public int serializedSize() {
+    return Byte.BYTES + Integer.BYTES + rowCount * 2 * Long.BYTES;
+  }
+
+  @Override
+  public void serializeToWAL(IWALByteBufferView buffer) {
+    WALWriteUtils.write(TSDataType.INT64, buffer);
+    buffer.putInt(rowCount);
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      buffer.putLong(getTime(rowIdx));
+      buffer.putLong(getLong(rowIdx));
+    }
+  }
+
+  public static LongTVList deserialize(DataInputStream stream) throws IOException {
+    LongTVList tvList = new LongTVList();
+    int rowCount = stream.readInt();
+    long[] times = new long[rowCount];
+    long[] values = new long[rowCount];
+    for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) {
+      times[rowIdx] = stream.readLong();
+      values[rowIdx] = stream.readLong();
+    }
+    tvList.putLongs(times, values, null, 0, rowCount);
+    return tvList;
+  }
 }
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 550bc333f7..56ac6e1205 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
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.utils.datastructure;
 
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.db.wal.buffer.WALEntryValue;
 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;
@@ -28,7 +29,9 @@ import org.apache.iotdb.tsfile.read.common.TimeRange;
 import org.apache.iotdb.tsfile.read.reader.IPointReader;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
+import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -38,7 +41,7 @@ import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 import static org.apache.iotdb.tsfile.utils.RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
 import static org.apache.iotdb.tsfile.utils.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
 
-public abstract class TVList {
+public abstract class TVList implements WALEntryValue {
 
   protected static final int SMALL_ARRAY_LENGTH = 32;
   protected static final String ERR_DATATYPE_NOT_CONSISTENT = "DataType not consistent";
@@ -615,7 +618,26 @@ public abstract class TVList {
 
   public abstract TSDataType getDataType();
 
-  public long getLastTime() {
-    return getTime(rowCount - 1);
+  public static TVList deserialize(DataInputStream stream) throws IOException {
+    TSDataType dataType = ReadWriteIOUtils.readDataType(stream);
+    switch (dataType) {
+      case TEXT:
+        return BinaryTVList.deserialize(stream);
+      case FLOAT:
+        return FloatTVList.deserialize(stream);
+      case INT32:
+        return IntTVList.deserialize(stream);
+      case INT64:
+        return LongTVList.deserialize(stream);
+      case DOUBLE:
+        return DoubleTVList.deserialize(stream);
+      case BOOLEAN:
+        return BooleanTVList.deserialize(stream);
+      case VECTOR:
+        return AlignedTVList.deserialize(stream);
+      default:
+        break;
+    }
+    return null;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/io/BatchLogReader.java b/server/src/main/java/org/apache/iotdb/db/utils/writelog/BatchLogReader.java
similarity index 98%
rename from server/src/main/java/org/apache/iotdb/db/writelog/io/BatchLogReader.java
rename to server/src/main/java/org/apache/iotdb/db/utils/writelog/BatchLogReader.java
index dbbbe59be0..296e20a361 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/io/BatchLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/writelog/BatchLogReader.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.writelog.io;
+package org.apache.iotdb.db.utils.writelog;
 
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/io/ILogReader.java b/server/src/main/java/org/apache/iotdb/db/utils/writelog/ILogReader.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/writelog/io/ILogReader.java
rename to server/src/main/java/org/apache/iotdb/db/utils/writelog/ILogReader.java
index 956ad3f1c8..ccdb22350c 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/io/ILogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/writelog/ILogReader.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.writelog.io;
+package org.apache.iotdb.db.utils.writelog;
 
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/io/ILogWriter.java b/server/src/main/java/org/apache/iotdb/db/utils/writelog/ILogWriter.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/writelog/io/ILogWriter.java
rename to server/src/main/java/org/apache/iotdb/db/utils/writelog/ILogWriter.java
index 4a1ad9f80a..4e97768fb0 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/io/ILogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/writelog/ILogWriter.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.writelog.io;
+package org.apache.iotdb.db.utils.writelog;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/io/LogWriter.java b/server/src/main/java/org/apache/iotdb/db/utils/writelog/LogWriter.java
similarity index 98%
rename from server/src/main/java/org/apache/iotdb/db/writelog/io/LogWriter.java
rename to server/src/main/java/org/apache/iotdb/db/utils/writelog/LogWriter.java
index c648192d6a..6ca3d3a99c 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/io/LogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/writelog/LogWriter.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.writelog.io;
+package org.apache.iotdb.db.utils.writelog;
 
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/io/MultiFileLogReader.java b/server/src/main/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReader.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/writelog/io/MultiFileLogReader.java
rename to server/src/main/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReader.java
index 29d93dc4b9..ab344242c9 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/io/MultiFileLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReader.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.writelog.io;
+package org.apache.iotdb.db.utils.writelog;
 
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/io/SingleFileLogReader.java b/server/src/main/java/org/apache/iotdb/db/utils/writelog/SingleFileLogReader.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/writelog/io/SingleFileLogReader.java
rename to server/src/main/java/org/apache/iotdb/db/utils/writelog/SingleFileLogReader.java
index 9530cd6e03..5e2ad6edbe 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/io/SingleFileLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/writelog/SingleFileLogReader.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.writelog.io;
+package org.apache.iotdb.db.utils.writelog;
 
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
@@ -114,7 +114,7 @@ public class SingleFileLogReader implements ILogReader {
       fileCorrupted = true;
       return false;
     }
-    return true;
+    return batchLogReader != null && batchLogReader.hasNext();
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/WALManager.java b/server/src/main/java/org/apache/iotdb/db/wal/WALManager.java
new file mode 100644
index 0000000000..e03936cf68
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/WALManager.java
@@ -0,0 +1,226 @@
+/*
+ * 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.wal;
+
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.concurrent.ThreadName;
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.service.IService;
+import org.apache.iotdb.commons.service.ServiceType;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.conf.directories.FolderManager;
+import org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategyType;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.wal.node.IWALNode;
+import org.apache.iotdb.db.wal.node.WALFakeNode;
+import org.apache.iotdb.db.wal.node.WALNode;
+import org.apache.iotdb.db.wal.utils.WALMode;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/** This class is used to manage all wal nodes */
+public class WALManager implements IService {
+  private static final Logger logger = LoggerFactory.getLogger(WALManager.class);
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  public static final long DELETE_WAL_FILES_PERIOD_IN_MS = config.getDeleteWalFilesPeriodInMs();
+  private static final int MAX_WAL_NODE_NUM =
+      config.getMaxWalNodesNum() > 0 ? config.getMaxWalNodesNum() : config.getWalDirs().length * 2;
+
+  /** manage wal folders */
+  private FolderManager folderManager;
+  /** protect concurrent safety of wal nodes, including walNodes, nodeCursor and nodeIdCounter */
+  private final Lock nodesLock = new ReentrantLock();
+  // region these variables should be protected by nodesLock
+  /** wal nodes, the max number of wal nodes is MAX_WAL_NUM */
+  private final List<WALNode> walNodes = new ArrayList<>(MAX_WAL_NODE_NUM);
+  /** help allocate node for users */
+  private int nodeCursor = -1;
+  /** each wal node has a unique long value identifier */
+  private long nodeIdCounter = -1;
+  // endregion
+  /** single thread to delete old .wal files */
+  private ScheduledExecutorService walDeleteThread;
+
+  private WALManager() {}
+
+  /** Apply for a wal node */
+  public IWALNode applyForWALNode() {
+    if (config.getWalMode() == WALMode.DISABLE) {
+      return WALFakeNode.getSuccessInstance();
+    }
+
+    WALNode selectedNode;
+    nodesLock.lock();
+    try {
+      if (walNodes.size() < MAX_WAL_NODE_NUM) {
+        nodeIdCounter++;
+        String identifier = String.valueOf(nodeIdCounter);
+        String folder;
+        // get wal folder
+        try {
+          folder = folderManager.getNextFolder();
+        } catch (DiskSpaceInsufficientException e) {
+          logger.error("All disks of wal folders are full, change system mode to read-only.", e);
+          config.setReadOnly(true);
+          return WALFakeNode.getFailureInstance(e);
+        }
+        folder = folder + File.separator + identifier;
+        // create new wal node
+        try {
+          selectedNode = new WALNode(identifier, folder);
+        } catch (FileNotFoundException e) {
+          logger.error("Fail to create wal node", e);
+          return WALFakeNode.getFailureInstance(e);
+        }
+        walNodes.add(selectedNode);
+      } else {
+        // select next wal node by sequence order
+        nodeCursor = (nodeCursor + 1) % MAX_WAL_NODE_NUM;
+        selectedNode = walNodes.get(nodeCursor);
+      }
+    } finally {
+      nodesLock.unlock();
+    }
+    return selectedNode;
+  }
+
+  @Override
+  public void start() throws StartupException {
+    if (config.getWalMode() == WALMode.DISABLE) {
+      return;
+    }
+
+    try {
+      folderManager =
+          new FolderManager(
+              Arrays.asList(config.getWalDirs()), DirectoryStrategyType.SEQUENCE_STRATEGY);
+      walDeleteThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(ThreadName.WAL_DELETE.getName());
+      walDeleteThread.scheduleAtFixedRate(
+          this::deleteOutdatedFiles,
+          DELETE_WAL_FILES_PERIOD_IN_MS,
+          DELETE_WAL_FILES_PERIOD_IN_MS,
+          TimeUnit.MILLISECONDS);
+    } catch (Exception e) {
+      throw new StartupException(this.getID().getName(), e.getMessage());
+    }
+  }
+
+  public void deleteOutdatedWALFiles() {
+    Future<?> future = walDeleteThread.submit(this::deleteOutdatedFiles);
+    try {
+      future.get();
+    } catch (ExecutionException e) {
+      logger.warn("Exception occurs when deleting wal files", e);
+    } catch (InterruptedException e) {
+      logger.warn("Interrupted when deleting wal files", e);
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  private void deleteOutdatedFiles() {
+    for (WALNode walNode : getNodesSnapshot()) {
+      walNode.deleteOutdatedFiles();
+    }
+  }
+
+  private List<WALNode> getNodesSnapshot() {
+    List<WALNode> snapshot;
+    if (walNodes.size() < MAX_WAL_NODE_NUM) {
+      nodesLock.lock();
+      try {
+        snapshot = new ArrayList<>(walNodes);
+      } finally {
+        nodesLock.unlock();
+      }
+    } else {
+      snapshot = walNodes;
+    }
+    return snapshot;
+  }
+
+  @Override
+  public void stop() {
+    if (config.getWalMode() == WALMode.DISABLE) {
+      return;
+    }
+    if (walDeleteThread != null) {
+      shutdownThread(walDeleteThread, ThreadName.WAL_DELETE);
+    }
+    clear();
+  }
+
+  private void shutdownThread(ExecutorService thread, ThreadName threadName) {
+    thread.shutdown();
+    try {
+      if (!thread.awaitTermination(30, TimeUnit.SECONDS)) {
+        logger.warn("Waiting thread {} to be terminated is timeout", threadName.getName());
+      }
+    } catch (InterruptedException e) {
+      logger.warn("Thread {} still doesn't exit after 30s", threadName.getName());
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  @TestOnly
+  public void clear() {
+    nodesLock.lock();
+    try {
+      nodeCursor = -1;
+      nodeIdCounter = -1;
+      for (WALNode walNode : walNodes) {
+        walNode.close();
+      }
+      walNodes.clear();
+    } finally {
+      nodesLock.unlock();
+    }
+  }
+
+  @Override
+  public ServiceType getID() {
+    return ServiceType.WAL_SERVICE;
+  }
+
+  public static WALManager getInstance() {
+    return InstanceHolder.INSTANCE;
+  }
+
+  private static class InstanceHolder {
+    private InstanceHolder() {}
+
+    private static final WALManager INSTANCE = new WALManager();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/AbstractWALBuffer.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/AbstractWALBuffer.java
new file mode 100644
index 0000000000..cc689b083e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/AbstractWALBuffer.java
@@ -0,0 +1,76 @@
+/*
+ * 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.wal.buffer;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.wal.io.ILogWriter;
+import org.apache.iotdb.db.wal.io.WALWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public abstract class AbstractWALBuffer implements IWALBuffer {
+  private static final Logger logger = LoggerFactory.getLogger(AbstractWALBuffer.class);
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  /** use size limit to control WALEntry number in each file */
+  protected static final long FILE_SIZE_THRESHOLD = config.getWalFileSizeThresholdInByte();
+
+  /** WALNode identifier of this buffer */
+  protected final String identifier;
+  /** directory to store .wal files */
+  protected final String logDirectory;
+  /** current wal file version id */
+  protected final AtomicInteger currentWALFileVersion = new AtomicInteger();
+  /** current wal file log writer */
+  protected volatile ILogWriter currentWALFileWriter;
+
+  public AbstractWALBuffer(String identifier, String logDirectory) throws FileNotFoundException {
+    this.identifier = identifier;
+    this.logDirectory = logDirectory;
+    File logDirFile = SystemFileFactory.INSTANCE.getFile(logDirectory);
+    if (!logDirFile.exists() && logDirFile.mkdirs()) {
+      logger.info("create folder {} for wal buffer-{}.", logDirectory, identifier);
+    }
+    currentWALFileWriter =
+        new WALWriter(
+            SystemFileFactory.INSTANCE.getFile(
+                logDirectory, WALWriter.getLogFileName(currentWALFileVersion.get())));
+  }
+
+  @Override
+  public int getCurrentWALFileVersion() {
+    return currentWALFileVersion.get();
+  }
+
+  /** Notice: only called by syncBufferThread and old log writer will be closed by this function. */
+  protected void rollLogWriter() throws IOException {
+    currentWALFileWriter.close();
+    File nextLogFile =
+        SystemFileFactory.INSTANCE.getFile(
+            logDirectory, WALWriter.getLogFileName(currentWALFileVersion.incrementAndGet()));
+    currentWALFileWriter = new WALWriter(nextLogFile);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/IWALBuffer.java
similarity index 59%
copy from server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
copy to server/src/main/java/org/apache/iotdb/db/wal/buffer/IWALBuffer.java
index 542eb6448f..9feef0963e 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/IWALBuffer.java
@@ -16,31 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.wal.buffer;
 
-package org.apache.iotdb.db.engine.flush;
+import org.apache.iotdb.commons.utils.TestOnly;
 
-import org.apache.iotdb.db.engine.memtable.IMemTable;
+/** Currently, there are 2 buffer types, including wal rolling buffer and wal segmented buffer. */
+public interface IWALBuffer extends AutoCloseable {
+  /**
+   * Write WALEntry into wal buffer.
+   *
+   * @param walEntry info will be written into wal buffer
+   */
+  void write(WALEntry walEntry);
 
-import java.io.IOException;
+  /** Get current log version id */
+  int getCurrentWALFileVersion();
 
-public interface FlushListener {
+  @Override
+  void close();
 
-  void onFlushStart(IMemTable memTable) throws IOException;
-
-  void onFlushEnd(IMemTable memTable);
-
-  class EmptyListener implements FlushListener {
-
-    public static final EmptyListener INSTANCE = new EmptyListener();
-
-    @Override
-    public void onFlushStart(IMemTable memTable) {
-      // do nothing
-    }
-
-    @Override
-    public void onFlushEnd(IMemTable memTable) {
-      // do nothing
-    }
-  }
+  @TestOnly
+  boolean isAllWALEntriesConsumed();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/IWALByteBufferView.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/IWALByteBufferView.java
new file mode 100644
index 0000000000..977ac51dfc
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/IWALByteBufferView.java
@@ -0,0 +1,52 @@
+/*
+ * 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.wal.buffer;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This ByteBuffer view provides blocking writing interface for wal to serialize huge object with
+ * limited memory usage. This interface should behave like {@link ByteBuffer} and don't guarantee
+ * the concurrent safety.
+ */
+public interface IWALByteBufferView {
+  /** Like {@link ByteBuffer#put(byte)}. */
+  void put(byte b);
+
+  /** Like {@link ByteBuffer#put(byte[])}. */
+  void put(byte[] src);
+
+  /** Like {@link ByteBuffer#putChar(char)}. */
+  void putChar(char value);
+
+  /** Like {@link ByteBuffer#putShort(short)}. */
+  void putShort(short value);
+
+  /** Like {@link ByteBuffer#putInt(int)}. */
+  void putInt(int value);
+
+  /** Like {@link ByteBuffer#putLong(long)}. */
+  void putLong(long value);
+
+  /** Like {@link ByteBuffer#putFloat(float)}. */
+  void putFloat(float value);
+
+  /** Like {@link ByteBuffer#putDouble(double)}. */
+  void putDouble(double value);
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/SignalWALEntry.java
similarity index 51%
copy from server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
copy to server/src/main/java/org/apache/iotdb/db/wal/buffer/SignalWALEntry.java
index 6dcb92f64c..c3fc7b1b68 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/SignalWALEntry.java
@@ -16,32 +16,36 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.engine.cq;
+package org.apache.iotdb.db.wal.buffer;
 
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.io.SingleFileLogReader;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 
-import java.io.File;
-import java.io.IOException;
+/** This class provides a signal to help wal buffer dealing with some special cases */
+public class SignalWALEntry extends WALEntry {
+  private final SignalType signalType;
 
-public class CQLogReader implements AutoCloseable {
-
-  private final SingleFileLogReader logReader;
+  public SignalWALEntry(SignalType signalType) {
+    this(signalType, false);
+  }
 
-  public CQLogReader(File logFile) throws IOException {
-    logReader = new SingleFileLogReader(logFile);
+  public SignalWALEntry(SignalType signalType, boolean wait) {
+    super(Integer.MIN_VALUE, new DeletePlan(), wait);
+    this.signalType = signalType;
   }
 
-  public boolean hasNext() {
-    return !logReader.isFileCorrupted() && logReader.hasNext();
+  @Override
+  public boolean isSignal() {
+    return true;
   }
 
-  public PhysicalPlan next() {
-    return logReader.next();
+  public SignalType getSignalType() {
+    return signalType;
   }
 
-  @Override
-  public void close() {
-    logReader.close();
+  public enum SignalType {
+    /** signal wal buffer has been closed */
+    CLOSE_SIGNAL,
+    /** signal wal buffer to roll wal log writer */
+    ROLL_WAL_LOG_WRITER_SIGNAL,
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALBuffer.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALBuffer.java
new file mode 100644
index 0000000000..a218d48d6c
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALBuffer.java
@@ -0,0 +1,454 @@
+/*
+ * 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.wal.buffer;
+
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.utils.MmapUtil;
+import org.apache.iotdb.db.wal.exception.WALNodeClosedException;
+import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * This buffer guarantees the concurrent safety and uses double buffers mechanism to accelerate
+ * writes and avoid waiting for buffer syncing to disk.
+ */
+public class WALBuffer extends AbstractWALBuffer {
+  private static final Logger logger = LoggerFactory.getLogger(WALBuffer.class);
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private static final int WAL_BUFFER_SIZE = config.getWalBufferSize();
+  private static final long FSYNC_WAL_DELAY_IN_MS = config.getFsyncWalDelayInMs();
+  private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity();
+
+  /** whether close method is called */
+  private volatile boolean isClosed = false;
+  /** WALEntries */
+  private final BlockingQueue<WALEntry> walEntries = new ArrayBlockingQueue<>(QUEUE_CAPACITY);
+  /** lock to provide synchronization for double buffers mechanism, protecting buffers status */
+  private final Lock buffersLock = new ReentrantLock();
+  /** condition to guarantee correctness of switching buffers */
+  private final Condition idleBufferReadyCondition = buffersLock.newCondition();
+  // region these variables should be protected by buffersLock
+  /** two buffers switch between three statuses (there is always 1 buffer working) */
+  // buffer in working status, only updated by serializeThread
+  private volatile ByteBuffer workingBuffer;
+  // buffer in idle status
+  private volatile ByteBuffer idleBuffer;
+  // buffer in syncing status, serializeThread makes sure no more writes to syncingBuffer
+  private volatile ByteBuffer syncingBuffer;
+  // endregion
+  /** single thread to serialize WALEntry to workingBuffer */
+  private final ExecutorService serializeThread;
+  /** single thread to sync syncingBuffer to disk */
+  private final ExecutorService syncBufferThread;
+
+  public WALBuffer(String identifier, String logDirectory) throws FileNotFoundException {
+    super(identifier, logDirectory);
+    allocateBuffers();
+    serializeThread =
+        IoTDBThreadPoolFactory.newSingleThreadExecutor(
+            ThreadName.WAL_SERIALIZE.getName() + "(node-" + identifier + ")");
+    syncBufferThread =
+        IoTDBThreadPoolFactory.newSingleThreadExecutor(
+            ThreadName.WAL_SYNC.getName() + "(node-" + identifier + ")");
+    // start receiving serialize tasks
+    serializeThread.submit(new SerializeTask());
+  }
+
+  private void allocateBuffers() {
+    try {
+      workingBuffer = ByteBuffer.allocateDirect(WAL_BUFFER_SIZE / 2);
+      idleBuffer = ByteBuffer.allocateDirect(WAL_BUFFER_SIZE / 2);
+    } catch (OutOfMemoryError e) {
+      logger.error("Fail to allocate wal node-{}'s buffer because out of memory.", identifier, e);
+      close();
+      throw e;
+    }
+  }
+
+  @Override
+  public void write(WALEntry walEntry) {
+    if (isClosed) {
+      logger.error(
+          "Fail to write WALEntry into wal node-{} because this node is closed.", identifier);
+      walEntry.getWalFlushListener().fail(new WALNodeClosedException(identifier));
+      return;
+    }
+    // just add this WALEntry to queue
+    try {
+      walEntries.put(walEntry);
+    } catch (InterruptedException e) {
+      logger.warn("Interrupted when waiting for adding WALEntry to buffer.");
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  // region Task of serializeThread
+  /** This task serializes WALEntry to workingBuffer and will call fsync at last. */
+  private class SerializeTask implements Runnable {
+    private final IWALByteBufferView byteBufferVew = new ByteBufferView();
+    private final List<WALFlushListener> fsyncListeners = new LinkedList<>();
+
+    @Override
+    public void run() {
+      try {
+        serialize();
+      } finally {
+        serializeThread.submit(new SerializeTask());
+      }
+    }
+
+    /** In order to control memory usage of blocking queue, get 1 and then serialize 1 */
+    private void serialize() {
+      boolean rollWAlFileWriter = false;
+      int batchSize = 0;
+
+      // try to get first WALEntry with blocking interface
+      try {
+        WALEntry firstWALEntry = walEntries.take();
+        if (!firstWALEntry.isSignal()) {
+          try {
+            firstWALEntry.serialize(byteBufferVew);
+            ++batchSize;
+            fsyncListeners.add(firstWALEntry.getWalFlushListener());
+          } catch (Exception e) {
+            logger.error(
+                "Fail to serialize WALEntry to wal node-{}'s buffer, discard it.", identifier, e);
+            firstWALEntry.getWalFlushListener().fail(e);
+          }
+        } else {
+          switch (((SignalWALEntry) firstWALEntry).getSignalType()) {
+            case ROLL_WAL_LOG_WRITER_SIGNAL:
+              rollWAlFileWriter = true;
+              fsyncListeners.add(firstWALEntry.getWalFlushListener());
+              break;
+            case CLOSE_SIGNAL:
+            default:
+              break;
+          }
+        }
+      } catch (InterruptedException e) {
+        logger.warn(
+            "Interrupted when waiting for taking WALEntry from blocking queue to serialize.");
+        Thread.currentThread().interrupt();
+      }
+
+      // for better fsync performance, sleep a while to enlarge write batch
+      if (FSYNC_WAL_DELAY_IN_MS > 0) {
+        try {
+          Thread.sleep(FSYNC_WAL_DELAY_IN_MS);
+        } catch (InterruptedException e) {
+          logger.warn("Interrupted when sleeping a while to enlarge wal write batch.");
+          Thread.currentThread().interrupt();
+        }
+      }
+
+      // try to get more WALEntries with non-blocking interface to enlarge write batch
+      while (walEntries.peek() != null && batchSize < QUEUE_CAPACITY) {
+        WALEntry walEntry = walEntries.poll();
+        if (!walEntry.isSignal()) {
+          try {
+            walEntry.serialize(byteBufferVew);
+          } catch (Exception e) {
+            logger.error(
+                "Fail to serialize WALEntry to wal node-{}'s buffer, discard it.", identifier, e);
+            walEntry.getWalFlushListener().fail(e);
+            continue;
+          }
+          ++batchSize;
+          fsyncListeners.add(walEntry.getWalFlushListener());
+        } else {
+          switch (((SignalWALEntry) walEntry).getSignalType()) {
+            case ROLL_WAL_LOG_WRITER_SIGNAL:
+              rollWAlFileWriter = true;
+              fsyncListeners.add(walEntry.getWalFlushListener());
+              break;
+            case CLOSE_SIGNAL:
+            default:
+              break;
+          }
+          break;
+        }
+      }
+
+      // call fsync at last and set fsyncListeners
+      if (batchSize > 0 || rollWAlFileWriter) {
+        fsyncWorkingBuffer(fsyncListeners, rollWAlFileWriter);
+      }
+    }
+  }
+
+  /**
+   * This view uses workingBuffer lock-freely because workingBuffer is only updated by
+   * serializeThread and this class is only used by serializeThread.
+   */
+  private class ByteBufferView implements IWALByteBufferView {
+    private void ensureEnoughSpace(int bytesNum) {
+      if (workingBuffer.remaining() < bytesNum) {
+        rollBuffer();
+      }
+    }
+
+    private void rollBuffer() {
+      syncWorkingBuffer();
+    }
+
+    @Override
+    public void put(byte b) {
+      ensureEnoughSpace(Byte.BYTES);
+      workingBuffer.put(b);
+    }
+
+    @Override
+    public void put(byte[] src) {
+      int offset = 0;
+      while (true) {
+        int leftCapacity = workingBuffer.remaining();
+        int needCapacity = src.length - offset;
+        if (leftCapacity >= needCapacity) {
+          workingBuffer.put(src, offset, needCapacity);
+          break;
+        } else {
+          workingBuffer.put(src, offset, leftCapacity);
+          offset += leftCapacity;
+          rollBuffer();
+        }
+      }
+    }
+
+    @Override
+    public void putChar(char value) {
+      ensureEnoughSpace(Character.BYTES);
+      workingBuffer.putChar(value);
+    }
+
+    @Override
+    public void putShort(short value) {
+      ensureEnoughSpace(Short.BYTES);
+      workingBuffer.putShort(value);
+    }
+
+    @Override
+    public void putInt(int value) {
+      ensureEnoughSpace(Integer.BYTES);
+      workingBuffer.putInt(value);
+    }
+
+    @Override
+    public void putLong(long value) {
+      ensureEnoughSpace(Long.BYTES);
+      workingBuffer.putLong(value);
+    }
+
+    @Override
+    public void putFloat(float value) {
+      ensureEnoughSpace(Float.BYTES);
+      workingBuffer.putFloat(value);
+    }
+
+    @Override
+    public void putDouble(double value) {
+      ensureEnoughSpace(Double.BYTES);
+      workingBuffer.putDouble(value);
+    }
+  }
+
+  /** Notice: this method only called when buffer is exhausted by SerializeTask. */
+  private void syncWorkingBuffer() {
+    switchWorkingBufferToFlushing();
+    syncBufferThread.submit(new SyncBufferTask(false));
+  }
+
+  /** Notice: this method only called at the last of SerializeTask. */
+  private void fsyncWorkingBuffer(
+      List<WALFlushListener> fsyncListeners, boolean rollWAlFileWriter) {
+    switchWorkingBufferToFlushing();
+    syncBufferThread.submit(new SyncBufferTask(true, rollWAlFileWriter, fsyncListeners));
+  }
+
+  // only called by serializeThread
+  private void switchWorkingBufferToFlushing() {
+    buffersLock.lock();
+    try {
+      while (idleBuffer == null) {
+        idleBufferReadyCondition.await();
+      }
+      syncingBuffer = workingBuffer;
+      workingBuffer = idleBuffer;
+      workingBuffer.clear();
+      idleBuffer = null;
+    } catch (InterruptedException e) {
+      logger.warn("Interrupted When waiting for available working buffer.");
+      Thread.currentThread().interrupt();
+    } finally {
+      buffersLock.unlock();
+    }
+  }
+  // endregion
+
+  // region Task of syncBufferThread
+  /**
+   * This task syncs syncingBuffer to disk. The precondition is that syncingBuffer cannot be null.
+   */
+  private class SyncBufferTask implements Runnable {
+    private final boolean force;
+    private final boolean rollWAlFileWriter;
+    private final List<WALFlushListener> fsyncListeners;
+
+    public SyncBufferTask(boolean force) {
+      this(force, false, Collections.emptyList());
+    }
+
+    public SyncBufferTask(
+        boolean force, boolean rollWAlFileWriter, List<WALFlushListener> fsyncListeners) {
+      this.force = force;
+      this.rollWAlFileWriter = rollWAlFileWriter;
+      this.fsyncListeners = fsyncListeners == null ? Collections.emptyList() : fsyncListeners;
+    }
+
+    @Override
+    public void run() {
+      // flush buffer to os
+      try {
+        currentWALFileWriter.write(syncingBuffer);
+      } catch (Throwable e) {
+        logger.error(
+            "Fail to sync wal node-{}'s buffer, change system mode to read-only.", identifier, e);
+        config.setReadOnly(true);
+      } finally {
+        switchSyncingBufferToIdle();
+      }
+
+      // force os cache to the storage device
+      if (force) {
+        try {
+          currentWALFileWriter.force();
+        } catch (IOException e) {
+          logger.error(
+              "Fail to fsync wal node-{}'s log writer, change system mode to read-only.",
+              identifier,
+              e);
+          for (WALFlushListener fsyncListener : fsyncListeners) {
+            fsyncListener.fail(e);
+          }
+          config.setReadOnly(true);
+        }
+        for (WALFlushListener fsyncListener : fsyncListeners) {
+          fsyncListener.succeed();
+        }
+      }
+
+      // try to roll log writer
+      try {
+        if (rollWAlFileWriter || (force && currentWALFileWriter.size() >= FILE_SIZE_THRESHOLD)) {
+          rollLogWriter();
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Fail to roll wal node-{}'s log writer, change system mode to read-only.",
+            identifier,
+            e);
+        config.setReadOnly(true);
+      }
+    }
+  }
+
+  // only called by syncBufferThread
+  private void switchSyncingBufferToIdle() {
+    buffersLock.lock();
+    try {
+      // No need to judge whether idleBuffer is null because syncingBuffer is not null
+      // and there is only one buffer can be null between syncingBuffer and idleBuffer
+      idleBuffer = syncingBuffer;
+      syncingBuffer = null;
+      idleBufferReadyCondition.signal();
+    } finally {
+      buffersLock.unlock();
+    }
+  }
+  // endregion
+
+  @Override
+  public void close() {
+    isClosed = true;
+    // first waiting serialize and sync tasks finished, then release all resources
+    if (serializeThread != null) {
+      // add close signal WALEntry to notify serializeThread
+      walEntries.add(new SignalWALEntry(SignalWALEntry.SignalType.CLOSE_SIGNAL));
+      shutdownThread(serializeThread, ThreadName.WAL_SERIALIZE);
+    }
+    if (syncBufferThread != null) {
+      shutdownThread(syncBufferThread, ThreadName.WAL_SYNC);
+    }
+
+    if (currentWALFileWriter != null) {
+      try {
+        currentWALFileWriter.close();
+      } catch (IOException e) {
+        logger.error("Fail to close wal node-{}'s log writer.", identifier, e);
+      }
+    }
+
+    if (workingBuffer != null) {
+      MmapUtil.clean((MappedByteBuffer) workingBuffer);
+    }
+    if (idleBuffer != null) {
+      MmapUtil.clean((MappedByteBuffer) workingBuffer);
+    }
+    if (syncingBuffer != null) {
+      MmapUtil.clean((MappedByteBuffer) syncingBuffer);
+    }
+  }
+
+  private void shutdownThread(ExecutorService thread, ThreadName threadName) {
+    thread.shutdown();
+    try {
+      if (!thread.awaitTermination(30, TimeUnit.SECONDS)) {
+        logger.warn("Waiting thread {} to be terminated is timeout", threadName.getName());
+      }
+    } catch (InterruptedException e) {
+      logger.warn("Thread {} still doesn't exit after 30s", threadName.getName());
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  @Override
+  public boolean isAllWALEntriesConsumed() {
+    return walEntries.isEmpty();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntry.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntry.java
new file mode 100644
index 0000000000..705916eeae
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntry.java
@@ -0,0 +1,219 @@
+/*
+ * 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.wal.buffer;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.memtable.AbstractMemTable;
+import org.apache.iotdb.db.engine.memtable.IMemTable;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertRowNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertTabletNode;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.utils.SerializedSize;
+import org.apache.iotdb.db.wal.utils.WALMode;
+import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * WALEntry is the basic element of .wal file, including type, memTable id, and specific
+ * value(physical plan or memTable snapshot).
+ */
+public class WALEntry implements SerializedSize {
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  /** wal entry type 1 byte, memTable id 4 bytes */
+  private static final int FIXED_SERIALIZED_SIZE = Byte.BYTES + Integer.BYTES;
+
+  /** type of value */
+  private final WALEntryType type;
+  /** memTable id */
+  private final int memTableId;
+  /** value(physical plan or memTable snapshot) */
+  private final WALEntryValue value;
+  /** extra info for InsertTabletPlan type value */
+  private TabletInfo tabletInfo;
+
+  /**
+   * listen whether this WALEntry has been written to the filesystem, null iff this WALEntry is
+   * deserialized from .wal file
+   */
+  private final WALFlushListener walFlushListener;
+
+  public WALEntry(int memTableId, WALEntryValue value) {
+    this(memTableId, value, config.getWalMode() == WALMode.SYNC);
+    if (value instanceof InsertTabletPlan) {
+      tabletInfo = new TabletInfo(0, ((InsertTabletPlan) value).getRowCount());
+    } else if (value instanceof InsertTabletNode) {
+      tabletInfo = new TabletInfo(0, ((InsertTabletNode) value).getRowCount());
+    }
+  }
+
+  public WALEntry(int memTableId, InsertTabletPlan value, int tabletStart, int tabletEnd) {
+    this(memTableId, value, config.getWalMode() == WALMode.SYNC);
+    tabletInfo = new TabletInfo(tabletStart, tabletEnd);
+  }
+
+  public WALEntry(int memTableId, InsertTabletNode value, int tabletStart, int tabletEnd) {
+    this(memTableId, value, config.getWalMode() == WALMode.SYNC);
+    tabletInfo = new TabletInfo(tabletStart, tabletEnd);
+  }
+
+  public WALEntry(int memTableId, WALEntryValue value, boolean wait) {
+    this.memTableId = memTableId;
+    this.value = value;
+    if (value instanceof InsertRowPlan) {
+      this.type = WALEntryType.INSERT_ROW_PLAN;
+    } else if (value instanceof InsertTabletPlan) {
+      this.type = WALEntryType.INSERT_TABLET_PLAN;
+    } else if (value instanceof DeletePlan) {
+      this.type = WALEntryType.DELETE_PLAN;
+    } else if (value instanceof IMemTable) {
+      this.type = WALEntryType.MEMORY_TABLE_SNAPSHOT;
+    } else if (value instanceof InsertRowNode) {
+      this.type = WALEntryType.INSERT_ROW_NODE;
+    } else if (value instanceof InsertTabletNode) {
+      this.type = WALEntryType.INSERT_TABLET_NODE;
+    } else {
+      throw new RuntimeException("Unknown WALEntry type");
+    }
+    walFlushListener = new WALFlushListener(wait);
+  }
+
+  private WALEntry(WALEntryType type, int memTableId, WALEntryValue value) {
+    this.type = type;
+    this.memTableId = memTableId;
+    this.value = value;
+    this.walFlushListener = null;
+  }
+
+  @Override
+  public int serializedSize() {
+    return FIXED_SERIALIZED_SIZE + value.serializedSize();
+  }
+
+  public void serialize(IWALByteBufferView buffer) {
+    buffer.put(type.getCode());
+    buffer.putInt(memTableId);
+    switch (type) {
+      case INSERT_TABLET_PLAN:
+        ((InsertTabletPlan) value)
+            .serializeToWAL(buffer, tabletInfo.tabletStart, tabletInfo.tabletEnd);
+        break;
+      case INSERT_TABLET_NODE:
+        ((InsertTabletNode) value)
+            .serializeToWAL(buffer, tabletInfo.tabletStart, tabletInfo.tabletEnd);
+        break;
+      case INSERT_ROW_PLAN:
+      case INSERT_ROW_NODE:
+      case DELETE_PLAN:
+      case MEMORY_TABLE_SNAPSHOT:
+        value.serializeToWAL(buffer);
+        break;
+    }
+  }
+
+  public static WALEntry deserialize(DataInputStream stream)
+      throws IllegalPathException, IOException {
+    byte typeNum = stream.readByte();
+    WALEntryType type = WALEntryType.valueOf(typeNum);
+    if (type == null) {
+      throw new IOException("unrecognized wal entry type " + typeNum);
+    }
+
+    int memTableId = stream.readInt();
+    WALEntryValue value = null;
+    switch (type) {
+      case INSERT_ROW_PLAN:
+        value = (InsertRowPlan) PhysicalPlan.Factory.create(stream);
+        break;
+      case INSERT_TABLET_PLAN:
+        value = (InsertTabletPlan) PhysicalPlan.Factory.create(stream);
+        break;
+      case DELETE_PLAN:
+        value = (DeletePlan) PhysicalPlan.Factory.create(stream);
+        break;
+      case MEMORY_TABLE_SNAPSHOT:
+        value = AbstractMemTable.Factory.create(stream);
+        break;
+      case INSERT_ROW_NODE:
+        // TODO
+        break;
+      case INSERT_TABLET_NODE:
+        // TODO
+        break;
+    }
+    return new WALEntry(type, memTableId, value);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof WALEntry)) {
+      return false;
+    }
+    WALEntry other = (WALEntry) obj;
+    return this.type == other.type
+        && this.memTableId == other.memTableId
+        && Objects.equals(this.value, other.value);
+  }
+
+  public WALEntryType getType() {
+    return type;
+  }
+
+  public int getMemTableId() {
+    return memTableId;
+  }
+
+  public WALEntryValue getValue() {
+    return value;
+  }
+
+  public WALFlushListener getWalFlushListener() {
+    return walFlushListener;
+  }
+
+  public boolean isSignal() {
+    return false;
+  }
+
+  private static class TabletInfo {
+    /** start row of insert tablet */
+    private final int tabletStart;
+    /** end row of insert tablet */
+    private final int tabletEnd;
+
+    public TabletInfo(int tabletStart, int tabletEnd) {
+      this.tabletStart = tabletStart;
+      this.tabletEnd = tabletEnd;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java
new file mode 100644
index 0000000000..c3e108fd2d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java
@@ -0,0 +1,55 @@
+/*
+ * 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.wal.buffer;
+
+/** Type of {@link WALEntry} */
+public enum WALEntryType {
+  /** {@link org.apache.iotdb.db.qp.physical.crud.InsertRowPlan} */
+  INSERT_ROW_PLAN((byte) 0),
+  /** {@link org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan} */
+  INSERT_TABLET_PLAN((byte) 1),
+  /** {@link org.apache.iotdb.db.qp.physical.crud.DeletePlan} */
+  DELETE_PLAN((byte) 2),
+  /** snapshot of {@link org.apache.iotdb.db.engine.memtable.IMemTable} */
+  MEMORY_TABLE_SNAPSHOT((byte) 3),
+  /** {@link org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertRowNode} */
+  INSERT_ROW_NODE((byte) 4),
+  /** {@link org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertTabletNode} */
+  INSERT_TABLET_NODE((byte) 5),
+  ;
+
+  private final byte code;
+
+  WALEntryType(byte code) {
+    this.code = code;
+  }
+
+  public byte getCode() {
+    return code;
+  }
+
+  public static WALEntryType valueOf(byte code) {
+    for (WALEntryType type : WALEntryType.values()) {
+      if (type.code == code) {
+        return type;
+      }
+    }
+    return null;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryValue.java
similarity index 58%
copy from server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
copy to server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryValue.java
index 542eb6448f..95b3ddc20d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryValue.java
@@ -16,31 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.wal.buffer;
 
-package org.apache.iotdb.db.engine.flush;
+import org.apache.iotdb.db.utils.SerializedSize;
 
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-
-import java.io.IOException;
-
-public interface FlushListener {
-
-  void onFlushStart(IMemTable memTable) throws IOException;
-
-  void onFlushEnd(IMemTable memTable);
-
-  class EmptyListener implements FlushListener {
-
-    public static final EmptyListener INSTANCE = new EmptyListener();
-
-    @Override
-    public void onFlushStart(IMemTable memTable) {
-      // do nothing
-    }
-
-    @Override
-    public void onFlushEnd(IMemTable memTable) {
-      // do nothing
-    }
-  }
+/** A class implements this interface can be written into .wal file. */
+public interface WALEntryValue extends SerializedSize {
+  /**
+   * Serialize using {@link IWALByteBufferView}, which encapsulates some actions to deal with {@link
+   * java.nio.BufferOverflowException} occurs in {@link java.nio.ByteBuffer}.
+   */
+  void serializeToWAL(IWALByteBufferView buffer);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/Checkpoint.java b/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/Checkpoint.java
new file mode 100644
index 0000000000..b21f4e9eb0
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/Checkpoint.java
@@ -0,0 +1,102 @@
+/*
+ * 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.wal.checkpoint;
+
+import org.apache.iotdb.db.utils.SerializedSize;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Checkpoint is the basic element of .checkpoint file, including type, number of memTables, and
+ * brief information of each memTable.
+ */
+public class Checkpoint implements SerializedSize {
+  /** checkpoint type 1 byte, checkpoint number 4 bytes */
+  private static final int FIXED_SERIALIZED_SIZE = Byte.BYTES + Integer.BYTES;
+
+  /** checkpoint type */
+  private final CheckpointType type;
+  /** memTable information */
+  private final List<MemTableInfo> memTableInfos;
+
+  public Checkpoint(CheckpointType type, List<MemTableInfo> memTableInfos) {
+    this.type = type;
+    this.memTableInfos = memTableInfos;
+  }
+
+  @Override
+  public int serializedSize() {
+    int size = FIXED_SERIALIZED_SIZE;
+    for (MemTableInfo memTableInfo : memTableInfos) {
+      size += memTableInfo.serializedSize();
+    }
+    return size;
+  }
+
+  public void serialize(ByteBuffer buffer) {
+    buffer.put(type.getCode());
+    buffer.putInt(memTableInfos.size());
+    for (MemTableInfo memTableInfo : memTableInfos) {
+      memTableInfo.serialize(buffer);
+    }
+  }
+
+  public static Checkpoint deserialize(DataInputStream stream) throws IOException {
+    byte typeNum = stream.readByte();
+    CheckpointType type = CheckpointType.valueOf(typeNum);
+    if (type == null) {
+      throw new IOException("unrecognized checkpoint type " + typeNum);
+    }
+    int cnt = stream.readInt();
+    List<MemTableInfo> memTableInfos = new ArrayList<>(cnt);
+    for (int i = 0; i < cnt; ++i) {
+      MemTableInfo memTableInfo = MemTableInfo.deserialize(stream);
+      memTableInfos.add(memTableInfo);
+    }
+    return new Checkpoint(type, memTableInfos);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof Checkpoint)) {
+      return false;
+    }
+    Checkpoint other = (Checkpoint) obj;
+    return this.type == other.type && Objects.equals(this.memTableInfos, other.memTableInfos);
+  }
+
+  public CheckpointType getType() {
+    return type;
+  }
+
+  public List<MemTableInfo> getMemTableInfos() {
+    return memTableInfos;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/CheckpointManager.java b/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/CheckpointManager.java
new file mode 100644
index 0000000000..65740be655
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/CheckpointManager.java
@@ -0,0 +1,257 @@
+/*
+ * 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.wal.checkpoint;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.wal.io.CheckpointWriter;
+import org.apache.iotdb.db.wal.io.ILogWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/** This class is used to manage checkpoints of one wal node */
+public class CheckpointManager implements AutoCloseable {
+  /** use size limit to control WALEntry number in each file */
+  public static final long LOG_SIZE_LIMIT = 3 * 1024 * 1024;
+
+  private static final Logger logger = LoggerFactory.getLogger(CheckpointManager.class);
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  /** WALNode identifier of this checkpoint manager */
+  protected final String identifier;
+  /** directory to store .checkpoint file */
+  protected final String logDirectory;
+  /**
+   * protect concurrent safety of checkpoint info, including memTableId2Info, cachedByteBuffer,
+   * currentLogVersion and currentLogWriter
+   */
+  private final Lock infoLock = new ReentrantLock();
+  // region these variables should be protected by infoLock
+  /** memTable id -> memTable info */
+  private final Map<Integer, MemTableInfo> memTableId2Info = new HashMap<>();
+  /** cache the biggest byte buffer to serialize checkpoint */
+  private volatile ByteBuffer cachedByteBuffer;
+  /** current checkpoint file version id, only updated by fsyncAndDeleteThread */
+  private int currentCheckPointFileVersion = 0;
+  /** current checkpoint file log writer, only updated by fsyncAndDeleteThread */
+  private ILogWriter currentLogWriter;
+  // endregion
+
+  public CheckpointManager(String identifier, String logDirectory) throws FileNotFoundException {
+    this.identifier = identifier;
+    this.logDirectory = logDirectory;
+    File logDirFile = SystemFileFactory.INSTANCE.getFile(logDirectory);
+    if (!logDirFile.exists() && logDirFile.mkdirs()) {
+      logger.info("create folder {} for wal buffer-{}.", logDirectory, identifier);
+    }
+    currentLogWriter =
+        new CheckpointWriter(
+            SystemFileFactory.INSTANCE.getFile(
+                logDirectory, CheckpointWriter.getLogFileName(currentCheckPointFileVersion)));
+    makeGlobalInfoCP();
+  }
+
+  /**
+   * make checkpoint for global memTables' info, this checkpoint only exists in the beginning of
+   * each checkpoint file
+   */
+  private void makeGlobalInfoCP() {
+    infoLock.lock();
+    try {
+      Checkpoint checkpoint =
+          new Checkpoint(
+              CheckpointType.GLOBAL_MEMORY_TABLE_INFO, new ArrayList<>(memTableId2Info.values()));
+      logByCachedByteBuffer(checkpoint);
+    } finally {
+      infoLock.unlock();
+    }
+  }
+
+  /** make checkpoint for create memTable info */
+  public void makeCreateMemTableCP(MemTableInfo memTableInfo) {
+    infoLock.lock();
+    try {
+      memTableId2Info.put(memTableInfo.getMemTableId(), memTableInfo);
+      Checkpoint checkpoint =
+          new Checkpoint(
+              CheckpointType.CREATE_MEMORY_TABLE, Collections.singletonList(memTableInfo));
+      logByCachedByteBuffer(checkpoint);
+    } finally {
+      infoLock.unlock();
+    }
+  }
+
+  /** make checkpoint for flush memTable info */
+  public void makeFlushMemTableCP(int memTableId) {
+    infoLock.lock();
+    try {
+      MemTableInfo memTableInfo = memTableId2Info.remove(memTableId);
+      if (memTableInfo == null) {
+        return;
+      }
+      Checkpoint checkpoint =
+          new Checkpoint(
+              CheckpointType.FLUSH_MEMORY_TABLE, Collections.singletonList(memTableInfo));
+      logByCachedByteBuffer(checkpoint);
+    } finally {
+      infoLock.unlock();
+    }
+  }
+
+  private void logByCachedByteBuffer(Checkpoint checkpoint) {
+    // make sure cached ByteBuffer has enough capacity
+    int estimateSize = checkpoint.serializedSize();
+    if (cachedByteBuffer == null || estimateSize > cachedByteBuffer.capacity()) {
+      cachedByteBuffer = ByteBuffer.allocate(estimateSize);
+    }
+    checkpoint.serialize(cachedByteBuffer);
+
+    try {
+      currentLogWriter.write(cachedByteBuffer);
+    } catch (IOException e) {
+      logger.error("Fail to make checkpoint: {}", checkpoint, e);
+    } finally {
+      cachedByteBuffer.clear();
+    }
+
+    fsyncCheckpointFile();
+  }
+
+  // region Task to fsync checkpoint file
+  /** Fsync checkpoints to the disk */
+  private void fsyncCheckpointFile() {
+    infoLock.lock();
+    try {
+      try {
+        currentLogWriter.force();
+      } catch (IOException e) {
+        logger.error(
+            "Fail to fsync wal node-{}'s checkpoint writer, change system mode to read-only.",
+            identifier,
+            e);
+        config.setReadOnly(true);
+      }
+
+      try {
+        if (tryRollingLogWriter()) {
+          // first log global memTables' info, then delete old checkpoint file
+          makeGlobalInfoCP();
+          currentLogWriter.force();
+          File oldFile =
+              SystemFileFactory.INSTANCE.getFile(
+                  logDirectory, CheckpointWriter.getLogFileName(currentCheckPointFileVersion - 1));
+          oldFile.delete();
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Fail to roll wal node-{}'s checkpoint writer, change system mode to read-only.",
+            identifier,
+            e);
+        config.setReadOnly(true);
+      }
+    } finally {
+      infoLock.unlock();
+    }
+  }
+
+  private boolean tryRollingLogWriter() throws IOException {
+    if (currentLogWriter.size() < LOG_SIZE_LIMIT) {
+      return false;
+    }
+    currentLogWriter.close();
+    currentCheckPointFileVersion++;
+    File nextLogFile =
+        SystemFileFactory.INSTANCE.getFile(
+            logDirectory, CheckpointWriter.getLogFileName(currentCheckPointFileVersion));
+    currentLogWriter = new CheckpointWriter(nextLogFile);
+    return true;
+  }
+  // endregion
+
+  public MemTableInfo getOldestMemTableInfo() {
+    // find oldest memTable
+    List<MemTableInfo> memTableInfos;
+    infoLock.lock();
+    try {
+      memTableInfos = new ArrayList<>(memTableId2Info.values());
+    } finally {
+      infoLock.unlock();
+    }
+    if (memTableInfos.isEmpty()) {
+      return null;
+    }
+    MemTableInfo oldestMemTableInfo = memTableInfos.get(0);
+    for (MemTableInfo memTableInfo : memTableInfos) {
+      if (oldestMemTableInfo.getFirstFileVersionId() > memTableInfo.getFirstFileVersionId()) {
+        oldestMemTableInfo = memTableInfo;
+      }
+    }
+    return oldestMemTableInfo;
+  }
+
+  /**
+   * Get version id of first valid .wal file
+   *
+   * @return Return {@link Integer#MIN_VALUE} if no file is valid
+   */
+  public int getFirstValidWALVersionId() {
+    List<MemTableInfo> memTableInfos;
+    infoLock.lock();
+    try {
+      memTableInfos = new ArrayList<>(memTableId2Info.values());
+    } finally {
+      infoLock.unlock();
+    }
+    int firstValidVersionId = memTableInfos.isEmpty() ? Integer.MIN_VALUE : Integer.MAX_VALUE;
+    for (MemTableInfo memTableInfo : memTableInfos) {
+      firstValidVersionId = Math.min(firstValidVersionId, memTableInfo.getFirstFileVersionId());
+    }
+    return firstValidVersionId;
+  }
+
+  @Override
+  public void close() {
+    infoLock.lock();
+    try {
+      if (currentLogWriter != null) {
+        try {
+          currentLogWriter.close();
+        } catch (IOException e) {
+          logger.error("Fail to close wal node-{}'s checkpoint writer.", identifier, e);
+        }
+      }
+    } finally {
+      infoLock.unlock();
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java b/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/CheckpointType.java
similarity index 55%
copy from server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
copy to server/src/main/java/org/apache/iotdb/db/wal/checkpoint/CheckpointType.java
index 6dcb92f64c..a73aa03b8d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/CheckpointType.java
@@ -16,32 +16,34 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.engine.cq;
-
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.io.SingleFileLogReader;
-
-import java.io.File;
-import java.io.IOException;
-
-public class CQLogReader implements AutoCloseable {
-
-  private final SingleFileLogReader logReader;
-
-  public CQLogReader(File logFile) throws IOException {
-    logReader = new SingleFileLogReader(logFile);
-  }
-
-  public boolean hasNext() {
-    return !logReader.isFileCorrupted() && logReader.hasNext();
+package org.apache.iotdb.db.wal.checkpoint;
+
+/** Type of {@link Checkpoint} */
+public enum CheckpointType {
+  /** record all existing memtables' info */
+  GLOBAL_MEMORY_TABLE_INFO((byte) 0),
+  /** record create info of one memtable */
+  CREATE_MEMORY_TABLE((byte) 1),
+  /** record flush info of one memtable */
+  FLUSH_MEMORY_TABLE((byte) 2),
+  ;
+
+  private final byte code;
+
+  CheckpointType(byte code) {
+    this.code = code;
   }
 
-  public PhysicalPlan next() {
-    return logReader.next();
+  public byte getCode() {
+    return code;
   }
 
-  @Override
-  public void close() {
-    logReader.close();
+  public static CheckpointType valueOf(byte code) {
+    for (CheckpointType type : CheckpointType.values()) {
+      if (type.code == code) {
+        return type;
+      }
+    }
+    return null;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/MemTableInfo.java b/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/MemTableInfo.java
new file mode 100644
index 0000000000..6e73333034
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/checkpoint/MemTableInfo.java
@@ -0,0 +1,113 @@
+/*
+ * 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.wal.checkpoint;
+
+import org.apache.iotdb.db.engine.memtable.IMemTable;
+import org.apache.iotdb.db.utils.SerializedSize;
+import org.apache.iotdb.db.wal.buffer.WALEntry;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+/**
+ * MemTableInfo records brief info of one memtable, including memTable id, tsFile path, and .wal
+ * file version id of its first {@link WALEntry}.
+ */
+public class MemTableInfo implements SerializedSize {
+  /** memTable id 4 bytes, first version id 4 bytes */
+  private static final int FIXED_SERIALIZED_SIZE = Integer.BYTES * 2;
+
+  /** memTable */
+  private IMemTable memTable;
+  /** memTable id */
+  private int memTableId;
+  /** path of the tsFile which this memTable will be flushed to */
+  private String tsFilePath;
+  /** version id of the file where this memTable's first WALEntry is located */
+  private volatile int firstFileVersionId;
+
+  private MemTableInfo() {}
+
+  public MemTableInfo(IMemTable memTable, String tsFilePath, int firstFileVersionId) {
+    this.memTable = memTable;
+    this.memTableId = memTable.getMemTableId();
+    this.tsFilePath = tsFilePath;
+    this.firstFileVersionId = firstFileVersionId;
+  }
+
+  @Override
+  public int serializedSize() {
+    return FIXED_SERIALIZED_SIZE + ReadWriteIOUtils.sizeToWrite(tsFilePath);
+  }
+
+  public void serialize(ByteBuffer buffer) {
+    buffer.putInt(memTableId);
+    ReadWriteIOUtils.write(tsFilePath, buffer);
+    buffer.putInt(firstFileVersionId);
+  }
+
+  public static MemTableInfo deserialize(DataInputStream stream) throws IOException {
+    MemTableInfo memTableInfo = new MemTableInfo();
+    memTableInfo.memTableId = stream.readInt();
+    memTableInfo.tsFilePath = ReadWriteIOUtils.readString(stream);
+    memTableInfo.firstFileVersionId = stream.readInt();
+    return memTableInfo;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof MemTableInfo)) {
+      return false;
+    }
+    MemTableInfo other = (MemTableInfo) obj;
+    return this.memTableId == other.memTableId
+        && Objects.equals(this.tsFilePath, other.tsFilePath)
+        && this.firstFileVersionId == other.firstFileVersionId;
+  }
+
+  public IMemTable getMemTable() {
+    return memTable;
+  }
+
+  public int getMemTableId() {
+    return memTableId;
+  }
+
+  public String getTsFilePath() {
+    return tsFilePath;
+  }
+
+  public int getFirstFileVersionId() {
+    return firstFileVersionId;
+  }
+
+  /** */
+  public void setFirstFileVersionId(int firstFileVersionId) {
+    this.firstFileVersionId = firstFileVersionId;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java b/server/src/main/java/org/apache/iotdb/db/wal/exception/WALException.java
similarity index 56%
copy from server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
copy to server/src/main/java/org/apache/iotdb/db/wal/exception/WALException.java
index 6dcb92f64c..8f2ccebf21 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cq/CQLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/exception/WALException.java
@@ -16,32 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.engine.cq;
+package org.apache.iotdb.db.wal.exception;
 
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.io.SingleFileLogReader;
+import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.rpc.TSStatusCode;
 
-import java.io.File;
-import java.io.IOException;
-
-public class CQLogReader implements AutoCloseable {
-
-  private final SingleFileLogReader logReader;
-
-  public CQLogReader(File logFile) throws IOException {
-    logReader = new SingleFileLogReader(logFile);
-  }
-
-  public boolean hasNext() {
-    return !logReader.isFileCorrupted() && logReader.hasNext();
+public class WALException extends IoTDBException {
+  public WALException(Throwable cause) {
+    super(cause, TSStatusCode.WRITE_AHEAD_LOG_ERROR.getStatusCode());
+    this.initCause(cause);
   }
 
-  public PhysicalPlan next() {
-    return logReader.next();
+  public WALException(String msg) {
+    super(msg, TSStatusCode.WRITE_AHEAD_LOG_ERROR.getStatusCode());
   }
 
-  @Override
-  public void close() {
-    logReader.close();
+  public WALException(String message, Throwable cause) {
+    super(message + cause.getMessage(), TSStatusCode.WRITE_AHEAD_LOG_ERROR.getStatusCode());
+    this.initCause(cause);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java b/server/src/main/java/org/apache/iotdb/db/wal/exception/WALNodeClosedException.java
similarity index 59%
copy from server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
copy to server/src/main/java/org/apache/iotdb/db/wal/exception/WALNodeClosedException.java
index 542eb6448f..bd9a4323fe 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/exception/WALNodeClosedException.java
@@ -16,31 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.wal.exception;
 
-package org.apache.iotdb.db.engine.flush;
-
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-
-import java.io.IOException;
-
-public interface FlushListener {
-
-  void onFlushStart(IMemTable memTable) throws IOException;
-
-  void onFlushEnd(IMemTable memTable);
-
-  class EmptyListener implements FlushListener {
-
-    public static final EmptyListener INSTANCE = new EmptyListener();
-
-    @Override
-    public void onFlushStart(IMemTable memTable) {
-      // do nothing
-    }
-
-    @Override
-    public void onFlushEnd(IMemTable memTable) {
-      // do nothing
-    }
+public class WALNodeClosedException extends WALException {
+  public WALNodeClosedException(String nodeIdentifier) {
+    super(String.format("wal node-%s has been closed", nodeIdentifier));
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java b/server/src/main/java/org/apache/iotdb/db/wal/exception/WALRecoverException.java
similarity index 59%
copy from server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
copy to server/src/main/java/org/apache/iotdb/db/wal/exception/WALRecoverException.java
index 542eb6448f..ce836315ef 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/FlushListener.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/exception/WALRecoverException.java
@@ -16,31 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.wal.exception;
 
-package org.apache.iotdb.db.engine.flush;
-
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-
-import java.io.IOException;
-
-public interface FlushListener {
-
-  void onFlushStart(IMemTable memTable) throws IOException;
-
-  void onFlushEnd(IMemTable memTable);
-
-  class EmptyListener implements FlushListener {
-
-    public static final EmptyListener INSTANCE = new EmptyListener();
-
-    @Override
-    public void onFlushStart(IMemTable memTable) {
-      // do nothing
-    }
+public class WALRecoverException extends WALException {
+  public WALRecoverException(Throwable cause) {
+    super(cause);
+  }
 
-    @Override
-    public void onFlushEnd(IMemTable memTable) {
-      // do nothing
-    }
+  public WALRecoverException(String message, Throwable cause) {
+    super(message, cause);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/io/CheckpointReader.java b/server/src/main/java/org/apache/iotdb/db/wal/io/CheckpointReader.java
new file mode 100644
index 0000000000..b2cc91f687
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/io/CheckpointReader.java
@@ -0,0 +1,63 @@
+/*
+ * 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.wal.io;
+
+import org.apache.iotdb.db.wal.checkpoint.Checkpoint;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+/** CheckpointReader is used to read all checkpoints from .checkpoint file. */
+public class CheckpointReader {
+  private static final Logger logger = LoggerFactory.getLogger(CheckpointReader.class);
+
+  private final File logFile;
+
+  public CheckpointReader(File logFile) {
+    this.logFile = logFile;
+  }
+
+  /**
+   * Read all checkpoints from .checkpoint file.
+   *
+   * @return checkpoints
+   */
+  public List<Checkpoint> readAll() {
+    List<Checkpoint> checkpoints = new LinkedList<>();
+    try (DataInputStream logStream =
+        new DataInputStream(new BufferedInputStream(new FileInputStream(logFile)))) {
+      while (logStream.available() > 0) {
+        Checkpoint checkpoint = Checkpoint.deserialize(logStream);
+        checkpoints.add(checkpoint);
+      }
+    } catch (IOException e) {
+      logger.warn(
+          "Meet error when reading checkpoint file {}, skip broken checkpoints", logFile, e);
+    }
+    return checkpoints;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/io/CheckpointWriter.java b/server/src/main/java/org/apache/iotdb/db/wal/io/CheckpointWriter.java
new file mode 100644
index 0000000000..8fa34c0e6a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/io/CheckpointWriter.java
@@ -0,0 +1,61 @@
+/*
+ * 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.wal.io;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.db.wal.checkpoint.Checkpoint;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/** CheckpointWriter writes the binary {@link Checkpoint} into .checkpoint file. */
+public class CheckpointWriter extends LogWriter {
+  public static final String FILE_SUFFIX = IoTDBConstant.WAL_CHECKPOINT_FILE_SUFFIX;
+  public static final Pattern CHECKPOINT_FILE_NAME_PATTERN =
+      Pattern.compile("_(?<versionId>\\d+)\\.checkpoint");
+
+  /** Return true when this file is .checkpoint file */
+  public static boolean checkpointFilenameFilter(File dir, String name) {
+    return CHECKPOINT_FILE_NAME_PATTERN.matcher(name).find();
+  }
+
+  /**
+   * Parse version id from filename
+   *
+   * @return Return {@link Integer#MIN_VALUE} when this file is not .checkpoint file
+   */
+  public static int parseVersionId(String filename) {
+    Matcher matcher = CHECKPOINT_FILE_NAME_PATTERN.matcher(filename);
+    if (matcher.find()) {
+      return Integer.parseInt(matcher.group("versionId"));
+    }
+    return Integer.MIN_VALUE;
+  }
+
+  /** Get .checkpoint filename */
+  public static String getLogFileName(long version) {
+    return FILE_PREFIX + version + FILE_SUFFIX;
+  }
+
+  public CheckpointWriter(File logFile) throws FileNotFoundException {
+    super(logFile);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/io/ILogWriter.java b/server/src/main/java/org/apache/iotdb/db/wal/io/ILogWriter.java
new file mode 100644
index 0000000000..9d26c17fa4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/io/ILogWriter.java
@@ -0,0 +1,61 @@
+/*
+ * 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.wal.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public interface ILogWriter extends Closeable {
+  /**
+   * Write given logs to a persistent medium. NOTICE: the logs may be cached in the storage device,
+   * if the storage device you are using do not guarantee strong persistence, and you want the logs
+   * to be persisted immediately, please call {@link #force()} after calling this method. Notice: do
+   * not flip the buffer before calling this method
+   *
+   * @param buffer content that have been converted to bytes
+   * @throws IOException if an I/O error occurs
+   */
+  void write(ByteBuffer buffer) throws IOException;
+
+  /**
+   * Forces any updates to this file to be written to the storage device that contains it.
+   *
+   * @throws IOException if an I/O error occurs
+   */
+  void force() throws IOException;
+
+  /**
+   * Forces any updates to this file to be written to the storage device that contains it.
+   *
+   * @param metaData If <tt>true</tt> then this method is required to force changes to both the
+   *     file's content and metadata to be written to storage; otherwise, it needs only force
+   *     content changes to be written
+   * @throws IOException if an I/O error occurs
+   */
+  void force(boolean metaData) throws IOException;
+
+  /**
+   * Returns the current size of this file.
+   *
+   * @return size
+   * @throws IOException if an I/O error occurs
+   */
+  long size() throws IOException;
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/io/LogWriter.java b/server/src/main/java/org/apache/iotdb/db/wal/io/LogWriter.java
new file mode 100644
index 0000000000..2b4c7c8cb8
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/io/LogWriter.java
@@ -0,0 +1,96 @@
+/*
+ * 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.wal.io;
+
+import org.apache.iotdb.db.wal.buffer.WALEntry;
+import org.apache.iotdb.db.wal.checkpoint.Checkpoint;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.FileChannel;
+
+/**
+ * LogWriter writes the binary logs into a file, including writing {@link WALEntry} into .wal file
+ * and writing {@link Checkpoint} into .checkpoint file.
+ */
+public abstract class LogWriter implements ILogWriter {
+  public static final String FILE_PREFIX = "_";
+  private static final Logger logger = LoggerFactory.getLogger(LogWriter.class);
+
+  private final File logFile;
+  private final FileOutputStream logStream;
+  private final FileChannel logChannel;
+
+  private long size;
+
+  public LogWriter(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    this.logStream = new FileOutputStream(logFile, true);
+    this.logChannel = this.logStream.getChannel();
+  }
+
+  @Override
+  public void write(ByteBuffer buffer) throws IOException {
+    size += buffer.position();
+    buffer.flip();
+    try {
+      logChannel.write(buffer);
+    } catch (ClosedChannelException e) {
+      logger.warn("Cannot write to {}", logFile, e);
+    }
+  }
+
+  @Override
+  public void force() throws IOException {
+    force(true);
+  }
+
+  @Override
+  public void force(boolean metaData) throws IOException {
+    if (logChannel != null && logChannel.isOpen()) {
+      logChannel.force(metaData);
+    }
+  }
+
+  @Override
+  public long size() throws IOException {
+    return size;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (logChannel != null) {
+      try {
+        if (logChannel.isOpen()) {
+          logChannel.force(true);
+        }
+      } finally {
+        logChannel.close();
+        logStream.close();
+      }
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/io/WALReader.java b/server/src/main/java/org/apache/iotdb/db/wal/io/WALReader.java
new file mode 100644
index 0000000000..a229a8a616
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/io/WALReader.java
@@ -0,0 +1,114 @@
+/*
+ * 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.wal.io;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.wal.buffer.WALEntry;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+/**
+ * The usage of WALReader is like {@link Iterator}, which aims to control the memory usage of
+ * reader.
+ */
+public class WALReader implements Closeable {
+  private static final Logger logger = LoggerFactory.getLogger(WALReader.class);
+  /** 1/10 of .wal file size as buffer size */
+  private static final int STREAM_BUFFER_SIZE =
+      (int) IoTDBDescriptor.getInstance().getConfig().getWalFileSizeThresholdInByte() / 10;
+  /** 1000 as default batch limit */
+  private static final int BATCH_LIMIT = 1_000;
+
+  private final File logFile;
+  private final DataInputStream logStream;
+  private final List<WALEntry> walEntries;
+
+  private Iterator<WALEntry> itr = null;
+  private boolean fileCorrupted = false;
+
+  public WALReader(File logFile) throws FileNotFoundException {
+    this.logFile = logFile;
+    this.logStream =
+        new DataInputStream(
+            new BufferedInputStream(new FileInputStream(logFile), STREAM_BUFFER_SIZE));
+    this.walEntries = new LinkedList<>();
+  }
+
+  /** Like {@link Iterator#hasNext()} */
+  public boolean hasNext() {
+    if (itr != null && itr.hasNext()) {
+      return true;
+    }
+    // read WALEntries from log stream
+    try {
+      if (fileCorrupted) {
+        return false;
+      }
+      walEntries.clear();
+      while (walEntries.size() < BATCH_LIMIT) {
+        WALEntry walEntry = WALEntry.deserialize(logStream);
+        walEntries.add(walEntry);
+      }
+    } catch (EOFException e) {
+      // reach end of wal file
+      fileCorrupted = true;
+    } catch (IllegalPathException e) {
+      fileCorrupted = true;
+      logger.warn(
+          "WALEntry of wal file {} contains illegal path, skip illegal WALEntries.", logFile, e);
+    } catch (Exception e) {
+      fileCorrupted = true;
+      logger.warn("Fail to read WALEntry from wal file {}, skip broken WALEntries.", logFile, e);
+    }
+
+    if (walEntries.size() != 0) {
+      itr = walEntries.iterator();
+      return true;
+    }
+    return false;
+  }
+
+  /** Like {@link Iterator#next()} */
+  public WALEntry next() {
+    if (itr == null) {
+      throw new NoSuchElementException();
+    }
+    return itr.next();
+  }
+
+  @Override
+  public void close() throws IOException {
+    logStream.close();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/io/WALWriter.java b/server/src/main/java/org/apache/iotdb/db/wal/io/WALWriter.java
new file mode 100644
index 0000000000..34f620297f
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/io/WALWriter.java
@@ -0,0 +1,60 @@
+/*
+ * 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.wal.io;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.db.wal.buffer.WALEntry;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/** WALWriter writes the binary {@link WALEntry} into .wal file. */
+public class WALWriter extends LogWriter {
+  public static final String FILE_SUFFIX = IoTDBConstant.WAL_FILE_SUFFIX;
+  public static final Pattern WAL_FILE_NAME_PATTERN = Pattern.compile("_(?<versionId>\\d+)\\.wal");
+
+  /** Return true when this file is .wal file */
+  public static boolean walFilenameFilter(File dir, String name) {
+    return WAL_FILE_NAME_PATTERN.matcher(name).find();
+  }
+
+  /**
+   * Parse version id from filename
+   *
+   * @return Return {@link Integer#MIN_VALUE} when this file is not .wal file
+   */
+  public static int parseVersionId(String filename) {
+    Matcher matcher = WAL_FILE_NAME_PATTERN.matcher(filename);
+    if (matcher.find()) {
+      return Integer.parseInt(matcher.group("versionId"));
+    }
+    return Integer.MIN_VALUE;
+  }
+
+  /** Get .wal filename */
+  public static String getLogFileName(long version) {
+    return FILE_PREFIX + version + FILE_SUFFIX;
+  }
+
+  public WALWriter(File logFile) throws FileNotFoundException {
+    super(logFile);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/node/IWALNode.java b/server/src/main/java/org/apache/iotdb/db/wal/node/IWALNode.java
new file mode 100644
index 0000000000..a3134e2810
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/node/IWALNode.java
@@ -0,0 +1,52 @@
+/*
+ * 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.wal.node;
+
+import org.apache.iotdb.db.engine.flush.FlushListener;
+import org.apache.iotdb.db.engine.memtable.IMemTable;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertRowNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertTabletNode;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
+
+/** This interface provides uniform interface for writing wal and making checkpoints. */
+public interface IWALNode extends FlushListener, AutoCloseable {
+  /** Log InsertRowPlan */
+  WALFlushListener log(int memTableId, InsertRowPlan insertRowPlan);
+
+  /** Log InsertRowNode */
+  WALFlushListener log(int memTableId, InsertRowNode insertRowNode);
+
+  /** Log InsertTabletPlan */
+  WALFlushListener log(int memTableId, InsertTabletPlan insertTabletPlan, int start, int end);
+
+  /** Log InsertTabletNode */
+  WALFlushListener log(int memTableId, InsertTabletNode insertTabletNode, int start, int end);
+
+  /** Log DeletePlan */
+  WALFlushListener log(int memTableId, DeletePlan deletePlan);
+
+  /** Callback when memTable created */
+  void onMemTableCreated(IMemTable memTable, String targetTsFile);
+
+  @Override
+  void close();
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/node/WALFakeNode.java b/server/src/main/java/org/apache/iotdb/db/wal/node/WALFakeNode.java
new file mode 100644
index 0000000000..f220769da8
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/node/WALFakeNode.java
@@ -0,0 +1,118 @@
+/*
+ * 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.wal.node;
+
+import org.apache.iotdb.db.engine.memtable.IMemTable;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertRowNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertTabletNode;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.wal.exception.WALException;
+import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
+
+/** This class provides fake wal node when wal is disabled or exception happens. */
+public class WALFakeNode implements IWALNode {
+  private final WALFlushListener.Status status;
+  private final Exception cause;
+
+  private WALFakeNode(WALFlushListener.Status status) {
+    this(status, null);
+  }
+
+  public WALFakeNode(WALFlushListener.Status status, Exception cause) {
+    this.status = status;
+    this.cause = cause;
+  }
+
+  @Override
+  public WALFlushListener log(int memTableId, InsertRowPlan insertRowPlan) {
+    return getResult();
+  }
+
+  @Override
+  public WALFlushListener log(int memTableId, InsertRowNode insertRowNode) {
+    return getResult();
+  }
+
+  @Override
+  public WALFlushListener log(
+      int memTableId, InsertTabletPlan insertTabletPlan, int start, int end) {
+    return getResult();
+  }
+
+  @Override
+  public WALFlushListener log(
+      int memTableId, InsertTabletNode insertTabletNode, int start, int end) {
+    return getResult();
+  }
+
+  @Override
+  public WALFlushListener log(int memTableId, DeletePlan deletePlan) {
+    return getResult();
+  }
+
+  private WALFlushListener getResult() {
+    WALFlushListener walFlushListener = new WALFlushListener(false);
+    switch (status) {
+      case SUCCESS:
+        walFlushListener.succeed();
+        break;
+      case FAILURE:
+        walFlushListener.fail(cause);
+        break;
+    }
+    return walFlushListener;
+  }
+
+  @Override
+  public void onMemTableFlushStarted(IMemTable memTable) {
+    // do nothing
+  }
+
+  @Override
+  public void onMemTableFlushed(IMemTable memTable) {
+    // do nothing
+  }
+
+  @Override
+  public void onMemTableCreated(IMemTable memTable, String targetTsFile) {
+    // do nothing
+  }
+
+  @Override
+  public void close() {
+    // do nothing
+  }
+
+  public static WALFakeNode getFailureInstance(Exception e) {
+    return new WALFakeNode(
+        WALFlushListener.Status.FAILURE,
+        new WALException("Cannot write wal into a fake node. ", e));
+  }
+
+  public static WALFakeNode getSuccessInstance() {
+    return WALFakeNodeHolder.SUCCESS_INSTANCE;
+  }
+
+  private static class WALFakeNodeHolder {
+    private static final WALFakeNode SUCCESS_INSTANCE =
+        new WALFakeNode(WALFlushListener.Status.SUCCESS);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/node/WALNode.java b/server/src/main/java/org/apache/iotdb/db/wal/node/WALNode.java
new file mode 100644
index 0000000000..33dcaa3bb4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/node/WALNode.java
@@ -0,0 +1,382 @@
+/*
+ * 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.wal.node;
+
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.flush.FlushStatus;
+import org.apache.iotdb.db.engine.memtable.IMemTable;
+import org.apache.iotdb.db.engine.storagegroup.VirtualStorageGroupProcessor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertRowNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertTabletNode;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.wal.buffer.IWALBuffer;
+import org.apache.iotdb.db.wal.buffer.SignalWALEntry;
+import org.apache.iotdb.db.wal.buffer.WALBuffer;
+import org.apache.iotdb.db.wal.buffer.WALEntry;
+import org.apache.iotdb.db.wal.checkpoint.CheckpointManager;
+import org.apache.iotdb.db.wal.checkpoint.MemTableInfo;
+import org.apache.iotdb.db.wal.io.WALWriter;
+import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.utils.TsFileUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/** This class encapsulates {@link IWALBuffer} and {@link CheckpointManager}. */
+public class WALNode implements IWALNode {
+  public static final Pattern WAL_NODE_FOLDER_PATTERN = Pattern.compile("(?<nodeIdentifier>\\d+)");
+
+  private static final Logger logger = LoggerFactory.getLogger(WALNode.class);
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private static final long WAL_FILE_TTL_IN_MS = config.getWalFileTTLInMs();
+  private static final long MEM_TABLE_SNAPSHOT_THRESHOLD_IN_BYTE =
+      config.getWalMemTableSnapshotThreshold();
+  private static final int MAX_WAL_MEM_TABLE_SNAPSHOT_NUM = config.getMaxWalMemTableSnapshotNum();
+
+  /** unique identifier of this WALNode */
+  private final String identifier;
+  /** directory to store this node's files */
+  private final String logDirectory;
+  /** wal buffer */
+  private final IWALBuffer buffer;
+  /** manage checkpoints */
+  private final CheckpointManager checkpointManager;
+  /**
+   * memTable id -> memTable snapshot count, used to avoid write amplification caused by frequent
+   * snapshot
+   */
+  private final Map<Integer, Integer> memTableSnapshotCount = new ConcurrentHashMap<>();
+
+  public WALNode(String identifier, String logDirectory) throws FileNotFoundException {
+    this.identifier = identifier;
+    this.logDirectory = logDirectory;
+    File logDirFile = SystemFileFactory.INSTANCE.getFile(logDirectory);
+    if (!logDirFile.exists() && logDirFile.mkdirs()) {
+      logger.info("create folder {} for wal node-{}.", logDirectory, identifier);
+    }
+    this.buffer = new WALBuffer(identifier, logDirectory);
+    this.checkpointManager = new CheckpointManager(identifier, logDirectory);
+  }
+
+  /** Return true when this folder wal node folder */
+  public static boolean walNodeFolderNameFilter(File dir, String name) {
+    return WAL_NODE_FOLDER_PATTERN.matcher(name).find();
+  }
+
+  @Override
+  public WALFlushListener log(int memTableId, InsertRowPlan insertRowPlan) {
+    WALEntry walEntry = new WALEntry(memTableId, insertRowPlan);
+    return log(walEntry);
+  }
+
+  @Override
+  public WALFlushListener log(int memTableId, InsertRowNode insertRowNode) {
+    WALEntry walEntry = new WALEntry(memTableId, insertRowNode);
+    return log(walEntry);
+  }
+
+  @Override
+  public WALFlushListener log(
+      int memTableId, InsertTabletPlan insertTabletPlan, int start, int end) {
+    WALEntry walEntry = new WALEntry(memTableId, insertTabletPlan, start, end);
+    return log(walEntry);
+  }
+
+  @Override
+  public WALFlushListener log(
+      int memTableId, InsertTabletNode insertTabletNode, int start, int end) {
+    WALEntry walEntry = new WALEntry(memTableId, insertTabletNode, start, end);
+    return log(walEntry);
+  }
+
+  @Override
+  public WALFlushListener log(int memTableId, DeletePlan deletePlan) {
+    WALEntry walEntry = new WALEntry(memTableId, deletePlan);
+    return log(walEntry);
+  }
+
+  private WALFlushListener log(WALEntry walEntry) {
+    buffer.write(walEntry);
+    return walEntry.getWalFlushListener();
+  }
+
+  @Override
+  public void onMemTableFlushStarted(IMemTable memTable) {
+    // do nothing
+  }
+
+  @Override
+  public void onMemTableFlushed(IMemTable memTable) {
+    if (memTable.isSignalMemTable()) {
+      return;
+    }
+    memTableSnapshotCount.remove(memTable.getMemTableId());
+    checkpointManager.makeFlushMemTableCP(memTable.getMemTableId());
+  }
+
+  @Override
+  public void onMemTableCreated(IMemTable memTable, String targetTsFile) {
+    if (memTable.isSignalMemTable()) {
+      return;
+    }
+    // use current log version id as first file version id
+    int firstFileVersionId = buffer.getCurrentWALFileVersion();
+    MemTableInfo memTableInfo = new MemTableInfo(memTable, targetTsFile, firstFileVersionId);
+    checkpointManager.makeCreateMemTableCP(memTableInfo);
+  }
+
+  // region Task to delete outdated .wal files
+  /** Delete outdated .wal files */
+  public void deleteOutdatedFiles() {
+    try {
+      new DeleteOutdatedFileTask().run();
+    } catch (Exception e) {
+      logger.error("Fail to delete wal node-{}'s outdated files.", identifier, e);
+    }
+  }
+
+  private class DeleteOutdatedFileTask implements Runnable {
+    /** .wal files whose version ids are less than first valid version id should be deleted */
+    private int firstValidVersionId;
+
+    @Override
+    public void run() {
+      // init firstValidVersionId
+      firstValidVersionId = checkpointManager.getFirstValidWALVersionId();
+      if (firstValidVersionId == Integer.MIN_VALUE) {
+        // roll wal log writer to delete current wal file
+        WALEntry rollWALFileSignal =
+            new SignalWALEntry(SignalWALEntry.SignalType.ROLL_WAL_LOG_WRITER_SIGNAL, true);
+        WALFlushListener walFlushListener = log(rollWALFileSignal);
+        if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
+          logger.error(
+              "Fail to trigger rolling wal node-{}'s wal file log writer.",
+              identifier,
+              walFlushListener.getCause());
+        }
+        // update firstValidVersionId
+        firstValidVersionId = checkpointManager.getFirstValidWALVersionId();
+        if (firstValidVersionId == Integer.MIN_VALUE) {
+          firstValidVersionId = buffer.getCurrentWALFileVersion();
+        }
+      }
+
+      // delete outdated files
+      File[] filesToDelete = deleteOutdatedFiles();
+
+      // exceed time limit, update first valid version id by snapshotting or flushing memTable,
+      // then delete old .wal files again
+      if (filesToDelete != null && filesToDelete.length == 0) {
+        File firstWALFile =
+            SystemFileFactory.INSTANCE.getFile(
+                logDirectory, WALWriter.getLogFileName(firstValidVersionId));
+        if (firstWALFile.exists()) {
+          long fileCreatedTime = Long.MAX_VALUE;
+          try {
+            fileCreatedTime =
+                Files.readAttributes(firstWALFile.toPath(), BasicFileAttributes.class)
+                    .creationTime()
+                    .toMillis();
+          } catch (IOException e) {
+            logger.warn("Fail to get creation time of wal file {}", firstWALFile, e);
+          }
+          long currentTime = System.currentTimeMillis();
+          if (fileCreatedTime + WAL_FILE_TTL_IN_MS < currentTime) {
+            snapshotOrFlushMemTable();
+            run();
+          }
+        }
+      }
+    }
+
+    private File[] deleteOutdatedFiles() {
+      File directory = SystemFileFactory.INSTANCE.getFile(logDirectory);
+      File[] filesToDelete = directory.listFiles(this::filterFilesToDelete);
+      if (filesToDelete != null) {
+        for (File file : filesToDelete) {
+          if (!file.delete()) {
+            logger.info("Fail to delete outdated wal file {} of wal node-{}.", file, identifier);
+          }
+        }
+      }
+      return filesToDelete;
+    }
+
+    private boolean filterFilesToDelete(File dir, String name) {
+      Pattern pattern = WALWriter.WAL_FILE_NAME_PATTERN;
+      Matcher matcher = pattern.matcher(name);
+      boolean toDelete = false;
+      if (matcher.find()) {
+        int versionId = Integer.parseInt(matcher.group("versionId"));
+        toDelete = versionId < firstValidVersionId;
+      }
+      return toDelete;
+    }
+
+    private void snapshotOrFlushMemTable() {
+      // find oldest memTable
+      MemTableInfo oldestMemTableInfo = checkpointManager.getOldestMemTableInfo();
+      if (oldestMemTableInfo == null) {
+        return;
+      }
+      IMemTable oldestMemTable = oldestMemTableInfo.getMemTable();
+
+      // get memTable's virtual storage group processor
+      File oldestTsFile =
+          FSFactoryProducer.getFSFactory().getFile(oldestMemTableInfo.getTsFilePath());
+      VirtualStorageGroupProcessor vsgProcessor;
+      try {
+        vsgProcessor =
+            StorageEngine.getInstance()
+                .getProcessorByVSGId(
+                    new PartialPath(TsFileUtils.getStorageGroup(oldestTsFile)),
+                    TsFileUtils.getVirtualStorageGroupId(oldestTsFile));
+      } catch (IllegalPathException | StorageEngineException e) {
+        logger.error("Fail to get virtual storage group processor for {}", oldestTsFile, e);
+        return;
+      }
+
+      // snapshot or flush memTable
+      int snapshotCount = memTableSnapshotCount.getOrDefault(oldestMemTable.getMemTableId(), 0);
+      if (snapshotCount >= MAX_WAL_MEM_TABLE_SNAPSHOT_NUM
+          || oldestMemTable.getTVListsRamCost() > MEM_TABLE_SNAPSHOT_THRESHOLD_IN_BYTE) {
+        flushMemTable(vsgProcessor, oldestTsFile, oldestMemTable);
+      } else {
+        snapshotMemTable(vsgProcessor, oldestTsFile, oldestMemTableInfo);
+      }
+    }
+
+    private void flushMemTable(
+        VirtualStorageGroupProcessor vsgProcessor, File tsFile, IMemTable memTable) {
+      boolean shouldWait = true;
+      if (memTable.getFlushStatus() == FlushStatus.WORKING) {
+        shouldWait =
+            vsgProcessor.submitAFlushTask(
+                TsFileUtils.getTimePartition(tsFile), TsFileUtils.isSequence(tsFile));
+        logger.info(
+            "WAL node-{} flushes memTable-{} to TsFile {}, memTable size is {}.",
+            identifier,
+            memTable.getMemTableId(),
+            tsFile,
+            memTable.getTVListsRamCost());
+      }
+
+      // it's fine to wait until memTable has been flushed, because deleting files is not urgent.
+      if (shouldWait) {
+        long sleepTime = 0;
+        while (memTable.getFlushStatus() != FlushStatus.FLUSHED) {
+          try {
+            Thread.sleep(1_000);
+            sleepTime += 1_000;
+            if (sleepTime > 10_000) {
+              logger.warn("Waiting too long for memTable flush to be done.");
+              break;
+            }
+          } catch (InterruptedException e) {
+            logger.warn("Interrupted when waiting for memTable flush to be done.");
+            Thread.currentThread().interrupt();
+          }
+        }
+      }
+    }
+
+    private void snapshotMemTable(
+        VirtualStorageGroupProcessor vsgProcessor, File tsFile, MemTableInfo memTableInfo) {
+      IMemTable memTable = memTableInfo.getMemTable();
+      if (memTable.getFlushStatus() != FlushStatus.WORKING) {
+        return;
+      }
+
+      // update snapshot count
+      memTableSnapshotCount.compute(memTable.getMemTableId(), (k, v) -> v == null ? 1 : v + 1);
+      // roll wal log writer to make sure first version id will be updated
+      WALEntry rollWALFileSignal =
+          new SignalWALEntry(SignalWALEntry.SignalType.ROLL_WAL_LOG_WRITER_SIGNAL, true);
+      WALFlushListener fileRolledListener = log(rollWALFileSignal);
+      if (fileRolledListener.waitForResult() == WALFlushListener.Status.FAILURE) {
+        logger.error("Fail to roll wal log writer.", fileRolledListener.getCause());
+        return;
+      }
+      logger.info("Version id is {}", memTableInfo.getFirstFileVersionId());
+      // update first version id first to make sure snapshot is in the files ≥ current log
+      // version
+      memTableInfo.setFirstFileVersionId(buffer.getCurrentWALFileVersion());
+      logger.info("Version id is {}", memTableInfo.getFirstFileVersionId());
+
+      // get vsg write lock to make sure no more writes to the memTable
+      vsgProcessor.writeLock(
+          "CheckpointManager$DeleteOutdatedFileTask.snapshotOrFlushOldestMemTable");
+      try {
+        // log snapshot in a new .wal file
+        WALEntry walEntry = new WALEntry(memTable.getMemTableId(), memTable, true);
+        WALFlushListener flushListener = log(walEntry);
+
+        // wait until getting the result
+        // it's low-risk to block writes awhile because this memTable accumulates slowly
+        if (flushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
+          logger.error("Fail to snapshot memTable of {}", tsFile, flushListener.getCause());
+        }
+        logger.info(
+            "WAL node-{} snapshots memTable-{} to wal files, memTable size is {}.",
+            identifier,
+            memTable.getMemTableId(),
+            memTable.getTVListsRamCost());
+      } finally {
+        vsgProcessor.writeUnlock();
+      }
+    }
+  }
+  // endregion
+
+  @Override
+  public void close() {
+    buffer.close();
+    checkpointManager.close();
+  }
+
+  @TestOnly
+  boolean isAllWALEntriesConsumed() {
+    return buffer.isAllWALEntriesConsumed();
+  }
+
+  @TestOnly
+  int getCurrentLogVersion() {
+    return buffer.getCurrentWALFileVersion();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/CheckpointRecoverUtils.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/CheckpointRecoverUtils.java
new file mode 100644
index 0000000000..20d63c1e79
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/CheckpointRecoverUtils.java
@@ -0,0 +1,78 @@
+/*
+ * 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.wal.recover;
+
+import org.apache.iotdb.db.wal.checkpoint.Checkpoint;
+import org.apache.iotdb.db.wal.checkpoint.MemTableInfo;
+import org.apache.iotdb.db.wal.io.CheckpointReader;
+import org.apache.iotdb.db.wal.io.CheckpointWriter;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class CheckpointRecoverUtils {
+  private CheckpointRecoverUtils() {}
+
+  /** Recover memTable information from checkpoint folder */
+  public static Map<Integer, MemTableInfo> recoverMemTableInfo(File logDirectory) {
+    // find all .checkpoint file
+    File[] checkpointFiles = logDirectory.listFiles(CheckpointWriter::checkpointFilenameFilter);
+    if (checkpointFiles == null) {
+      return Collections.emptyMap();
+    }
+    Arrays.sort(
+        checkpointFiles,
+        Comparator.comparingInt(file -> CheckpointWriter.parseVersionId(((File) file).getName()))
+            .reversed());
+    // find last valid .checkpoint file and load checkpoints from it
+    List<Checkpoint> checkpoints = null;
+    for (File checkpointFile : checkpointFiles) {
+      checkpoints = new CheckpointReader(checkpointFile).readAll();
+      if (!checkpoints.isEmpty()) {
+        break;
+      }
+    }
+    if (checkpoints == null || checkpoints.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    // recover memTables information by checkpoints
+    Map<Integer, MemTableInfo> memTableId2Info = new HashMap<>();
+    for (Checkpoint checkpoint : checkpoints) {
+      switch (checkpoint.getType()) {
+        case GLOBAL_MEMORY_TABLE_INFO:
+        case CREATE_MEMORY_TABLE:
+          for (MemTableInfo memTableInfo : checkpoint.getMemTableInfos()) {
+            memTableId2Info.put(memTableInfo.getMemTableId(), memTableInfo);
+          }
+          break;
+        case FLUSH_MEMORY_TABLE:
+          for (MemTableInfo memTableInfo : checkpoint.getMemTableInfos()) {
+            memTableId2Info.remove(memTableInfo.getMemTableId());
+          }
+          break;
+      }
+    }
+    return memTableId2Info;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/WALNodeRecoverTask.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/WALNodeRecoverTask.java
new file mode 100644
index 0000000000..0620c5b25d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/WALNodeRecoverTask.java
@@ -0,0 +1,151 @@
+/*
+ * 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.wal.recover;
+
+import org.apache.iotdb.db.utils.FileUtils;
+import org.apache.iotdb.db.wal.buffer.WALEntry;
+import org.apache.iotdb.db.wal.checkpoint.MemTableInfo;
+import org.apache.iotdb.db.wal.io.WALReader;
+import org.apache.iotdb.db.wal.io.WALWriter;
+import org.apache.iotdb.db.wal.recover.file.UnsealedTsFileRecoverPerformer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+/** This task is responsible for the recovery of one wal node. */
+public class WALNodeRecoverTask implements Runnable {
+  private static final Logger logger = LoggerFactory.getLogger(WALNodeRecoverTask.class);
+  private static final WALRecoverManager walRecoverManger = WALRecoverManager.getInstance();
+
+  /** this directory store one wal node's .wal and .checkpoint files */
+  private final File logDirectory;
+  /** latch to collect all nodes' recovery end information */
+  private final CountDownLatch allNodesRecoveredLatch;
+  /** version id of first valid .wal file */
+  private int firstValidVersionId = Integer.MAX_VALUE;
+
+  private Map<Integer, MemTableInfo> memTableId2Info;
+  private Map<Integer, UnsealedTsFileRecoverPerformer> memTableId2RecoverPerformer;
+
+  public WALNodeRecoverTask(File logDirectory, CountDownLatch allNodesRecoveredLatch) {
+    this.logDirectory = logDirectory;
+    this.allNodesRecoveredLatch = allNodesRecoveredLatch;
+  }
+
+  @Override
+  public void run() {
+    try {
+      recoverInfoFromCheckpoints();
+      recoverTsFiles();
+    } catch (Exception e) {
+      for (UnsealedTsFileRecoverPerformer recoverPerformer : memTableId2RecoverPerformer.values()) {
+        recoverPerformer.getRecoverListener().fail(e);
+      }
+    } finally {
+      allNodesRecoveredLatch.countDown();
+      for (UnsealedTsFileRecoverPerformer recoverPerformer : memTableId2RecoverPerformer.values()) {
+        try {
+          if (!recoverPerformer.canWrite()) {
+            recoverPerformer.close();
+          }
+        } catch (Exception e) {
+          // continue
+        }
+      }
+    }
+    // delete this wal node folder
+    FileUtils.deleteDirectory(logDirectory);
+  }
+
+  private void recoverInfoFromCheckpoints() {
+    // parse memTables information
+    memTableId2Info = CheckpointRecoverUtils.recoverMemTableInfo(logDirectory);
+    memTableId2RecoverPerformer = new HashMap<>();
+    // update firstValidVersionId and get recover performer from WALRecoverManager
+    for (MemTableInfo memTableInfo : memTableId2Info.values()) {
+      firstValidVersionId = Math.min(firstValidVersionId, memTableInfo.getFirstFileVersionId());
+
+      File tsFile = new File(memTableInfo.getTsFilePath());
+      UnsealedTsFileRecoverPerformer recoverPerformer =
+          walRecoverManger.removeRecoverPerformer(tsFile.getAbsolutePath());
+      memTableId2RecoverPerformer.put(memTableInfo.getMemTableId(), recoverPerformer);
+    }
+  }
+
+  private void recoverTsFiles() {
+    if (memTableId2RecoverPerformer.isEmpty()) {
+      return;
+    }
+    // make preparation for recovery
+    for (UnsealedTsFileRecoverPerformer recoverPerformer : memTableId2RecoverPerformer.values()) {
+      try {
+        recoverPerformer.startRecovery();
+      } catch (Exception e) {
+        recoverPerformer.getRecoverListener().fail(e);
+      }
+    }
+    // find all valid .wal files
+    File[] walFiles =
+        logDirectory.listFiles(
+            (dir, name) -> WALWriter.parseVersionId(name) >= firstValidVersionId);
+    if (walFiles == null) {
+      return;
+    }
+    Arrays.sort(
+        walFiles, Comparator.comparingInt(file -> WALWriter.parseVersionId(file.getName())));
+    // read .wal files and redo logs
+    for (File walFile : walFiles) {
+      try (WALReader walReader = new WALReader(walFile)) {
+        while (walReader.hasNext()) {
+          WALEntry walEntry = walReader.next();
+          if (!memTableId2Info.containsKey(walEntry.getMemTableId())) {
+            continue;
+          }
+
+          UnsealedTsFileRecoverPerformer recoverPerformer =
+              memTableId2RecoverPerformer.get(walEntry.getMemTableId());
+          if (recoverPerformer != null) {
+            recoverPerformer.redoLog(walEntry);
+          } else {
+            logger.warn(
+                "Fail to find TsFile recover performer for wal entry in TsFile {}", walFile);
+          }
+        }
+      } catch (Exception e) {
+        logger.warn("Fail to read wal logs from {}, skip them", walFile, e);
+      }
+    }
+    // end recovering all recover performers
+    for (UnsealedTsFileRecoverPerformer recoverPerformer : memTableId2RecoverPerformer.values()) {
+      try {
+        recoverPerformer.endRecovery();
+        recoverPerformer.getRecoverListener().succeed();
+      } catch (Exception e) {
+        recoverPerformer.getRecoverListener().fail(e);
+      }
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/WALRecoverManager.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/WALRecoverManager.java
new file mode 100644
index 0000000000..d0c5edb7ae
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/WALRecoverManager.java
@@ -0,0 +1,172 @@
+/*
+ * 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.wal.recover;
+
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.concurrent.ThreadName;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.exception.StorageGroupProcessorException;
+import org.apache.iotdb.db.wal.WALManager;
+import org.apache.iotdb.db.wal.exception.WALRecoverException;
+import org.apache.iotdb.db.wal.node.WALNode;
+import org.apache.iotdb.db.wal.recover.file.UnsealedTsFileRecoverPerformer;
+import org.apache.iotdb.db.wal.utils.listener.WALRecoverListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+
+/** First set allVsgScannedLatch, then call recover method. */
+public class WALRecoverManager {
+  private static final Logger logger = LoggerFactory.getLogger(WALManager.class);
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  /** start recovery after all virtual storage groups have submitted unsealed zero-level TsFiles */
+  private volatile CountDownLatch allVsgScannedLatch;
+  /** threads to recover wal nodes */
+  private ExecutorService recoverThreadPool;
+  /** stores all UnsealedTsFileRecoverPerformer submitted by virtual storage group processors */
+  private final Map<String, UnsealedTsFileRecoverPerformer> absolutePath2RecoverPerformer =
+      new ConcurrentHashMap<>();
+
+  private WALRecoverManager() {}
+
+  public void recover() throws WALRecoverException {
+    try {
+      // collect wal nodes' information
+      List<File> walNodeDirs = new ArrayList<>();
+      for (String walDir : config.getWalDirs()) {
+        File walDirFile = SystemFileFactory.INSTANCE.getFile(walDir);
+        File[] nodeDirs = walDirFile.listFiles(WALNode::walNodeFolderNameFilter);
+        if (nodeDirs == null) {
+          continue;
+        }
+        for (File nodeDir : nodeDirs) {
+          if (nodeDir.isDirectory()) {
+            walNodeDirs.add(nodeDir);
+          }
+        }
+      }
+      // wait until all virtual storage groups have submitted their unsealed TsFiles,
+      // which means walRecoverManger.addRecoverPerformer method won't be call anymore
+      try {
+        allVsgScannedLatch.await();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new WALRecoverException("Fail to recover wal.", e);
+      }
+      // recover each wal node's TsFiles
+      if (!walNodeDirs.isEmpty()) {
+        recoverThreadPool =
+            IoTDBThreadPoolFactory.newCachedThreadPool(ThreadName.WAL_RECOVER.getName());
+        CountDownLatch allNodesRecoveredLatch = new CountDownLatch(walNodeDirs.size());
+        for (File walNodeDir : walNodeDirs) {
+          recoverThreadPool.submit(new WALNodeRecoverTask(walNodeDir, allNodesRecoveredLatch));
+        }
+
+        try {
+          allNodesRecoveredLatch.await();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new WALRecoverException("Fail to recover wal.", e);
+        }
+      }
+      // deal with remaining TsFiles which don't have wal
+      for (UnsealedTsFileRecoverPerformer recoverPerformer :
+          absolutePath2RecoverPerformer.values()) {
+        try {
+          recoverPerformer.startRecovery();
+          // skip redo logs because it doesn't belong to any wal node
+          recoverPerformer.endRecovery();
+          recoverPerformer.getRecoverListener().succeed();
+        } catch (StorageGroupProcessorException | IOException e) {
+          logger.error(
+              "Fail to recover unsealed TsFile {}, skip it.",
+              recoverPerformer.getTsFileAbsolutePath(),
+              e);
+          recoverPerformer.getRecoverListener().fail(e);
+        }
+      }
+    } catch (Exception e) {
+      for (UnsealedTsFileRecoverPerformer recoverPerformer :
+          absolutePath2RecoverPerformer.values()) {
+        recoverPerformer.getRecoverListener().fail(e);
+      }
+    } finally {
+      for (UnsealedTsFileRecoverPerformer recoverPerformer :
+          absolutePath2RecoverPerformer.values()) {
+        try {
+          if (!recoverPerformer.canWrite()) {
+            recoverPerformer.close();
+          }
+        } catch (Exception e) {
+          // continue
+        }
+      }
+      clear();
+    }
+  }
+
+  public WALRecoverListener addRecoverPerformer(UnsealedTsFileRecoverPerformer recoverPerformer) {
+    absolutePath2RecoverPerformer.put(recoverPerformer.getTsFileAbsolutePath(), recoverPerformer);
+    return recoverPerformer.getRecoverListener();
+  }
+
+  UnsealedTsFileRecoverPerformer removeRecoverPerformer(String absolutePath) {
+    return absolutePath2RecoverPerformer.remove(absolutePath);
+  }
+
+  public CountDownLatch getAllVsgScannedLatch() {
+    return allVsgScannedLatch;
+  }
+
+  public void setAllVsgScannedLatch(CountDownLatch allVsgScannedLatch) {
+    this.allVsgScannedLatch = allVsgScannedLatch;
+  }
+
+  @TestOnly
+  public void clear() {
+    absolutePath2RecoverPerformer.clear();
+    if (recoverThreadPool != null) {
+      recoverThreadPool.shutdown();
+      recoverThreadPool = null;
+    }
+  }
+
+  public static WALRecoverManager getInstance() {
+    return InstanceHolder.INSTANCE;
+  }
+
+  private static class InstanceHolder {
+    private InstanceHolder() {}
+
+    private static final WALRecoverManager INSTANCE = new WALRecoverManager();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/AbstractTsFileRecoverPerformer.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/AbstractTsFileRecoverPerformer.java
new file mode 100644
index 0000000000..abd411ab0a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/AbstractTsFileRecoverPerformer.java
@@ -0,0 +1,126 @@
+/*
+ * 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.wal.recover.file;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageGroupProcessorException;
+import org.apache.iotdb.db.utils.FileLoaderUtils;
+import org.apache.iotdb.tsfile.exception.NotCompatibleTsFileException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+
+import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.RESOURCE_SUFFIX;
+
+/** This class is used to help recover TsFile */
+public abstract class AbstractTsFileRecoverPerformer implements Closeable {
+  private static final Logger logger =
+      LoggerFactory.getLogger(AbstractTsFileRecoverPerformer.class);
+
+  /** TsFile which needs recovery */
+  protected final TsFileResource tsFileResource;
+  /** this writer will be open when .resource file doesn't exist */
+  protected RestorableTsFileIOWriter writer;
+
+  public AbstractTsFileRecoverPerformer(TsFileResource tsFileResource) {
+    this.tsFileResource = tsFileResource;
+  }
+
+  /**
+   * Recover TsFile with RestorableTsFileIOWriter, including load .resource file (reconstruct when
+   * necessary) and truncate the file to remaining corrected data. <br>
+   * Notice: this method may open a {@link RestorableTsFileIOWriter}, remember to close it.
+   */
+  protected void recoverWithWriter() throws StorageGroupProcessorException, IOException {
+    File tsFile = tsFileResource.getTsFile();
+    if (!tsFile.exists()) {
+      logger.error("TsFile {} is missing, will skip its recovery.", tsFile);
+      return;
+    }
+
+    if (tsFileResource.resourceFileExists()) {
+      // .resource file exists, just deserialize it into memory
+      loadResourceFile();
+      return;
+    }
+
+    // try to remove corrupted part of the TsFile
+    try {
+      writer = new RestorableTsFileIOWriter(tsFile);
+    } catch (NotCompatibleTsFileException e) {
+      boolean result = tsFile.delete();
+      logger.warn(
+          "TsFile {} is incompatible. Try to delete it and delete result is {}", tsFile, result);
+      throw new StorageGroupProcessorException(e);
+    } catch (IOException e) {
+      throw new StorageGroupProcessorException(e);
+    }
+
+    // reconstruct .resource file when TsFile is complete
+    if (!writer.hasCrashed()) {
+      try {
+        reconstructResourceFile();
+      } catch (IOException e) {
+        throw new StorageGroupProcessorException(
+            "Failed recover the resource file: " + tsFile + RESOURCE_SUFFIX + e);
+      }
+    }
+  }
+
+  private void loadResourceFile() throws IOException {
+    try {
+      tsFileResource.deserialize();
+    } catch (IOException e) {
+      logger.warn(
+          "Cannot deserialize .resource file of {}, try to reconstruct it.",
+          tsFileResource.getTsFile(),
+          e);
+      reconstructResourceFile();
+    }
+  }
+
+  protected void reconstructResourceFile() throws IOException {
+    try (TsFileSequenceReader reader =
+        new TsFileSequenceReader(tsFileResource.getTsFile().getAbsolutePath())) {
+      FileLoaderUtils.updateTsFileResource(reader, tsFileResource);
+    }
+    tsFileResource.serialize();
+  }
+
+  public boolean hasCrashed() {
+    return writer != null && writer.hasCrashed();
+  }
+
+  public boolean canWrite() {
+    return writer != null && writer.canWrite();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (writer != null) {
+      writer.close();
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/SealedTsFileRecoverPerformer.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/SealedTsFileRecoverPerformer.java
new file mode 100644
index 0000000000..bc9d3ac0ca
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/SealedTsFileRecoverPerformer.java
@@ -0,0 +1,54 @@
+/*
+ * 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.wal.recover.file;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageGroupProcessorException;
+
+import java.io.IOException;
+
+import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.RESOURCE_SUFFIX;
+
+/** This class is used to help recover all sealed TsFiles, except unsealed TsFile at zero level. */
+public class SealedTsFileRecoverPerformer extends AbstractTsFileRecoverPerformer {
+  public SealedTsFileRecoverPerformer(TsFileResource tsFileResource) {
+    super(tsFileResource);
+  }
+
+  /**
+   * Recover sealed TsFile, including load .resource file (reconstruct when necessary) and truncate
+   * the file to remaining corrected data
+   */
+  public void recover() throws StorageGroupProcessorException, IOException {
+    super.recoverWithWriter();
+
+    if (hasCrashed()) {
+      writer.endFile();
+      try {
+        reconstructResourceFile();
+      } catch (IOException e) {
+        throw new StorageGroupProcessorException(
+            "Failed recover the resource file: "
+                + tsFileResource.getTsFilePath()
+                + RESOURCE_SUFFIX
+                + e);
+      }
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java
new file mode 100644
index 0000000000..47318b0c5d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java
@@ -0,0 +1,159 @@
+/*
+ * 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.wal.recover.file;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+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.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.service.IoTDB;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This class helps redo wal logs into a TsFile. Notice: You should update time map in {@link
+ * TsFileResource} before using this class to avoid duplicated insertion and this class doesn't
+ * guarantee concurrency safety.
+ */
+public class TsFilePlanRedoer {
+  private static final Logger logger = LoggerFactory.getLogger(TsFilePlanRedoer.class);
+
+  private final TsFileResource tsFileResource;
+  /** only unsequence file tolerates duplicated data */
+  private final boolean sequence;
+  /** virtual storage group's idTable of this tsFile */
+  private final IDTable idTable;
+  /** store data when redoing logs */
+  private IMemTable recoveryMemTable = new PrimitiveMemTable();
+
+  public TsFilePlanRedoer(TsFileResource tsFileResource, boolean sequence, IDTable idTable) {
+    this.tsFileResource = tsFileResource;
+    this.sequence = sequence;
+    this.idTable = idTable;
+  }
+
+  void redoDelete(DeletePlan deletePlan) throws IOException, MetadataException {
+    List<PartialPath> paths = deletePlan.getPaths();
+    for (PartialPath path : paths) {
+      for (PartialPath device : IoTDB.schemaProcessor.getBelongedDevices(path)) {
+        recoveryMemTable.delete(
+            path, device, deletePlan.getDeleteStartTime(), deletePlan.getDeleteEndTime());
+      }
+      tsFileResource
+          .getModFile()
+          .write(
+              new Deletion(
+                  path,
+                  tsFileResource.getTsFileSize(),
+                  deletePlan.getDeleteStartTime(),
+                  deletePlan.getDeleteEndTime()));
+    }
+  }
+
+  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+  void redoInsert(InsertPlan plan) throws WriteProcessException, QueryProcessException {
+    if (tsFileResource != null) {
+      String deviceId =
+          plan.isAligned()
+              ? plan.getDevicePath().getDevicePath().getFullPath()
+              : plan.getDevicePath().getFullPath();
+      // orders of insert plan is guaranteed by storage engine, just check time in the file
+      // the last chunk group may contain the same data with the logs, ignore such logs in seq file
+      long lastEndTime = tsFileResource.getEndTime(deviceId);
+      if (lastEndTime != Long.MIN_VALUE && lastEndTime >= plan.getMinTime() && sequence) {
+        return;
+      }
+    }
+
+    plan.setMeasurementMNodes(new IMeasurementMNode[plan.getMeasurements().length]);
+    try {
+      if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
+        idTable.getSeriesSchemas(plan);
+      } else {
+        IoTDB.schemaProcessor.getSeriesSchemasAndReadLockDevice(plan);
+        plan.setDeviceID(DeviceIDFactory.getInstance().getDeviceID(plan.getDevicePath()));
+      }
+    } catch (IOException | MetadataException e) {
+      throw new QueryProcessException("can't replay insert logs, ", e);
+    }
+
+    // mark failed plan manually
+    checkDataTypeAndMarkFailed(plan.getMeasurementMNodes(), plan);
+    if (plan instanceof InsertRowPlan) {
+      if (plan.isAligned()) {
+        recoveryMemTable.insertAlignedRow((InsertRowPlan) plan);
+      } else {
+        recoveryMemTable.insert((InsertRowPlan) plan);
+      }
+    } else {
+      if (plan.isAligned()) {
+        recoveryMemTable.insertAlignedTablet(
+            (InsertTabletPlan) plan, 0, ((InsertTabletPlan) plan).getRowCount());
+      } else {
+        recoveryMemTable.insertTablet(
+            (InsertTabletPlan) plan, 0, ((InsertTabletPlan) plan).getRowCount());
+      }
+    }
+  }
+
+  private void checkDataTypeAndMarkFailed(final IMeasurementMNode[] mNodes, InsertPlan tPlan) {
+    for (int i = 0; i < mNodes.length; i++) {
+      if (mNodes[i] == null) {
... 8510 lines suppressed ...