You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/07/29 06:05:09 UTC

[GitHub] [iotdb] qiaojialin commented on a change in pull request #3447: [ISSUE 3445] New compaction strategy and compaction scheduling strategy

qiaojialin commented on a change in pull request #3447:
URL: https://github.com/apache/iotdb/pull/3447#discussion_r678124825



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossCompactionStrategy.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.cross;
+
+import org.apache.iotdb.db.engine.compaction.cross.inplace.InplaceCompactionRecoverTask;
+import org.apache.iotdb.db.engine.compaction.cross.inplace.InplaceCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.cross.inplace.InplaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.cross.inplace.manage.CrossSpaceMergeResource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList;
+
+import java.io.File;
+import java.util.List;
+
+public enum CrossCompactionStrategy {
+  INPLACE_COMPACTION;
+
+  public AbstractCrossSpaceCompactionTask getCompactionTask(
+      String logicalStorageGroupName,
+      String virtualStorageGroupName,
+      long timePartitionId,
+      CrossSpaceMergeResource mergeResource,
+      String storageGroupDir,
+      TsFileResourceList seqTsFileResourceList,
+      TsFileResourceList unSeqTsFileResourceList,
+      List<TsFileResource> selectedSeqTsFileResourceList,
+      List<TsFileResource> selectedUnSeqTsFileResourceList,
+      int concurrentMergeCount) {
+    switch (this) {
+      case INPLACE_COMPACTION:
+      default:
+        return new InplaceCompactionTask(
+            logicalStorageGroupName,
+            virtualStorageGroupName,
+            timePartitionId,
+            mergeResource,
+            storageGroupDir,
+            seqTsFileResourceList,
+            unSeqTsFileResourceList,
+            selectedSeqTsFileResourceList,
+            selectedUnSeqTsFileResourceList,
+            concurrentMergeCount);
+    }
+  }
+
+  public AbstractCrossSpaceCompactionTask getCompactionRecoverTask(

Review comment:
       ```suggestion
     public AbstractCrossSpaceCompactionRecoverTask getCompactionRecoverTask(
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1829,15 +1771,23 @@ public void delete(PartialPath path, long startTime, long endTime, long planInde
       logDeletion(startTime, endTime, path);
 
       Deletion deletion = new Deletion(path, MERGE_MOD_START_VERSION_NUM, startTime, endTime);
-      if (tsFileManagement.mergingModification != null) {
-        tsFileManagement.mergingModification.write(deletion);
-        updatedModFiles.add(tsFileManagement.mergingModification);
-      }
+      //      if (tsFileResourceManager.mergingModification != null) {

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -438,7 +439,14 @@ public boolean resourceFileExists() {
 
   public synchronized ModificationFile getModFile() {

Review comment:
       remove synchronized and use double check

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java
##########
@@ -159,23 +137,32 @@ public ServiceType getID() {
     return ServiceType.COMPACTION_SERVICE;
   }
 
-  public void submitTask(String storageGroupName, Callable<Void> compactionMergeTask)
+  public void submitTask(
+      String fullStorageGroupName, long timePartition, Callable<Void> compactionMergeTask)
       throws RejectedExecutionException {
     if (pool != null && !pool.isTerminated()) {
-      Future<Void> future = pool.submit(compactionMergeTask);
-      storageGroupTasks
-          .computeIfAbsent(storageGroupName, k -> new ConcurrentSkipListSet<>())
-          .add(future);
+      synchronized (CompactionScheduler.currentTaskNum) {

Review comment:
       synchronize this mehod?

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/task/AbstractCompactionTask.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.db.engine.compaction.CompactionScheduler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Callable;
+
+import static org.apache.iotdb.db.engine.compaction.CompactionScheduler.currentTaskNum;
+
+/**
+ * AbstractCompactionTask is the base class for all compaction task, it carries out the execution of
+ * compaction. AbstractCompactionTask uses a template method, it execute the abstract function
+ * <i>doCompaction</i> implemented by subclass, and decrease the currentTaskNum in
+ * CompactionScheduler when the <i>doCompaction</i> finish.
+ */
+public abstract class AbstractCompactionTask implements Callable<Void> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(AbstractCompactionTask.class);
+  protected String fullStorageGroupName;
+  protected long timePartition;
+
+  public AbstractCompactionTask(String fullStorageGroupName, long timePartition) {
+    this.fullStorageGroupName = fullStorageGroupName;
+    this.timePartition = timePartition;
+  }
+
+  protected abstract void doCompaction() throws Exception;
+
+  @Override
+  public Void call() throws Exception {
+    try {
+      doCompaction();
+    } catch (Exception e) {
+      LOGGER.warn(e.getMessage(), e);

Review comment:
       add some information
   
   meet error in doCompaction
   ```suggestion
         LOGGER.error(e.getMessage(), e);
   ```

##########
File path: cluster/src/main/java/org/apache/iotdb/cluster/log/logtypes/CloseFileLog.java
##########
@@ -90,7 +90,7 @@ public long getPartitionId() {
   @Override
   public String toString() {
     return "CloseFileLog{"
-        + "storageGroupName='"
+        + "fullStorageGroupName='"

Review comment:
       is this meant?

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -393,18 +394,17 @@ public StorageGroupProcessor(
     this.fileFlushPolicy = fileFlushPolicy;
 
     storageGroupSysDir = SystemFileFactory.INSTANCE.getFile(systemDir, virtualStorageGroupId);
+    this.tsFileResourceManager =
+        new TsFileResourceManager(
+            logicalStorageGroupName, virtualStorageGroupId, storageGroupSysDir.getPath());
     if (storageGroupSysDir.mkdirs()) {
       logger.info(
           "Storage Group system Directory {} doesn't exist, create it",
           storageGroupSysDir.getPath());
     } else if (!storageGroupSysDir.exists()) {
       logger.error("create Storage Group system Directory {} failed", storageGroupSysDir.getPath());
     }
-    this.tsFileManagement =
-        IoTDBDescriptor.getInstance()
-            .getConfig()
-            .getCompactionStrategy()
-            .getTsFileManagement(logicalStorageGroupName, storageGroupSysDir.getAbsolutePath());
+    // TODO: new TsFileManagement

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.cross.AbstractCrossSpaceCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTaskFactory;
+import org.apache.iotdb.db.engine.compaction.inner.AbstractInnerSpaceCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTaskFactory;
+import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionSelector;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * CompactionScheduler schedule and submit the compaction task periodically, and it count the total

Review comment:
       ```suggestion
    * CompactionScheduler schedules and submits the compaction task periodically, and it counts the total
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/CompactionUtils.java
##########
@@ -477,9 +484,79 @@ private static void modifyChunkMetaDataWithCache(
     for (Modification modification : modifications) {
       if (modification.getPath().matchFullPath(seriesPath)) {
         seriesModifications.add(modification);
-        usedModifications.add(modification);
       }
     }
     modifyChunkMetaData(chunkMetadataList, seriesModifications);
   }
+
+  private static boolean isFileListHasModifications(
+      Set<TsFileSequenceReader> readers, Map<String, List<Modification>> modificationCache) {
+    for (TsFileSequenceReader reader : readers) {
+      if (!getModifications(reader, modificationCache).isEmpty()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static List<Modification> getModifications(
+      TsFileSequenceReader reader, Map<String, List<Modification>> modificationCache) {
+    return modificationCache.computeIfAbsent(
+        reader.getFileName(),
+        fileName ->
+            new LinkedList<>(
+                new ModificationFile(fileName + ModificationFile.FILE_SUFFIX).getModifications()));
+  }
+
+  public static ModificationFile getModificationByPath(String filePath) {

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java
##########
@@ -94,25 +89,8 @@ public void waitAndStop(long milliseconds) {
   @TestOnly
   public void waitAllCompactionFinish() {
     if (pool != null) {
-      File sgDir =
-          FSFactoryProducer.getFSFactory()
-              .getFile(
-                  FilePathUtils.regularizePath(
-                          IoTDBDescriptor.getInstance().getConfig().getSystemDir())
-                      + "storage_groups");
-      File[] subDirList = sgDir.listFiles();
-      if (subDirList != null) {
-        for (File subDir : subDirList) {
-          while (FSFactoryProducer.getFSFactory()
-              .getFile(
-                  subDir.getAbsoluteFile()
-                      + File.separator
-                      + subDir.getName()
-                      + COMPACTION_LOG_NAME)
-              .exists()) {
-            // wait
-          }
-        }
+      while (CompactionScheduler.currentTaskNum.get() > 0) {
+        // wait

Review comment:
       sleep

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1229,33 +1189,27 @@ private TsFileProcessor getOrCreateTsFileProcessorIntern(
       // build new processor
       res = newTsFileProcessor(sequence, timeRangeId);
       tsFileProcessorTreeMap.put(timeRangeId, res);
-      tsFileManagement.add(res.getTsFileResource(), sequence);
+      tsFileResourceManager.add(res.getTsFileResource(), sequence);
     }
 
     return res;
   }
 
   private TsFileProcessor newTsFileProcessor(boolean sequence, long timePartitionId)
       throws IOException, DiskSpaceInsufficientException {
-    DirectoryManager directoryManager = DirectoryManager.getInstance();

Review comment:
       compare with master

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/CompactionUtils.java
##########
@@ -67,6 +75,7 @@
 public class CompactionUtils {

Review comment:
       ```suggestion
   public class InnerSpaceCompactionUtils {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1307,8 +1261,8 @@ private String getNewTsFileName(long timePartitionId) {
     return getNewTsFileName(System.currentTimeMillis(), version, 0, 0);
   }
 
-  private String getNewTsFileName(long time, long version, int mergeCnt, int unSeqMergeCnt) {
-    return TsFileResource.getNewTsFileName(time, version, mergeCnt, unSeqMergeCnt);
+  private String getNewTsFileName(long time, long version, int mergeCnt, int unseqMergeCnt) {

Review comment:
       ```suggestion
     private String getNewTsFileName(long time, long version, int mergeCnt, int unseqCompactionCnt) {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -2124,12 +2050,10 @@ public int countUpgradeFiles() {
   /** upgrade all files belongs to this storage group */
   public void upgrade() {
     for (TsFileResource seqTsFileResource : upgradeSeqFileList) {
-      seqTsFileResource.setSeq(true);

Review comment:
       recover this

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/CompactionUtils.java
##########
@@ -477,9 +484,79 @@ private static void modifyChunkMetaDataWithCache(
     for (Modification modification : modifications) {
       if (modification.getPath().matchFullPath(seriesPath)) {
         seriesModifications.add(modification);
-        usedModifications.add(modification);
       }
     }
     modifyChunkMetaData(chunkMetadataList, seriesModifications);
   }
+
+  private static boolean isFileListHasModifications(

Review comment:
       check modification in chunk level

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1449,9 +1403,9 @@ public void syncDeleteDataFiles() {
       syncCloseAllWorkingTsFileProcessors();
       // normally, mergingModification is just need to be closed by after a merge task is finished.
       // we close it here just for IT test.
-      if (this.tsFileManagement.mergingModification != null) {
-        this.tsFileManagement.mergingModification.close();
-      }
+      //      if (this.tsFileResourceManager.mergingModification != null) {

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1902,11 +1852,21 @@ private void deleteDataInFiles(
         continue;
       }
 
-      deletion.setFileOffset(tsFileResource.getTsFileSize());
-      // write deletion into modification file
-      tsFileResource.getModFile().write(deletion);
-      // remember to close mod file
-      tsFileResource.getModFile().close();
+      if (tsFileResource.isMerging) {

Review comment:
       add design doc for delete in compaction

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_MERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_TIME_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_VERSION_INDEX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileNameGenerator {
+
+  private static FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
+  /**
+   * @param sequence whether the file is sequence
+   * @param logicalStorageGroup eg. "root.sg"
+   * @param virtualStorageGroup eg. "0"
+   * @param timePartitionId eg. 0
+   * @param time eg. 1623895965058
+   * @param version eg. 0
+   * @param innerSpaceCompactionCount the times of inner space compaction of this file
+   * @param crossSpaceCompactionCount the times of cross space compaction of this file
+   * @return a relative path of new tsfile, eg.
+   *     "data/data/sequence/root.sg/0/0/1623895965058-0-0-0.tsfile"
+   */
+  public static String generateNewTsFilePath(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePath(
+      String tsFileDir,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount) {
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePatWithMkdir(

Review comment:
       ```suggestion
     public static String generateNewTsFilePathWithMkdir(
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -2124,12 +2050,10 @@ public int countUpgradeFiles() {
   /** upgrade all files belongs to this storage group */
   public void upgrade() {
     for (TsFileResource seqTsFileResource : upgradeSeqFileList) {
-      seqTsFileResource.setSeq(true);
       seqTsFileResource.setUpgradeTsFileResourceCallBack(this::upgradeTsFileResourceCallBack);
       seqTsFileResource.doUpgrade();
     }
     for (TsFileResource unseqTsFileResource : upgradeUnseqFileList) {
-      unseqTsFileResource.setSeq(false);

Review comment:
       recover

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_MERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_TIME_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_VERSION_INDEX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileNameGenerator {
+
+  private static FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
+  /**
+   * @param sequence whether the file is sequence
+   * @param logicalStorageGroup eg. "root.sg"
+   * @param virtualStorageGroup eg. "0"
+   * @param timePartitionId eg. 0
+   * @param time eg. 1623895965058
+   * @param version eg. 0
+   * @param innerSpaceCompactionCount the times of inner space compaction of this file
+   * @param crossSpaceCompactionCount the times of cross space compaction of this file
+   * @return a relative path of new tsfile, eg.
+   *     "data/data/sequence/root.sg/0/0/1623895965058-0-0-0.tsfile"
+   */
+  public static String generateNewTsFilePath(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePath(
+      String tsFileDir,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount) {
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePatWithMkdir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException, IOException {

Review comment:
       ```suggestion
         throws DiskSpaceInsufficientException {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_MERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_TIME_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_VERSION_INDEX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileNameGenerator {
+
+  private static FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
+  /**
+   * @param sequence whether the file is sequence
+   * @param logicalStorageGroup eg. "root.sg"
+   * @param virtualStorageGroup eg. "0"
+   * @param timePartitionId eg. 0
+   * @param time eg. 1623895965058
+   * @param version eg. 0
+   * @param innerSpaceCompactionCount the times of inner space compaction of this file
+   * @param crossSpaceCompactionCount the times of cross space compaction of this file
+   * @return a relative path of new tsfile, eg.
+   *     "data/data/sequence/root.sg/0/0/1623895965058-0-0-0.tsfile"
+   */
+  public static String generateNewTsFilePath(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePath(
+      String tsFileDir,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount) {
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePatWithMkdir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException, IOException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    fsFactory.getFile(tsFileDir).mkdirs();
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  private static String generateTsFileDir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId)
+      throws DiskSpaceInsufficientException {
+    DirectoryManager directoryManager = DirectoryManager.getInstance();
+    String baseDir =
+        sequence
+            ? directoryManager.getNextFolderForSequenceFile()
+            : directoryManager.getNextFolderForUnSequenceFile();
+    return baseDir
+        + File.separator
+        + logicalStorageGroup
+        + File.separator
+        + virtualStorageGroup
+        + File.separator
+        + timePartitionId;
+  }
+
+  public static String generateNewTsFileName(
+      long time, long version, int innerSpaceCompactionCount, int crossSpaceCompactionCount) {
+    return time
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + version
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + innerSpaceCompactionCount
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + crossSpaceCompactionCount
+        + TsFileConstant.TSFILE_SUFFIX;
+  }
+
+  public static TsFileName getTsFileName(String fileName) throws IOException {
+    String[] fileNameParts =
+        fileName.split(FILE_NAME_SUFFIX_SEPARATOR)[FILE_NAME_SUFFIX_INDEX].split(
+            FILE_NAME_SEPARATOR);
+    if (fileNameParts.length != 4) {
+      throw new IOException("tsfile file name format is incorrect:" + fileName);
+    }
+    try {
+      TsFileName tsFileName =
+          new TsFileName(
+              Long.parseLong(fileNameParts[FILE_NAME_SUFFIX_TIME_INDEX]),
+              Long.parseLong(fileNameParts[FILE_NAME_SUFFIX_VERSION_INDEX]),
+              Integer.parseInt(fileNameParts[FILE_NAME_SUFFIX_MERGECNT_INDEX]),
+              Integer.parseInt(fileNameParts[FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX]));
+      return tsFileName;
+    } catch (NumberFormatException e) {
+      throw new IOException("tsfile file name format is incorrect:" + fileName);
+    }
+  }
+
+  public static TsFileResource modifyTsFileNameUnseqMergCnt(TsFileResource tsFileResource)

Review comment:
       ```suggestion
     public static TsFileResource increaseCrossCompactionCnt(TsFileResource tsFileResource)
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -813,6 +813,11 @@ public void updatePlanIndexes(long planIndex) {
     }
   }
 
+  public static int getCompactionCount(String fileName) throws IOException {

Review comment:
       ```suggestion
     public static int getInnerCompactionCount(String fileName) throws IOException {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -438,7 +439,14 @@ public boolean resourceFileExists() {
 
   public synchronized ModificationFile getModFile() {
     if (modFile == null) {
-      modFile = new ModificationFile(file.getPath() + ModificationFile.FILE_SUFFIX);
+      modFile = ModificationFile.getNormalMods(this);
+    }
+    return modFile;
+  }
+
+  public synchronized ModificationFile getCompactionModFile() {

Review comment:
       remove synchronized and use double check

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_MERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_TIME_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_VERSION_INDEX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileNameGenerator {
+
+  private static FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
+  /**
+   * @param sequence whether the file is sequence
+   * @param logicalStorageGroup eg. "root.sg"
+   * @param virtualStorageGroup eg. "0"
+   * @param timePartitionId eg. 0
+   * @param time eg. 1623895965058
+   * @param version eg. 0
+   * @param innerSpaceCompactionCount the times of inner space compaction of this file
+   * @param crossSpaceCompactionCount the times of cross space compaction of this file
+   * @return a relative path of new tsfile, eg.
+   *     "data/data/sequence/root.sg/0/0/1623895965058-0-0-0.tsfile"
+   */
+  public static String generateNewTsFilePath(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePath(
+      String tsFileDir,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount) {
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePatWithMkdir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException, IOException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    fsFactory.getFile(tsFileDir).mkdirs();
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  private static String generateTsFileDir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId)
+      throws DiskSpaceInsufficientException {
+    DirectoryManager directoryManager = DirectoryManager.getInstance();
+    String baseDir =
+        sequence
+            ? directoryManager.getNextFolderForSequenceFile()
+            : directoryManager.getNextFolderForUnSequenceFile();
+    return baseDir
+        + File.separator
+        + logicalStorageGroup
+        + File.separator
+        + virtualStorageGroup
+        + File.separator
+        + timePartitionId;
+  }
+
+  public static String generateNewTsFileName(
+      long time, long version, int innerSpaceCompactionCount, int crossSpaceCompactionCount) {
+    return time
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + version
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + innerSpaceCompactionCount
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + crossSpaceCompactionCount
+        + TsFileConstant.TSFILE_SUFFIX;
+  }
+
+  public static TsFileName getTsFileName(String fileName) throws IOException {
+    String[] fileNameParts =
+        fileName.split(FILE_NAME_SUFFIX_SEPARATOR)[FILE_NAME_SUFFIX_INDEX].split(
+            FILE_NAME_SEPARATOR);
+    if (fileNameParts.length != 4) {
+      throw new IOException("tsfile file name format is incorrect:" + fileName);
+    }
+    try {
+      TsFileName tsFileName =
+          new TsFileName(
+              Long.parseLong(fileNameParts[FILE_NAME_SUFFIX_TIME_INDEX]),
+              Long.parseLong(fileNameParts[FILE_NAME_SUFFIX_VERSION_INDEX]),
+              Integer.parseInt(fileNameParts[FILE_NAME_SUFFIX_MERGECNT_INDEX]),
+              Integer.parseInt(fileNameParts[FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX]));
+      return tsFileName;
+    } catch (NumberFormatException e) {
+      throw new IOException("tsfile file name format is incorrect:" + fileName);
+    }
+  }
+
+  public static TsFileResource modifyTsFileNameUnseqMergCnt(TsFileResource tsFileResource)
+      throws IOException {
+    File tsFile = tsFileResource.getTsFile();
+    String path = tsFile.getParent();
+    TsFileName tsFileName = getTsFileName(tsFileResource.getTsFile().getName());
+    tsFileName.setUnSeqMergeCnt(tsFileName.getUnSeqMergeCnt() + 1);
+    tsFileResource.setFile(
+        new File(
+            path,
+            tsFileName.time
+                + FILE_NAME_SEPARATOR
+                + tsFileName.version
+                + FILE_NAME_SEPARATOR
+                + tsFileName.mergeCnt
+                + FILE_NAME_SEPARATOR
+                + tsFileName.unSeqMergeCnt
+                + TSFILE_SUFFIX));
+    return tsFileResource;
+  }
+
+  public static File modifyTsFileNameUnseqMergCnt(File tsFile) throws IOException {

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_MERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_TIME_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_VERSION_INDEX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileNameGenerator {
+
+  private static FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
+  /**
+   * @param sequence whether the file is sequence
+   * @param logicalStorageGroup eg. "root.sg"
+   * @param virtualStorageGroup eg. "0"
+   * @param timePartitionId eg. 0
+   * @param time eg. 1623895965058
+   * @param version eg. 0
+   * @param innerSpaceCompactionCount the times of inner space compaction of this file
+   * @param crossSpaceCompactionCount the times of cross space compaction of this file
+   * @return a relative path of new tsfile, eg.
+   *     "data/data/sequence/root.sg/0/0/1623895965058-0-0-0.tsfile"
+   */
+  public static String generateNewTsFilePath(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePath(
+      String tsFileDir,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount) {
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePatWithMkdir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException, IOException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    fsFactory.getFile(tsFileDir).mkdirs();
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  private static String generateTsFileDir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId)
+      throws DiskSpaceInsufficientException {
+    DirectoryManager directoryManager = DirectoryManager.getInstance();
+    String baseDir =
+        sequence
+            ? directoryManager.getNextFolderForSequenceFile()
+            : directoryManager.getNextFolderForUnSequenceFile();
+    return baseDir
+        + File.separator
+        + logicalStorageGroup
+        + File.separator
+        + virtualStorageGroup
+        + File.separator
+        + timePartitionId;
+  }
+
+  public static String generateNewTsFileName(
+      long time, long version, int innerSpaceCompactionCount, int crossSpaceCompactionCount) {
+    return time
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + version
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + innerSpaceCompactionCount
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + crossSpaceCompactionCount
+        + TsFileConstant.TSFILE_SUFFIX;
+  }
+
+  public static TsFileName getTsFileName(String fileName) throws IOException {
+    String[] fileNameParts =
+        fileName.split(FILE_NAME_SUFFIX_SEPARATOR)[FILE_NAME_SUFFIX_INDEX].split(
+            FILE_NAME_SEPARATOR);
+    if (fileNameParts.length != 4) {
+      throw new IOException("tsfile file name format is incorrect:" + fileName);
+    }
+    try {
+      TsFileName tsFileName =
+          new TsFileName(
+              Long.parseLong(fileNameParts[FILE_NAME_SUFFIX_TIME_INDEX]),
+              Long.parseLong(fileNameParts[FILE_NAME_SUFFIX_VERSION_INDEX]),
+              Integer.parseInt(fileNameParts[FILE_NAME_SUFFIX_MERGECNT_INDEX]),
+              Integer.parseInt(fileNameParts[FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX]));
+      return tsFileName;
+    } catch (NumberFormatException e) {
+      throw new IOException("tsfile file name format is incorrect:" + fileName);
+    }
+  }
+
+  public static TsFileResource modifyTsFileNameUnseqMergCnt(TsFileResource tsFileResource)
+      throws IOException {
+    File tsFile = tsFileResource.getTsFile();
+    String path = tsFile.getParent();
+    TsFileName tsFileName = getTsFileName(tsFileResource.getTsFile().getName());
+    tsFileName.setUnSeqMergeCnt(tsFileName.getUnSeqMergeCnt() + 1);
+    tsFileResource.setFile(
+        new File(
+            path,
+            tsFileName.time
+                + FILE_NAME_SEPARATOR
+                + tsFileName.version
+                + FILE_NAME_SEPARATOR
+                + tsFileName.mergeCnt
+                + FILE_NAME_SEPARATOR
+                + tsFileName.unSeqMergeCnt
+                + TSFILE_SUFFIX));
+    return tsFileResource;
+  }
+
+  public static File modifyTsFileNameUnseqMergCnt(File tsFile) throws IOException {
+    String path = tsFile.getParent();
+    TsFileName tsFileName = getTsFileName(tsFile.getName());
+    tsFileName.setUnSeqMergeCnt(tsFileName.getUnSeqMergeCnt() + 1);
+    return new File(
+        path,
+        tsFileName.time
+            + FILE_NAME_SEPARATOR
+            + tsFileName.version
+            + FILE_NAME_SEPARATOR
+            + tsFileName.mergeCnt
+            + FILE_NAME_SEPARATOR
+            + tsFileName.unSeqMergeCnt
+            + TSFILE_SUFFIX);
+  }
+
+  public static File modifyTsFileNameMergeCnt(File tsFile) throws IOException {
+    String path = tsFile.getParent();
+    TsFileName tsFileName = getTsFileName(tsFile.getName());
+    tsFileName.setMergeCnt(tsFileName.getMergeCnt() + 1);
+    return new File(
+        path,
+        tsFileName.time
+            + FILE_NAME_SEPARATOR
+            + tsFileName.version
+            + FILE_NAME_SEPARATOR
+            + tsFileName.mergeCnt
+            + FILE_NAME_SEPARATOR
+            + tsFileName.unSeqMergeCnt
+            + TSFILE_SUFFIX);
+  }
+
+  public static File getInnerCompactionFileName(List<TsFileResource> tsFileResources)
+      throws IOException {
+    long minTime = Long.MAX_VALUE;
+    long minVersion = Long.MAX_VALUE;
+    long maxInnerMergeCount = Long.MIN_VALUE;
+    long maxCrossMergeCount = Long.MIN_VALUE;
+    for (TsFileResource resource : tsFileResources) {
+      TsFileName tsFileName = getTsFileName(resource.getTsFile().getName());
+      minTime = Math.min(tsFileName.time, minTime);
+      minVersion = Math.min(tsFileName.version, minVersion);
+      maxInnerMergeCount = Math.max(tsFileName.mergeCnt, maxInnerMergeCount);
+      maxCrossMergeCount = Math.max(tsFileName.unSeqMergeCnt, maxCrossMergeCount);
+    }
+    return new File(
+        tsFileResources.get(0).getTsFile().getParent(),
+        minTime
+            + FILE_NAME_SEPARATOR
+            + minVersion
+            + FILE_NAME_SEPARATOR
+            + (maxInnerMergeCount + 1)
+            + FILE_NAME_SEPARATOR
+            + maxCrossMergeCount
+            + TSFILE_SUFFIX);
+  }
+
+  public static class TsFileName {
+
+    private long time;
+    private long version;
+    private int mergeCnt;
+    private int unSeqMergeCnt;

Review comment:
       ```suggestion
       private int crossCompactionCnt;
   ```

##########
File path: server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionOverlapType.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.utils;
+
+public enum CompactionOverlapType {
+  FILE_NO_OVERLAP,
+  FILE_OVERLAP_CHUNK_NO_OVERLAP,
+  FILE_OVERLAP_CHUNK_OVERLAP_PAGE_NO_OVERLAP,

Review comment:
       ```suggestion
     CHUNK_OVERLAP_PAGE_NO_OVERLAP,
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResourceManager.java
##########
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.exception.WriteLockFailedException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+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.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileResourceManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TsFileResourceManager.class);
+  private String storageGroupName;
+  private String virtualStorageGroup;
+  private String storageGroupDir;
+
+  /** Serialize queries, delete resource files, compaction cleanup files */
+  private final ReadWriteLock resourceListLock = new ReentrantReadWriteLock();
+
+  private String writeLockHolder;
+  // time partition -> double linked list of tsfiles
+  private Map<Long, TsFileResourceList> sequenceFiles = new TreeMap<>();
+  private Map<Long, TsFileResourceList> unsequenceFiles = new TreeMap<>();
+
+  private List<TsFileResource> sequenceRecoverTsFileResources = new ArrayList<>();
+  private List<TsFileResource> unsequenceRecoverTsFileResources = new ArrayList<>();
+
+  public TsFileResourceManager(
+      String storageGroupName, String virtualStorageGroup, String storageGroupDir) {
+    this.storageGroupName = storageGroupName;
+    this.storageGroupDir = storageGroupDir;
+    this.virtualStorageGroup = virtualStorageGroup;
+  }
+
+  public List<TsFileResource> getTsFileList(boolean sequence) {
+    readLock();
+    try {
+      List<TsFileResource> allResources = new ArrayList<>();
+      Map<Long, TsFileResourceList> chosenMap = sequence ? sequenceFiles : unsequenceFiles;
+      for (Map.Entry<Long, TsFileResourceList> entry : chosenMap.entrySet()) {
+        allResources.addAll(entry.getValue().getArrayList());
+      }
+      return allResources;
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public TsFileResourceList getSequenceListByTimePartition(long timePartition) {
+    return sequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+  }
+
+  public TsFileResourceList getUnsequenceListByTimePartition(long timePartition) {
+    return unsequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+  }
+
+  public Iterator<TsFileResource> getIterator(boolean sequence) {
+    readLock();
+    try {
+      return getTsFileList(sequence).iterator();
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public void remove(TsFileResource tsFileResource, boolean sequence) {
+    writeLock("remove");
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      for (Map.Entry<Long, TsFileResourceList> entry : selectedMap.entrySet()) {
+        if (entry.getValue().contains(tsFileResource)) {
+          entry.getValue().remove(tsFileResource);
+          break;
+        }
+      }
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public void removeAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
+    for (TsFileResource resource : tsFileResourceList) {
+      remove(resource, sequence);
+    }
+  }
+
+  /**
+   * insert tsFileResource to a target pos(targetPos = insertPos + 1) e.g. if insertPos = 0, then to
+   * the first, if insert Pos = 1, then to the second.
+   */
+  public void insert(TsFileResource tsFileResource, boolean sequence, int insertPos) {
+    writeLock("add");
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      TsFileResourceList tsFileResources =
+          selectedMap.computeIfAbsent(
+              tsFileResource.getTimePartition(), o -> new TsFileResourceList());
+      tsFileResources.set(insertPos, tsFileResource);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public void add(TsFileResource tsFileResource, boolean sequence) {
+    writeLock("add");
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      selectedMap
+          .computeIfAbsent(tsFileResource.getTimePartition(), o -> new TsFileResourceList())
+          .add(tsFileResource);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public void addRecover(TsFileResource tsFileResource, boolean sequence) {
+    if (sequence) {
+      sequenceRecoverTsFileResources.add(tsFileResource);
+    } else {
+      unsequenceRecoverTsFileResources.add(tsFileResource);
+    }
+  }
+
+  public void addAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
+    writeLock("add");
+    try {
+      for (TsFileResource resource : tsFileResourceList) {
+        add(resource, sequence);
+      }
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public boolean contains(TsFileResource tsFileResource, boolean sequence) {
+    readLock();
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      TsFileResourceList list = selectedMap.getOrDefault(tsFileResource.getTimePartition(), null);
+      return list != null && list.contains(tsFileResource);
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public void clear() {
+    writeLock("clear");
+    try {
+      sequenceFiles.clear();
+      unsequenceFiles.clear();
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public boolean isEmpty(boolean sequence) {
+    readLock();
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      for (Map.Entry<Long, TsFileResourceList> entry : selectedMap.entrySet()) {
+        if (!entry.getValue().isEmpty()) {
+          return false;
+        }
+      }
+      return true;
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public int size(boolean sequence) {
+    readLock();
+    try {
+      int totalSize = 0;
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      for (Map.Entry<Long, TsFileResourceList> entry : selectedMap.entrySet()) {
+        totalSize += entry.getValue().size();
+      }
+      return totalSize;
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public void readLock() {
+    resourceListLock.readLock().lock();
+  }
+
+  public void readUnlock() {
+    resourceListLock.readLock().unlock();
+  }
+
+  public void writeLock(String holder) {
+    resourceListLock.writeLock().lock();
+    writeLockHolder = holder;
+  }
+
+  /**
+   * Acquire write lock with timeout, {@link WriteLockFailedException} will be thrown after timeout.
+   * The unit of timeout is ms.
+   */
+  public void writeLockWithTimeout(String holder, long timeout) throws WriteLockFailedException {
+    try {
+      if (resourceListLock.writeLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
+        writeLockHolder = holder;
+      } else {
+        throw new WriteLockFailedException(
+            String.format("cannot get write lock in %d ms", timeout));
+      }
+    } catch (InterruptedException e) {
+      LOGGER.warn(e.getMessage(), e);
+      Thread.interrupted();
+      throw new WriteLockFailedException("thread is interrupted");
+    }
+  }
+
+  public void writeUnlock() {
+    resourceListLock.writeLock().unlock();
+    writeLockHolder = "";
+  }
+
+  public String getStorageGroupName() {
+    return storageGroupName;
+  }
+
+  public String getStorageGroupDir() {
+    return storageGroupDir;
+  }
+
+  public void setStorageGroupDir(String storageGroupDir) {
+    this.storageGroupDir = storageGroupDir;
+  }
+
+  public Set<Long> getTimePartitions() {
+    readLock();
+    try {
+      Set<Long> timePartitions = new HashSet<>(sequenceFiles.keySet());
+      return timePartitions;
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public String getVirtualStorageGroup() {
+    return virtualStorageGroup;
+  }
+
+  public void setVirtualStorageGroup(String virtualStorageGroup) {
+    this.virtualStorageGroup = virtualStorageGroup;
+  }
+
+  public List<TsFileResource> getSequenceRecoverTsFileResources() {
+    return sequenceRecoverTsFileResources;
+  }
+
+  public List<TsFileResource> getUnsequenceRecoverTsFileResources() {
+    return unsequenceRecoverTsFileResources;
+  }
+
+  // ({systemTime}-{versionNum}-{compactionNum}-{mergeNum}.tsfile)

Review comment:
       ```suggestion
     // ({systemTime}-{versionNum}-{innerCompactionNum}-{crossCompactionNum}.tsfile)
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResourceManager.java
##########
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.exception.WriteLockFailedException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+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.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileResourceManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TsFileResourceManager.class);
+  private String storageGroupName;
+  private String virtualStorageGroup;
+  private String storageGroupDir;
+
+  /** Serialize queries, delete resource files, compaction cleanup files */
+  private final ReadWriteLock resourceListLock = new ReentrantReadWriteLock();
+
+  private String writeLockHolder;
+  // time partition -> double linked list of tsfiles
+  private Map<Long, TsFileResourceList> sequenceFiles = new TreeMap<>();
+  private Map<Long, TsFileResourceList> unsequenceFiles = new TreeMap<>();
+
+  private List<TsFileResource> sequenceRecoverTsFileResources = new ArrayList<>();
+  private List<TsFileResource> unsequenceRecoverTsFileResources = new ArrayList<>();
+
+  public TsFileResourceManager(
+      String storageGroupName, String virtualStorageGroup, String storageGroupDir) {
+    this.storageGroupName = storageGroupName;
+    this.storageGroupDir = storageGroupDir;
+    this.virtualStorageGroup = virtualStorageGroup;
+  }
+
+  public List<TsFileResource> getTsFileList(boolean sequence) {
+    readLock();
+    try {
+      List<TsFileResource> allResources = new ArrayList<>();
+      Map<Long, TsFileResourceList> chosenMap = sequence ? sequenceFiles : unsequenceFiles;
+      for (Map.Entry<Long, TsFileResourceList> entry : chosenMap.entrySet()) {
+        allResources.addAll(entry.getValue().getArrayList());
+      }
+      return allResources;
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public TsFileResourceList getSequenceListByTimePartition(long timePartition) {
+    return sequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+  }
+
+  public TsFileResourceList getUnsequenceListByTimePartition(long timePartition) {
+    return unsequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+  }
+
+  public Iterator<TsFileResource> getIterator(boolean sequence) {
+    readLock();
+    try {
+      return getTsFileList(sequence).iterator();
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public void remove(TsFileResource tsFileResource, boolean sequence) {
+    writeLock("remove");
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      for (Map.Entry<Long, TsFileResourceList> entry : selectedMap.entrySet()) {
+        if (entry.getValue().contains(tsFileResource)) {
+          entry.getValue().remove(tsFileResource);
+          break;
+        }
+      }
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public void removeAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
+    for (TsFileResource resource : tsFileResourceList) {
+      remove(resource, sequence);
+    }
+  }
+
+  /**
+   * insert tsFileResource to a target pos(targetPos = insertPos + 1) e.g. if insertPos = 0, then to

Review comment:
       ```suggestion
      * insert tsFileResource to a target pos(targetPos = insertPos) e.g. if insertPos = 0, then to
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_MERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_TIME_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_VERSION_INDEX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileNameGenerator {
+
+  private static FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
+  /**
+   * @param sequence whether the file is sequence
+   * @param logicalStorageGroup eg. "root.sg"
+   * @param virtualStorageGroup eg. "0"
+   * @param timePartitionId eg. 0
+   * @param time eg. 1623895965058
+   * @param version eg. 0
+   * @param innerSpaceCompactionCount the times of inner space compaction of this file
+   * @param crossSpaceCompactionCount the times of cross space compaction of this file
+   * @return a relative path of new tsfile, eg.
+   *     "data/data/sequence/root.sg/0/0/1623895965058-0-0-0.tsfile"
+   */
+  public static String generateNewTsFilePath(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePath(
+      String tsFileDir,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount) {
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  public static String generateNewTsFilePatWithMkdir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId,
+      long time,
+      long version,
+      int innerSpaceCompactionCount,
+      int crossSpaceCompactionCount)
+      throws DiskSpaceInsufficientException, IOException {
+    String tsFileDir =
+        generateTsFileDir(sequence, logicalStorageGroup, virtualStorageGroup, timePartitionId);
+    fsFactory.getFile(tsFileDir).mkdirs();
+    return tsFileDir
+        + File.separator
+        + generateNewTsFileName(
+            time, version, innerSpaceCompactionCount, crossSpaceCompactionCount);
+  }
+
+  private static String generateTsFileDir(
+      boolean sequence,
+      String logicalStorageGroup,
+      String virtualStorageGroup,
+      long timePartitionId)
+      throws DiskSpaceInsufficientException {
+    DirectoryManager directoryManager = DirectoryManager.getInstance();
+    String baseDir =
+        sequence
+            ? directoryManager.getNextFolderForSequenceFile()
+            : directoryManager.getNextFolderForUnSequenceFile();
+    return baseDir
+        + File.separator
+        + logicalStorageGroup
+        + File.separator
+        + virtualStorageGroup
+        + File.separator
+        + timePartitionId;
+  }
+
+  public static String generateNewTsFileName(
+      long time, long version, int innerSpaceCompactionCount, int crossSpaceCompactionCount) {
+    return time
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + version
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + innerSpaceCompactionCount
+        + IoTDBConstant.FILE_NAME_SEPARATOR
+        + crossSpaceCompactionCount
+        + TsFileConstant.TSFILE_SUFFIX;
+  }
+
+  public static TsFileName getTsFileName(String fileName) throws IOException {
+    String[] fileNameParts =
+        fileName.split(FILE_NAME_SUFFIX_SEPARATOR)[FILE_NAME_SUFFIX_INDEX].split(
+            FILE_NAME_SEPARATOR);
+    if (fileNameParts.length != 4) {
+      throw new IOException("tsfile file name format is incorrect:" + fileName);
+    }
+    try {
+      TsFileName tsFileName =
+          new TsFileName(
+              Long.parseLong(fileNameParts[FILE_NAME_SUFFIX_TIME_INDEX]),
+              Long.parseLong(fileNameParts[FILE_NAME_SUFFIX_VERSION_INDEX]),
+              Integer.parseInt(fileNameParts[FILE_NAME_SUFFIX_MERGECNT_INDEX]),
+              Integer.parseInt(fileNameParts[FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX]));
+      return tsFileName;
+    } catch (NumberFormatException e) {
+      throw new IOException("tsfile file name format is incorrect:" + fileName);
+    }
+  }
+
+  public static TsFileResource modifyTsFileNameUnseqMergCnt(TsFileResource tsFileResource)
+      throws IOException {
+    File tsFile = tsFileResource.getTsFile();
+    String path = tsFile.getParent();
+    TsFileName tsFileName = getTsFileName(tsFileResource.getTsFile().getName());
+    tsFileName.setUnSeqMergeCnt(tsFileName.getUnSeqMergeCnt() + 1);
+    tsFileResource.setFile(
+        new File(
+            path,
+            tsFileName.time
+                + FILE_NAME_SEPARATOR
+                + tsFileName.version
+                + FILE_NAME_SEPARATOR
+                + tsFileName.mergeCnt
+                + FILE_NAME_SEPARATOR
+                + tsFileName.unSeqMergeCnt
+                + TSFILE_SUFFIX));
+    return tsFileResource;
+  }
+
+  public static File modifyTsFileNameUnseqMergCnt(File tsFile) throws IOException {
+    String path = tsFile.getParent();
+    TsFileName tsFileName = getTsFileName(tsFile.getName());
+    tsFileName.setUnSeqMergeCnt(tsFileName.getUnSeqMergeCnt() + 1);
+    return new File(
+        path,
+        tsFileName.time
+            + FILE_NAME_SEPARATOR
+            + tsFileName.version
+            + FILE_NAME_SEPARATOR
+            + tsFileName.mergeCnt
+            + FILE_NAME_SEPARATOR
+            + tsFileName.unSeqMergeCnt
+            + TSFILE_SUFFIX);
+  }
+
+  public static File modifyTsFileNameMergeCnt(File tsFile) throws IOException {
+    String path = tsFile.getParent();
+    TsFileName tsFileName = getTsFileName(tsFile.getName());
+    tsFileName.setMergeCnt(tsFileName.getMergeCnt() + 1);
+    return new File(
+        path,
+        tsFileName.time
+            + FILE_NAME_SEPARATOR
+            + tsFileName.version
+            + FILE_NAME_SEPARATOR
+            + tsFileName.mergeCnt
+            + FILE_NAME_SEPARATOR
+            + tsFileName.unSeqMergeCnt
+            + TSFILE_SUFFIX);
+  }
+
+  public static File getInnerCompactionFileName(List<TsFileResource> tsFileResources)
+      throws IOException {
+    long minTime = Long.MAX_VALUE;
+    long minVersion = Long.MAX_VALUE;
+    long maxInnerMergeCount = Long.MIN_VALUE;
+    long maxCrossMergeCount = Long.MIN_VALUE;
+    for (TsFileResource resource : tsFileResources) {
+      TsFileName tsFileName = getTsFileName(resource.getTsFile().getName());
+      minTime = Math.min(tsFileName.time, minTime);
+      minVersion = Math.min(tsFileName.version, minVersion);
+      maxInnerMergeCount = Math.max(tsFileName.mergeCnt, maxInnerMergeCount);
+      maxCrossMergeCount = Math.max(tsFileName.unSeqMergeCnt, maxCrossMergeCount);
+    }
+    return new File(
+        tsFileResources.get(0).getTsFile().getParent(),
+        minTime
+            + FILE_NAME_SEPARATOR
+            + minVersion
+            + FILE_NAME_SEPARATOR
+            + (maxInnerMergeCount + 1)
+            + FILE_NAME_SEPARATOR
+            + maxCrossMergeCount
+            + TSFILE_SUFFIX);
+  }
+
+  public static class TsFileName {
+
+    private long time;
+    private long version;
+    private int mergeCnt;

Review comment:
       ```suggestion
       private int innerCompactionCnt;
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_MERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_TIME_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_UNSEQMERGECNT_INDEX;
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_VERSION_INDEX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileNameGenerator {
+
+  private static FSFactory fsFactory = FSFactoryProducer.getFSFactory();
+
+  /**
+   * @param sequence whether the file is sequence
+   * @param logicalStorageGroup eg. "root.sg"
+   * @param virtualStorageGroup eg. "0"
+   * @param timePartitionId eg. 0
+   * @param time eg. 1623895965058
+   * @param version eg. 0
+   * @param innerSpaceCompactionCount the times of inner space compaction of this file
+   * @param crossSpaceCompactionCount the times of cross space compaction of this file
+   * @return a relative path of new tsfile, eg.
+   *     "data/data/sequence/root.sg/0/0/1623895965058-0-0-0.tsfile"
+   */
+  public static String generateNewTsFilePath(

Review comment:
       check who use this previous

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -1137,6 +1139,7 @@ private void loadFile(File file, OperateFilePlan plan) throws QueryProcessExcept
         StorageEngine.getInstance().loadNewTsFile(resource);
       }
     } catch (Exception e) {
+      e.printStackTrace();

Review comment:
       use logger

##########
File path: server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionOverlapType.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.utils;
+
+public enum CompactionOverlapType {
+  FILE_NO_OVERLAP,
+  FILE_OVERLAP_CHUNK_NO_OVERLAP,
+  FILE_OVERLAP_CHUNK_OVERLAP_PAGE_NO_OVERLAP,
+  FILE_OVERLAP_CHUNK_OVERLAP_PAGE_OVERLAP

Review comment:
       ```suggestion
     PAGE_OVERLAP
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResourceManager.java
##########
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.storagegroup;
+
+import org.apache.iotdb.db.exception.WriteLockFailedException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+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.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+public class TsFileResourceManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TsFileResourceManager.class);
+  private String storageGroupName;
+  private String virtualStorageGroup;
+  private String storageGroupDir;
+
+  /** Serialize queries, delete resource files, compaction cleanup files */
+  private final ReadWriteLock resourceListLock = new ReentrantReadWriteLock();
+
+  private String writeLockHolder;
+  // time partition -> double linked list of tsfiles
+  private Map<Long, TsFileResourceList> sequenceFiles = new TreeMap<>();
+  private Map<Long, TsFileResourceList> unsequenceFiles = new TreeMap<>();
+
+  private List<TsFileResource> sequenceRecoverTsFileResources = new ArrayList<>();
+  private List<TsFileResource> unsequenceRecoverTsFileResources = new ArrayList<>();
+
+  public TsFileResourceManager(
+      String storageGroupName, String virtualStorageGroup, String storageGroupDir) {
+    this.storageGroupName = storageGroupName;
+    this.storageGroupDir = storageGroupDir;
+    this.virtualStorageGroup = virtualStorageGroup;
+  }
+
+  public List<TsFileResource> getTsFileList(boolean sequence) {
+    readLock();
+    try {
+      List<TsFileResource> allResources = new ArrayList<>();
+      Map<Long, TsFileResourceList> chosenMap = sequence ? sequenceFiles : unsequenceFiles;
+      for (Map.Entry<Long, TsFileResourceList> entry : chosenMap.entrySet()) {
+        allResources.addAll(entry.getValue().getArrayList());
+      }
+      return allResources;
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public TsFileResourceList getSequenceListByTimePartition(long timePartition) {
+    return sequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+  }
+
+  public TsFileResourceList getUnsequenceListByTimePartition(long timePartition) {
+    return unsequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+  }
+
+  public Iterator<TsFileResource> getIterator(boolean sequence) {
+    readLock();
+    try {
+      return getTsFileList(sequence).iterator();
+    } finally {
+      readUnlock();
+    }
+  }
+
+  public void remove(TsFileResource tsFileResource, boolean sequence) {
+    writeLock("remove");
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      for (Map.Entry<Long, TsFileResourceList> entry : selectedMap.entrySet()) {
+        if (entry.getValue().contains(tsFileResource)) {
+          entry.getValue().remove(tsFileResource);
+          break;
+        }
+      }
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public void removeAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
+    for (TsFileResource resource : tsFileResourceList) {
+      remove(resource, sequence);
+    }
+  }
+
+  /**
+   * insert tsFileResource to a target pos(targetPos = insertPos + 1) e.g. if insertPos = 0, then to
+   * the first, if insert Pos = 1, then to the second.
+   */
+  public void insert(TsFileResource tsFileResource, boolean sequence, int insertPos) {
+    writeLock("add");
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      TsFileResourceList tsFileResources =
+          selectedMap.computeIfAbsent(
+              tsFileResource.getTimePartition(), o -> new TsFileResourceList());
+      tsFileResources.set(insertPos, tsFileResource);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public void add(TsFileResource tsFileResource, boolean sequence) {
+    writeLock("add");
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      selectedMap
+          .computeIfAbsent(tsFileResource.getTimePartition(), o -> new TsFileResourceList())
+          .add(tsFileResource);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public void addRecover(TsFileResource tsFileResource, boolean sequence) {

Review comment:
       ```suggestion
     public void addForRecover(TsFileResource tsFileResource, boolean sequence) {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org