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 2021/06/11 06:36:24 UTC

[iotdb] branch clear_merge_code updated: Clear compaction code (#3305)

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

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


The following commit(s) were added to refs/heads/clear_merge_code by this push:
     new ce36fe9  Clear compaction code (#3305)
ce36fe9 is described below

commit ce36fe905a827a0ff593ffc0b4c9f56624e7e089
Author: zhanglingzhe0820 <44...@qq.com>
AuthorDate: Fri Jun 11 14:35:52 2021 +0800

    Clear compaction code (#3305)
---
 .../cluster/log/snapshot/PullSnapshotTaskTest.java |   2 +-
 .../apache/iotdb/cluster/query/BaseQueryTest.java  |   2 +-
 docs/SystemDesign/StorageEngine/MergeManager.md    |  10 +-
 docs/zh/SystemDesign/StorageEngine/MergeManager.md |  10 +-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   4 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |   5 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   5 +-
 .../engine/compaction/AbstractCompactionTask.java  |  28 +
 ...tionTaskManager.java => CompactionManager.java} | 110 ++-
 .../engine/compaction/CompactionRecoverTask.java   |  23 +
 .../iotdb/db/engine/compaction/CompactionTask.java |  42 +
 .../db/engine/compaction/TsFileManagement.java     | 687 ++++++++++-------
 .../CrossSpaceCompactionExecutor.java              |  16 +
 .../inplace/InplaceCompactionExecutor.java         | 285 +++++++
 .../inplace}/manage/MergeContext.java              |   2 +-
 .../inplace}/manage/MergeFuture.java               |   6 +-
 .../inplace}/manage/MergeManager.java              |  22 +-
 .../inplace}/manage/MergeManagerMBean.java         |   2 +-
 .../inplace}/manage/MergeResource.java             |   8 +-
 .../inplace}/manage/MergeThreadPool.java           |  10 +-
 .../inplace}/recover/LogAnalyzer.java              |  22 +-
 .../inplace}/recover/MergeLogger.java              |   4 +-
 .../selector/IFileQueryMemMeasurement.java         |   2 +-
 .../inplace}/selector/IMergeFileSelector.java      |   2 +-
 .../inplace}/selector/IMergePathSelector.java      |   2 +-
 .../selector/MaxFileMergeFileSelector.java         |   4 +-
 .../selector/MaxSeriesMergeFileSelector.java       |   4 +-
 .../inplace}/selector/MergeFileStrategy.java       |   2 +-
 .../inplace}/selector/NaivePathSelector.java       |   2 +-
 .../inplace}/task/MergeCallback.java               |   2 +-
 .../inplace}/task/MergeFileTask.java               |   8 +-
 .../inplace}/task/MergeMultiChunkTask.java         |  14 +-
 .../inplace}/task/MergeTask.java                   |  13 +-
 .../inplace}/task/RecoverMergeTask.java            |  14 +-
 .../CompactionStrategy.java                        |  14 +-
 .../InnerSpaceCompactionExecutor.java              |  16 +
 .../level/LevelCompactionExecutor.java             | 328 ++++++++
 .../no/NoCompactionTsFileManagement.java           |  49 ++
 .../utils/CompactionLogAnalyzer.java               |  19 +-
 .../utils/CompactionLogger.java                    |   4 +-
 .../utils/CompactionUtils.java                     |   4 +-
 .../level/LevelCompactionTsFileManagement.java     | 842 ---------------------
 .../no/NoCompactionTsFileManagement.java           | 290 -------
 .../apache/iotdb/db/engine/merge/package-info.java |  25 -
 .../engine/storagegroup/StorageGroupProcessor.java | 216 ++----
 .../virtualSg/VirtualStorageGroupManager.java      |   9 -
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |   4 +-
 .../java/org/apache/iotdb/db/service/IoTDB.java    |   6 +-
 .../db/sync/sender/manage/SyncFileManager.java     |   2 +-
 .../java/org/apache/iotdb/db/utils/MergeUtils.java |   2 +-
 .../db/engine/compaction/CompactionChunkTest.java  |   4 +-
 .../compaction/LevelCompactionCacheTest.java       |  15 +-
 ...tTest.java => LevelCompactionExecutorTest.java} |  94 ++-
 .../engine/compaction/LevelCompactionLogTest.java  |  17 +-
 .../compaction/LevelCompactionMergeTest.java       |  79 +-
 .../engine/compaction/LevelCompactionModsTest.java |  14 +-
 .../compaction/LevelCompactionMoreDataTest.java    |  17 +-
 .../compaction/LevelCompactionRecoverTest.java     | 128 ++--
 .../compaction/LevelCompactionSelectorTest.java    |  17 +-
 .../NoCompactionTsFileManagementTest.java          |   2 +-
 .../MaxFileMergeFileSelectorTest.java              |   8 +-
 .../MaxSeriesMergeFileSelectorTest.java            |   6 +-
 .../crossSpaceCompaction}/MergeLogTest.java        |   6 +-
 .../crossSpaceCompaction}/MergeManagerTest.java    |   8 +-
 .../crossSpaceCompaction}/MergeOverLapTest.java    |   6 +-
 .../crossSpaceCompaction}/MergePerfTest.java       |   6 +-
 .../crossSpaceCompaction}/MergeTaskTest.java       |   6 +-
 .../crossSpaceCompaction}/MergeTest.java           |   4 +-
 .../crossSpaceCompaction}/MergeUpgradeTest.java    |   6 +-
 .../storagegroup/StorageGroupProcessorTest.java    |   4 +-
 .../iotdb/db/integration/IOTDBGroupByIT.java       |   2 +-
 .../iotdb/db/integration/IoTDBAlignByDeviceIT.java |   2 +-
 .../db/integration/IoTDBDeleteTimeseriesIT.java    |   2 +-
 .../iotdb/db/integration/IoTDBGroupByMonthIT.java  |   2 +-
 .../iotdb/db/integration/IoTDBLargeDataIT.java     |   2 +-
 .../db/integration/IoTDBLevelCompactionIT.java     |   2 +-
 ...IoTDBLoadExternalTsFileWithTimePartitionIT.java |   2 +-
 .../db/integration/IoTDBLoadExternalTsfileIT.java  |   2 +-
 .../apache/iotdb/db/integration/IoTDBMergeIT.java  |   2 +-
 .../iotdb/db/integration/IoTDBMultiDeviceIT.java   |   2 +-
 .../IoTDBMultiOverlappedChunkInUnseqIT.java        |   2 +-
 .../db/integration/IoTDBMultiOverlappedPageIT.java |   2 +-
 .../iotdb/db/integration/IoTDBMultiSeriesIT.java   |   2 +-
 .../db/integration/IoTDBMultiStatementsIT.java     |   2 +-
 .../db/integration/IoTDBNewTsFileCompactionIT.java |   8 +-
 .../db/integration/IoTDBOverlappedPageIT.java      |   2 +-
 .../iotdb/db/integration/IoTDBRestartIT.java       |   4 +-
 .../iotdb/db/integration/IoTDBSeriesReaderIT.java  |   2 +-
 .../aggregation/IoTDBAggregationIT.java            |   2 +-
 .../aggregation/IoTDBAggregationLargeDataIT.java   |   2 +-
 .../aggregation/IoTDBAggregationSmallDataIT.java   |   2 +-
 .../query/reader/series/SeriesReaderTestUtil.java  |   2 +-
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |   4 +-
 server/src/test/resources/logback.xml              |   2 +-
 .../apache/iotdb/session/pool/SessionPoolTest.java |   2 +-
 95 files changed, 1688 insertions(+), 2023 deletions(-)

diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskTest.java
index f4933ed..31c32d9 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/log/snapshot/PullSnapshotTaskTest.java
@@ -36,7 +36,7 @@ import org.apache.iotdb.cluster.server.member.DataGroupMember;
 import org.apache.iotdb.cluster.utils.IOUtils;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StartupException;
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/BaseQueryTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/BaseQueryTest.java
index eb04872..d7ec130 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/query/BaseQueryTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/BaseQueryTest.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.cluster.common.TestUtils;
 import org.apache.iotdb.cluster.server.member.BaseMember;
 import org.apache.iotdb.cluster.server.monitor.NodeStatusManager;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
diff --git a/docs/SystemDesign/StorageEngine/MergeManager.md b/docs/SystemDesign/StorageEngine/MergeManager.md
index fdc4332..f868e3d 100644
--- a/docs/SystemDesign/StorageEngine/MergeManager.md
+++ b/docs/SystemDesign/StorageEngine/MergeManager.md
@@ -45,25 +45,25 @@ There are two options when recovering, one is to continue the previous progress,
 
 ## Related code
 
-* org.apache.iotdb.db.engine.merge.BaseFileSelector
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.BaseFileSelector
 
     The base class for the file selection process, which specifies the basic framework for selecting files and methods for calculating file memory consumption in different situations. All custom file selection strategies need to inherit this class
     
-* org.apache.iotdb.db.engine.merge.IRecoverMergeTask
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.IRecoverMergeTask
   
     The interface class of the recover process, which specifies the recoverMerge interface. All custom merge recovery strategies must inherit this class.
 
 In addition, each custom MergeTask needs to inherit the Callable \<void\> interface to ensure that it can be called back
 
-* org.apache.iotdb.db.engine.merge.manage.MergeContext
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeContext
 
     Common context classes in the Merge process
 
-* org.apache.iotdb.db.engine.merge.manage.MergeManager
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager
 
     The thread pool class in the Merge process, which manages the operation of multiple merge tasks
 
-* org.apache.iotdb.db.engine.merge.manage.MergeResource
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource
 
     Resource class in the Merge process, responsible for managing files, readers, writers, measurement Schemas, modifications, and other resources during the merge process
 
diff --git a/docs/zh/SystemDesign/StorageEngine/MergeManager.md b/docs/zh/SystemDesign/StorageEngine/MergeManager.md
index 6f7cbd7..17af6f3 100644
--- a/docs/zh/SystemDesign/StorageEngine/MergeManager.md
+++ b/docs/zh/SystemDesign/StorageEngine/MergeManager.md
@@ -47,25 +47,25 @@
 
 ## 相关代码
 
-* org.apache.iotdb.db.engine.merge.BaseFileSelector
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.BaseFileSelector
 
     文件选择过程的基类,规定了选择文件的基本框架,及在不同情况下计算文件内存消耗的方法,所有自定义的文件选择策略均需继承此类
     
-* org.apache.iotdb.db.engine.merge.IRecoverMergeTask
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.IRecoverMergeTask
   
     recover 过程的接口类,规定了 recoverMerge 接口,所有自定义的merge恢复策略均需继承此类
 
 此外,每一个自定义的MergeTask均需继承Callable\<void\>接口,以保证可以被回调
 
-* org.apache.iotdb.db.engine.merge.manage.MergeContext
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeContext
 
     Merge 过程中的公用上下文类
 
-* org.apache.iotdb.db.engine.merge.manage.MergeManager
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager
 
     Merge 过程中的线程池类,管理了多个 merge task 的运行
 
