You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/01/31 05:16:32 UTC

[iotdb] branch master updated: [IOTDB-2478] Try to fix `IoTDBRemovePartitionIT` and `CompactionSchedulerTest` (#4967)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d331975  [IOTDB-2478] Try to fix `IoTDBRemovePartitionIT` and `CompactionSchedulerTest` (#4967)
d331975 is described below

commit d3319754b2f9e057ca99e921daf3423506b762c5
Author: Liu Xuxin <37...@users.noreply.github.com>
AuthorDate: Mon Jan 31 13:15:57 2022 +0800

    [IOTDB-2478] Try to fix `IoTDBRemovePartitionIT` and `CompactionSchedulerTest` (#4967)
---
 .../db/integration/IoTDBRemovePartitionIT.java     |  20 +-
 .../engine/compaction/CompactionTaskManager.java   |   5 +
 .../storagegroup/VirtualStorageGroupProcessor.java |   4 +-
 .../datastructure/FixedPriorityBlockingQueue.java  |   9 +-
 .../engine/compaction/CompactionSchedulerTest.java | 483 +++++++++++++++------
 5 files changed, 378 insertions(+), 143 deletions(-)

diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBRemovePartitionIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBRemovePartitionIT.java
index 1ae8258..dc4990d 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBRemovePartitionIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBRemovePartitionIT.java
@@ -19,6 +19,7 @@
 
 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.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
@@ -26,11 +27,11 @@ import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.itbase.category.LocalStandaloneTest;
 import org.apache.iotdb.jdbc.Config;
 
-import ch.qos.logback.classic.Level;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.sql.Connection;
@@ -47,14 +48,12 @@ import static org.junit.Assert.assertFalse;
 @Category({LocalStandaloneTest.class})
 public class IoTDBRemovePartitionIT {
 
+  private static final Logger logger = LoggerFactory.getLogger(IoTDBRemovePartitionIT.class);
+
   private static int partitionInterval = 100;
 
   @Before
   public void setUp() throws Exception {
-    ch.qos.logback.classic.Logger rootLogger =
-        (ch.qos.logback.classic.Logger)
-            LoggerFactory.getLogger(ch.qos.logback.classic.Logger.ROOT_LOGGER_NAME);
-    rootLogger.setLevel(Level.toLevel("trace"));
     EnvironmentUtils.envSetUp();
     StorageEngine.setEnablePartition(true);
     StorageEngine.setTimePartitionInterval(partitionInterval);
@@ -70,6 +69,7 @@ public class IoTDBRemovePartitionIT {
 
   @Test
   public void testRemoveNoPartition() throws IllegalPathException {
+    logger.warn("running testRemoveNoPartition");
     StorageEngine.getInstance()
         .removePartitions(
             new PartialPath("root.test1"), (storageGroupName, timePartitionId) -> false);
@@ -94,6 +94,7 @@ public class IoTDBRemovePartitionIT {
 
   @Test
   public void testRemovePartialPartition() throws IllegalPathException {
+    logger.warn("running testRemovePartialPartition");
     StorageEngine.getInstance()
         .removePartitions(
             new PartialPath("root.test1"),
@@ -133,6 +134,7 @@ public class IoTDBRemovePartitionIT {
 
   @Test
   public void testRemoveAllPartition() throws IllegalPathException {
+    logger.warn("running testRemoveAllPartition");
     StorageEngine.getInstance()
         .removePartitions(
             new PartialPath("root.test1"), (storageGroupName, timePartitionId) -> true);
@@ -151,6 +153,7 @@ public class IoTDBRemovePartitionIT {
 
   @Test
   public void testSQLRemovePartition() {
+    logger.warn("running testSQLRemovePartition");
     try (Connection connection =
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
@@ -172,6 +175,7 @@ public class IoTDBRemovePartitionIT {
 
   @Test
   public void testRemoveOnePartitionAndInsertData() {
+    logger.warn("running testRemoveOnePartitionAndInsertData");
     try (Connection connection =
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
@@ -196,6 +200,7 @@ public class IoTDBRemovePartitionIT {
 
   @Test
   public void testRemovePartitionAndInsertUnSeqDataAndMerge() {
+    logger.warn("running testRemovePartitionAndInsertUnSeqDataAndMerge");
     try (Connection connection =
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
@@ -225,6 +230,7 @@ public class IoTDBRemovePartitionIT {
 
   @Test
   public void testRemovePartitionAndInsertUnSeqDataAndUnSeqDataMerge() {
+    logger.warn("running testRemovePartitionAndInsertUnSeqDataAndUnSeqDataMerge");
     try (Connection connection =
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
@@ -254,6 +260,7 @@ public class IoTDBRemovePartitionIT {
 
   @Test
   public void testFlushAndRemoveOnePartitionAndInsertData() {
+    logger.warn("running testFlushAndRemoveOnePartitionAndInsertData");
     try (Connection connection =
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
@@ -312,6 +319,9 @@ public class IoTDBRemovePartitionIT {
       }
     }
     Class.forName(Config.JDBC_DRIVER_NAME);
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableUnseqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableCrossSpaceCompaction(false);
     try (Connection connection =
             DriverManager.getConnection(
                 Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
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/CompactionTaskManager.java
index b1ca103..e39eaaf 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/CompactionTaskManager.java
@@ -346,8 +346,13 @@ public class CompactionTaskManager implements IService {
    * corresponding storage group.
    */
   public void abortCompaction(String fullStorageGroupName) {
+    logger.warn("Aborting compaction task for {}", fullStorageGroupName);
     Set<Future<Void>> subTasks =
         storageGroupTasks.getOrDefault(fullStorageGroupName, Collections.emptySet());
+    logger.warn("Compaction task queue is {}", candidateCompactionTaskQueue);
+    candidateCompactionTaskQueue.clear();
+    logger.warn("the task map is {}", storageGroupTasks);
+    logger.warn("the size of compaction task of {} is {}", fullStorageGroupName, subTasks.size());
     Iterator<Future<Void>> subIterator = subTasks.iterator();
     while (subIterator.hasNext()) {
       Future<Void> next = subIterator.next();
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
index 29cf85b..5b07b32 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
@@ -3054,8 +3054,10 @@ public class VirtualStorageGroupProcessor {
     // this requires blocking all other activities
     writeLock("removePartitions");
     try {
+      tsFileManager.setAllowCompaction(false);
       // abort ongoing comapctions and merges
-      CompactionTaskManager.getInstance().abortCompaction(logicalStorageGroupName);
+      CompactionTaskManager.getInstance()
+          .abortCompaction(logicalStorageGroupName + "-" + virtualStorageGroupId);
       // close all working files that should be removed
       removePartitions(filter, workSequenceTsFileProcessors.entrySet(), true);
       removePartitions(filter, workUnsequenceTsFileProcessors.entrySet(), false);
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FixedPriorityBlockingQueue.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FixedPriorityBlockingQueue.java
index 2256a61..78b2572 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FixedPriorityBlockingQueue.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FixedPriorityBlockingQueue.java
@@ -137,7 +137,9 @@ public class FixedPriorityBlockingQueue<T> {
     final ReentrantLock lock = this.lock;
     lock.lock();
     try {
-      queue.clear();
+      while (queue.size() != 0) {
+        this.pollLast();
+      }
     } finally {
       lock.unlock();
     }
@@ -180,4 +182,9 @@ public class FixedPriorityBlockingQueue<T> {
   public interface PollLastHook<T> {
     void apply(T x);
   }
+
+  @Override
+  public String toString() {
+    return queue.toString();
+  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionSchedulerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionSchedulerTest.java
index d03f2e7..0de1228 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionSchedulerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionSchedulerTest.java
@@ -29,7 +29,6 @@ import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
 import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.service.IoTDB;
@@ -47,7 +46,6 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -59,7 +57,7 @@ import static org.junit.Assert.fail;
 
 public class CompactionSchedulerTest {
   private static final Logger logger = LoggerFactory.getLogger(CompactionSchedulerTest.class);
-  static final String COMPACTION_TEST_SG = "root.compactionSchedulerTest";
+  static final String COMPACTION_TEST_SG = "root.compactionSchedulerTest-";
   private static final boolean oldEnableInnerSeqCompaction =
       IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
   private static final boolean oldEnableInnerUnseqCompaction =
@@ -70,21 +68,21 @@ public class CompactionSchedulerTest {
   static final long SCHEDULE_AGAIN_TIME = 30_000;
   static final String[] fullPaths =
       new String[] {
-        COMPACTION_TEST_SG + ".device0.sensor0",
-        COMPACTION_TEST_SG + ".device0.sensor1",
-        COMPACTION_TEST_SG + ".device0.sensor2",
-        COMPACTION_TEST_SG + ".device0.sensor3",
-        COMPACTION_TEST_SG + ".device0.sensor4",
-        COMPACTION_TEST_SG + ".device0.sensor5",
-        COMPACTION_TEST_SG + ".device0.sensor6",
-        COMPACTION_TEST_SG + ".device0.sensor7",
-        COMPACTION_TEST_SG + ".device0.sensor8",
-        COMPACTION_TEST_SG + ".device0.sensor9",
-        COMPACTION_TEST_SG + ".device1.sensor0",
-        COMPACTION_TEST_SG + ".device1.sensor1",
-        COMPACTION_TEST_SG + ".device1.sensor2",
-        COMPACTION_TEST_SG + ".device1.sensor3",
-        COMPACTION_TEST_SG + ".device1.sensor4",
+        ".device0.sensor0",
+        ".device0.sensor1",
+        ".device0.sensor2",
+        ".device0.sensor3",
+        ".device0.sensor4",
+        ".device0.sensor5",
+        ".device0.sensor6",
+        ".device0.sensor7",
+        ".device0.sensor8",
+        ".device0.sensor9",
+        ".device1.sensor0",
+        ".device1.sensor1",
+        ".device1.sensor2",
+        ".device1.sensor3",
+        ".device1.sensor4",
       };
 
   @Before
@@ -92,20 +90,6 @@ public class CompactionSchedulerTest {
     CompactionClearUtils.clearAllCompactionFiles();
     EnvironmentUtils.cleanAllDir();
     IoTDB.metaManager.init();
-    try {
-      IoTDB.metaManager.setStorageGroup(new PartialPath(COMPACTION_TEST_SG));
-    } catch (Exception e) {
-      logger.error("exception occurs", e);
-    }
-    for (String fullPath : fullPaths) {
-      PartialPath path = new PartialPath(fullPath);
-      IoTDB.metaManager.createTimeseries(
-          path,
-          TSDataType.INT64,
-          TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
-          TSFileDescriptor.getInstance().getConfig().getCompressor(),
-          Collections.emptyMap());
-    }
     File basicOutputDir = new File(TestConstant.BASE_OUTPUT_PATH);
     IoTDBDescriptor.getInstance().getConfig().setCompactionPriority(CompactionPriority.INNER_CROSS);
     if (!basicOutputDir.exists()) {
@@ -150,7 +134,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=50 max_compaction_candidate_file_num=100
    */
   @Test
-  public void test1() throws IOException, IllegalPathException, InterruptedException {
+  public void test1() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test1");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -168,30 +152,44 @@ public class CompactionSchedulerTest {
     IoTDBDescriptor.getInstance()
         .getConfig()
         .setTargetCompactionFileSize(2L * 1024L * 1024L * 1024L);
+    String sgName = COMPACTION_TEST_SG + "test1";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -271,7 +269,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=50 max_compaction_candidate_file_num=100
    */
   @Test
-  public void test2() throws IOException, IllegalPathException, InterruptedException {
+  public void test2() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test2");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -289,30 +287,44 @@ public class CompactionSchedulerTest {
     IoTDBDescriptor.getInstance()
         .getConfig()
         .setCrossCompactionMemoryBudget(2 * 1024 * 1024L * 1024L);
-
+    String sgName = COMPACTION_TEST_SG + "test2";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -383,7 +395,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=50 max_compaction_candidate_file_num=100
    */
   @Test
-  public void test3() throws IOException, IllegalPathException, InterruptedException {
+  public void test3() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test3");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -397,30 +409,44 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(100);
+    String sgName = COMPACTION_TEST_SG + "test3";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
-
       CompactionTaskManager.getInstance().restart();
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -477,7 +503,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=50 max_compaction_candidate_file_num=100
    */
   @Test
-  public void test4() throws IOException, IllegalPathException, InterruptedException {
+  public void test4() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test4");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -494,29 +520,44 @@ public class CompactionSchedulerTest {
     IoTDBDescriptor.getInstance()
         .getConfig()
         .setTargetCompactionFileSize(2L * 1024L * 1024L * 1024L);
+    String sgName = COMPACTION_TEST_SG + "test4";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -559,7 +600,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=1 max_compaction_candidate_file_num=100
    */
   @Test
-  public void test5() throws IOException, IllegalPathException, InterruptedException {
+  public void test5() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test5");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -574,30 +615,44 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(100);
+    String sgName = COMPACTION_TEST_SG + "test5";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -672,7 +727,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=1 max_compaction_candidate_file_num=100
    */
   @Test
-  public void test6() throws IOException, IllegalPathException, InterruptedException {
+  public void test6() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test6");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -687,30 +742,44 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(100);
-
+    String sgName = COMPACTION_TEST_SG + "test6";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -773,7 +842,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=1 max_compaction_candidate_file_num=100
    */
   @Test
-  public void test7() throws IOException, IllegalPathException, InterruptedException {
+  public void test7() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test7");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -787,30 +856,44 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(100);
-
+    String sgName = COMPACTION_TEST_SG + "test7";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -872,7 +955,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=1 max_compaction_candidate_file_num=100
    */
   @Test
-  public void test8() throws IOException, IllegalPathException, InterruptedException {
+  public void test8() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test8");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -886,30 +969,46 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(100);
+    String sgName = COMPACTION_TEST_SG + "test8";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -955,7 +1054,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=50 max_compaction_candidate_file_num=2
    */
   @Test
-  public void test9() throws IOException, IllegalPathException, InterruptedException {
+  public void test9() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test9");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -969,30 +1068,46 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(2);
+    String sgName = COMPACTION_TEST_SG + "test9";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -1051,7 +1166,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=50 max_compaction_candidate_file_num=2
    */
   @Test
-  public void test10() throws IOException, IllegalPathException, InterruptedException {
+  public void test10() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test10");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -1068,30 +1183,46 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(2);
+    String sgName = COMPACTION_TEST_SG + "test10";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -1157,7 +1288,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=50 max_compaction_candidate_file_num=2
    */
   @Test
-  public void test11() throws IOException, IllegalPathException, InterruptedException {
+  public void test11() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test11");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -1171,30 +1302,46 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(2);
+    String sgName = COMPACTION_TEST_SG + "test11";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -1217,7 +1364,6 @@ public class CompactionSchedulerTest {
           e.printStackTrace();
         }
       }
-      //      assertEquals(100, tsFileManager.getTsFileList(false).size());
       CompactionScheduler.scheduleCompaction(tsFileManager, 0);
       CompactionTaskManager.getInstance().submitTaskFromTaskQueue();
       totalWaitingTime = 0;
@@ -1258,7 +1404,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=50 max_compaction_candidate_file_num=2
    */
   @Test
-  public void test12() throws IOException, IllegalPathException, InterruptedException {
+  public void test12() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test12");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -1272,30 +1418,46 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(2);
+    String sgName = COMPACTION_TEST_SG + "test12";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -1361,7 +1523,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=1 max_compaction_candidate_file_num=2
    */
   @Test
-  public void test14() throws IOException, IllegalPathException, InterruptedException {
+  public void test14() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test14");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -1375,29 +1537,46 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(2);
+    String sgName = COMPACTION_TEST_SG + "test13";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -1461,7 +1640,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=1 max_compaction_candidate_file_num=2
    */
   @Test
-  public void test15() throws IOException, IllegalPathException, InterruptedException {
+  public void test15() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test15");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -1475,30 +1654,46 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(2);
+    String sgName = COMPACTION_TEST_SG + "test14";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);
@@ -1562,7 +1757,7 @@ public class CompactionSchedulerTest {
    * compaction_concurrent_thread=1 max_compaction_candidate_file_num=2
    */
   @Test
-  public void test16() throws IOException, IllegalPathException, InterruptedException {
+  public void test16() throws IOException, MetadataException, InterruptedException {
     logger.warn("Running test16");
     boolean prevEnableSeqSpaceCompaction =
         IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
@@ -1576,30 +1771,46 @@ public class CompactionSchedulerTest {
     int prevMaxCompactionCandidateFileNum =
         IoTDBDescriptor.getInstance().getConfig().getMaxCompactionCandidateFileNum();
     IoTDBDescriptor.getInstance().getConfig().setMaxCompactionCandidateFileNum(2);
+    String sgName = COMPACTION_TEST_SG + "test16";
+    try {
+      IoTDB.metaManager.setStorageGroup(new PartialPath(sgName));
+    } catch (Exception e) {
+      logger.error("exception occurs", e);
+    }
     try {
       CompactionTaskManager.getInstance().restart();
-
-      TsFileManager tsFileManager = new TsFileManager(COMPACTION_TEST_SG, "0", "target");
+      TsFileManager tsFileManager = new TsFileManager(sgName, "0", "target");
+      Set<String> fullPath = new HashSet<>();
+      for (String device : fullPaths) {
+        fullPath.add(sgName + device);
+        PartialPath path = new PartialPath(sgName + device);
+        IoTDB.metaManager.createTimeseries(
+            path,
+            TSDataType.INT64,
+            TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder()),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+      }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(true, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 100, tsFileResource);
         tsFileManager.add(tsFileResource, true);
       }
       for (int i = 0; i < 100; i++) {
-        Set<String> fullPath = new HashSet<>(Arrays.asList(fullPaths));
+
         List<List<Long>> chunkPagePointsNum = new ArrayList<>();
         List<Long> pagePointsNum = new ArrayList<>();
         pagePointsNum.add(100L);
         chunkPagePointsNum.add(pagePointsNum);
         TsFileResource tsFileResource =
-            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, COMPACTION_TEST_SG);
+            CompactionFileGeneratorUtils.generateTsFileResource(false, i + 1, sgName);
         CompactionFileGeneratorUtils.writeTsFile(
             fullPath, chunkPagePointsNum, 100 * i + 50, tsFileResource);
         tsFileManager.add(tsFileResource, false);