-* org.apache.iotdb.db.engine.merge.manage.MergeResource
+* org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource
 
     Merge 过程中的资源类,负责管理 merge 过程中的 files,readers,writers,measurementSchemas,modifications 等资源
 
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 61b8997..e884eeb 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
@@ -19,8 +19,8 @@
 package org.apache.iotdb.db.conf;
 
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
-import org.apache.iotdb.db.engine.merge.selector.MergeFileStrategy;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.MergeFileStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel;
 import org.apache.iotdb.db.exception.LoadConfigurationException;
 import org.apache.iotdb.db.metadata.MManager;
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 6077a3d..80cf9b3 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
@@ -19,7 +19,7 @@
 package org.apache.iotdb.db.conf;
 
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.utils.FilePathUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -481,8 +481,7 @@ public class IoTDBDescriptor {
                   "merge_interval_sec", Long.toString(conf.getMergeIntervalSec()))));
       conf.setFullMerge(
           Boolean.parseBoolean(
-              properties.getProperty(
-                  "force_full_merge", Boolean.toString(conf.isFullMerge()))));
+              properties.getProperty("force_full_merge", Boolean.toString(conf.isFullMerge()))));
       conf.setCompactionThreadNum(
           Integer.parseInt(
               properties.getProperty(
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 96e288a..298ce25 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
@@ -22,6 +22,7 @@ import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.ServerConfigConsistent;
+import org.apache.iotdb.db.engine.compaction.CompactionManager;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.flush.CloseFileListener;
 import org.apache.iotdb.db.engine.flush.FlushListener;
@@ -737,9 +738,7 @@ public class StorageEngine implements IService {
       throw new StorageEngineException("Current system mode is read only, does not support merge");
     }
 
-    for (VirtualStorageGroupManager virtualStorageGroupManager : processorMap.values()) {
-      virtualStorageGroupManager.mergeAll(isFullMerge);
-    }
+    CompactionManager.getInstance().mergeAll(isFullMerge);
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/AbstractCompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/AbstractCompactionTask.java
new file mode 100644
index 0000000..ba04748
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/AbstractCompactionTask.java
@@ -0,0 +1,28 @@
+package org.apache.iotdb.db.engine.compaction;
+
+import java.util.concurrent.Callable;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.CrossSpaceCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.InplaceCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.InnerSpaceCompactionExecutor;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.CloseCompactionMergeCallBack;
+
+public abstract class AbstractCompactionTask implements Callable<Void> {
+  protected TsFileManagement tsFileManagement;
+  protected CloseCompactionMergeCallBack closeCompactionMergeCallBack;
+  protected CrossSpaceCompactionExecutor crossSpaceCompactionExecutor;
+  protected InnerSpaceCompactionExecutor innerSpaceCompactionExecutor;
+
+  public AbstractCompactionTask(
+      TsFileManagement tsFileManagement,
+      CloseCompactionMergeCallBack closeCompactionMergeCallBack) {
+    this.tsFileManagement = tsFileManagement;
+    this.closeCompactionMergeCallBack = closeCompactionMergeCallBack;
+    this.crossSpaceCompactionExecutor = new InplaceCompactionExecutor(tsFileManagement);
+    this.innerSpaceCompactionExecutor =
+        IoTDBDescriptor.getInstance()
+            .getConfig()
+            .getCompactionStrategy()
+            .getInnerSpaceCompactionExecutor(tsFileManagement);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionManager.java
similarity index 62%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionManager.java
index 53dc9cf..a29f5d1 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionManager.java
@@ -19,17 +19,7 @@
 
 package org.apache.iotdb.db.engine.compaction;
 
-import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
-import org.apache.iotdb.db.concurrent.ThreadName;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.service.IService;
-import org.apache.iotdb.db.service.ServiceType;
-import org.apache.iotdb.db.utils.FilePathUtils;
-import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
 
 import java.io.File;
 import java.util.Collections;
@@ -42,21 +32,30 @@ import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
+import org.apache.iotdb.db.service.IService;
+import org.apache.iotdb.db.service.ServiceType;
+import org.apache.iotdb.db.utils.FilePathUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** CompactionTaskManager provides a ThreadPool to queue and run all compaction tasks. */
-public class CompactionTaskManager implements IService {
+public class CompactionManager implements IService {
 
-  private static final Logger logger =
-      LoggerFactory.getLogger(CompactionTaskManager.class);
-  private static final CompactionTaskManager INSTANCE =
-      new CompactionTaskManager();
-  private ExecutorService pool;
+  private static final Logger logger = LoggerFactory.getLogger(CompactionManager.class);
+  private static final CompactionManager INSTANCE = new CompactionManager();
+  private ScheduledExecutorService pool;
   private Map<String, Set<Future<Void>>> storageGroupTasks = new ConcurrentHashMap<>();
+  private final Set<StorageGroupProcessor> storageGroupProcessorSet = new ConcurrentSkipListSet<>();
 
-  public static CompactionTaskManager getInstance() {
+  public static CompactionManager getInstance() {
     return INSTANCE;
   }
 
@@ -159,16 +158,79 @@ public class CompactionTaskManager implements IService {
     return ServiceType.COMPACTION_SERVICE;
   }
 
-  public void submitTask(String storageGroupName, Callable<Void> compactionMergeTask)
-      throws RejectedExecutionException {
-    if (pool != null && !pool.isTerminated()) {
-      Future<Void> future = pool.submit(compactionMergeTask);
+  public void init(
+      String storageGroupName,
+      TsFileManagement tsFileManagement,
+      StorageGroupProcessor storageGroupProcessor) {
+    logger.info("{} submit a compaction recover task", storageGroupName);
+    try {
+      Callable<Void> compactionRecoverTask =
+          new CompactionRecoverTask(tsFileManagement, this::closeCompactionMergeCallBack);
+      Future<Void> future = pool.submit(compactionRecoverTask);
       storageGroupTasks
           .computeIfAbsent(storageGroupName, k -> new ConcurrentSkipListSet<>())
           .add(future);
+    } catch (RejectedExecutionException e) {
+      this.closeCompactionMergeCallBack(storageGroupName);
+      logger.info("{} compaction submit task failed", storageGroupName, e);
+    }
+    storageGroupProcessorSet.add(storageGroupProcessor);
+    pool.scheduleWithFixedDelay(
+        () -> mergeAll(IoTDBDescriptor.getInstance().getConfig().isFullMerge()),
+        30,
+        0,
+        TimeUnit.SECONDS);
+  }
+
+  public void mergeAll(boolean isFullMerge) {
+    for (StorageGroupProcessor storageGroupProcessor : storageGroupProcessorSet) {
+      storageGroupProcessor.writeLock();
+      try {
+        for (long timePartitionId :
+            storageGroupProcessor.partitionLatestFlushedTimeForEachDevice.keySet()) {
+          CompactionManager.getInstance()
+              .startCompaction(
+                  storageGroupProcessor.getLogicalStorageGroupName(),
+                  timePartitionId,
+                  isFullMerge,
+                  storageGroupProcessor.tsFileManagement);
+        }
+      } finally {
+        storageGroupProcessor.writeUnlock();
+      }
     }
   }
 
+  public void startCompaction(
+      String storageGroupName,
+      long timePartition,
+      boolean fullMerge,
+      TsFileManagement tsFileManagement) {
+    if (!storageGroupTasks.containsKey(storageGroupName)
+        || storageGroupTasks.get(storageGroupName).isEmpty()) {
+      logger.info("{} submit a compaction task", storageGroupName);
+      try {
+        Callable<Void> compactionTask =
+            new CompactionTask(
+                tsFileManagement, timePartition, fullMerge, this::closeCompactionMergeCallBack);
+        Future<Void> future = pool.submit(compactionTask);
+        storageGroupTasks
+            .computeIfAbsent(storageGroupName, k -> new ConcurrentSkipListSet<>())
+            .add(future);
+      } catch (RejectedExecutionException e) {
+        this.closeCompactionMergeCallBack(storageGroupName);
+        logger.info("{} compaction submit task failed", storageGroupName, e);
+      }
+    } else {
+      logger.info("{} last compaction merge task is working, skip current merge", storageGroupName);
+    }
+  }
+
+  /** close compaction merge callback, to release some locks */
+  private void closeCompactionMergeCallBack(String storageGroupName) {
+    this.storageGroupTasks.remove(storageGroupName);
+  }
+
   /**
    * Abort all compactions of a storage group. The caller must acquire the write lock of the
    * corresponding storage group.
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionRecoverTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionRecoverTask.java
new file mode 100644
index 0000000..6479255
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionRecoverTask.java
@@ -0,0 +1,23 @@
+package org.apache.iotdb.db.engine.compaction;
+
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.CloseCompactionMergeCallBack;
+
+public class CompactionRecoverTask extends AbstractCompactionTask {
+  public CompactionRecoverTask(
+      TsFileManagement tsFileManagement,
+      CloseCompactionMergeCallBack closeCompactionMergeCallBack) {
+    super(tsFileManagement, closeCompactionMergeCallBack);
+  }
+
+  @Override
+  public Void call() {
+    recover();
+    closeCompactionMergeCallBack.call();
+    return null;
+  }
+
+  private void recover() {
+    crossSpaceCompactionExecutor.recover();
+    innerSpaceCompactionExecutor.recover();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTask.java
new file mode 100644
index 0000000..a3baac2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTask.java
@@ -0,0 +1,42 @@
+package org.apache.iotdb.db.engine.compaction;
+
+import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.CloseCompactionMergeCallBack;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class CompactionTask extends AbstractCompactionTask {
+  private final boolean enableUnseqCompaction =
+      IoTDBDescriptor.getInstance().getConfig().isEnableUnseqCompaction();
+
+  private long timePartition;
+  private boolean isFullMerge;
+
+  public CompactionTask(
+      TsFileManagement tsFileManagement,
+      long timePartition,
+      boolean isFullMerge,
+      CloseCompactionMergeCallBack closeCompactionMergeCallBack) {
+    super(tsFileManagement, closeCompactionMergeCallBack);
+    this.timePartition = timePartition;
+    this.isFullMerge = isFullMerge;
+  }
+
+  @Override
+  public Void call() {
+    merge(timePartition, isFullMerge);
+    closeCompactionMergeCallBack.call(tsFileManagement.storageGroupName);
+    return null;
+  }
+
+  private void merge(long timePartition, boolean isFullMerge) {
+    List<List<TsFileResource>> sequenceTsFileResources = tsFileListPair.left;
+    List<List<TsFileResource>> unSequenceTsFileResources = tsFileListPair.right;
+    if (enableUnseqCompaction) {
+      crossSpaceCompactionExecutor.doCrossSpaceCompaction(isFullMerge, timePartition);
+    }
+    innerSpaceCompactionExecutor.doInnerSpaceCompaction(true, timePartition);
+    innerSpaceCompactionExecutor.doInnerSpaceCompaction(false, timePartition);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileManagement.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileManagement.java
index 439c50d..b11f9c8 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileManagement.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileManagement.java
@@ -19,43 +19,36 @@
 
 package org.apache.iotdb.db.engine.compaction;
 
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.cache.ChunkCache;
-import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.selector.IMergeFileSelector;
-import org.apache.iotdb.db.engine.merge.selector.MaxFileMergeFileSelector;
-import org.apache.iotdb.db.engine.merge.selector.MaxSeriesMergeFileSelector;
-import org.apache.iotdb.db.engine.merge.selector.MergeFileStrategy;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
-import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.modification.ModificationFile;
-import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.CloseCompactionMergeCallBack;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
 
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.concurrent.Callable;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.cache.ChunkCache;
+import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
-import static org.apache.iotdb.db.engine.merge.task.MergeTask.MERGE_SUFFIX;
-import static org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.MERGING_MODIFICATION_FILE_NAME;
-import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
-
-public abstract class TsFileManagement {
+public class TsFileManagement {
 
   private static final Logger logger = LoggerFactory.getLogger(TsFileManagement.class);
-  protected String storageGroupName;
-  protected String storageGroupDir;
+  public String storageGroupName;
+  public String storageGroupSysDir;
 
   /** Serialize queries, delete resource files, compaction cleanup files */
   private final ReadWriteLock compactionMergeLock = new ReentrantReadWriteLock();
@@ -66,20 +59,22 @@ public abstract class TsFileManagement {
    */
   public ModificationFile mergingModification;
 
-  /** whether execute merge chunk in this task */
-  protected boolean isMergeExecutedInCurrentTask = false;
+  // First map is partition list; Second list is level list; Third list is file list in level;
+  private final Map<Long, List<SortedSet<TsFileResource>>> sequenceTsFileResources =
+      new HashMap<>();
+  private final Map<Long, List<List<TsFileResource>>> unSequenceTsFileResources = new HashMap<>();
 
-  protected boolean isForceFullMerge = IoTDBDescriptor.getInstance().getConfig().isFullMerge();
-  private final int maxOpenFileNumInEachUnseqCompaction =
-      IoTDBDescriptor.getInstance().getConfig().getMaxOpenFileNumInEachUnseqCompaction();
+  private final List<TsFileResource> sequenceRecoverTsFileResources = new ArrayList<>();
+  private final List<TsFileResource> unSequenceRecoverTsFileResources = new ArrayList<>();
 
-  public TsFileManagement(String storageGroupName, String storageGroupDir) {
-    this.storageGroupName = storageGroupName;
-    this.storageGroupDir = storageGroupDir;
-  }
+  private final int seqLevelNum =
+      Math.max(IoTDBDescriptor.getInstance().getConfig().getSeqLevelNum(), 1);
+  private final int unseqLevelNum =
+      Math.max(IoTDBDescriptor.getInstance().getConfig().getUnseqLevelNum(), 1);
 
-  public void setFullMerge(boolean forceFullMerge) {
-    isForceFullMerge = forceFullMerge;
+  public TsFileManagement(String storageGroupName, String storageGroupSysDir) {
+    this.storageGroupName = storageGroupName;
+    this.storageGroupSysDir = storageGroupSysDir;
   }
 
   /**
@@ -87,304 +82,452 @@ public abstract class TsFileManagement {
    * getTsFileListByTimePartition instead
    */
   @Deprecated
-  public abstract List<TsFileResource> getTsFileList(boolean sequence);
-
-  /** get the TsFile list in sequence by time partition */
-  public abstract List<TsFileResource> getTsFileListByTimePartition(
-      boolean sequence, long timePartition);
-
-  /** get the TsFile list iterator in sequence */
-  public abstract Iterator<TsFileResource> getIterator(boolean sequence);
-
-  /** remove one TsFile from list */
-  public abstract void remove(TsFileResource tsFileResource, boolean sequence);
-
-  /** remove some TsFiles from list */
-  public abstract void removeAll(List<TsFileResource> tsFileResourceList, boolean sequence);
-
-  /** add one TsFile to list */
-  public abstract void add(TsFileResource tsFileResource, boolean sequence);
-
-  /** add one TsFile to list for recover */
-  public abstract void addRecover(TsFileResource tsFileResource, boolean sequence);
-
-  /** add some TsFiles to list */
-  public abstract void addAll(List<TsFileResource> tsFileResourceList, boolean sequence);
-
-  /** is one TsFile contained in list */
-  public abstract boolean contains(TsFileResource tsFileResource, boolean sequence);
-
-  /** clear list */
-  public abstract void clear();
-
-  /** is the list empty */
-  public abstract boolean isEmpty(boolean sequence);
-
-  /** return TsFile list size */
-  public abstract int size(boolean sequence);
-
-  /** recover TsFile list */
-  public abstract void recover();
-
-  /** fork current TsFile list (call this before merge) */
-  public abstract void forkCurrentFileList(long timePartition) throws IOException;
-
-  protected void readLock() {
-    compactionMergeLock.readLock().lock();
-  }
-
-  protected void readUnLock() {
-    compactionMergeLock.readLock().unlock();
+  public List<TsFileResource> getTsFileList(boolean sequence) {
+    readLock();
+    try {
+      List<TsFileResource> result = new ArrayList<>();
+      if (sequence) {
+        for (long timePartition : sequenceTsFileResources.keySet()) {
+          result.addAll(getTsFileListByTimePartition(true, timePartition));
+        }
+      } else {
+        for (long timePartition : unSequenceTsFileResources.keySet()) {
+          result.addAll(getTsFileListByTimePartition(false, timePartition));
+        }
+      }
+      return result;
+    } finally {
+      readUnLock();
+    }
   }
 
-  public void writeLock() {
-    compactionMergeLock.writeLock().lock();
+  /** get the TsFile list in sequence by time partition and level */
+  public List<TsFileResource> getTsFileListByTimePartitionAndLevel(
+      boolean sequence, long timePartition, int level) {
+    readLock();
+    try {
+      if (sequence) {
+        List<SortedSet<TsFileResource>> sequenceTsFileList =
+            sequenceTsFileResources.get(timePartition);
+        return new ArrayList<>(sequenceTsFileList.get(level));
+      } else {
+        List<List<TsFileResource>> unSequenceTsFileList =
+            unSequenceTsFileResources.get(timePartition);
+        return new ArrayList<>(unSequenceTsFileList.get(level));
+      }
+    } finally {
+      readUnLock();
+    }
   }
 
-  public void writeUnlock() {
-    compactionMergeLock.writeLock().unlock();
+  /**
+   * get the closed TsFile list in the format of tree in sequence by time partition, used by level
+   * compaction
+   */
+  public List<List<TsFileResource>> getClosedTsFileListByTimePartition(
+      boolean sequence, long timePartition) {
+    readLock();
+    try {
+      List<List<TsFileResource>> result = new ArrayList<>();
+      if (sequence) {
+        List<SortedSet<TsFileResource>> sequenceTsFileList =
+            sequenceTsFileResources.get(timePartition);
+        for (SortedSet<TsFileResource> tsFileResources : sequenceTsFileList) {
+          result.add(new ArrayList<>(tsFileResources));
+        }
+      } else {
+        List<List<TsFileResource>> unSequenceTsFileList =
+            unSequenceTsFileResources.get(timePartition);
+        for (List<TsFileResource> tsFileResources : unSequenceTsFileList) {
+          result.add(new ArrayList<>(tsFileResources));
+        }
+      }
+      return result;
+    } finally {
+      readUnLock();
+    }
   }
 
-  public boolean tryWriteLock() {
-    return compactionMergeLock.writeLock().tryLock();
+  /** get the TsFile list in sequence by time partition */
+  public List<TsFileResource> getTsFileListByTimePartition(boolean sequence, long timePartition) {
+    readLock();
+    try {
+      List<TsFileResource> result = new ArrayList<>();
+      if (sequence) {
+        List<SortedSet<TsFileResource>> sequenceTsFileList =
+            sequenceTsFileResources.get(timePartition);
+        for (int i = sequenceTsFileList.size() - 1; i >= 0; i--) {
+          result.addAll(sequenceTsFileList.get(i));
+        }
+      } else {
+        List<List<TsFileResource>> unSequenceTsFileList =
+            unSequenceTsFileResources.get(timePartition);
+        for (int i = unSequenceTsFileList.size() - 1; i >= 0; i--) {
+          result.addAll(unSequenceTsFileList.get(i));
+        }
+      }
+      return result;
+    } finally {
+      readUnLock();
+    }
   }
 
-  protected abstract void merge(long timePartition);
-
-  public class CompactionMergeTask implements Callable<Void> {
-
-    private CloseCompactionMergeCallBack closeCompactionMergeCallBack;
-    private long timePartitionId;
-
-    public CompactionMergeTask(
-        CloseCompactionMergeCallBack closeCompactionMergeCallBack, long timePartitionId) {
-      this.closeCompactionMergeCallBack = closeCompactionMergeCallBack;
-      this.timePartitionId = timePartitionId;
+  /** get the TsFile list iterator in sequence */
+  public Iterator<TsFileResource> getIterator(boolean sequence) {
+    readLock();
+    try {
+      return getTsFileList(sequence).iterator();
+    } finally {
+      readUnLock();
     }
+  }
 
-    @Override
-    public Void call() {
-      merge(timePartitionId);
-      closeCompactionMergeCallBack.call(isMergeExecutedInCurrentTask, timePartitionId);
-      return null;
+  /** remove one TsFile from list */
+  public void remove(TsFileResource tsFileResource, boolean sequence) {
+    writeLock();
+    try {
+      if (sequence) {
+        for (SortedSet<TsFileResource> sequenceTsFileResource :
+            sequenceTsFileResources.get(tsFileResource.getTimePartition())) {
+          sequenceTsFileResource.remove(tsFileResource);
+        }
+      } else {
+        for (List<TsFileResource> unSequenceTsFileResource :
+            unSequenceTsFileResources.get(tsFileResource.getTimePartition())) {
+          unSequenceTsFileResource.remove(tsFileResource);
+        }
+      }
+    } finally {
+      writeUnlock();
     }
   }
 
-  public class LevelCompactionRecoverTask implements Callable<Void> {
-
-    private CloseCompactionMergeCallBack closeCompactionMergeCallBack;
-
-    public LevelCompactionRecoverTask(CloseCompactionMergeCallBack closeCompactionMergeCallBack) {
-      this.closeCompactionMergeCallBack = closeCompactionMergeCallBack;
+  /** remove some TsFiles from list */
+  public void removeAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
+    writeLock();
+    try {
+      if (sequence) {
+        for (List<SortedSet<TsFileResource>> partitionSequenceTsFileResource :
+            sequenceTsFileResources.values()) {
+          for (SortedSet<TsFileResource> levelTsFileResource : partitionSequenceTsFileResource) {
+            levelTsFileResource.removeAll(tsFileResourceList);
+          }
+        }
+      } else {
+        for (List<List<TsFileResource>> partitionUnSequenceTsFileResource :
+            unSequenceTsFileResources.values()) {
+          for (List<TsFileResource> levelTsFileResource : partitionUnSequenceTsFileResource) {
+            levelTsFileResource.removeAll(tsFileResourceList);
+          }
+        }
+      }
+    } finally {
+      writeUnlock();
     }
+  }
 
-    @Override
-    public Void call() {
-      recover();
-      // in recover logic, we do not have to start next compaction task, and in this case the param
-      // time partition is useless, we can just pass 0L
-      closeCompactionMergeCallBack.call(false, 0L);
-      return null;
+  /** add one TsFile to list */
+  public void add(TsFileResource tsFileResource, boolean sequence) {
+    writeLock();
+    try {
+      long timePartitionId = tsFileResource.getTimePartition();
+      int level = getMergeLevel(tsFileResource.getTsFile());
+      if (sequence) {
+        if (level <= seqLevelNum - 1) {
+          // current file has normal level
+          sequenceTsFileResources
+              .computeIfAbsent(timePartitionId, this::newSequenceTsFileResources)
+              .get(level)
+              .add(tsFileResource);
+        } else {
+          // current file has too high level
+          sequenceTsFileResources
+              .computeIfAbsent(timePartitionId, this::newSequenceTsFileResources)
+              .get(seqLevelNum - 1)
+              .add(tsFileResource);
+        }
+      } else {
+        if (level <= unseqLevelNum - 1) {
+          // current file has normal level
+          unSequenceTsFileResources
+              .computeIfAbsent(timePartitionId, this::newUnSequenceTsFileResources)
+              .get(level)
+              .add(tsFileResource);
+        } else {
+          // current file has too high level
+          unSequenceTsFileResources
+              .computeIfAbsent(timePartitionId, this::newUnSequenceTsFileResources)
+              .get(unseqLevelNum - 1)
+              .add(tsFileResource);
+        }
+      }
+    } finally {
+      writeUnlock();
     }
   }
 
-  protected void doUnseqMerge(
-      boolean fullMerge,
-      List<TsFileResource> seqMergeList,
-      List<TsFileResource> unSeqMergeList) {
-
-    if (seqMergeList.isEmpty() || unSeqMergeList.isEmpty()) {
-      logger.info("{} no files to be merged, seqFiles={}, unseqFiles={}",
-          storageGroupName, seqMergeList.size(), unSeqMergeList.size());
-      return;
+  /** add one TsFile to list for recover */
+  public void addRecover(TsFileResource tsFileResource, boolean sequence) {
+    if (sequence) {
+      sequenceRecoverTsFileResources.add(tsFileResource);
+    } else {
+      unSequenceRecoverTsFileResources.add(tsFileResource);
     }
+  }
 
-    // the number of unseq files in one merge should not exceed maxOpenFileNumInEachUnseqCompaction
-    if (unSeqMergeList.size() > maxOpenFileNumInEachUnseqCompaction) {
-      unSeqMergeList = unSeqMergeList.subList(0, maxOpenFileNumInEachUnseqCompaction);
+  /** add some TsFiles to list */
+  public void addAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
+    writeLock();
+    try {
+      for (TsFileResource tsFileResource : tsFileResourceList) {
+        add(tsFileResource, sequence);
+      }
+    } finally {
+      writeUnlock();
     }
+  }
 
-    long timeLowerBound = System.currentTimeMillis() - IoTDBDescriptor.getInstance().getConfig().getDefaultTTL();
-    MergeResource mergeResource = new MergeResource(seqMergeList, unSeqMergeList, timeLowerBound);
-
-    IMergeFileSelector fileSelector = getMergeFileSelector(mergeResource);
+  /** is one TsFile contained in list */
+  public boolean contains(TsFileResource tsFileResource, boolean sequence) {
+    readLock();
     try {
-      List[] mergeFiles = fileSelector.select();
-      if (mergeFiles.length == 0) {
-        logger.info(
-            "{} cannot select merge candidates under the budget {}",
-            storageGroupName, IoTDBDescriptor.getInstance().getConfig().getMergeMemoryBudget());
-        return;
+      if (sequence) {
+        for (SortedSet<TsFileResource> sequenceTsFileResource :
+            sequenceTsFileResources.computeIfAbsent(
+                tsFileResource.getTimePartition(), this::newSequenceTsFileResources)) {
+          if (sequenceTsFileResource.contains(tsFileResource)) {
+            return true;
+          }
+        }
+      } else {
+        for (List<TsFileResource> unSequenceTsFileResource :
+            unSequenceTsFileResources.computeIfAbsent(
+                tsFileResource.getTimePartition(), this::newUnSequenceTsFileResources)) {
+          if (unSequenceTsFileResource.contains(tsFileResource)) {
+            return true;
+          }
+        }
       }
-
-      mergeResource.startMerging();
-
-      MergeTask mergeTask =
-          new MergeTask(
-              mergeResource,
-              storageGroupDir,
-              this::mergeEndAction,
-              fullMerge,
-              fileSelector.getConcurrentMergeNum(),
-              storageGroupName);
-
-      mergingModification =
-          new ModificationFile(storageGroupDir + File.separator + MERGING_MODIFICATION_FILE_NAME);
-
-      logger.info(
-          "{} start merge {} seqFiles, {} unseqFiles", storageGroupName,
-          mergeFiles[0].size(),
-          mergeFiles[1].size());
-
-      mergeTask.doMerge();
-
-    } catch (Exception e) {
-      logger.error("{} cannot select file for merge", storageGroupName, e);
+      return false;
+    } finally {
+      readUnLock();
     }
   }
 
-  private IMergeFileSelector getMergeFileSelector(MergeResource resource) {
-    long budget = IoTDBDescriptor.getInstance().getConfig().getMergeMemoryBudget();
-    MergeFileStrategy strategy = IoTDBDescriptor.getInstance().getConfig().getMergeFileStrategy();
-    switch (strategy) {
-      case MAX_FILE_NUM:
-        return new MaxFileMergeFileSelector(resource, budget);
-      case MAX_SERIES_NUM:
-        return new MaxSeriesMergeFileSelector(resource, budget);
-      default:
-        throw new UnsupportedOperationException("Unknown MergeFileStrategy " + strategy);
+  /** clear list */
+  public void clear() {
+    writeLock();
+    try {
+      sequenceTsFileResources.clear();
+      unSequenceTsFileResources.clear();
+    } finally {
+      writeUnlock();
     }
   }
 
-  /** acquire the write locks of the resource , the merge lock and the compaction lock */
-  private void doubleWriteLock(TsFileResource seqFile) {
-    boolean fileLockGot;
-    boolean compactionLockGot;
-    while (true) {
-      fileLockGot = seqFile.tryWriteLock();
-      compactionLockGot = tryWriteLock();
-
-      if (fileLockGot && compactionLockGot) {
-        break;
-      } else {
-        // did not get all of them, release the gotten one and retry
-        if (compactionLockGot) {
-          writeUnlock();
+  /** is the list empty */
+  @SuppressWarnings("squid:S3776")
+  public boolean isEmpty(boolean sequence) {
+    readLock();
+    try {
+      if (sequence) {
+        for (List<SortedSet<TsFileResource>> partitionSequenceTsFileResource :
+            sequenceTsFileResources.values()) {
+          for (SortedSet<TsFileResource> sequenceTsFileResource : partitionSequenceTsFileResource) {
+            if (!sequenceTsFileResource.isEmpty()) {
+              return false;
+            }
+          }
         }
-        if (fileLockGot) {
-          seqFile.writeUnlock();
+      } else {
+        for (List<List<TsFileResource>> partitionUnSequenceTsFileResource :
+            unSequenceTsFileResources.values()) {
+          for (List<TsFileResource> unSequenceTsFileResource : partitionUnSequenceTsFileResource) {
+            if (!unSequenceTsFileResource.isEmpty()) {
+              return false;
+            }
+          }
         }
       }
+      return true;
+    } finally {
+      readUnLock();
     }
   }
 
-  /** release the write locks of the resource , the merge lock and the compaction lock */
-  private void doubleWriteUnlock(TsFileResource seqFile) {
-    writeUnlock();
-    seqFile.writeUnlock();
-  }
-
-  private void removeUnseqFiles(List<TsFileResource> unseqFiles) {
-    writeLock();
+  /** return TsFile list size */
+  public int size(boolean sequence) {
+    readLock();
     try {
-      removeAll(unseqFiles, false);
-      // clean cache
-      if (IoTDBDescriptor.getInstance().getConfig().isMetaDataCacheEnable()) {
-        ChunkCache.getInstance().clear();
-        TimeSeriesMetadataCache.getInstance().clear();
+      int result = 0;
+      if (sequence) {
+        for (List<SortedSet<TsFileResource>> partitionSequenceTsFileResource :
+            sequenceTsFileResources.values()) {
+          for (int i = seqLevelNum - 1; i >= 0; i--) {
+            result += partitionSequenceTsFileResource.get(i).size();
+          }
+        }
+      } else {
+        for (List<List<TsFileResource>> partitionUnSequenceTsFileResource :
+            unSequenceTsFileResources.values()) {
+          for (int i = unseqLevelNum - 1; i >= 0; i--) {
+            result += partitionUnSequenceTsFileResource.get(i).size();
+          }
+        }
       }
+      return result;
     } finally {
-      writeUnlock();
+      readUnLock();
     }
+  }
 
-    for (TsFileResource unseqFile : unseqFiles) {
-      unseqFile.writeLock();
-      try {
-        unseqFile.remove();
-      } finally {
-        unseqFile.writeUnlock();
-      }
+  public void deleteLevelFilesInDisk(Collection<TsFileResource> mergeTsFiles) {
+    logger.debug("{} [compaction] merge starts to delete real file", storageGroupName);
+    for (TsFileResource mergeTsFile : mergeTsFiles) {
+      deleteLevelFile(mergeTsFile);
+      logger.info(
+          "{} [Compaction] delete TsFile {}", storageGroupName, mergeTsFile.getTsFilePath());
     }
   }
 
-  @SuppressWarnings("squid:S1141")
-  private void updateMergeModification(TsFileResource seqFile) {
-    try {
-      // remove old modifications and write modifications generated during merge
-      seqFile.removeModFile();
-      if (mergingModification != null) {
-        for (Modification modification : mergingModification.getModifications()) {
-          // we have to set modification offset to MAX_VALUE, as the offset of source chunk may
-          // change after compaction
-          modification.setFileOffset(Long.MAX_VALUE);
-          seqFile.getModFile().write(modification);
+  public void deleteLevelFilesInList(
+      long timePartitionId, Collection<TsFileResource> mergeTsFiles, int level, boolean sequence) {
+    logger.debug("{} [compaction] merge starts to delete file list", storageGroupName);
+    if (sequence) {
+      if (sequenceTsFileResources.containsKey(timePartitionId)) {
+        if (sequenceTsFileResources.get(timePartitionId).size() > level) {
+          sequenceTsFileResources.get(timePartitionId).get(level).removeAll(mergeTsFiles);
         }
-        try {
-          seqFile.getModFile().close();
-        } catch (IOException e) {
-          logger.error(
-              "Cannot close the ModificationFile {}", seqFile.getModFile().getFilePath(), e);
+      }
+    } else {
+      if (unSequenceTsFileResources.containsKey(timePartitionId)) {
+        if (unSequenceTsFileResources.get(timePartitionId).size() > level) {
+          unSequenceTsFileResources.get(timePartitionId).get(level).removeAll(mergeTsFiles);
         }
       }
-    } catch (IOException e) {
-      logger.error(
-          "{} cannot clean the ModificationFile of {} after merge",
-          storageGroupName,
-          seqFile.getTsFile(),
-          e);
     }
   }
 
-  private void removeMergingModification() {
+  private void deleteLevelFile(TsFileResource seqFile) {
+    seqFile.writeLock();
     try {
-      if (mergingModification != null) {
-        mergingModification.remove();
-        mergingModification = null;
-      }
+      ChunkCache.getInstance().clear();
+      TimeSeriesMetadataCache.getInstance().clear();
+      FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile.getTsFilePath());
+      seqFile.setDeleted(true);
+      seqFile.delete();
     } catch (IOException e) {
-      logger.error("{} cannot remove merging modification ", storageGroupName, e);
+      logger.error(e.getMessage(), e);
+    } finally {
+      seqFile.writeUnlock();
     }
   }
 
-  public void mergeEndAction(
-      List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles, File mergeLog) {
-    logger.info("{} a merge task is ending...", storageGroupName);
+  public static int getMergeLevel(File file) {
+    String mergeLevelStr =
+        file.getPath()
+            .substring(file.getPath().lastIndexOf(FILE_NAME_SEPARATOR) + 1)
+            .replaceAll(TSFILE_SUFFIX, "");
+    return Integer.parseInt(mergeLevelStr);
+  }
 
-    if (Thread.currentThread().isInterrupted() || unseqFiles.isEmpty()) {
-      // merge task abort, or merge runtime exception arose, just end this merge
-      logger.info("{} a merge task abnormally ends", storageGroupName);
-      return;
+  private List<SortedSet<TsFileResource>> newSequenceTsFileResources(Long k) {
+    List<SortedSet<TsFileResource>> newSequenceTsFileResources = new ArrayList<>();
+    for (int i = 0; i < seqLevelNum; i++) {
+      newSequenceTsFileResources.add(
+          new TreeSet<>(
+              (o1, o2) -> {
+                try {
+                  int rangeCompare =
+                      Long.compare(
+                          Long.parseLong(o1.getTsFile().getParentFile().getName()),
+                          Long.parseLong(o2.getTsFile().getParentFile().getName()));
+                  return rangeCompare == 0
+                      ? compareFileName(o1.getTsFile(), o2.getTsFile())
+                      : rangeCompare;
+                } catch (NumberFormatException e) {
+                  return compareFileName(o1.getTsFile(), o2.getTsFile());
+                }
+              }));
     }
-    removeUnseqFiles(unseqFiles);
-
-    for (TsFileResource seqFile : seqFiles) {
-      // get both seqFile lock and merge lock
-      doubleWriteLock(seqFile);
-
-      try {
-        // if meet error(like file not found) in merge task, the .merge file may not be deleted
-        File mergedFile =
-            FSFactoryProducer.getFSFactory().getFile(seqFile.getTsFilePath() + MERGE_SUFFIX);
-        if (mergedFile.exists()) {
-          if (!mergedFile.delete()) {
-            logger.warn("Delete file {} failed", mergedFile);
-          }
+    return newSequenceTsFileResources;
+  }
+
+  private List<List<TsFileResource>> newUnSequenceTsFileResources(Long k) {
+    List<List<TsFileResource>> newUnSequenceTsFileResources = new ArrayList<>();
+    for (int i = 0; i < unseqLevelNum; i++) {
+      newUnSequenceTsFileResources.add(new ArrayList<>());
+    }
+    return newUnSequenceTsFileResources;
+  }
+
+  public TsFileResource getRecoverTsFileResource(String filePath, boolean isSeq)
+      throws IOException {
+    if (isSeq) {
+      for (TsFileResource tsFileResource : sequenceRecoverTsFileResources) {
+        if (Files.isSameFile(tsFileResource.getTsFile().toPath(), new File(filePath).toPath())) {
+          return tsFileResource;
+        }
+      }
+    } else {
+      for (TsFileResource tsFileResource : unSequenceRecoverTsFileResources) {
+        if (Files.isSameFile(tsFileResource.getTsFile().toPath(), new File(filePath).toPath())) {
+          return tsFileResource;
         }
-        updateMergeModification(seqFile);
-      } finally {
-        doubleWriteUnlock(seqFile);
       }
     }
+    logger.error("cannot get tsfile resource path: {}", filePath);
+    throw new IOException();
+  }
 
-    try {
-      removeMergingModification();
-      Files.delete(mergeLog.toPath());
-    } catch (IOException e) {
-      logger.error(
-          "{} a merge task ends but cannot delete log {}", storageGroupName, mergeLog.toPath());
+  public TsFileResource getTsFileResource(String filePath, boolean isSeq) throws IOException {
+    if (isSeq) {
+      for (List<SortedSet<TsFileResource>> tsFileResourcesWithLevel :
+          sequenceTsFileResources.values()) {
+        for (SortedSet<TsFileResource> tsFileResources : tsFileResourcesWithLevel) {
+          for (TsFileResource tsFileResource : tsFileResources) {
+            if (Files.isSameFile(
+                tsFileResource.getTsFile().toPath(), new File(filePath).toPath())) {
+              return tsFileResource;
+            }
+          }
+        }
+      }
+    } else {
+      for (List<List<TsFileResource>> tsFileResourcesWithLevel :
+          unSequenceTsFileResources.values()) {
+        for (List<TsFileResource> tsFileResources : tsFileResourcesWithLevel) {
+          for (TsFileResource tsFileResource : tsFileResources) {
+            if (Files.isSameFile(
+                tsFileResource.getTsFile().toPath(), new File(filePath).toPath())) {
+              return tsFileResource;
+            }
+          }
+        }
+      }
     }
+    logger.error("cannot get tsfile resource path: {}", filePath);
+    throw new IOException();
+  }
+
+  public void readLock() {
+    compactionMergeLock.readLock().lock();
+  }
 
-    logger.info("{} a merge task ends", storageGroupName);
+  public void readUnLock() {
+    compactionMergeLock.readLock().unlock();
+  }
+
+  public void writeLock() {
+    compactionMergeLock.writeLock().lock();
+  }
+
+  public void writeUnlock() {
+    compactionMergeLock.writeLock().unlock();
+  }
+
+  public boolean tryWriteLock() {
+    return compactionMergeLock.writeLock().tryLock();
   }
 
   // ({systemTime}-{versionNum}-{mergeNum}.tsfile)
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/CrossSpaceCompactionExecutor.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/CrossSpaceCompactionExecutor.java
new file mode 100644
index 0000000..fee7733
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/CrossSpaceCompactionExecutor.java
@@ -0,0 +1,16 @@
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
+
+import org.apache.iotdb.db.engine.compaction.TsFileManagement;
+
+public abstract class CrossSpaceCompactionExecutor {
+
+  protected TsFileManagement tsFileManagement;
+
+  public CrossSpaceCompactionExecutor(TsFileManagement tsFileManagement) {
+    this.tsFileManagement = tsFileManagement;
+  }
+
+  public abstract void recover();
+
+  public abstract void doCrossSpaceCompaction(boolean fullMerge, long timePartition);
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/InplaceCompactionExecutor.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/InplaceCompactionExecutor.java
new file mode 100644
index 0000000..3845a28
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/InplaceCompactionExecutor.java
@@ -0,0 +1,285 @@
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace;
+
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask.MERGE_SUFFIX;
+import static org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.MERGING_MODIFICATION_FILE_NAME;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.cache.ChunkCache;
+import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
+import org.apache.iotdb.db.engine.compaction.TsFileManagement;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.CrossSpaceCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.IMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.MaxFileMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.MaxSeriesMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.MergeFileStrategy;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.RecoverMergeTask;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class InplaceCompactionExecutor extends CrossSpaceCompactionExecutor {
+  private static final Logger logger = LoggerFactory.getLogger(InplaceCompactionExecutor.class);
+
+  private final int unseqLevelNum =
+      Math.max(IoTDBDescriptor.getInstance().getConfig().getUnseqLevelNum(), 1);
+  private final int maxOpenFileNumInEachUnseqCompaction =
+      IoTDBDescriptor.getInstance().getConfig().getMaxOpenFileNumInEachUnseqCompaction();
+
+  public InplaceCompactionExecutor(TsFileManagement tsFileManagement) {
+    super(tsFileManagement);
+  }
+
+  @Override
+  public void recover() {
+    // MERGE TODO: unify mods solution of inplace compaction and level compaction
+    try {
+      // recover mods
+      File mergingMods =
+          SystemFileFactory.INSTANCE.getFile(
+              tsFileManagement.storageGroupSysDir, MERGING_MODIFICATION_FILE_NAME);
+      if (mergingMods.exists()) {
+        this.tsFileManagement.mergingModification = new ModificationFile(mergingMods.getPath());
+      }
+      RecoverMergeTask recoverMergeTask =
+          new RecoverMergeTask(
+              new ArrayList<>(tsFileManagement.getTsFileList(true)),
+              tsFileManagement.getTsFileList(false),
+              tsFileManagement.storageGroupSysDir,
+              this::mergeEndAction,
+              IoTDBDescriptor.getInstance().getConfig().isFullMerge(),
+              tsFileManagement.storageGroupName);
+      logger.info("{} a RecoverMergeTask starts...", tsFileManagement.storageGroupName);
+      recoverMergeTask.recoverMerge(
+          IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot());
+      if (!IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot()) {
+        tsFileManagement.mergingModification.remove();
+      }
+    } catch (MetadataException | IOException e) {
+      logger.error("{} inplace compaction recover failed", tsFileManagement.storageGroupName, e);
+    }
+  }
+
+  @Override
+  public void doCrossSpaceCompaction(boolean fullMerge, long timePartition) {
+    List<TsFileResource> seqMergeList =
+        tsFileManagement.getTsFileListByTimePartition(true, timePartition);
+    List<TsFileResource> unSeqMergeList =
+        tsFileManagement.getTsFileListByTimePartitionAndLevel(
+            false, timePartition, unseqLevelNum - 1);
+    if (seqMergeList.isEmpty() || unSeqMergeList.isEmpty()) {
+      logger.info(
+          "{} no files to be merged, seqFiles={}, unseqFiles={}",
+          tsFileManagement.storageGroupName,
+          seqMergeList.size(),
+          unSeqMergeList.size());
+      return;
+    }
+
+    // the number of unseq files in one merge should not exceed maxOpenFileNumInEachUnseqCompaction
+    if (unSeqMergeList.size() > maxOpenFileNumInEachUnseqCompaction) {
+      unSeqMergeList = unSeqMergeList.subList(0, maxOpenFileNumInEachUnseqCompaction);
+    }
+
+    long timeLowerBound =
+        System.currentTimeMillis() - IoTDBDescriptor.getInstance().getConfig().getDefaultTTL();
+    MergeResource mergeResource = new MergeResource(seqMergeList, unSeqMergeList, timeLowerBound);
+
+    IMergeFileSelector fileSelector = getMergeFileSelector(mergeResource);
+    try {
+      List[] mergeFiles = fileSelector.select();
+      if (mergeFiles.length == 0) {
+        logger.info(
+            "{} cannot select merge candidates under the budget {}",
+            tsFileManagement.storageGroupName,
+            IoTDBDescriptor.getInstance().getConfig().getMergeMemoryBudget());
+        return;
+      }
+
+      mergeResource.startMerging();
+
+      MergeTask mergeTask =
+          new MergeTask(
+              mergeResource,
+              tsFileManagement.storageGroupSysDir,
+              this::mergeEndAction,
+              fullMerge,
+              fileSelector.getConcurrentMergeNum(),
+              tsFileManagement.storageGroupName);
+
+      tsFileManagement.mergingModification =
+          new ModificationFile(
+              tsFileManagement.storageGroupSysDir
+                  + File.separator
+                  + MERGING_MODIFICATION_FILE_NAME);
+
+      logger.info(
+          "{} start merge {} seqFiles, {} unseqFiles",
+          tsFileManagement.storageGroupName,
+          mergeFiles[0].size(),
+          mergeFiles[1].size());
+
+      mergeTask.doMerge();
+
+    } catch (Exception e) {
+      logger.error("{} cannot select file for merge", tsFileManagement.storageGroupName, e);
+    }
+  }
+
+  private IMergeFileSelector getMergeFileSelector(MergeResource resource) {
+    long budget = IoTDBDescriptor.getInstance().getConfig().getMergeMemoryBudget();
+    MergeFileStrategy strategy = IoTDBDescriptor.getInstance().getConfig().getMergeFileStrategy();
+    switch (strategy) {
+      case MAX_FILE_NUM:
+        return new MaxFileMergeFileSelector(resource, budget);
+      case MAX_SERIES_NUM:
+        return new MaxSeriesMergeFileSelector(resource, budget);
+      default:
+        throw new UnsupportedOperationException("Unknown MergeFileStrategy " + strategy);
+    }
+  }
+
+  /** acquire the write locks of the resource , the merge lock and the compaction lock */
+  private void doubleWriteLock(TsFileResource seqFile) {
+    boolean fileLockGot;
+    boolean compactionLockGot;
+    while (true) {
+      fileLockGot = seqFile.tryWriteLock();
+      compactionLockGot = tsFileManagement.tryWriteLock();
+
+      if (fileLockGot && compactionLockGot) {
+        break;
+      } else {
+        // did not get all of them, release the gotten one and retry
+        if (compactionLockGot) {
+          tsFileManagement.writeUnlock();
+        }
+        if (fileLockGot) {
+          seqFile.writeUnlock();
+        }
+      }
+    }
+  }
+
+  /** release the write locks of the resource , the merge lock and the compaction lock */
+  private void doubleWriteUnlock(TsFileResource seqFile) {
+    tsFileManagement.writeUnlock();
+    seqFile.writeUnlock();
+  }
+
+  private void removeUnseqFiles(List<TsFileResource> unseqFiles) {
+    tsFileManagement.writeLock();
+    try {
+      tsFileManagement.removeAll(unseqFiles, false);
+      // clean cache
+      if (IoTDBDescriptor.getInstance().getConfig().isMetaDataCacheEnable()) {
+        ChunkCache.getInstance().clear();
+        TimeSeriesMetadataCache.getInstance().clear();
+      }
+    } finally {
+      tsFileManagement.writeUnlock();
+    }
+
+    for (TsFileResource unseqFile : unseqFiles) {
+      unseqFile.writeLock();
+      try {
+        unseqFile.remove();
+      } finally {
+        unseqFile.writeUnlock();
+      }
+    }
+  }
+
+  @SuppressWarnings("squid:S1141")
+  private void updateMergeModification(TsFileResource seqFile) {
+    try {
+      // remove old modifications and write modifications generated during merge
+      seqFile.removeModFile();
+      if (tsFileManagement.mergingModification != null) {
+        for (Modification modification : tsFileManagement.mergingModification.getModifications()) {
+          // we have to set modification offset to MAX_VALUE, as the offset of source chunk may
+          // change after compaction
+          modification.setFileOffset(Long.MAX_VALUE);
+          seqFile.getModFile().write(modification);
+        }
+        try {
+          seqFile.getModFile().close();
+        } catch (IOException e) {
+          logger.error(
+              "Cannot close the ModificationFile {}", seqFile.getModFile().getFilePath(), e);
+        }
+      }
+    } catch (IOException e) {
+      logger.error(
+          "{} cannot clean the ModificationFile of {} after merge",
+          tsFileManagement.storageGroupName,
+          seqFile.getTsFile(),
+          e);
+    }
+  }
+
+  private void removeMergingModification() {
+    try {
+      if (tsFileManagement.mergingModification != null) {
+        tsFileManagement.mergingModification.remove();
+        tsFileManagement.mergingModification = null;
+      }
+    } catch (IOException e) {
+      logger.error("{} cannot remove merging modification ", tsFileManagement.storageGroupName, e);
+    }
+  }
+
+  public void mergeEndAction(
+      List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles, File mergeLog) {
+    logger.info("{} a merge task is ending...", tsFileManagement.storageGroupName);
+
+    if (Thread.currentThread().isInterrupted() || unseqFiles.isEmpty()) {
+      // merge task abort, or merge runtime exception arose, just end this merge
+      logger.info("{} a merge task abnormally ends", tsFileManagement.storageGroupName);
+      return;
+    }
+    removeUnseqFiles(unseqFiles);
+
+    for (TsFileResource seqFile : seqFiles) {
+      // get both seqFile lock and merge lock
+      doubleWriteLock(seqFile);
+
+      try {
+        // if meet error(like file not found) in merge task, the .merge file may not be deleted
+        File mergedFile =
+            FSFactoryProducer.getFSFactory().getFile(seqFile.getTsFilePath() + MERGE_SUFFIX);
+        if (mergedFile.exists()) {
+          if (!mergedFile.delete()) {
+            logger.warn("Delete file {} failed", mergedFile);
+          }
+        }
+        updateMergeModification(seqFile);
+      } finally {
+        doubleWriteUnlock(seqFile);
+      }
+    }
+
+    try {
+      removeMergingModification();
+      Files.delete(mergeLog.toPath());
+    } catch (IOException e) {
+      logger.error(
+          "{} a merge task ends but cannot delete log {}",
+          tsFileManagement.storageGroupName,
+          mergeLog.toPath());
+    }
+
+    logger.info("{} a merge task ends", tsFileManagement.storageGroupName);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeContext.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeContext.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeContext.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeContext.java
index 803df49..f06c523 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeContext.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.manage;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage;
 
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.metadata.PartialPath;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeFuture.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeFuture.java
similarity index 92%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeFuture.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeFuture.java
index c8c3d67..c56cf01 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeFuture.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeFuture.java
@@ -17,10 +17,10 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.manage;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage;
 
-import org.apache.iotdb.db.engine.merge.task.MergeMultiChunkTask.MergeChunkHeapTask;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeMultiChunkTask.MergeChunkHeapTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeManager.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeManager.java
similarity index 94%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeManager.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeManager.java
index 84c6859..362dabc 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeManager.java
@@ -17,14 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.manage;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage;
 
 import org.apache.iotdb.db.concurrent.ThreadName;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.merge.task.MergeMultiChunkTask.MergeChunkHeapTask;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeMultiChunkTask.MergeChunkHeapTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.service.IService;
 import org.apache.iotdb.db.service.JMXService;
@@ -143,13 +143,6 @@ public class MergeManager implements IService, MergeManagerMBean {
           new MergeThreadPool(
               threadNum * chunkSubThreadNum,
               r -> new Thread(r, "MergeChunkSubThread-" + threadCnt.getAndIncrement()));
-      long mergeInterval = IoTDBDescriptor.getInstance().getConfig().getMergeIntervalSec();
-      if (mergeInterval > 0) {
-        timedMergeThreadPool =
-            Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "TimedMergeThread"));
-        timedMergeThreadPool.scheduleAtFixedRate(
-            this::mergeAll, mergeInterval, mergeInterval, TimeUnit.SECONDS);
-      }
 
       taskCleanerThreadPool =
           Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "MergeTaskCleaner"));
@@ -250,15 +243,6 @@ public class MergeManager implements IService, MergeManagerMBean {
     return ServiceType.MERGE_SERVICE;
   }
 
-  private void mergeAll() {
-    try {
-      StorageEngine.getInstance()
-          .mergeAll(IoTDBDescriptor.getInstance().getConfig().isFullMerge());
-    } catch (StorageEngineException e) {
-      logger.error("Cannot perform a global merge because", e);
-    }
-  }
-
   /**
    * Abort all merges of a storage group. The caller must acquire the write lock of the
    * corresponding storage group.
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeManagerMBean.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeManagerMBean.java
similarity index 91%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeManagerMBean.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeManagerMBean.java
index 645f51f..c9f8c13 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeManagerMBean.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeManagerMBean.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.manage;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage;
 
 public interface MergeManagerMBean {
   void printMergeStatus();
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeResource.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeResource.java
index 050d83e..4b95f8c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeResource.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.manage;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage;
 
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -49,7 +49,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 
-import static org.apache.iotdb.db.engine.merge.task.MergeTask.MERGE_SUFFIX;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask.MERGE_SUFFIX;
 
 /**
  * MergeResource manages files and caches of readers, writers, MeasurementSchemas and modifications
@@ -99,8 +99,8 @@ public class MergeResource {
   /**
    * startMerging() is called after selecting files
    *
-   * do not cache metadata until true candidates are chosen, or too much metadata will be
-   * cached during selection
+   * <p>do not cache metadata until true candidates are chosen, or too much metadata will be cached
+   * during selection
    */
   public void startMerging() {
     cacheDeviceMeta = true;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeThreadPool.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeThreadPool.java
similarity index 76%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeThreadPool.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeThreadPool.java
index 1815f30..ec8d4ca 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeThreadPool.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/manage/MergeThreadPool.java
@@ -17,12 +17,12 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.manage;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage;
 
-import org.apache.iotdb.db.engine.merge.manage.MergeFuture.MainMergeFuture;
-import org.apache.iotdb.db.engine.merge.manage.MergeFuture.SubMergeFuture;
-import org.apache.iotdb.db.engine.merge.task.MergeMultiChunkTask.MergeChunkHeapTask;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeFuture.MainMergeFuture;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeFuture.SubMergeFuture;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeMultiChunkTask.MergeChunkHeapTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 
 import java.util.concurrent.Callable;
 import java.util.concurrent.LinkedBlockingQueue;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/recover/LogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/recover/LogAnalyzer.java
similarity index 90%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/recover/LogAnalyzer.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/recover/LogAnalyzer.java
index e6f84fc..04c06a1 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/recover/LogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/recover/LogAnalyzer.java
@@ -17,11 +17,11 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.recover;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover;
 
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
@@ -42,14 +42,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import static org.apache.iotdb.db.engine.merge.recover.MergeLogger.STR_ALL_TS_END;
-import static org.apache.iotdb.db.engine.merge.recover.MergeLogger.STR_END;
-import static org.apache.iotdb.db.engine.merge.recover.MergeLogger.STR_MERGE_END;
-import static org.apache.iotdb.db.engine.merge.recover.MergeLogger.STR_MERGE_START;
-import static org.apache.iotdb.db.engine.merge.recover.MergeLogger.STR_SEQ_FILES;
-import static org.apache.iotdb.db.engine.merge.recover.MergeLogger.STR_START;
-import static org.apache.iotdb.db.engine.merge.recover.MergeLogger.STR_TIMESERIES;
-import static org.apache.iotdb.db.engine.merge.recover.MergeLogger.STR_UNSEQ_FILES;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger.STR_ALL_TS_END;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger.STR_END;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger.STR_MERGE_END;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger.STR_MERGE_START;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger.STR_SEQ_FILES;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger.STR_START;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger.STR_TIMESERIES;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger.STR_UNSEQ_FILES;
 
 /**
  * LogAnalyzer scans the "merge.log" file and recovers information such as files of last merge, the
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/recover/MergeLogger.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/recover/MergeLogger.java
similarity index 95%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/recover/MergeLogger.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/recover/MergeLogger.java
index 7932c01..6a2eb83 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/recover/MergeLogger.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/recover/MergeLogger.java
@@ -17,9 +17,9 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.recover;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover;
 
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.metadata.PartialPath;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IFileQueryMemMeasurement.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IFileQueryMemMeasurement.java
similarity index 93%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IFileQueryMemMeasurement.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IFileQueryMemMeasurement.java
index 48e70de..40183ac 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IFileQueryMemMeasurement.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IFileQueryMemMeasurement.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.selector;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector;
 
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IMergeFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IMergeFileSelector.java
similarity index 93%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IMergeFileSelector.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IMergeFileSelector.java
index 63fc04c..f858b40 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IMergeFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IMergeFileSelector.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.selector;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector;
 
 import org.apache.iotdb.db.exception.MergeException;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IMergePathSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IMergePathSelector.java
similarity index 92%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IMergePathSelector.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IMergePathSelector.java
index 207e67b..20bd984 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/IMergePathSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/IMergePathSelector.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.selector;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector;
 
 import org.apache.iotdb.db.metadata.PartialPath;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxFileMergeFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MaxFileMergeFileSelector.java
similarity index 98%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxFileMergeFileSelector.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MaxFileMergeFileSelector.java
index 3deb763..673d8a9 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxFileMergeFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MaxFileMergeFileSelector.java
@@ -17,10 +17,10 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.selector;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.MergeException;
 import org.apache.iotdb.db.utils.MergeUtils;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxSeriesMergeFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MaxSeriesMergeFileSelector.java
similarity index 95%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxSeriesMergeFileSelector.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MaxSeriesMergeFileSelector.java
index 70c7233..68fb1d3 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxSeriesMergeFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MaxSeriesMergeFileSelector.java
@@ -17,9 +17,9 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.selector;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector;
 
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.MergeException;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MergeFileStrategy.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MergeFileStrategy.java
similarity index 91%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MergeFileStrategy.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MergeFileStrategy.java
index 874a91b..b5fc410 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MergeFileStrategy.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/MergeFileStrategy.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.selector;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector;
 
 public enum MergeFileStrategy {
   MAX_SERIES_NUM,
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/NaivePathSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/NaivePathSelector.java
similarity index 95%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/selector/NaivePathSelector.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/NaivePathSelector.java
index cdef8c7..3ecc8ad 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/selector/NaivePathSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/selector/NaivePathSelector.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.selector;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector;
 
 import org.apache.iotdb.db.metadata.PartialPath;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeCallback.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeCallback.java
similarity index 94%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeCallback.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeCallback.java
index f7a43e6..f83f319 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeCallback.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeCallback.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.task;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task;
 
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeFileTask.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeFileTask.java
index 67bf9d8..9a259b9 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeFileTask.java
@@ -17,14 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.task;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.merge.manage.MergeContext;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.recover.MergeLogger;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeContext;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.query.control.FileReaderManager;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeMultiChunkTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeMultiChunkTask.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeMultiChunkTask.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeMultiChunkTask.java
index e44e248..7453ac6 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeMultiChunkTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeMultiChunkTask.java
@@ -17,16 +17,16 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.task;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement;
-import org.apache.iotdb.db.engine.merge.manage.MergeContext;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.recover.MergeLogger;
-import org.apache.iotdb.db.engine.merge.selector.IMergePathSelector;
-import org.apache.iotdb.db.engine.merge.selector.NaivePathSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeContext;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.IMergePathSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.NaivePathSelector;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.metadata.PartialPath;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeTask.java
similarity index 95%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeTask.java
index 89f4574..c442efc 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/MergeTask.java
@@ -17,13 +17,12 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.task;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task;
 
-import org.apache.iotdb.db.engine.merge.manage.MergeContext;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.recover.MergeLogger;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeContext;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
@@ -43,7 +42,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.concurrent.Callable;
 
 /**
  * MergeTask merges given seqFiles and unseqFiles into new ones, which basically consists of three
@@ -74,13 +72,11 @@ public class MergeTask {
       List<TsFileResource> unseqFiles,
       String storageGroupSysDir,
       MergeCallback callback,
-      String taskName,
       boolean fullMerge,
       String storageGroupName) {
     this.resource = new MergeResource(seqFiles, unseqFiles);
     this.storageGroupSysDir = storageGroupSysDir;
     this.callback = callback;
-    this.taskName = taskName;
     this.fullMerge = fullMerge;
     this.concurrentMergeSeriesNum = 1;
     this.storageGroupName = storageGroupName;
@@ -102,7 +98,6 @@ public class MergeTask {
     this.storageGroupName = storageGroupName;
   }
 
-
   private void abort() throws IOException {
     states = States.ABORTED;
     cleanUp(false);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/RecoverMergeTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/RecoverMergeTask.java
similarity index 95%
rename from server/src/main/java/org/apache/iotdb/db/engine/merge/task/RecoverMergeTask.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/RecoverMergeTask.java
index cb1e39e..ea351e0 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/RecoverMergeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/inplace/task/RecoverMergeTask.java
@@ -17,13 +17,13 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge.task;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.merge.recover.LogAnalyzer;
-import org.apache.iotdb.db.engine.merge.recover.LogAnalyzer.Status;
-import org.apache.iotdb.db.engine.merge.recover.MergeLogger;
-import org.apache.iotdb.db.engine.merge.selector.MaxSeriesMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.LogAnalyzer;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.LogAnalyzer.Status;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.recover.MergeLogger;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.MaxSeriesMergeFileSelector;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.PartialPath;
@@ -59,11 +59,9 @@ public class RecoverMergeTask extends MergeTask {
       List<TsFileResource> unseqFiles,
       String storageGroupSysDir,
       MergeCallback callback,
-      String taskName,
       boolean fullMerge,
       String storageGroupName) {
-    super(
-        seqFiles, unseqFiles, storageGroupSysDir, callback, taskName, fullMerge, storageGroupName);
+    super(seqFiles, unseqFiles, storageGroupSysDir, callback, fullMerge, storageGroupName);
   }
 
   public void recoverMerge(boolean continueMerge) throws IOException, MetadataException {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionStrategy.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/CompactionStrategy.java
similarity index 63%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionStrategy.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/CompactionStrategy.java
index 96ec9f9..3e69b4b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionStrategy.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/CompactionStrategy.java
@@ -17,22 +17,24 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.compaction;
+package org.apache.iotdb.db.engine.compaction.innerSpaceCompaction;
 
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
-import org.apache.iotdb.db.engine.compaction.no.NoCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.TsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.no.NoCompactionTsFileManagement;
 
 public enum CompactionStrategy {
   LEVEL_COMPACTION,
   NO_COMPACTION;
 
-  public TsFileManagement getTsFileManagement(String storageGroupName, String storageGroupDir) {
+  public InnerSpaceCompactionExecutor getInnerSpaceCompactionExecutor(
+      TsFileManagement tsFileManagement) {
     switch (this) {
       case LEVEL_COMPACTION:
-        return new LevelCompactionTsFileManagement(storageGroupName, storageGroupDir);
+        return new LevelCompactionExecutor(tsFileManagement);
       case NO_COMPACTION:
       default:
-        return new NoCompactionTsFileManagement(storageGroupName, storageGroupDir);
+        return new NoCompactionTsFileManagement(tsFileManagement);
     }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/InnerSpaceCompactionExecutor.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/InnerSpaceCompactionExecutor.java
new file mode 100644
index 0000000..6faf07b
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/InnerSpaceCompactionExecutor.java
@@ -0,0 +1,16 @@
+package org.apache.iotdb.db.engine.compaction.innerSpaceCompaction;
+
+import org.apache.iotdb.db.engine.compaction.TsFileManagement;
+
+public abstract class InnerSpaceCompactionExecutor {
+
+  protected TsFileManagement tsFileManagement;
+
+  public InnerSpaceCompactionExecutor(TsFileManagement tsFileManagement) {
+    this.tsFileManagement = tsFileManagement;
+  }
+
+  public abstract void recover();
+
+  public abstract void doInnerSpaceCompaction(boolean sequence, long timePartition);
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/level/LevelCompactionExecutor.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/level/LevelCompactionExecutor.java
new file mode 100644
index 0000000..0203d3a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/level/LevelCompactionExecutor.java
@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level;
+
+import static org.apache.iotdb.db.engine.compaction.TsFileManagement.getMergeLevel;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.SOURCE_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.TARGET_NAME;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.TsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.InnerSpaceCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogAnalyzer;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionUtils;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** The TsFileManagement for LEVEL_COMPACTION, use level struct to manage TsFile list */
+public class LevelCompactionExecutor extends InnerSpaceCompactionExecutor {
+
+  private static final Logger logger = LoggerFactory.getLogger(LevelCompactionExecutor.class);
+
+  public LevelCompactionExecutor(TsFileManagement tsFileManagement) {
+    super(tsFileManagement);
+  }
+
+  public void renameLevelFilesMods(
+      Collection<Modification> filterModification,
+      Collection<TsFileResource> mergeTsFiles,
+      TsFileResource targetTsFile)
+      throws IOException {
+    logger.debug(
+        "{} [compaction] merge starts to rename real file's mod",
+        tsFileManagement.storageGroupName);
+    List<Modification> modifications = new ArrayList<>();
+    for (TsFileResource mergeTsFile : mergeTsFiles) {
+      try (ModificationFile sourceModificationFile =
+          new ModificationFile(mergeTsFile.getTsFilePath() + ModificationFile.FILE_SUFFIX)) {
+        modifications.addAll(sourceModificationFile.getModifications());
+        if (sourceModificationFile.exists()) {
+          sourceModificationFile.remove();
+        }
+      }
+    }
+    modifications.removeAll(filterModification);
+    if (!modifications.isEmpty()) {
+      try (ModificationFile modificationFile =
+          new ModificationFile(targetTsFile.getTsFilePath() + ModificationFile.FILE_SUFFIX)) {
+        for (Modification modification : modifications) {
+          // we have to set modification offset to MAX_VALUE, as the offset of source chunk may
+          // change after compaction
+          modification.setFileOffset(Long.MAX_VALUE);
+          modificationFile.write(modification);
+        }
+      }
+    }
+  }
+
+  /** recover files */
+  @Override
+  @SuppressWarnings({"squid:S3776", "squid:S2142"})
+  public void recover() {
+    File logFile =
+        FSFactoryProducer.getFSFactory()
+            .getFile(
+                tsFileManagement.storageGroupSysDir,
+                tsFileManagement.storageGroupName + COMPACTION_LOG_NAME);
+    try {
+      if (logFile.exists()) {
+        CompactionLogAnalyzer logAnalyzer = new CompactionLogAnalyzer(logFile);
+        logAnalyzer.analyze();
+        Set<String> deviceSet = logAnalyzer.getDeviceSet();
+        List<String> sourceFileList = logAnalyzer.getSourceFiles();
+        long offset = logAnalyzer.getOffset();
+        String targetFile = logAnalyzer.getTargetFile();
+        boolean isSeq = logAnalyzer.isSeq();
+        if (targetFile == null || sourceFileList.isEmpty()) {
+          return;
+        }
+        File target = new File(targetFile);
+        if (deviceSet.isEmpty()) {
+          // if not in compaction, just delete the target file
+          if (target.exists()) {
+            Files.delete(target.toPath());
+          }
+          return;
+        }
+        // get tsfile resource from list, as they have been recovered in StorageGroupProcessor
+        TsFileResource targetResource =
+            tsFileManagement.getRecoverTsFileResource(targetFile, isSeq);
+        long timePartition = targetResource.getTimePartition();
+        List<TsFileResource> sourceTsFileResources = new ArrayList<>();
+        for (String file : sourceFileList) {
+          // get tsfile resource from list, as they have been recovered in StorageGroupProcessor
+          sourceTsFileResources.add(tsFileManagement.getTsFileResource(file, isSeq));
+        }
+        int level = getMergeLevel(new File(sourceFileList.get(0)));
+        RestorableTsFileIOWriter writer = new RestorableTsFileIOWriter(target);
+        // if not complete compaction, resume merge
+        if (writer.hasCrashed()) {
+          if (offset > 0) {
+            writer.getIOWriterOut().truncate(offset - 1);
+          }
+          writer.close();
+          CompactionLogger compactionLogger =
+              new CompactionLogger(
+                  tsFileManagement.storageGroupSysDir, tsFileManagement.storageGroupName);
+          List<Modification> modifications = new ArrayList<>();
+          CompactionUtils.merge(
+              targetResource,
+              sourceTsFileResources,
+              tsFileManagement.storageGroupName,
+              compactionLogger,
+              deviceSet,
+              isSeq,
+              modifications);
+          // complete compaction and delete source file
+          tsFileManagement.writeLock();
+          try {
+            if (Thread.currentThread().isInterrupted()) {
+              throw new InterruptedException(
+                  String.format("%s [Compaction] abort", tsFileManagement.storageGroupName));
+            }
+            tsFileManagement.add(targetResource, isSeq);
+            tsFileManagement.deleteLevelFilesInList(
+                timePartition, sourceTsFileResources, level, isSeq);
+          } finally {
+            tsFileManagement.writeUnlock();
+          }
+          tsFileManagement.deleteLevelFilesInDisk(sourceTsFileResources);
+          renameLevelFilesMods(modifications, sourceTsFileResources, targetResource);
+          compactionLogger.close();
+        } else {
+          writer.close();
+        }
+      }
+    } catch (IOException | IllegalPathException | InterruptedException e) {
+      logger.error("recover level tsfile management error ", e);
+    } finally {
+      if (logFile.exists()) {
+        try {
+          Files.delete(logFile.toPath());
+        } catch (IOException e) {
+          logger.error("delete level tsfile management log file error ", e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void doInnerSpaceCompaction(boolean sequence, long timePartition) {
+    List<List<TsFileResource>> mergeResources =
+        tsFileManagement.getClosedTsFileListByTimePartition(true, timePartition);
+    int currMaxLevel =
+        sequence
+            ? IoTDBDescriptor.getInstance().getConfig().getSeqLevelNum()
+            : IoTDBDescriptor.getInstance().getConfig().getUnseqLevelNum();
+    int currMaxFileNumInEachLevel =
+        sequence
+            ? IoTDBDescriptor.getInstance().getConfig().getSeqFileNumInEachLevel()
+            : IoTDBDescriptor.getInstance().getConfig().getUnseqFileNumInEachLevel();
+    // wait until unseq merge has finished
+    long startTimeMillis = System.currentTimeMillis();
+    CompactionLogger compactionLogger = null;
+    try {
+      logger.info("{} start to filter compaction condition", tsFileManagement.storageGroupName);
+      for (int i = 0; i < currMaxLevel - 1; i++) {
+        List<TsFileResource> currLevelTsFileResource = mergeResources.get(i);
+        if (currMaxFileNumInEachLevel <= currLevelTsFileResource.size()) {
+          // just merge part of the file
+          compactionLogger =
+              new CompactionLogger(
+                  tsFileManagement.storageGroupSysDir, tsFileManagement.storageGroupName);
+          // log source file list and target file for recover
+          for (TsFileResource mergeResource : mergeResources.get(i)) {
+            compactionLogger.logFile(SOURCE_NAME, mergeResource.getTsFile());
+          }
+          File newLevelFile =
+              TsFileResource.modifyTsFileNameMergeCnt(mergeResources.get(i).get(0).getTsFile());
+          compactionLogger.logSequence(sequence);
+          compactionLogger.logFile(TARGET_NAME, newLevelFile);
+          List<TsFileResource> toMergeTsFiles =
+              mergeResources.get(i).subList(0, currMaxFileNumInEachLevel);
+          logger.info(
+              "{} [Compaction] merge level-{}'s {} TsFiles to next level",
+              tsFileManagement.storageGroupName,
+              i,
+              toMergeTsFiles.size());
+          for (TsFileResource toMergeTsFile : toMergeTsFiles) {
+            logger.info(
+                "{} [Compaction] start to merge TsFile {}",
+                tsFileManagement.storageGroupName,
+                toMergeTsFile);
+          }
+
+          TsFileResource newResource = new TsFileResource(newLevelFile);
+          List<Modification> modifications = new ArrayList<>();
+          // merge, read from source files and write to target file
+          CompactionUtils.merge(
+              newResource,
+              toMergeTsFiles,
+              tsFileManagement.storageGroupName,
+              compactionLogger,
+              new HashSet<>(),
+              sequence,
+              modifications);
+          logger.info(
+              "{} [Compaction] merged level-{}'s {} TsFiles to next level, and start to delete old files",
+              tsFileManagement.storageGroupName,
+              i,
+              toMergeTsFiles.size());
+          tsFileManagement.writeLock();
+          try {
+            if (Thread.currentThread().isInterrupted()) {
+              throw new InterruptedException(
+                  String.format("%s [Compaction] abort", tsFileManagement.storageGroupName));
+            }
+
+            tsFileManagement.add(newResource, sequence);
+            tsFileManagement.deleteLevelFilesInList(timePartition, toMergeTsFiles, i, sequence);
+            if (mergeResources.size() > i + 1) {
+              mergeResources.get(i + 1).add(newResource);
+            }
+          } finally {
+            tsFileManagement.writeUnlock();
+          }
+          tsFileManagement.deleteLevelFilesInDisk(toMergeTsFiles);
+          renameLevelFilesMods(modifications, toMergeTsFiles, newResource);
+          compactionLogger.close();
+          File logFile =
+              FSFactoryProducer.getFSFactory()
+                  .getFile(
+                      tsFileManagement.storageGroupSysDir,
+                      tsFileManagement.storageGroupName + COMPACTION_LOG_NAME);
+          if (logFile.exists()) {
+            Files.delete(logFile.toPath());
+          }
+          break;
+        }
+      }
+    } catch (Exception e) {
+      if (compactionLogger != null) {
+        try {
+          compactionLogger.close();
+        } catch (IOException ioException) {
+          logger.error(
+              "{} Compaction log close fail",
+              tsFileManagement.storageGroupName + COMPACTION_LOG_NAME);
+        }
+      }
+      restoreCompaction();
+      logger.error("Error occurred in Compaction Merge thread", e);
+    } finally {
+      // reset the merge working state to false
+      logger.info(
+          "{} [Compaction] merge end time isSeq = {}, consumption: {} ms",
+          tsFileManagement.storageGroupName,
+          sequence,
+          System.currentTimeMillis() - startTimeMillis);
+    }
+  }
+
+  /** restore the files back to the status before the compaction task is submitted */
+  private void restoreCompaction() {
+    File logFile =
+        FSFactoryProducer.getFSFactory()
+            .getFile(
+                tsFileManagement.storageGroupSysDir,
+                tsFileManagement.storageGroupName + COMPACTION_LOG_NAME);
+    try {
+      if (logFile.exists()) {
+        CompactionLogAnalyzer logAnalyzer = new CompactionLogAnalyzer(logFile);
+        logAnalyzer.analyze();
+        String targetFilePath = logAnalyzer.getTargetFile();
+        if (targetFilePath != null) {
+          File targetFile = new File(targetFilePath);
+          if (targetFile.exists()) {
+            if (!targetFile.delete()) {
+              logger.warn("Delete file {} failed", targetFile);
+            }
+          }
+        }
+      }
+    } catch (IOException e) {
+      logger.error("restore compaction failed", e);
+    } finally {
+      if (logFile.exists()) {
+        try {
+          Files.delete(logFile.toPath());
+        } catch (IOException e) {
+          logger.error("delete compaction log file error ", e);
+        }
+      }
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/no/NoCompactionTsFileManagement.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/no/NoCompactionTsFileManagement.java
new file mode 100644
index 0000000..15e3e11
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/no/NoCompactionTsFileManagement.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.no;
+
+import org.apache.iotdb.db.engine.compaction.TsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.InnerSpaceCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class NoCompactionTsFileManagement extends InnerSpaceCompactionExecutor {
+  private static final Logger logger = LoggerFactory.getLogger(LevelCompactionExecutor.class);
+
+  public NoCompactionTsFileManagement(TsFileManagement tsFileManagement) {
+    super(tsFileManagement);
+  }
+
+  @Override
+  public void recover() {
+    logger.info("no recover logic");
+  }
+
+  @Override
+  public void doInnerSpaceCompaction(
+      List<List<TsFileResource>> mergeResources, boolean sequence, long timePartition) {
+    logger.info("no compaction logic");
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionLogAnalyzer.java
similarity index 80%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionLogAnalyzer.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionLogAnalyzer.java
index 51662b2..e75c013 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionLogAnalyzer.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.compaction.utils;
+package org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -28,11 +28,10 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.FULL_MERGE;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.SEQUENCE_NAME;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.SOURCE_NAME;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.TARGET_NAME;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.UNSEQUENCE_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.SEQUENCE_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.SOURCE_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.TARGET_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.UNSEQUENCE_NAME;
 
 public class CompactionLogAnalyzer {
 
@@ -44,7 +43,6 @@ public class CompactionLogAnalyzer {
   private List<String> sourceFiles = new ArrayList<>();
   private String targetFile = null;
   private boolean isSeq = false;
-  private boolean fullMerge = false;
 
   public CompactionLogAnalyzer(File logFile) {
     this.logFile = logFile;
@@ -67,9 +65,6 @@ public class CompactionLogAnalyzer {
             currLine = bufferedReader.readLine();
             targetFile = currLine;
             break;
-          case FULL_MERGE:
-            fullMerge = true;
-            break;
           case SEQUENCE_NAME:
             isSeq = true;
             break;
@@ -105,8 +100,4 @@ public class CompactionLogAnalyzer {
   public boolean isSeq() {
     return isSeq;
   }
-
-  public boolean isFullMerge() {
-    return fullMerge;
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionLogger.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionLogger.java
similarity index 93%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionLogger.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionLogger.java
index 4f57653..522a54a 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionLogger.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionLogger.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.compaction.utils;
+package org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils;
 
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 
@@ -33,8 +33,6 @@ public class CompactionLogger {
   public static final String TARGET_NAME = "target";
   public static final String SEQUENCE_NAME = "sequence";
   public static final String UNSEQUENCE_NAME = "unsequence";
-  public static final String FULL_MERGE = "full merge";
-  public static final String MERGE_FINISHED = "merge finished";
 
   private BufferedWriter logStream;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionUtils.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionUtils.java
similarity index 99%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionUtils.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionUtils.java
index a6aa68c..fba5318 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/CompactionUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/innerSpaceCompaction/utils/CompactionUtils.java
@@ -17,11 +17,11 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.compaction.utils;
+package org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java
deleted file mode 100644
index 0b6730d..0000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java
+++ /dev/null
@@ -1,842 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.level;
-
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.cache.ChunkCache;
-import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.compaction.TsFileManagement;
-import org.apache.iotdb.db.engine.compaction.utils.CompactionLogAnalyzer;
-import org.apache.iotdb.db.engine.compaction.utils.CompactionLogger;
-import org.apache.iotdb.db.engine.compaction.utils.CompactionUtils;
-import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.modification.ModificationFile;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
-import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
-import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.SOURCE_NAME;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.TARGET_NAME;
-import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
-
-/** The TsFileManagement for LEVEL_COMPACTION, use level struct to manage TsFile list */
-public class LevelCompactionTsFileManagement extends TsFileManagement {
-
-  private static final Logger logger =
-      LoggerFactory.getLogger(LevelCompactionTsFileManagement.class);
-
-  private final int totalSeqLevelNum =
-      Math.max(IoTDBDescriptor.getInstance().getConfig().getSeqLevelNum(), 1);
-  private final int seqFileNumInEachLevel =
-      Math.max(IoTDBDescriptor.getInstance().getConfig().getSeqFileNumInEachLevel(), 1);
-  private final int unseqLevelNum =
-      Math.max(IoTDBDescriptor.getInstance().getConfig().getUnseqLevelNum(), 1);
-  private final int unseqFileNumInEachLevel =
-      Math.max(IoTDBDescriptor.getInstance().getConfig().getUnseqFileNumInEachLevel(), 1);
-
-  private final boolean enableUnseqCompaction =
-      IoTDBDescriptor.getInstance().getConfig().isEnableUnseqCompaction();
-
-  // First map is partition list; Second list is level list; Third list is file list in level;
-  private final Map<Long, List<SortedSet<TsFileResource>>> sequenceTsFileResources =
-      new HashMap<>();
-  private final Map<Long, List<List<TsFileResource>>> unSequenceTsFileResources = new HashMap<>();
-  private final List<List<TsFileResource>> forkedSequenceTsFileResources = new ArrayList<>();
-  private final List<List<TsFileResource>> forkedUnSequenceTsFileResources = new ArrayList<>();
-  private final List<TsFileResource> sequenceRecoverTsFileResources = new ArrayList<>();
-  private final List<TsFileResource> unSequenceRecoverTsFileResources = new ArrayList<>();
-
-  public LevelCompactionTsFileManagement(String storageGroupName, String storageGroupDir) {
-    super(storageGroupName, storageGroupDir);
-    clear();
-  }
-
-  public void renameLevelFilesMods(
-      Collection<Modification> filterModification,
-      Collection<TsFileResource> mergeTsFiles,
-      TsFileResource targetTsFile)
-      throws IOException {
-    logger.debug("{} [compaction] merge starts to rename real file's mod", storageGroupName);
-    List<Modification> modifications = new ArrayList<>();
-    for (TsFileResource mergeTsFile : mergeTsFiles) {
-      try (ModificationFile sourceModificationFile =
-          new ModificationFile(mergeTsFile.getTsFilePath() + ModificationFile.FILE_SUFFIX)) {
-        modifications.addAll(sourceModificationFile.getModifications());
-        if (sourceModificationFile.exists()) {
-          sourceModificationFile.remove();
-        }
-      }
-    }
-    modifications.removeAll(filterModification);
-    if (!modifications.isEmpty()) {
-      try (ModificationFile modificationFile =
-          new ModificationFile(targetTsFile.getTsFilePath() + ModificationFile.FILE_SUFFIX)) {
-        for (Modification modification : modifications) {
-          // we have to set modification offset to MAX_VALUE, as the offset of source chunk may
-          // change after compaction
-          modification.setFileOffset(Long.MAX_VALUE);
-          modificationFile.write(modification);
-        }
-      }
-    }
-  }
-
-  private void deleteLevelFilesInDisk(Collection<TsFileResource> mergeTsFiles) {
-    logger.debug("{} [compaction] merge starts to delete real file", storageGroupName);
-    for (TsFileResource mergeTsFile : mergeTsFiles) {
-      deleteLevelFile(mergeTsFile);
-      logger.info(
-          "{} [Compaction] delete TsFile {}", storageGroupName, mergeTsFile.getTsFilePath());
-    }
-  }
-
-  private void deleteLevelFilesInList(
-      long timePartitionId, Collection<TsFileResource> mergeTsFiles, int level, boolean sequence) {
-    logger.debug("{} [compaction] merge starts to delete file list", storageGroupName);
-    if (sequence) {
-      if (sequenceTsFileResources.containsKey(timePartitionId)) {
-        if (sequenceTsFileResources.get(timePartitionId).size() > level) {
-          sequenceTsFileResources.get(timePartitionId).get(level).removeAll(mergeTsFiles);
-        }
-      }
-    } else {
-      if (unSequenceTsFileResources.containsKey(timePartitionId)) {
-        if (unSequenceTsFileResources.get(timePartitionId).size() > level) {
-          unSequenceTsFileResources.get(timePartitionId).get(level).removeAll(mergeTsFiles);
-        }
-      }
-    }
-  }
-
-  private void deleteLevelFile(TsFileResource seqFile) {
-    seqFile.writeLock();
-    try {
-      ChunkCache.getInstance().clear();
-      TimeSeriesMetadataCache.getInstance().clear();
-      FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile.getTsFilePath());
-      seqFile.setDeleted(true);
-      seqFile.delete();
-    } catch (IOException e) {
-      logger.error(e.getMessage(), e);
-    } finally {
-      seqFile.writeUnlock();
-    }
-  }
-
-  @Deprecated
-  @Override
-  public List<TsFileResource> getTsFileList(boolean sequence) {
-    readLock();
-    try {
-      List<TsFileResource> result = new ArrayList<>();
-      if (sequence) {
-        for (long timePartition : sequenceTsFileResources.keySet()) {
-          result.addAll(getTsFileListByTimePartition(true, timePartition));
-        }
-      } else {
-        for (long timePartition : unSequenceTsFileResources.keySet()) {
-          result.addAll(getTsFileListByTimePartition(false, timePartition));
-        }
-      }
-      return result;
-    } finally {
-      readUnLock();
-    }
-  }
-
-  public List<TsFileResource> getTsFileListByTimePartition(boolean sequence, long timePartition) {
-    readLock();
-    try {
-      List<TsFileResource> result = new ArrayList<>();
-      if (sequence) {
-        List<SortedSet<TsFileResource>> sequenceTsFileList =
-            sequenceTsFileResources.get(timePartition);
-        for (int i = sequenceTsFileList.size() - 1; i >= 0; i--) {
-          result.addAll(sequenceTsFileList.get(i));
-        }
-      } else {
-        List<List<TsFileResource>> unSequenceTsFileList =
-            unSequenceTsFileResources.get(timePartition);
-        for (int i = unSequenceTsFileList.size() - 1; i >= 0; i--) {
-          result.addAll(unSequenceTsFileList.get(i));
-        }
-      }
-      return result;
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public Iterator<TsFileResource> getIterator(boolean sequence) {
-    readLock();
-    try {
-      return getTsFileList(sequence).iterator();
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public void remove(TsFileResource tsFileResource, boolean sequence) {
-    writeLock();
-    try {
-      if (sequence) {
-        for (SortedSet<TsFileResource> sequenceTsFileResource :
-            sequenceTsFileResources.get(tsFileResource.getTimePartition())) {
-          sequenceTsFileResource.remove(tsFileResource);
-        }
-      } else {
-        for (List<TsFileResource> unSequenceTsFileResource :
-            unSequenceTsFileResources.get(tsFileResource.getTimePartition())) {
-          unSequenceTsFileResource.remove(tsFileResource);
-        }
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public void removeAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
-    writeLock();
-    try {
-      if (sequence) {
-        for (List<SortedSet<TsFileResource>> partitionSequenceTsFileResource :
-            sequenceTsFileResources.values()) {
-          for (SortedSet<TsFileResource> levelTsFileResource : partitionSequenceTsFileResource) {
-            levelTsFileResource.removeAll(tsFileResourceList);
-          }
-        }
-      } else {
-        for (List<List<TsFileResource>> partitionUnSequenceTsFileResource :
-            unSequenceTsFileResources.values()) {
-          for (List<TsFileResource> levelTsFileResource : partitionUnSequenceTsFileResource) {
-            levelTsFileResource.removeAll(tsFileResourceList);
-          }
-        }
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public void add(TsFileResource tsFileResource, boolean sequence) {
-    writeLock();
-    try {
-      long timePartitionId = tsFileResource.getTimePartition();
-      int level = getMergeLevel(tsFileResource.getTsFile());
-      if (sequence) {
-        if (level <= totalSeqLevelNum - 1) {
-          // current file has normal level
-          sequenceTsFileResources
-              .computeIfAbsent(timePartitionId, this::newSequenceTsFileResources)
-              .get(level)
-              .add(tsFileResource);
-        } else {
-          // current file has too high level
-          sequenceTsFileResources
-              .computeIfAbsent(timePartitionId, this::newSequenceTsFileResources)
-              .get(totalSeqLevelNum - 1)
-              .add(tsFileResource);
-        }
-      } else {
-        if (level <= unseqLevelNum - 1) {
-          // current file has normal level
-          unSequenceTsFileResources
-              .computeIfAbsent(timePartitionId, this::newUnSequenceTsFileResources)
-              .get(level)
-              .add(tsFileResource);
-        } else {
-          // current file has too high level
-          unSequenceTsFileResources
-              .computeIfAbsent(timePartitionId, this::newUnSequenceTsFileResources)
-              .get(unseqLevelNum - 1)
-              .add(tsFileResource);
-        }
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public void addRecover(TsFileResource tsFileResource, boolean sequence) {
-    if (sequence) {
-      sequenceRecoverTsFileResources.add(tsFileResource);
-    } else {
-      unSequenceRecoverTsFileResources.add(tsFileResource);
-    }
-  }
-
-  @Override
-  public void addAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
-    writeLock();
-    try {
-      for (TsFileResource tsFileResource : tsFileResourceList) {
-        add(tsFileResource, sequence);
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public boolean contains(TsFileResource tsFileResource, boolean sequence) {
-    readLock();
-    try {
-      if (sequence) {
-        for (SortedSet<TsFileResource> sequenceTsFileResource :
-            sequenceTsFileResources.computeIfAbsent(
-                tsFileResource.getTimePartition(), this::newSequenceTsFileResources)) {
-          if (sequenceTsFileResource.contains(tsFileResource)) {
-            return true;
-          }
-        }
-      } else {
-        for (List<TsFileResource> unSequenceTsFileResource :
-            unSequenceTsFileResources.computeIfAbsent(
-                tsFileResource.getTimePartition(), this::newUnSequenceTsFileResources)) {
-          if (unSequenceTsFileResource.contains(tsFileResource)) {
-            return true;
-          }
-        }
-      }
-      return false;
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public void clear() {
-    writeLock();
-    try {
-      sequenceTsFileResources.clear();
-      unSequenceTsFileResources.clear();
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  @SuppressWarnings("squid:S3776")
-  public boolean isEmpty(boolean sequence) {
-    readLock();
-    try {
-      if (sequence) {
-        for (List<SortedSet<TsFileResource>> partitionSequenceTsFileResource :
-            sequenceTsFileResources.values()) {
-          for (SortedSet<TsFileResource> sequenceTsFileResource : partitionSequenceTsFileResource) {
-            if (!sequenceTsFileResource.isEmpty()) {
-              return false;
-            }
-          }
-        }
-      } else {
-        for (List<List<TsFileResource>> partitionUnSequenceTsFileResource :
-            unSequenceTsFileResources.values()) {
-          for (List<TsFileResource> unSequenceTsFileResource : partitionUnSequenceTsFileResource) {
-            if (!unSequenceTsFileResource.isEmpty()) {
-              return false;
-            }
-          }
-        }
-      }
-      return true;
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public int size(boolean sequence) {
-    readLock();
-    try {
-      int result = 0;
-      if (sequence) {
-        for (List<SortedSet<TsFileResource>> partitionSequenceTsFileResource :
-            sequenceTsFileResources.values()) {
-          for (int i = totalSeqLevelNum - 1; i >= 0; i--) {
-            result += partitionSequenceTsFileResource.get(i).size();
-          }
-        }
-      } else {
-        for (List<List<TsFileResource>> partitionUnSequenceTsFileResource :
-            unSequenceTsFileResources.values()) {
-          for (int i = unseqLevelNum - 1; i >= 0; i--) {
-            result += partitionUnSequenceTsFileResource.get(i).size();
-          }
-        }
-      }
-      return result;
-    } finally {
-      readUnLock();
-    }
-  }
-
-  /** recover files */
-  @Override
-  @SuppressWarnings({"squid:S3776", "squid:S2142"})
-  public void recover() {
-    File logFile =
-        FSFactoryProducer.getFSFactory()
-            .getFile(storageGroupDir, storageGroupName + COMPACTION_LOG_NAME);
-    try {
-      if (logFile.exists()) {
-        CompactionLogAnalyzer logAnalyzer = new CompactionLogAnalyzer(logFile);
-        logAnalyzer.analyze();
-        Set<String> deviceSet = logAnalyzer.getDeviceSet();
-        List<String> sourceFileList = logAnalyzer.getSourceFiles();
-        long offset = logAnalyzer.getOffset();
-        String targetFile = logAnalyzer.getTargetFile();
-        boolean fullMerge = logAnalyzer.isFullMerge();
-        boolean isSeq = logAnalyzer.isSeq();
-        if (targetFile == null || sourceFileList.isEmpty()) {
-          return;
-        }
-        File target = new File(targetFile);
-        if (deviceSet.isEmpty()) {
-          // if not in compaction, just delete the target file
-          if (target.exists()) {
-            Files.delete(target.toPath());
-          }
-          return;
-        }
-        if (fullMerge) {
-          // get tsfile resource from list, as they have been recovered in StorageGroupProcessor
-          TsFileResource targetTsFileResource = getRecoverTsFileResource(targetFile, isSeq);
-          long timePartition = targetTsFileResource.getTimePartition();
-          RestorableTsFileIOWriter writer = new RestorableTsFileIOWriter(target);
-          // if not complete compaction, resume merge
-          if (writer.hasCrashed()) {
-            if (offset > 0) {
-              writer.getIOWriterOut().truncate(offset - 1);
-            }
-            writer.close();
-            CompactionLogger compactionLogger =
-                new CompactionLogger(storageGroupDir, storageGroupName);
-            List<Modification> modifications = new ArrayList<>();
-            CompactionUtils.merge(
-                targetTsFileResource,
-                getTsFileList(isSeq),
-                storageGroupName,
-                compactionLogger,
-                deviceSet,
-                isSeq,
-                modifications);
-            compactionLogger.close();
-          } else {
-            writer.close();
-          }
-          // complete compaction and delete source file
-          deleteAllSubLevelFiles(isSeq, timePartition);
-        } else {
-          // get tsfile resource from list, as they have been recovered in StorageGroupProcessor
-          TsFileResource targetResource = getRecoverTsFileResource(targetFile, isSeq);
-          long timePartition = targetResource.getTimePartition();
-          List<TsFileResource> sourceTsFileResources = new ArrayList<>();
-          for (String file : sourceFileList) {
-            // get tsfile resource from list, as they have been recovered in StorageGroupProcessor
-            sourceTsFileResources.add(getTsFileResource(file, isSeq));
-          }
-          int level = getMergeLevel(new File(sourceFileList.get(0)));
-          RestorableTsFileIOWriter writer = new RestorableTsFileIOWriter(target);
-          // if not complete compaction, resume merge
-          if (writer.hasCrashed()) {
-            if (offset > 0) {
-              writer.getIOWriterOut().truncate(offset - 1);
-            }
-            writer.close();
-            CompactionLogger compactionLogger =
-                new CompactionLogger(storageGroupDir, storageGroupName);
-            List<Modification> modifications = new ArrayList<>();
-            CompactionUtils.merge(
-                targetResource,
-                sourceTsFileResources,
-                storageGroupName,
-                compactionLogger,
-                deviceSet,
-                isSeq,
-                modifications);
-            // complete compaction and delete source file
-            writeLock();
-            try {
-              if (Thread.currentThread().isInterrupted()) {
-                throw new InterruptedException(
-                    String.format("%s [Compaction] abort", storageGroupName));
-              }
-              int targetLevel = getMergeLevel(targetResource.getTsFile());
-              if (isSeq) {
-                sequenceTsFileResources.get(timePartition).get(targetLevel).add(targetResource);
-                sequenceRecoverTsFileResources.clear();
-              } else {
-                unSequenceTsFileResources.get(timePartition).get(targetLevel).add(targetResource);
-                unSequenceRecoverTsFileResources.clear();
-              }
-              deleteLevelFilesInList(timePartition, sourceTsFileResources, level, isSeq);
-            } finally {
-              writeUnlock();
-            }
-            deleteLevelFilesInDisk(sourceTsFileResources);
-            renameLevelFilesMods(modifications, sourceTsFileResources, targetResource);
-            compactionLogger.close();
-          } else {
-            writer.close();
-          }
-        }
-      }
-    } catch (IOException | IllegalPathException | InterruptedException e) {
-      logger.error("recover level tsfile management error ", e);
-    } finally {
-      if (logFile.exists()) {
-        try {
-          Files.delete(logFile.toPath());
-        } catch (IOException e) {
-          logger.error("delete level tsfile management log file error ", e);
-        }
-      }
-    }
-  }
-
-  private void deleteAllSubLevelFiles(boolean isSeq, long timePartition) {
-    if (isSeq) {
-      for (int level = 0; level < sequenceTsFileResources.get(timePartition).size(); level++) {
-        SortedSet<TsFileResource> currLevelMergeFile =
-            sequenceTsFileResources.get(timePartition).get(level);
-        deleteLevelFilesInDisk(currLevelMergeFile);
-        deleteLevelFilesInList(timePartition, currLevelMergeFile, level, isSeq);
-      }
-    } else {
-      for (int level = 0; level < unSequenceTsFileResources.get(timePartition).size(); level++) {
-        SortedSet<TsFileResource> currLevelMergeFile =
-            sequenceTsFileResources.get(timePartition).get(level);
-        deleteLevelFilesInDisk(currLevelMergeFile);
-        deleteLevelFilesInList(timePartition, currLevelMergeFile, level, isSeq);
-      }
-    }
-  }
-
-  @Override
-  public void forkCurrentFileList(long timePartition) {
-    readLock();
-    try {
-      forkTsFileList(
-          forkedSequenceTsFileResources,
-          sequenceTsFileResources.computeIfAbsent(timePartition, this::newSequenceTsFileResources),
-          totalSeqLevelNum);
-      // we have to copy all unseq file
-      forkTsFileList(
-          forkedUnSequenceTsFileResources,
-          unSequenceTsFileResources.computeIfAbsent(
-              timePartition, this::newUnSequenceTsFileResources),
-          unseqLevelNum + 1);
-    } finally {
-      readUnLock();
-    }
-  }
-
-  private void forkTsFileList(
-      List<List<TsFileResource>> forkedTsFileResources, List rawTsFileResources, int currMaxLevel) {
-    forkedTsFileResources.clear();
-    for (int i = 0; i < currMaxLevel - 1; i++) {
-      List<TsFileResource> forkedLevelTsFileResources = new ArrayList<>();
-      Collection<TsFileResource> levelRawTsFileResources =
-          (Collection<TsFileResource>) rawTsFileResources.get(i);
-      for (TsFileResource tsFileResource : levelRawTsFileResources) {
-        if (tsFileResource.isClosed()) {
-          forkedLevelTsFileResources.add(tsFileResource);
-        }
-      }
-      forkedTsFileResources.add(forkedLevelTsFileResources);
-    }
-  }
-
-  @Override
-  protected void merge(long timePartition) {
-    // do unseq compaction if has an unseq file in max unseq level
-    if (enableUnseqCompaction && forkedUnSequenceTsFileResources.get(unseqLevelNum - 1).size() > 0) {
-      doUnseqMerge(isForceFullMerge, getTsFileListByTimePartition(true, timePartition),
-          forkedUnSequenceTsFileResources.get(unseqLevelNum - 1));
-    }
-
-    doLevelCompaction(forkedSequenceTsFileResources, true, timePartition, totalSeqLevelNum,
-        seqFileNumInEachLevel);
-
-    doLevelCompaction(forkedUnSequenceTsFileResources, false, timePartition, unseqLevelNum,
-        unseqFileNumInEachLevel);
-
-  }
-
-  @SuppressWarnings("squid:S3776") //MERGE TODO: move to a LevelCompactionExecutor
-  private boolean doLevelCompaction(
-      List<List<TsFileResource>> mergeResources, // each level is a List<TsFileResource>
-      boolean sequence,
-      long timePartition,
-      int currMaxLevel,
-      int currMaxFileNumInEachLevel) {
-    // wait until unseq merge has finished
-    long startTimeMillis = System.currentTimeMillis();
-    // whether execute merge chunk in the loop below
-    boolean isMergeExecutedInCurrentTask = false;
-    CompactionLogger compactionLogger = null;
-    try {
-      logger.info("{} start to filter compaction condition", storageGroupName);
-      for (int i = 0; i < currMaxLevel - 1; i++) {
-        List<TsFileResource> currLevelTsFileResource = mergeResources.get(i);
-        if (currMaxFileNumInEachLevel <= currLevelTsFileResource.size()) {
-          // just merge part of the file
-          isMergeExecutedInCurrentTask = true;
-          compactionLogger = new CompactionLogger(storageGroupDir, storageGroupName);
-          // log source file list and target file for recover
-          for (TsFileResource mergeResource : mergeResources.get(i)) {
-            compactionLogger.logFile(SOURCE_NAME, mergeResource.getTsFile());
-          }
-          File newLevelFile =
-              TsFileResource.modifyTsFileNameMergeCnt(mergeResources.get(i).get(0).getTsFile());
-          compactionLogger.logSequence(sequence);
-          compactionLogger.logFile(TARGET_NAME, newLevelFile);
-          List<TsFileResource> toMergeTsFiles =
-              mergeResources.get(i).subList(0, currMaxFileNumInEachLevel);
-          logger.info(
-              "{} [Compaction] merge level-{}'s {} TsFiles to next level",
-              storageGroupName,
-              i,
-              toMergeTsFiles.size());
-          for (TsFileResource toMergeTsFile : toMergeTsFiles) {
-            logger.info(
-                "{} [Compaction] start to merge TsFile {}", storageGroupName, toMergeTsFile);
-          }
-
-          TsFileResource newResource = new TsFileResource(newLevelFile);
-          List<Modification> modifications = new ArrayList<>();
-          // merge, read from source files and write to target file
-          CompactionUtils.merge(
-              newResource,
-              toMergeTsFiles,
-              storageGroupName,
-              compactionLogger,
-              new HashSet<>(),
-              sequence,
-              modifications);
-          logger.info(
-              "{} [Compaction] merged level-{}'s {} TsFiles to next level, and start to delete old files",
-              storageGroupName,
-              i,
-              toMergeTsFiles.size());
-          writeLock();
-          try {
-            if (Thread.currentThread().isInterrupted()) {
-              throw new InterruptedException(
-                  String.format("%s [Compaction] abort", storageGroupName));
-            }
-
-            if (sequence) {
-              sequenceTsFileResources.get(timePartition).get(i + 1).add(newResource);
-            } else {
-              unSequenceTsFileResources.get(timePartition).get(i + 1).add(newResource);
-            }
-            deleteLevelFilesInList(timePartition, toMergeTsFiles, i, sequence);
-            if (mergeResources.size() > i + 1) {
-              mergeResources.get(i + 1).add(newResource);
-            }
-          } finally {
-            writeUnlock();
-          }
-          deleteLevelFilesInDisk(toMergeTsFiles);
-          renameLevelFilesMods(modifications, toMergeTsFiles, newResource);
-          compactionLogger.close();
-          File logFile =
-              FSFactoryProducer.getFSFactory()
-                  .getFile(storageGroupDir, storageGroupName + COMPACTION_LOG_NAME);
-          if (logFile.exists()) {
-            Files.delete(logFile.toPath());
-          }
-          break;
-        }
-      }
-    } catch (Exception e) {
-      if (compactionLogger != null) {
-        try {
-          compactionLogger.close();
-        } catch (IOException ioException) {
-          logger.error("{} Compaction log close fail", storageGroupName + COMPACTION_LOG_NAME);
-        }
-      }
-      restoreCompaction();
-      logger.error("Error occurred in Compaction Merge thread", e);
-    } finally {
-      // reset the merge working state to false
-      logger.info(
-          "{} [Compaction] merge end time isSeq = {}, consumption: {} ms",
-          storageGroupName,
-          sequence,
-          System.currentTimeMillis() - startTimeMillis);
-    }
-    return isMergeExecutedInCurrentTask;
-  }
-
-  private List<SortedSet<TsFileResource>> newSequenceTsFileResources(Long k) {
-    List<SortedSet<TsFileResource>> newSequenceTsFileResources = new ArrayList<>();
-    for (int i = 0; i < totalSeqLevelNum; i++) {
-      newSequenceTsFileResources.add(
-          new TreeSet<>(
-              (o1, o2) -> {
-                try {
-                  int rangeCompare =
-                      Long.compare(
-                          Long.parseLong(o1.getTsFile().getParentFile().getName()),
-                          Long.parseLong(o2.getTsFile().getParentFile().getName()));
-                  return rangeCompare == 0
-                      ? compareFileName(o1.getTsFile(), o2.getTsFile())
-                      : rangeCompare;
-                } catch (NumberFormatException e) {
-                  return compareFileName(o1.getTsFile(), o2.getTsFile());
-                }
-              }));
-    }
-    return newSequenceTsFileResources;
-  }
-
-  private List<List<TsFileResource>> newUnSequenceTsFileResources(Long k) {
-    List<List<TsFileResource>> newUnSequenceTsFileResources = new ArrayList<>();
-    for (int i = 0; i < unseqLevelNum; i++) {
-      newUnSequenceTsFileResources.add(new ArrayList<>());
-    }
-    return newUnSequenceTsFileResources;
-  }
-
-  public static int getMergeLevel(File file) {
-    String mergeLevelStr =
-        file.getPath()
-            .substring(file.getPath().lastIndexOf(FILE_NAME_SEPARATOR) + 1)
-            .replaceAll(TSFILE_SUFFIX, "");
-    return Integer.parseInt(mergeLevelStr);
-  }
-
-  private TsFileResource getRecoverTsFileResource(String filePath, boolean isSeq)
-      throws IOException {
-    if (isSeq) {
-      for (TsFileResource tsFileResource : sequenceRecoverTsFileResources) {
-        if (Files.isSameFile(tsFileResource.getTsFile().toPath(), new File(filePath).toPath())) {
-          return tsFileResource;
-        }
-      }
-    } else {
-      for (TsFileResource tsFileResource : unSequenceRecoverTsFileResources) {
-        if (Files.isSameFile(tsFileResource.getTsFile().toPath(), new File(filePath).toPath())) {
-          return tsFileResource;
-        }
-      }
-    }
-    logger.error("cannot get tsfile resource path: {}", filePath);
-    throw new IOException();
-  }
-
-  private TsFileResource getTsFileResource(String filePath, boolean isSeq) throws IOException {
-    if (isSeq) {
-      for (List<SortedSet<TsFileResource>> tsFileResourcesWithLevel :
-          sequenceTsFileResources.values()) {
-        for (SortedSet<TsFileResource> tsFileResources : tsFileResourcesWithLevel) {
-          for (TsFileResource tsFileResource : tsFileResources) {
-            if (Files.isSameFile(
-                tsFileResource.getTsFile().toPath(), new File(filePath).toPath())) {
-              return tsFileResource;
-            }
-          }
-        }
-      }
-    } else {
-      for (List<List<TsFileResource>> tsFileResourcesWithLevel :
-          unSequenceTsFileResources.values()) {
-        for (List<TsFileResource> tsFileResources : tsFileResourcesWithLevel) {
-          for (TsFileResource tsFileResource : tsFileResources) {
-            if (Files.isSameFile(
-                tsFileResource.getTsFile().toPath(), new File(filePath).toPath())) {
-              return tsFileResource;
-            }
-          }
-        }
-      }
-    }
-    logger.error("cannot get tsfile resource path: {}", filePath);
-    throw new IOException();
-  }
-
-  /** restore the files back to the status before the compaction task is submitted */
-  private void restoreCompaction() {
-    File logFile =
-        FSFactoryProducer.getFSFactory()
-            .getFile(storageGroupDir, storageGroupName + COMPACTION_LOG_NAME);
-    try {
-      if (logFile.exists()) {
-        CompactionLogAnalyzer logAnalyzer = new CompactionLogAnalyzer(logFile);
-        logAnalyzer.analyze();
-        String targetFilePath = logAnalyzer.getTargetFile();
-        if (targetFilePath != null) {
-          File targetFile = new File(targetFilePath);
-          if (targetFile.exists()) {
-            if (!targetFile.delete()) {
-              logger.warn("Delete file {} failed", targetFile);
-            }
-          }
-        }
-      }
-    } catch (IOException e) {
-      logger.error("restore compaction failed", e);
-    } finally {
-      if (logFile.exists()) {
-        try {
-          Files.delete(logFile.toPath());
-        } catch (IOException e) {
-          logger.error("delete compaction log file error ", e);
-        }
-      }
-    }
-  }
-
-  @TestOnly
-  public Map<Long, List<SortedSet<TsFileResource>>> getSequenceTsFileResources() {
-    return sequenceTsFileResources;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/no/NoCompactionTsFileManagement.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/no/NoCompactionTsFileManagement.java
deleted file mode 100644
index 5c3d8b1..0000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/no/NoCompactionTsFileManagement.java
+++ /dev/null
@@ -1,290 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.no;
-
-import org.apache.iotdb.db.engine.compaction.TsFileManagement;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-public class NoCompactionTsFileManagement extends TsFileManagement {
-
-  private static final Logger logger = LoggerFactory.getLogger(NoCompactionTsFileManagement.class);
-  // includes sealed and unsealed sequence TsFiles
-  private final Map<Long, TreeSet<TsFileResource>> sequenceFileTreeSetMap = new TreeMap<>();
-
-  // includes sealed and unsealed unSequence TsFiles
-  private final Map<Long, List<TsFileResource>> unSequenceFileListMap = new TreeMap<>();
-
-  public NoCompactionTsFileManagement(String storageGroupName, String storageGroupDir) {
-    super(storageGroupName, storageGroupDir);
-  }
-
-  @Deprecated
-  @Override
-  public List<TsFileResource> getTsFileList(boolean sequence) {
-    readLock();
-    try {
-      List<TsFileResource> result = new ArrayList<>();
-      if (sequence) {
-        for (TreeSet<TsFileResource> tsFileResourceTreeSet : sequenceFileTreeSetMap.values()) {
-          result.addAll(tsFileResourceTreeSet);
-        }
-      } else {
-        for (List<TsFileResource> tsFileResourceList : unSequenceFileListMap.values()) {
-          result.addAll(tsFileResourceList);
-        }
-      }
-      return result;
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public List<TsFileResource> getTsFileListByTimePartition(boolean sequence, long timePartition) {
-    readLock();
-    try {
-      if (sequence) {
-        return new ArrayList<>(
-            sequenceFileTreeSetMap.getOrDefault(timePartition, newSequenceTsFileResources(0L)));
-      } else {
-        return new ArrayList<>(
-            unSequenceFileListMap.getOrDefault(timePartition, Collections.emptyList()));
-      }
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public Iterator<TsFileResource> getIterator(boolean sequence) {
-    readLock();
-    try {
-      return getTsFileList(sequence).iterator();
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public void remove(TsFileResource tsFileResource, boolean sequence) {
-    writeLock();
-    try {
-      if (sequence) {
-        TreeSet<TsFileResource> sequenceFileTreeSet =
-            sequenceFileTreeSetMap.get(tsFileResource.getTimePartition());
-        sequenceFileTreeSet.remove(tsFileResource);
-      } else {
-        List<TsFileResource> unSequenceFileList =
-            unSequenceFileListMap.get(tsFileResource.getTimePartition());
-        unSequenceFileList.remove(tsFileResource);
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public void removeAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
-    writeLock();
-    try {
-      if (tsFileResourceList.size() > 0) {
-        tsFileResourceList.sort((o1, o2) -> (int) (o1.getTimePartition() - o2.getTimePartition()));
-        if (sequence) {
-          long currTimePartition = tsFileResourceList.get(0).getTimePartition();
-          int startIndex = 0;
-          for (int i = 1; i < tsFileResourceList.size(); i++) {
-            TsFileResource tsFileResource = tsFileResourceList.get(i);
-            if (tsFileResource.getTimePartition() != currTimePartition) {
-              sequenceFileTreeSetMap
-                  .get(currTimePartition)
-                  .removeAll(tsFileResourceList.subList(startIndex, i));
-              currTimePartition = tsFileResource.getTimePartition();
-              startIndex = i;
-            }
-          }
-          sequenceFileTreeSetMap
-              .get(currTimePartition)
-              .removeAll(tsFileResourceList.subList(startIndex, tsFileResourceList.size()));
-        } else {
-          long currTimePartition = tsFileResourceList.get(0).getTimePartition();
-          int startIndex = 0;
-          for (int i = 1; i < tsFileResourceList.size(); i++) {
-            TsFileResource tsFileResource = tsFileResourceList.get(i);
-            if (tsFileResource.getTimePartition() != currTimePartition) {
-              unSequenceFileListMap
-                  .get(currTimePartition)
-                  .removeAll(tsFileResourceList.subList(startIndex, i));
-              currTimePartition = tsFileResource.getTimePartition();
-              startIndex = i;
-            }
-          }
-          unSequenceFileListMap
-              .get(currTimePartition)
-              .removeAll(tsFileResourceList.subList(startIndex, tsFileResourceList.size()));
-        }
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public void add(TsFileResource tsFileResource, boolean sequence) {
-    writeLock();
-    try {
-      long timePartitionId = tsFileResource.getTimePartition();
-      if (sequence) {
-        sequenceFileTreeSetMap
-            .computeIfAbsent(timePartitionId, this::newSequenceTsFileResources)
-            .add(tsFileResource);
-      } else {
-        unSequenceFileListMap
-            .computeIfAbsent(timePartitionId, this::newUnSequenceTsFileResources)
-            .add(tsFileResource);
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public void addRecover(TsFileResource tsFileResource, boolean sequence) {
-    logger.info("{} do not need to recover", storageGroupName);
-  }
-
-  @Override
-  public void addAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
-    writeLock();
-    try {
-      for (TsFileResource tsFileResource : tsFileResourceList) {
-        add(tsFileResource, sequence);
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public boolean contains(TsFileResource tsFileResource, boolean sequence) {
-    readLock();
-    try {
-      if (sequence) {
-        return sequenceFileTreeSetMap
-            .getOrDefault(tsFileResource.getTimePartition(), newSequenceTsFileResources(0L))
-            .contains(tsFileResource);
-      } else {
-        return unSequenceFileListMap
-            .getOrDefault(tsFileResource.getTimePartition(), new ArrayList<>())
-            .contains(tsFileResource);
-      }
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public void clear() {
-    writeLock();
-    try {
-      sequenceFileTreeSetMap.clear();
-      unSequenceFileListMap.clear();
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public boolean isEmpty(boolean sequence) {
-    readLock();
-    try {
-      if (sequence) {
-        for (Set<TsFileResource> sequenceFileTreeSet : sequenceFileTreeSetMap.values()) {
-          if (!sequenceFileTreeSet.isEmpty()) {
-            return false;
-          }
-        }
-      } else {
-        for (List<TsFileResource> unSequenceFileList : unSequenceFileListMap.values()) {
-          if (!unSequenceFileList.isEmpty()) {
-            return false;
-          }
-        }
-      }
-      return true;
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public int size(boolean sequence) {
-    readLock();
-    try {
-      int result = 0;
-      if (sequence) {
-        for (Set<TsFileResource> sequenceFileTreeSet : sequenceFileTreeSetMap.values()) {
-          result += sequenceFileTreeSet.size();
-        }
-      } else {
-        for (List<TsFileResource> unSequenceFileList : unSequenceFileListMap.values()) {
-          result += unSequenceFileList.size();
-        }
-      }
-      return result;
-    } finally {
-      readUnLock();
-    }
-  }
-
-  @Override
-  public void recover() {
-    logger.info("{} no recover logic", storageGroupName);
-  }
-
-  @Override
-  public void forkCurrentFileList(long timePartition) {
-    logger.info("{} do not need fork", storageGroupName);
-  }
-
-  @Override
-  protected void merge(long timePartition) {
-    logger.info("{} no merge logic", storageGroupName);
-  }
-
-  private TreeSet<TsFileResource> newSequenceTsFileResources(Long k) {
-    return new TreeSet<>((o1, o2) -> compareFileName(o1.getTsFile(), o2.getTsFile()));
-  }
-
-  private List<TsFileResource> newUnSequenceTsFileResources(Long k) {
-    return new ArrayList<>();
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/package-info.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/package-info.java
deleted file mode 100644
index 566e074..0000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/package-info.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/**
- * Package merge contains classes supporting merge functionality (also compaction in other systems)
- * that compacts several data files into a new one, making it more ordered and removing duplicated
- * data.
- */
-package org.apache.iotdb.db.engine.merge;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index 75b2175..912fa60 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -18,20 +18,49 @@
  */
 package org.apache.iotdb.db.engine.storagegroup;
 
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask.MERGE_SUFFIX;
+import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.TEMP_SUFFIX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+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;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionManager;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.flush.CloseFileListener;
 import org.apache.iotdb.db.engine.flush.FlushListener;
 import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
-import org.apache.iotdb.db.engine.merge.task.RecoverMergeTask;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
@@ -81,43 +110,9 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
-
-import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
 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;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
-import static org.apache.iotdb.db.engine.merge.task.MergeTask.MERGE_SUFFIX;
-import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.TEMP_SUFFIX;
-import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
-
 /**
  * For sequence data, a StorageGroupProcessor has some TsFileProcessors, in which there is only one
  * TsFileProcessor in the working status. <br>
@@ -203,7 +198,7 @@ public class StorageGroupProcessor {
    * with timestamp less than or equals to the device's latestFlushedTime should go into an
    * unsequential file.
    */
-  private Map<Long, Map<String, Long>> partitionLatestFlushedTimeForEachDevice = new HashMap<>();
+  public Map<Long, Map<String, Long>> partitionLatestFlushedTimeForEachDevice = new HashMap<>();
 
   /** used to record the latest flush time while upgrading and inserting */
   private Map<Long, Map<String, Long>> newlyFlushedPartitionLatestFlushedTimeForEachDevice =
@@ -220,7 +215,7 @@ public class StorageGroupProcessor {
   private String logicalStorageGroupName;
   private File storageGroupSysDir;
   // manage seqFileList and unSeqFileList
-  private TsFileManagement tsFileManagement;
+  public TsFileManagement tsFileManagement;
   /**
    * time partition id -> version controller which assigns a version for each MemTable and
    * deletion/update such that after they are persisted, the order of insertions, deletions and
@@ -378,10 +373,7 @@ public class StorageGroupProcessor {
       logger.error("create Storage Group system Directory {} failed", storageGroupSysDir.getPath());
     }
     this.tsFileManagement =
-        IoTDBDescriptor.getInstance()
-            .getConfig()
-            .getCompactionStrategy()
-            .getTsFileManagement(logicalStorageGroupName, storageGroupSysDir.getAbsolutePath());
+        new TsFileManagement(logicalStorageGroupName, storageGroupSysDir.getAbsolutePath());
 
     ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
     executorService.scheduleWithFixedDelay(
@@ -392,6 +384,10 @@ public class StorageGroupProcessor {
     recover();
   }
 
+  public Map<Long, Map<String, Long>> getPartitionLatestFlushedTimeForEachDevice() {
+    return partitionLatestFlushedTimeForEachDevice;
+  }
+
   public String getLogicalStorageGroupName() {
     return logicalStorageGroupName;
   }
@@ -446,37 +442,6 @@ public class StorageGroupProcessor {
         recoverTsFiles(value, false);
       }
 
-      String unseqMergeTaskName =
-          logicalStorageGroupName + "-" + virtualStorageGroupId + "-" + System.currentTimeMillis();
-      File mergingMods =
-          SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, MERGING_MODIFICATION_FILE_NAME);
-      if (mergingMods.exists()) {
-        this.tsFileManagement.mergingModification = new ModificationFile(mergingMods.getPath());
-      }
-
-      // MERGE TODO: only pass the tsfileManagement into RecoverMergeTask
-      RecoverMergeTask recoverMergeTask =
-          new RecoverMergeTask(
-              new ArrayList<>(tsFileManagement.getTsFileList(true)),
-              tsFileManagement.getTsFileList(false),
-              storageGroupSysDir.getPath(),
-              tsFileManagement::mergeEndAction,
-              unseqMergeTaskName,
-              IoTDBDescriptor.getInstance().getConfig().isFullMerge(),
-              logicalStorageGroupName + "-" + virtualStorageGroupId);
-      logger.info(
-          "{} - {} a RecoverMergeTask {} starts...",
-          logicalStorageGroupName,
-          virtualStorageGroupId,
-          unseqMergeTaskName);
-      recoverMergeTask.recoverMerge(
-          IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot());
-      if (!IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot()) {
-        mergingMods.delete();
-      }
-
-      // MERGE TODO: move unseqMergeRecover into TsFileManagement.CompactionRecoverTask()
-      recoverLevelCompaction();
       for (TsFileResource resource : tsFileManagement.getTsFileList(true)) {
         long partitionNum = resource.getTimePartition();
         updatePartitionFileVersion(partitionNum, resource.getVersion());
@@ -494,7 +459,7 @@ public class StorageGroupProcessor {
         updatePartitionFileVersion(partitionNum, resource.getVersion());
       }
       updateLatestFlushedTime();
-    } catch (IOException | MetadataException e) {
+    } catch (IOException e) {
       throw new StorageGroupProcessorException(e);
     }
 
@@ -515,38 +480,8 @@ public class StorageGroupProcessor {
       globalLatestFlushedTimeForEachDevice.putAll(endTimeMap);
     }
 
-    if (IoTDBDescriptor.getInstance().getConfig().isEnableContinuousCompaction()) {
-      for (long timePartitionId : timePartitionIdVersionControllerMap.keySet()) {
-        executeCompaction(timePartitionId, IoTDBDescriptor.getInstance().getConfig().isFullMerge());
-      }
-    }
-  }
-
-  private void recoverLevelCompaction() {
-    if (!CompactionTaskManager.getInstance().isTerminated()) {
-      isCompactionWorking = true;
-      logger.info(
-          "{} - {} submit a compaction recover merge task",
-          logicalStorageGroupName,
-          virtualStorageGroupId);
-      try {
-        CompactionTaskManager.getInstance()
-            .submitTask(
-                logicalStorageGroupName,
-                tsFileManagement.new LevelCompactionRecoverTask(this::closeCompactionMergeCallBack));
-      } catch (RejectedExecutionException e) {
-        this.closeCompactionMergeCallBack(false, 0);
-        logger.error(
-            "{} - {} compaction submit task failed",
-            logicalStorageGroupName,
-            virtualStorageGroupId,
-            e);
-      }
-    } else {
-      logger.error(
-          "{} compaction pool not started ,recover failed",
-          logicalStorageGroupName + "-" + virtualStorageGroupId);
-    }
+    CompactionManager.getInstance()
+        .init(logicalStorageGroupName, tsFileManagement, this);
   }
 
   private void updatePartitionFileVersion(long partitionNum, long fileVersion) {
@@ -700,7 +635,7 @@ public class StorageGroupProcessor {
       RestorableTsFileIOWriter writer;
       try {
         // this tsfile is not zero level, no need to perform redo wal
-        if (LevelCompactionTsFileManagement.getMergeLevel(tsFileResource.getTsFile()) > 0) {
+        if (TsFileManagement.getMergeLevel(tsFileResource.getTsFile()) > 0) {
           writer =
               recoverPerformer.recover(false, this::getWalDirectByteBuffer, this::releaseWalBuffer);
           if (writer.hasCrashed()) {
@@ -1971,48 +1906,12 @@ public class StorageGroupProcessor {
         "signal closing storage group condition in {}",
         logicalStorageGroupName + "-" + virtualStorageGroupId);
 
-    executeCompaction(
-        tsFileProcessor.getTimeRangeId(),
-        IoTDBDescriptor.getInstance().getConfig().isFullMerge());
-  }
-
-  private void executeCompaction(long timePartition, boolean fullMerge) {
-    if (!isCompactionWorking && !CompactionTaskManager.getInstance().isTerminated()) {
-      isCompactionWorking = true;
-      logger.info(
-          "{} submit a compaction merge task",
-          logicalStorageGroupName + "-" + virtualStorageGroupId);
-      try {
-        // fork and filter current tsfile, then commit then to compaction merge
-        tsFileManagement.forkCurrentFileList(timePartition);
-        tsFileManagement.setFullMerge(fullMerge);
-        CompactionTaskManager.getInstance()
-            .submitTask(
-                logicalStorageGroupName,
-                tsFileManagement
-                .new CompactionMergeTask(this::closeCompactionMergeCallBack, timePartition));
-      } catch (IOException | RejectedExecutionException e) {
-        this.closeCompactionMergeCallBack(false, timePartition);
-        logger.error(
-            "{} compaction submit task failed",
-            logicalStorageGroupName + "-" + virtualStorageGroupId,
-            e);
-      }
-    } else {
-      logger.info(
-          "{} last compaction merge task is working, skip current merge",
-          logicalStorageGroupName + "-" + virtualStorageGroupId);
-    }
-  }
-
-  /** close compaction merge callback, to release some locks */
-  private void closeCompactionMergeCallBack(boolean isMerge, long timePartitionId) {
-    if (isMerge && IoTDBDescriptor.getInstance().getConfig().isEnableContinuousCompaction()) {
-      executeCompaction(
-          timePartitionId, IoTDBDescriptor.getInstance().getConfig().isFullMerge());
-    } else {
-      this.isCompactionWorking = false;
-    }
+    CompactionManager.getInstance()
+        .startCompaction(
+            logicalStorageGroupName,
+            tsFileProcessor.getTimeRangeId(),
+            IoTDBDescriptor.getInstance().getConfig().isFullMerge(),
+            tsFileManagement);
   }
 
   /**
@@ -2112,17 +2011,6 @@ public class StorageGroupProcessor {
     resources.clear();
   }
 
-  public void merge(boolean isFullMerge) {
-    writeLock();
-    try {
-      for (long timePartitionId : partitionLatestFlushedTimeForEachDevice.keySet()) {
-        executeCompaction(timePartitionId, isFullMerge);
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
   /**
    * Load a new tsfile to storage group processor. Tne file may have overlap with other files.
    *
@@ -2844,7 +2732,7 @@ public class StorageGroupProcessor {
     writeLock();
     try {
       // abort ongoing comapctions and merges
-      CompactionTaskManager.getInstance().abortCompaction(logicalStorageGroupName);
+      CompactionManager.getInstance().abortCompaction(logicalStorageGroupName);
       MergeManager.getINSTANCE().abortMerge(logicalStorageGroupName);
       // close all working files that should be removed
       removePartitions(filter, workSequenceTsFileProcessors.entrySet());
@@ -2983,7 +2871,7 @@ public class StorageGroupProcessor {
   @FunctionalInterface
   public interface CloseCompactionMergeCallBack {
 
-    void call(boolean isMergeExecutedInCurrentTask, long timePartitionId);
+    void call(String storageGroupName);
   }
 
   @FunctionalInterface
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/virtualSg/VirtualStorageGroupManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/virtualSg/VirtualStorageGroupManager.java
index e8d1f77..74e3f72 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/virtualSg/VirtualStorageGroupManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/virtualSg/VirtualStorageGroupManager.java
@@ -294,15 +294,6 @@ public class VirtualStorageGroupManager {
     }
   }
 
-  /** push mergeAll operation down to all virtual storage group processors */
-  public void mergeAll(boolean isFullMerge) {
-    for (StorageGroupProcessor storageGroupProcessor : virtualStorageGroupProcessor) {
-      if (storageGroupProcessor != null) {
-        storageGroupProcessor.merge(isFullMerge);
-      }
-    }
-  }
-
   /** push syncDeleteDataFiles operation down to all virtual storage group processors */
   public void syncDeleteDataFiles() {
     for (StorageGroupProcessor storageGroupProcessor : virtualStorageGroupProcessor) {
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 8918a2b..0f3f00c 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
@@ -30,9 +30,9 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager.TaskStatus;
 import org.apache.iotdb.db.engine.flush.pool.FlushTaskPoolManager;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager.TaskStatus;
 import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.TimePartitionFilter;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
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 e4458ff..4b22f2a 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
@@ -25,9 +25,9 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.cost.statistic.Measurement;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.CacheHitRatioMonitor;
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionManager;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
 import org.apache.iotdb.db.engine.flush.FlushManager;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
 import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.metadata.MManager;
@@ -108,7 +108,7 @@ public class IoTDB implements IoTDBMBean {
     registerManager.register(TVListAllocator.getInstance());
     registerManager.register(CacheHitRatioMonitor.getInstance());
     registerManager.register(MergeManager.getINSTANCE());
-    registerManager.register(CompactionTaskManager.getInstance());
+    registerManager.register(CompactionManager.getInstance());
     JMXService.registerMBean(getInstance(), mbeanName);
     registerManager.register(StorageEngine.getInstance());
     registerManager.register(TemporaryQueryDataFileService.getInstance());
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java
index 48ca327..3ab6ca1 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java
@@ -19,7 +19,7 @@
 package org.apache.iotdb.db.sync.sender.manage;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.service.IoTDB;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
index 89d1968..a6fc45b 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
@@ -19,7 +19,7 @@
 
 package org.apache.iotdb.db.utils;
 
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.metadata.PartialPath;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionChunkTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionChunkTest.java
index 5e7276a..7efa465 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionChunkTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionChunkTest.java
@@ -21,8 +21,8 @@ package org.apache.iotdb.db.engine.compaction;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.utils.CompactionUtils;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionUtils;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionCacheTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionCacheTest.java
index d632cb9..77c4a7c 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionCacheTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionCacheTest.java
@@ -24,7 +24,7 @@ import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache.TimeSeriesMetadataCacheKey;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement.CompactionMergeTask;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
@@ -69,8 +69,8 @@ public class LevelCompactionCacheTest extends LevelCompactionTest {
 
   @Test
   public void testCompactionChunkCache() throws IOException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
     TsFileResource tsFileResource = seqResources.get(1);
     TsFileSequenceReader reader = new TsFileSequenceReader(tsFileResource.getTsFilePath());
     List<Path> paths = reader.getAllPaths();
@@ -90,12 +90,11 @@ public class LevelCompactionCacheTest extends LevelCompactionTest {
     ChunkCache.getInstance().get(firstChunkMetadata);
     TimeSeriesMetadataCache.getInstance().get(firstTimeSeriesMetadataCacheKey, allSensors);
 
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     CompactionMergeTask compactionMergeTask =
-        levelCompactionTsFileManagement
-        .new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
+        levelCompactionExecutor.new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
     compactionMergeWorking = true;
     compactionMergeTask.call();
     while (compactionMergeWorking) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionTsFileManagementTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionExecutorTest.java
similarity index 64%
rename from server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionTsFileManagementTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionExecutorTest.java
index 7375bad..9f49401 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionTsFileManagementTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionExecutorTest.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.engine.compaction;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
@@ -40,7 +40,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class LevelCompactionTsFileManagementTest extends LevelCompactionTest {
+public class LevelCompactionExecutorTest extends LevelCompactionTest {
 
   File tempSGDir;
 
@@ -62,19 +62,19 @@ public class LevelCompactionTsFileManagementTest extends LevelCompactionTest {
   /** just compaction once */
   @Test
   public void testAddRemoveAndIterator() {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
     for (TsFileResource tsFileResource : seqResources) {
-      levelCompactionTsFileManagement.add(tsFileResource, true);
+      levelCompactionExecutor.add(tsFileResource, true);
     }
-    levelCompactionTsFileManagement.addAll(seqResources, false);
-    assertEquals(6, levelCompactionTsFileManagement.getTsFileList(true).size());
-    assertEquals(6, levelCompactionTsFileManagement.getTsFileList(false).size());
-    assertEquals(6, levelCompactionTsFileManagement.size(true));
-    assertEquals(6, levelCompactionTsFileManagement.size(false));
-    assertTrue(levelCompactionTsFileManagement.contains(seqResources.get(0), true));
+    levelCompactionExecutor.addAll(seqResources, false);
+    assertEquals(6, levelCompactionExecutor.getTsFileList(true).size());
+    assertEquals(6, levelCompactionExecutor.getTsFileList(false).size());
+    assertEquals(6, levelCompactionExecutor.size(true));
+    assertEquals(6, levelCompactionExecutor.size(false));
+    assertTrue(levelCompactionExecutor.contains(seqResources.get(0), true));
     assertFalse(
-        levelCompactionTsFileManagement.contains(
+        levelCompactionExecutor.contains(
             new TsFileResource(
                 new File(
                     TestConstant.BASE_OUTPUT_PATH.concat(
@@ -87,9 +87,9 @@ public class LevelCompactionTsFileManagementTest extends LevelCompactionTest {
                             + 0
                             + ".tsfile"))),
             false));
-    assertTrue(levelCompactionTsFileManagement.contains(seqResources.get(0), false));
+    assertTrue(levelCompactionExecutor.contains(seqResources.get(0), false));
     assertFalse(
-        levelCompactionTsFileManagement.contains(
+        levelCompactionExecutor.contains(
             new TsFileResource(
                 new File(
                     TestConstant.BASE_OUTPUT_PATH.concat(
@@ -102,29 +102,25 @@ public class LevelCompactionTsFileManagementTest extends LevelCompactionTest {
                             + 0
                             + ".tsfile"))),
             false));
-    assertFalse(levelCompactionTsFileManagement.isEmpty(true));
-    assertFalse(levelCompactionTsFileManagement.isEmpty(false));
-    levelCompactionTsFileManagement.remove(
-        levelCompactionTsFileManagement.getTsFileList(true).get(0), true);
-    levelCompactionTsFileManagement.remove(
-        levelCompactionTsFileManagement.getTsFileList(false).get(0), false);
-    assertEquals(5, levelCompactionTsFileManagement.getTsFileList(true).size());
-    levelCompactionTsFileManagement.removeAll(
-        levelCompactionTsFileManagement.getTsFileList(false), false);
-    assertEquals(0, levelCompactionTsFileManagement.getTsFileList(false).size());
+    assertFalse(levelCompactionExecutor.isEmpty(true));
+    assertFalse(levelCompactionExecutor.isEmpty(false));
+    levelCompactionExecutor.remove(levelCompactionExecutor.getTsFileList(true).get(0), true);
+    levelCompactionExecutor.remove(levelCompactionExecutor.getTsFileList(false).get(0), false);
+    assertEquals(5, levelCompactionExecutor.getTsFileList(true).size());
+    levelCompactionExecutor.removeAll(levelCompactionExecutor.getTsFileList(false), false);
+    assertEquals(0, levelCompactionExecutor.getTsFileList(false).size());
     long count = 0;
-    Iterator<TsFileResource> iterator = levelCompactionTsFileManagement.getIterator(true);
+    Iterator<TsFileResource> iterator = levelCompactionExecutor.getIterator(true);
     while (iterator.hasNext()) {
       iterator.next();
       count++;
     }
     assertEquals(5, count);
-    levelCompactionTsFileManagement.removeAll(
-        levelCompactionTsFileManagement.getTsFileList(true), true);
-    assertEquals(0, levelCompactionTsFileManagement.getTsFileList(true).size());
-    assertTrue(levelCompactionTsFileManagement.isEmpty(true));
-    assertTrue(levelCompactionTsFileManagement.isEmpty(false));
-    levelCompactionTsFileManagement.add(
+    levelCompactionExecutor.removeAll(levelCompactionExecutor.getTsFileList(true), true);
+    assertEquals(0, levelCompactionExecutor.getTsFileList(true).size());
+    assertTrue(levelCompactionExecutor.isEmpty(true));
+    assertTrue(levelCompactionExecutor.isEmpty(false));
+    levelCompactionExecutor.add(
         new TsFileResource(
             new File(
                 TestConstant.BASE_OUTPUT_PATH.concat(
@@ -137,7 +133,7 @@ public class LevelCompactionTsFileManagementTest extends LevelCompactionTest {
                         + 0
                         + ".tsfile"))),
         true);
-    levelCompactionTsFileManagement.add(
+    levelCompactionExecutor.add(
         new TsFileResource(
             new File(
                 TestConstant.BASE_OUTPUT_PATH.concat(
@@ -150,32 +146,31 @@ public class LevelCompactionTsFileManagementTest extends LevelCompactionTest {
                         + 0
                         + ".tsfile"))),
         false);
-    assertEquals(1, levelCompactionTsFileManagement.size(true));
-    assertEquals(1, levelCompactionTsFileManagement.size(false));
-    levelCompactionTsFileManagement.clear();
-    assertEquals(0, levelCompactionTsFileManagement.size(true));
-    assertEquals(0, levelCompactionTsFileManagement.size(false));
+    assertEquals(1, levelCompactionExecutor.size(true));
+    assertEquals(1, levelCompactionExecutor.size(false));
+    levelCompactionExecutor.clear();
+    assertEquals(0, levelCompactionExecutor.size(true));
+    assertEquals(0, levelCompactionExecutor.size(false));
   }
 
   @Test
   public void testIteratorRemove() {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
     for (TsFileResource tsFileResource : seqResources) {
-      levelCompactionTsFileManagement.add(tsFileResource, true);
+      levelCompactionExecutor.add(tsFileResource, true);
     }
-    levelCompactionTsFileManagement.addAll(seqResources, false);
-    assertEquals(6, levelCompactionTsFileManagement.getTsFileList(true).size());
+    levelCompactionExecutor.addAll(seqResources, false);
+    assertEquals(6, levelCompactionExecutor.getTsFileList(true).size());
 
-    Iterator<TsFileResource> tsFileResourceIterator =
-        levelCompactionTsFileManagement.getIterator(true);
+    Iterator<TsFileResource> tsFileResourceIterator = levelCompactionExecutor.getIterator(true);
     tsFileResourceIterator.next();
     try {
       tsFileResourceIterator.remove();
     } catch (UnsupportedOperationException e) {
       // pass
     }
-    assertEquals(6, levelCompactionTsFileManagement.getTsFileList(true).size());
+    assertEquals(6, levelCompactionExecutor.getTsFileList(true).size());
 
     TsFileResource tsFileResource1 =
         new TsFileResource(
@@ -201,8 +196,8 @@ public class LevelCompactionTsFileManagementTest extends LevelCompactionTest {
                         + IoTDBConstant.FILE_NAME_SEPARATOR
                         + 0
                         + ".tsfile")));
-    levelCompactionTsFileManagement.add(tsFileResource1, true);
-    levelCompactionTsFileManagement.add(tsFileResource2, true);
+    levelCompactionExecutor.add(tsFileResource1, true);
+    levelCompactionExecutor.add(tsFileResource2, true);
     TsFileResource tsFileResource3 =
         new TsFileResource(
             new File(
@@ -215,9 +210,8 @@ public class LevelCompactionTsFileManagementTest extends LevelCompactionTest {
                         + IoTDBConstant.FILE_NAME_SEPARATOR
                         + 0
                         + ".tsfile")));
-    levelCompactionTsFileManagement.add(tsFileResource3, true);
-    Iterator<TsFileResource> tsFileResourceIterator2 =
-        levelCompactionTsFileManagement.getIterator(true);
+    levelCompactionExecutor.add(tsFileResource3, true);
+    Iterator<TsFileResource> tsFileResourceIterator2 = levelCompactionExecutor.getIterator(true);
     int count = 0;
     while (tsFileResourceIterator2.hasNext()) {
       count++;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionLogTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionLogTest.java
index 2170fbd..6ca34b6 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionLogTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionLogTest.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.engine.compaction;
 
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement.CompactionMergeTask;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
@@ -35,7 +35,7 @@ import org.junit.Test;
 import java.io.File;
 import java.io.IOException;
 
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
 import static org.junit.Assert.assertFalse;
 
 public class LevelCompactionLogTest extends LevelCompactionTest {
@@ -60,14 +60,13 @@ public class LevelCompactionLogTest extends LevelCompactionTest {
 
   @Test
   public void testCompactionLog() {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     CompactionMergeTask compactionMergeTask =
-        levelCompactionTsFileManagement
-        .new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
+        levelCompactionExecutor.new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
     compactionMergeWorking = true;
     compactionMergeTask.call();
     while (compactionMergeWorking) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionMergeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionMergeTest.java
index 579fd88..ccb5861 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionMergeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionMergeTest.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.engine.compaction;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement.CompactionMergeTask;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
@@ -76,14 +76,13 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
   /** just compaction once */
   @Test
   public void testCompactionMergeOnce() throws IllegalPathException, IOException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     CompactionMergeTask compactionMergeTask =
-        levelCompactionTsFileManagement
-        .new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
+        levelCompactionExecutor.new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
     compactionMergeWorking = true;
     compactionMergeTask.call();
     while (compactionMergeWorking) {
@@ -100,7 +99,7 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -120,14 +119,13 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
     int prevSeqLevelNum = IoTDBDescriptor.getInstance().getConfig().getSeqLevelNum();
     IoTDBDescriptor.getInstance().getConfig().setSeqFileNumInEachLevel(2);
     IoTDBDescriptor.getInstance().getConfig().setSeqLevelNum(2);
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     CompactionMergeTask compactionMergeTask =
-        levelCompactionTsFileManagement
-        .new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
+        levelCompactionExecutor.new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
     compactionMergeWorking = true;
     compactionMergeTask.call();
     while (compactionMergeWorking) {
@@ -144,7 +142,7 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -172,8 +170,8 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
         IoTDBDescriptor.getInstance().getConfig().getMergePagePointNumberThreshold();
     IoTDBDescriptor.getInstance().getConfig().setMergePagePointNumberThreshold(1);
 
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
     TsFileResource forthSeqTsFileResource = seqResources.get(3);
     PartialPath path =
         new PartialPath(
@@ -187,12 +185,11 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
           new Deletion(path, forthSeqTsFileResource.getTsFileSize() / 10, 300, 310);
       sourceModificationFile.write(modification);
     }
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     CompactionMergeTask compactionMergeTask =
-        levelCompactionTsFileManagement
-        .new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
+        levelCompactionExecutor.new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
     compactionMergeWorking = true;
     compactionMergeTask.call();
     while (compactionMergeWorking) {
@@ -204,7 +201,7 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -217,7 +214,7 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
     }
     assertEquals(489, count);
 
-    List<TsFileResource> tsFileResourceList = levelCompactionTsFileManagement.getTsFileList(true);
+    List<TsFileResource> tsFileResourceList = levelCompactionExecutor.getTsFileList(true);
     for (TsFileResource tsFileResource : tsFileResourceList) {
       tsFileResource.getModFile().remove();
       tsFileResource.remove();
@@ -232,21 +229,20 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
         IoTDBDescriptor.getInstance().getConfig().getMergeChunkPointNumberThreshold();
     IoTDBDescriptor.getInstance().getConfig().setMergeChunkPointNumberThreshold(1);
 
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     CompactionMergeTask compactionMergeTask =
-        levelCompactionTsFileManagement
-        .new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
+        levelCompactionExecutor.new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
     compactionMergeWorking = true;
     compactionMergeTask.call();
     while (compactionMergeWorking) {
       // wait
     }
     TsFileResource newTsFileResource =
-        levelCompactionTsFileManagement.getTsFileListByTimePartition(true, 0).get(0);
+        levelCompactionExecutor.getTsFileListByTimePartition(true, 0).get(0);
     TsFileSequenceReader tsFileSequenceReader =
         new TsFileSequenceReader(newTsFileResource.getTsFilePath());
     Map<String, List<ChunkMetadata>> sensorChunkMetadataListMap =
@@ -269,21 +265,20 @@ public class LevelCompactionMergeTest extends LevelCompactionTest {
         IoTDBDescriptor.getInstance().getConfig().getMergeChunkPointNumberThreshold();
     IoTDBDescriptor.getInstance().getConfig().setMergeChunkPointNumberThreshold(100000);
 
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     CompactionMergeTask compactionMergeTask =
-        levelCompactionTsFileManagement
-        .new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
+        levelCompactionExecutor.new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
     compactionMergeWorking = true;
     compactionMergeTask.call();
     while (compactionMergeWorking) {
       // wait
     }
     TsFileResource newTsFileResource =
-        levelCompactionTsFileManagement.getTsFileListByTimePartition(true, 0).get(0);
+        levelCompactionExecutor.getTsFileListByTimePartition(true, 0).get(0);
     TsFileSequenceReader tsFileSequenceReader =
         new TsFileSequenceReader(newTsFileResource.getTsFilePath());
     Map<String, List<ChunkMetadata>> sensorChunkMetadataListMap =
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionModsTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionModsTest.java
index e3dd45f..aee34e8 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionModsTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionModsTest.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.engine.compaction;
 
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
@@ -63,8 +63,8 @@ public class LevelCompactionModsTest extends LevelCompactionTest {
 
   @Test
   public void testCompactionMods() throws IllegalPathException, IOException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
     TsFileResource sourceTsFileResource = seqResources.get(0);
     TsFileResource targetTsFileResource = seqResources.get(1);
     List<Modification> filterModifications = new ArrayList<>();
@@ -80,7 +80,7 @@ public class LevelCompactionModsTest extends LevelCompactionTest {
     }
     List<TsFileResource> sourceTsFileResources = new ArrayList<>();
     sourceTsFileResources.add(sourceTsFileResource);
-    levelCompactionTsFileManagement.renameLevelFilesMods(
+    levelCompactionExecutor.renameLevelFilesMods(
         filterModifications, sourceTsFileResources, targetTsFileResource);
     try (ModificationFile targetModificationFile =
         new ModificationFile(targetTsFileResource.getTsFilePath() + ModificationFile.FILE_SUFFIX)) {
@@ -96,8 +96,8 @@ public class LevelCompactionModsTest extends LevelCompactionTest {
    */
   @Test
   public void testCompactionModsByOffset() throws IllegalPathException, IOException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
     TsFileResource sourceTsFileResource = seqResources.get(0);
     TsFileResource targetTsFileResource = seqResources.get(1);
     List<Modification> filterModifications = new ArrayList<>();
@@ -123,7 +123,7 @@ public class LevelCompactionModsTest extends LevelCompactionTest {
     }
     List<TsFileResource> sourceTsFileResources = new ArrayList<>();
     sourceTsFileResources.add(sourceTsFileResource);
-    levelCompactionTsFileManagement.renameLevelFilesMods(
+    levelCompactionExecutor.renameLevelFilesMods(
         filterModifications, sourceTsFileResources, targetTsFileResource);
     try (ModificationFile targetModificationFile =
         new ModificationFile(targetTsFileResource.getTsFilePath() + ModificationFile.FILE_SUFFIX)) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionMoreDataTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionMoreDataTest.java
index 75e576b..939df03 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionMoreDataTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionMoreDataTest.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.engine.compaction;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement.CompactionMergeTask;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
@@ -194,14 +194,13 @@ public class LevelCompactionMoreDataTest extends LevelCompactionTest {
   // test file compaction larger than 1024 sensor
   @Test
   public void testSensorWithTwoOrThreeNode() throws IllegalPathException, IOException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     CompactionMergeTask compactionMergeTask =
-        levelCompactionTsFileManagement
-        .new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
+        levelCompactionExecutor.new CompactionMergeTask(this::closeCompactionMergeCallBack, 0);
     compactionMergeWorking = true;
     compactionMergeTask.call();
     while (compactionMergeWorking) {
@@ -218,7 +217,7 @@ public class LevelCompactionMoreDataTest extends LevelCompactionTest {
             path,
             measurementSchemas[2688].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionRecoverTest.java
index 7adffc2..5ae57c3 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionRecoverTest.java
@@ -21,9 +21,9 @@ package org.apache.iotdb.db.engine.compaction;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
-import org.apache.iotdb.db.engine.compaction.utils.CompactionLogger;
-import org.apache.iotdb.db.engine.compaction.utils.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionUtils;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
@@ -54,9 +54,9 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.SOURCE_NAME;
-import static org.apache.iotdb.db.engine.compaction.utils.CompactionLogger.TARGET_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.COMPACTION_LOG_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.SOURCE_NAME;
+import static org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.utils.CompactionLogger.TARGET_NAME;
 import static org.junit.Assert.assertEquals;
 
 public class LevelCompactionRecoverTest extends LevelCompactionTest {
@@ -81,10 +81,10 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
   /** compaction recover merge finished */
   @Test
   public void testCompactionMergeRecoverMergeFinished() throws IOException, IllegalPathException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
     QueryContext context = new QueryContext();
     PartialPath path =
         new PartialPath(
@@ -96,7 +96,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -139,8 +139,8 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
         true,
         new ArrayList<>());
     compactionLogger.close();
-    levelCompactionTsFileManagement.addRecover(targetTsFileResource, true);
-    levelCompactionTsFileManagement.recover();
+    levelCompactionExecutor.addRecover(targetTsFileResource, true);
+    levelCompactionExecutor.recover();
     context = new QueryContext();
     path =
         new PartialPath(
@@ -152,7 +152,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -172,10 +172,10 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
   @Test
   public void testCompactionMergeRecoverMergeFinishedAndDeleteOneOffset()
       throws IOException, IllegalPathException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
     QueryContext context = new QueryContext();
     PartialPath path =
         new PartialPath(
@@ -187,7 +187,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -254,8 +254,8 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
     }
     logStream.close();
 
-    levelCompactionTsFileManagement.addRecover(targetTsFileResource, true);
-    levelCompactionTsFileManagement.recover();
+    levelCompactionExecutor.addRecover(targetTsFileResource, true);
+    levelCompactionExecutor.recover();
     context = new QueryContext();
     path =
         new PartialPath(
@@ -267,7 +267,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -287,10 +287,10 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
   @Test
   public void testCompactionMergeRecoverMergeFinishedAndDeleteOneDeviceWithOffset()
       throws IOException, IllegalPathException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
     QueryContext context = new QueryContext();
     PartialPath path =
         new PartialPath(
@@ -302,7 +302,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -375,8 +375,8 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
     out.truncate(Long.parseLong(logs.get(logs.size() - 1).split(" ")[1]) - 1);
     out.close();
 
-    levelCompactionTsFileManagement.addRecover(targetTsFileResource, true);
-    levelCompactionTsFileManagement.recover();
+    levelCompactionExecutor.addRecover(targetTsFileResource, true);
+    levelCompactionExecutor.recover();
     context = new QueryContext();
     path =
         new PartialPath(
@@ -388,7 +388,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -408,10 +408,10 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
   @Test
   public void testCompactionMergeRecoverMergeFinishedUnseq()
       throws IOException, IllegalPathException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(seqResources, false);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(seqResources, false);
     QueryContext context = new QueryContext();
     PartialPath path =
         new PartialPath(
@@ -423,7 +423,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -466,8 +466,8 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
         false,
         new ArrayList<>());
     compactionLogger.close();
-    levelCompactionTsFileManagement.addRecover(targetTsFileResource, false);
-    levelCompactionTsFileManagement.recover();
+    levelCompactionExecutor.addRecover(targetTsFileResource, false);
+    levelCompactionExecutor.recover();
     context = new QueryContext();
     path =
         new PartialPath(
@@ -479,7 +479,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(false),
+            levelCompactionExecutor.getTsFileList(false),
             new ArrayList<>(),
             null,
             null,
@@ -499,17 +499,17 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
   @Test
   public void testCompactionMergeRecoverMergeStartSourceLog()
       throws IOException, IllegalPathException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
     CompactionLogger compactionLogger =
         new CompactionLogger(tempSGDir.getPath(), COMPACTION_TEST_SG);
     compactionLogger.logFile(SOURCE_NAME, seqResources.get(0).getTsFile());
     compactionLogger.logFile(SOURCE_NAME, seqResources.get(1).getTsFile());
     compactionLogger.logFile(SOURCE_NAME, seqResources.get(2).getTsFile());
     compactionLogger.close();
-    levelCompactionTsFileManagement.recover();
+    levelCompactionExecutor.recover();
     QueryContext context = new QueryContext();
     PartialPath path =
         new PartialPath(
@@ -521,7 +521,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -541,10 +541,10 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
   @Test
   public void testCompactionMergeRecoverMergeStartSequenceLog()
       throws IOException, IllegalPathException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
     CompactionLogger compactionLogger =
         new CompactionLogger(tempSGDir.getPath(), COMPACTION_TEST_SG);
     compactionLogger.logFile(SOURCE_NAME, seqResources.get(0).getTsFile());
@@ -552,7 +552,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
     compactionLogger.logFile(SOURCE_NAME, seqResources.get(2).getTsFile());
     compactionLogger.logSequence(true);
     compactionLogger.close();
-    levelCompactionTsFileManagement.recover();
+    levelCompactionExecutor.recover();
     QueryContext context = new QueryContext();
     PartialPath path =
         new PartialPath(
@@ -564,7 +564,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -583,10 +583,10 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
   /** compaction recover merge start target file logged */
   @Test
   public void testCompactionMergeRecoverMergeStart() throws IOException, IllegalPathException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
     CompactionLogger compactionLogger =
         new CompactionLogger(tempSGDir.getPath(), COMPACTION_TEST_SG);
     compactionLogger.logFile(SOURCE_NAME, seqResources.get(0).getTsFile());
@@ -606,9 +606,9 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
                         + 0
                         + ".tsfile")));
     compactionLogger.logFile(TARGET_NAME, targetTsFileResource.getTsFile());
-    levelCompactionTsFileManagement.add(targetTsFileResource, true);
+    levelCompactionExecutor.add(targetTsFileResource, true);
     compactionLogger.close();
-    levelCompactionTsFileManagement.recover();
+    levelCompactionExecutor.recover();
     QueryContext context = new QueryContext();
     PartialPath path =
         new PartialPath(
@@ -620,7 +620,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
@@ -640,10 +640,10 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
   @Test
   public void testCompactionMergeRecoverMergeFinishedNoLog()
       throws IOException, IllegalPathException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
     CompactionLogger compactionLogger =
         new CompactionLogger(tempSGDir.getPath(), COMPACTION_TEST_SG);
     compactionLogger.logFile(SOURCE_NAME, seqResources.get(0).getTsFile());
@@ -671,9 +671,9 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
         new HashSet<>(),
         true,
         new ArrayList<>());
-    levelCompactionTsFileManagement.addRecover(targetTsFileResource, true);
+    levelCompactionExecutor.addRecover(targetTsFileResource, true);
     compactionLogger.close();
-    levelCompactionTsFileManagement.recover();
+    levelCompactionExecutor.recover();
     QueryContext context = new QueryContext();
     PartialPath path =
         new PartialPath(
@@ -685,7 +685,7 @@ public class LevelCompactionRecoverTest extends LevelCompactionTest {
             path,
             measurementSchemas[0].getType(),
             context,
-            levelCompactionTsFileManagement.getTsFileList(true),
+            levelCompactionExecutor.getTsFileList(true),
             new ArrayList<>(),
             null,
             null,
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionSelectorTest.java
index baa2502..c5aa248 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionSelectorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/LevelCompactionSelectorTest.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.engine.compaction;
 
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
@@ -60,17 +60,16 @@ public class LevelCompactionSelectorTest extends LevelCompactionTest {
   /** just compaction once */
   @Test
   public void testCompactionSelector() throws NoSuchFieldException, IllegalAccessException {
-    LevelCompactionTsFileManagement levelCompactionTsFileManagement =
-        new LevelCompactionTsFileManagement(COMPACTION_TEST_SG, tempSGDir.getPath());
-    levelCompactionTsFileManagement.addAll(seqResources, true);
-    levelCompactionTsFileManagement.addAll(unseqResources, false);
-    levelCompactionTsFileManagement.forkCurrentFileList(0);
+    LevelCompactionExecutor levelCompactionExecutor =
+        new LevelCompactionExecutor(COMPACTION_TEST_SG, tempSGDir.getPath());
+    levelCompactionExecutor.addAll(seqResources, true);
+    levelCompactionExecutor.addAll(unseqResources, false);
+    levelCompactionExecutor.forkCurrentFileList(0);
     Field fieldForkedSequenceTsFileResources =
-        LevelCompactionTsFileManagement.class.getDeclaredField("forkedSequenceTsFileResources");
+        LevelCompactionExecutor.class.getDeclaredField("forkedSequenceTsFileResources");
     fieldForkedSequenceTsFileResources.setAccessible(true);
     List<TsFileResource> forkedSequenceTsFileResources =
-        (List<TsFileResource>)
-            fieldForkedSequenceTsFileResources.get(levelCompactionTsFileManagement);
+        (List<TsFileResource>) fieldForkedSequenceTsFileResources.get(levelCompactionExecutor);
     assertEquals(2, forkedSequenceTsFileResources.size());
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/NoCompactionTsFileManagementTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/NoCompactionTsFileManagementTest.java
index 8ad9f18..3296bdd 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/NoCompactionTsFileManagementTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/NoCompactionTsFileManagementTest.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.engine.compaction;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.compaction.TsFileManagement.CompactionMergeTask;
-import org.apache.iotdb.db.engine.compaction.no.NoCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.no.NoCompactionTsFileManagement;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MaxFileMergeFileSelectorTest.java
similarity index 97%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MaxFileMergeFileSelectorTest.java
index 789d58d..1c68807 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MaxFileMergeFileSelectorTest.java
@@ -17,14 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.selector.IMergeFileSelector;
-import org.apache.iotdb.db.engine.merge.selector.MaxFileMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.IMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.MaxFileMergeFileSelector;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.engine.storagegroup.timeindex.ITimeIndex;
 import org.apache.iotdb.db.exception.MergeException;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxSeriesMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MaxSeriesMergeFileSelectorTest.java
similarity index 94%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MaxSeriesMergeFileSelectorTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MaxSeriesMergeFileSelectorTest.java
index c2b6e6c..8e26759 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxSeriesMergeFileSelectorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MaxSeriesMergeFileSelectorTest.java
@@ -17,10 +17,10 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.selector.MaxSeriesMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.MaxSeriesMergeFileSelector;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.MergeException;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeLogTest.java
similarity index 91%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeLogTest.java
index 1c0dfeb..d6e4387d 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeLogTest.java
@@ -17,12 +17,12 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeManagerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeManagerTest.java
similarity index 93%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MergeManagerTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeManagerTest.java
index 2232521..57d221c 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeManagerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeManagerTest.java
@@ -17,11 +17,11 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
-import org.apache.iotdb.db.engine.merge.task.MergeMultiChunkTask;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeMultiChunkTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 
 import com.google.common.util.concurrent.RateLimiter;
 import org.junit.Test;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeOverLapTest.java
similarity index 96%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeOverLapTest.java
index 0684f46..ba6684a 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeOverLapTest.java
@@ -18,12 +18,12 @@
  *
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergePerfTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergePerfTest.java
similarity index 93%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MergePerfTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergePerfTest.java
index 84d4c5f..147941e 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergePerfTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergePerfTest.java
@@ -17,12 +17,12 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeTaskTest.java
similarity index 98%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeTaskTest.java
index 5f7c036..dbe870d 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeTaskTest.java
@@ -17,13 +17,13 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.task.MergeTask;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeTest.java
similarity index 98%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeTest.java
index fa705ca..0d550c1 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeTest.java
@@ -17,14 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeUpgradeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeUpgradeTest.java
similarity index 96%
rename from server/src/test/java/org/apache/iotdb/db/engine/merge/MergeUpgradeTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeUpgradeTest.java
index 3971238..0da6505 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeUpgradeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/crossSpaceCompaction/MergeUpgradeTest.java
@@ -17,12 +17,12 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.merge;
+package org.apache.iotdb.db.engine.compaction.crossSpaceCompaction;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.merge.manage.MergeResource;
-import org.apache.iotdb.db.engine.merge.selector.MaxFileMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeResource;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.selector.MaxFileMergeFileSelector;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.MergeException;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
index 8ad49c7..b951ec3 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
@@ -22,9 +22,9 @@ import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.exception.StorageGroupProcessorException;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java
index 624977f..661bf1a 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAlignByDeviceIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAlignByDeviceIT.java
index a57d6d3..e903764 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAlignByDeviceIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAlignByDeviceIT.java
@@ -19,7 +19,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteTimeseriesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteTimeseriesIT.java
index 558d7ba..583bceb 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteTimeseriesIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeleteTimeseriesIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByMonthIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByMonthIT.java
index 76e0dba..de645e2 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByMonthIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByMonthIT.java
@@ -19,7 +19,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
index a0fbae6..3345bf1 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.integration;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLevelCompactionIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLevelCompactionIT.java
index b04b4e3..3dffefc 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLevelCompactionIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLevelCompactionIT.java
@@ -19,7 +19,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsFileWithTimePartitionIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsFileWithTimePartitionIT.java
index d991123..b32f917 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsFileWithTimePartitionIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsFileWithTimePartitionIT.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.integration;
 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.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java
index f1a149f..ed62deb 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.integration;
 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.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel;
 import org.apache.iotdb.db.engine.storagegroup.virtualSg.HashVirtualPartitioner;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMergeIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMergeIT.java
index 3daad5e..1f63405 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMergeIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMergeIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiDeviceIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiDeviceIT.java
index 5ffac74..566b181 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiDeviceIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiDeviceIT.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiOverlappedChunkInUnseqIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiOverlappedChunkInUnseqIT.java
index 8619705..5cb415c 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiOverlappedChunkInUnseqIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiOverlappedChunkInUnseqIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiOverlappedPageIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiOverlappedPageIT.java
index 87d6739..87d8fcf 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiOverlappedPageIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiOverlappedPageIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
index b74d43c..515562c 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.integration;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
index 633a0ee..81a4f3d 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiStatementsIT.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBNewTsFileCompactionIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBNewTsFileCompactionIT.java
index c327ef1..8e732dc 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBNewTsFileCompactionIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBNewTsFileCompactionIT.java
@@ -20,8 +20,8 @@ package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
-import org.apache.iotdb.db.engine.compaction.level.LevelCompactionTsFileManagement;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.level.LevelCompactionExecutor;
 import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.metadata.PartialPath;
@@ -1039,8 +1039,8 @@ public class IoTDBNewTsFileCompactionIT {
   private boolean waitForMergeFinish() throws StorageEngineException, InterruptedException {
     StorageGroupProcessor storageGroupProcessor =
         StorageEngine.getInstance().getProcessor(storageGroupPath);
-    LevelCompactionTsFileManagement tsFileManagement =
-        (LevelCompactionTsFileManagement) storageGroupProcessor.getTsFileManagement();
+    LevelCompactionExecutor tsFileManagement =
+        (LevelCompactionExecutor) storageGroupProcessor.getTsFileManagement();
 
     long startTime = System.nanoTime();
     // get the size of level 1's tsfile list to judge whether merge is finished
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBOverlappedPageIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBOverlappedPageIT.java
index d62c1fc..126e94b 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBOverlappedPageIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBOverlappedPageIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
index 289ce56..18f7bac 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
@@ -19,7 +19,7 @@
 package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionManager;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
@@ -379,7 +379,7 @@ public class IoTDBRestartIT {
     }
 
     try {
-      CompactionTaskManager.getInstance().waitAllCompactionFinish();
+      CompactionManager.getInstance().waitAllCompactionFinish();
       Thread.sleep(10000);
       EnvironmentUtils.restartDaemon();
     } catch (Exception e) {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
index 27b8abc..6f3b003 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.db.integration;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationIT.java b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationIT.java
index bfdfbf2..727054c 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration.aggregation;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationLargeDataIT.java b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationLargeDataIT.java
index 294f02e..20dbd35 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationLargeDataIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationLargeDataIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration.aggregation;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationSmallDataIT.java b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationSmallDataIT.java
index a077104..bf5c8c3 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationSmallDataIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationSmallDataIT.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.integration.aggregation;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.jdbc.IoTDBSQLException;
diff --git a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderTestUtil.java b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderTestUtil.java
index 2a9d808..6fb0ed7 100644
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderTestUtil.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderTestUtil.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.merge.manage.MergeManager;
+import org.apache.iotdb.db.engine.compaction.crossSpaceCompaction.inplace.manage.MergeManager;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.PartialPath;
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
index 9892780..8d61829 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
@@ -27,7 +27,7 @@ import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionManager;
 import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.TriggerManagementException;
@@ -86,7 +86,7 @@ public class EnvironmentUtils {
 
   public static void cleanEnv() throws IOException, StorageEngineException {
     // wait all compaction finished
-    CompactionTaskManager.getInstance().waitAllCompactionFinish();
+    CompactionManager.getInstance().waitAllCompactionFinish();
 
     // deregister all user defined classes
     try {
diff --git a/server/src/test/resources/logback.xml b/server/src/test/resources/logback.xml
index c962823..98a5f9a 100644
--- a/server/src/test/resources/logback.xml
+++ b/server/src/test/resources/logback.xml
@@ -40,7 +40,7 @@
     <!-- enable me if you want to monitor when files are opened and closed.
     <logger name="FileMonitor" level="info"/>
     -->
-    <logger name="org.apache.iotdb.db.engine.merge" level="INFO"/>
+    <logger name="org.apache.iotdb.db.engine.compaction.crossSpaceCompaction" level="INFO"/>
     <logger name="org.apache.iotdb.db.service.thrift.ThriftServiceThread" level="INFO"/>
     <logger name="org.eclipse.jetty.util.thread.QueuedThreadPool" level="INFO"/>
     <logger name="org.apache.iotdb.db.service.MetricsService" level="INFO"/>
diff --git a/session/src/test/java/org/apache/iotdb/session/pool/SessionPoolTest.java b/session/src/test/java/org/apache/iotdb/session/pool/SessionPoolTest.java
index 6b7d68f..706febf 100644
--- a/session/src/test/java/org/apache/iotdb/session/pool/SessionPoolTest.java
+++ b/session/src/test/java/org/apache/iotdb/session/pool/SessionPoolTest.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.session.pool;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.innerSpaceCompaction.CompactionStrategy;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.rpc.StatementExecutionException;