You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ca...@apache.org on 2023/01/29 08:12:50 UTC

[iotdb] branch rc/1.0.1 updated: [To rc/1.0.1] [IOTDB-5266] Refine the code of cross selector and fix the bug when using FileTImeIndex (#8899)

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

caogaofei pushed a commit to branch rc/1.0.1
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rc/1.0.1 by this push:
     new 573097af3f [To rc/1.0.1] [IOTDB-5266] Refine the code of cross selector and fix the bug when using FileTImeIndex (#8899)
573097af3f is described below

commit 573097af3f024ec30705e40c47fe18ea69ff9c69
Author: Zhang.Jinrui <xi...@gmail.com>
AuthorDate: Sun Jan 29 16:12:44 2023 +0800

    [To rc/1.0.1] [IOTDB-5266] Refine the code of cross selector and fix the bug when using FileTImeIndex (#8899)
---
 .../db/engine/compaction/CompactionScheduler.java  |   8 +-
 .../compaction/cross/ICrossSpaceSelector.java      |   4 +-
 .../cross/rewrite/CrossCompactionTaskResource.java | 119 +++
 .../rewrite/CrossSpaceCompactionCandidate.java     | 259 ++++++
 .../rewrite/CrossSpaceCompactionResource.java      |  74 --
 .../RewriteCrossSpaceCompactionSelector.java       | 340 ++------
 .../compaction/task/ICompactionSelector.java       |   4 +-
 .../db/engine/storagegroup/TsFileResource.java     |  19 +
 .../storagegroup/timeindex/DeviceTimeIndex.java    |   4 +
 .../apache/iotdb/db/exception/MergeException.java  |   2 +-
 .../CrossSpaceCompactionWithFastPerformerTest.java |  21 +-
 ...eCompactionWithFastPerformerValidationTest.java | 937 +++++++++++----------
 ...sSpaceCompactionWithReadPointPerformerTest.java |  21 +-
 ...actionWithReadPointPerformerValidationTest.java | 906 ++++++++++----------
 .../engine/compaction/cross/MergeUpgradeTest.java  |   4 +-
 .../cross/RewriteCompactionFileSelectorTest.java   | 121 +--
 16 files changed, 1508 insertions(+), 1335 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java
index aa4eb75f72..cae5d2fa49 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java
@@ -24,12 +24,12 @@ import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTask;
 import org.apache.iotdb.db.engine.compaction.cross.ICrossSpaceSelector;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
 import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
 import org.apache.iotdb.db.engine.compaction.performer.ICompactionPerformer;
 import org.apache.iotdb.db.engine.compaction.task.ICompactionSelector;
 import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.tsfile.utils.Pair;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -145,7 +145,7 @@ public class CompactionScheduler {
         config
             .getCrossCompactionSelector()
             .createInstance(logicalStorageGroupName, dataRegionId, timePartition, tsFileManager);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> taskList =
+    List<CrossCompactionTaskResource> taskList =
         crossSpaceCompactionSelector.selectCrossSpaceTask(
             tsFileManager.getSequenceListByTimePartition(timePartition),
             tsFileManager.getUnsequenceListByTimePartition(timePartition));
@@ -156,8 +156,8 @@ public class CompactionScheduler {
               new CrossSpaceCompactionTask(
                   timePartition,
                   tsFileManager,
-                  taskList.get(i).left,
-                  taskList.get(i).right,
+                  taskList.get(i).getSeqFiles(),
+                  taskList.get(i).getUnseqFiles(),
                   IoTDBDescriptor.getInstance()
                       .getConfig()
                       .getCrossCompactionPerformer()
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/ICrossSpaceSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/ICrossSpaceSelector.java
index 061e8d2f26..4a7668cc62 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/ICrossSpaceSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/ICrossSpaceSelector.java
@@ -18,13 +18,13 @@
  */
 package org.apache.iotdb.db.engine.compaction.cross;
 
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
 import org.apache.iotdb.db.engine.compaction.task.ICompactionSelector;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.tsfile.utils.Pair;
 
 import java.util.List;
 
 public interface ICrossSpaceSelector extends ICompactionSelector {
-  List<Pair<List<TsFileResource>, List<TsFileResource>>> selectCrossSpaceTask(
+  List<CrossCompactionTaskResource> selectCrossSpaceTask(
       List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossCompactionTaskResource.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossCompactionTaskResource.java
new file mode 100644
index 0000000000..129b7cb791
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossCompactionTaskResource.java
@@ -0,0 +1,119 @@
+/*
+ * 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.rewrite;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class CrossCompactionTaskResource {
+  private List<TsFileResource> seqFiles;
+  private List<TsFileResource> unseqFiles;
+
+  private long totalMemoryCost;
+  private long totalFileSize;
+  private long totalFileNums;
+
+  public CrossCompactionTaskResource() {
+    this.seqFiles = new ArrayList<>();
+    this.unseqFiles = new ArrayList<>();
+    this.totalMemoryCost = 0L;
+    this.totalFileSize = 0L;
+    this.totalFileNums = 0L;
+  }
+
+  public List<TsFileResource> getSeqFiles() {
+    return seqFiles;
+  }
+
+  // we need to unsure the files in seqFiles is ordered by the time range, that is, it should keep
+  // the
+  // order from candidates' seq file list.
+  public void sortSeqFiles(List<TsFileResource> seqFilesCandidates) {
+    Map<TsFileResource, Boolean> selectedFileMap = new HashMap<>();
+    for (TsFileResource selectedFile : this.seqFiles) {
+      selectedFileMap.put(selectedFile, true);
+    }
+    List<TsFileResource> sortedSeqFiles = new ArrayList<>();
+    for (TsFileResource file : seqFilesCandidates) {
+      if (selectedFileMap.containsKey(file)) {
+        sortedSeqFiles.add(file);
+      }
+    }
+    this.seqFiles = sortedSeqFiles;
+  }
+
+  public void putResources(
+      TsFileResource unseqFile, List<TsFileResource> seqFiles, long memoryCost) {
+    addUnseqFile(unseqFile);
+    addTargetSeqFiles(seqFiles);
+    increaseMemoryCost(memoryCost);
+  }
+
+  private void addUnseqFile(TsFileResource file) {
+    unseqFiles.add(file);
+    countStatistic(file);
+  }
+
+  private void addTargetSeqFiles(List<TsFileResource> targetSeqFiles) {
+    targetSeqFiles.forEach(this::addSeqFile);
+  }
+
+  private void addSeqFile(TsFileResource file) {
+    seqFiles.add(file);
+    countStatistic(file);
+  }
+
+  private void increaseMemoryCost(long newMemoryCost) {
+    this.totalMemoryCost += newMemoryCost;
+  }
+
+  private void countStatistic(TsFileResource file) {
+    totalFileSize += file.getTsFileSize();
+    totalFileNums += 1;
+  }
+
+  public List<TsFileResource> getUnseqFiles() {
+    return unseqFiles;
+  }
+
+  public long getTotalMemoryCost() {
+    return totalMemoryCost;
+  }
+
+  public long getTotalFileSize() {
+    return totalFileSize;
+  }
+
+  public long getTotalFileNums() {
+    return totalFileNums;
+  }
+
+  public boolean isValid() {
+    // Regarding current implementation of cross compaction task, the unseqFiles and seqFiles should
+    // not be empty.
+    // It should be changed once the task execution is optimized.
+    // See https://issues.apache.org/jira/browse/IOTDB-5263
+    return !unseqFiles.isEmpty() && !seqFiles.isEmpty();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossSpaceCompactionCandidate.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossSpaceCompactionCandidate.java
new file mode 100644
index 0000000000..bc17956bd7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossSpaceCompactionCandidate.java
@@ -0,0 +1,259 @@
+/*
+ * 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.rewrite;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
+import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex;
+import org.apache.iotdb.db.engine.storagegroup.timeindex.ITimeIndex;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * CrossSpaceCompactionResource manages files and caches of readers to avoid unnecessary object
+ * creations and file openings.
+ */
+public class CrossSpaceCompactionCandidate {
+  private List<TsFileResourceCandidate> seqFiles;
+  private List<TsFileResourceCandidate> unseqFiles;
+
+  private int nextUnseqFileIndex;
+  private CrossCompactionTaskResourceSplit nextSplit;
+  private long ttlLowerBound = Long.MIN_VALUE;
+
+  public CrossSpaceCompactionCandidate(
+      List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
+    init(seqFiles, unseqFiles);
+  }
+
+  public CrossSpaceCompactionCandidate(
+      List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles, long ttlLowerBound) {
+    this.ttlLowerBound = ttlLowerBound;
+    init(seqFiles, unseqFiles);
+  }
+
+  private void init(List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
+    this.seqFiles = copySeqResource(seqFiles);
+    // it is necessary that unseqFiles are all available
+    this.unseqFiles = filterUnseqResource(unseqFiles);
+    this.nextUnseqFileIndex = 0;
+  }
+
+  public boolean hasNextSplit() throws IOException {
+    if (nextUnseqFileIndex >= unseqFiles.size()) {
+      return false;
+    }
+    return prepareNextSplit();
+  }
+
+  public CrossCompactionTaskResourceSplit nextSplit() {
+    return nextSplit;
+  }
+
+  private boolean prepareNextSplit() throws IOException {
+    TsFileResourceCandidate unseqFile = unseqFiles.get(nextUnseqFileIndex);
+    List<TsFileResourceCandidate> ret = new ArrayList<>();
+
+    // The startTime and endTime of each device are different in one TsFile. So we need to do the
+    // check
+    // one by one. And we cannot skip any device in the unseq file because it may lead to omission
+    // of
+    // target seq file
+    for (DeviceInfo unseqDeviceInfo : unseqFile.getDevices()) {
+      for (TsFileResourceCandidate seqFile : seqFiles) {
+        if (!seqFile.containsDevice(unseqDeviceInfo.deviceId)) {
+          continue;
+        }
+        DeviceInfo seqDeviceInfo = seqFile.getDeviceInfoById(unseqDeviceInfo.deviceId);
+        // If the seqFile should be selected but its invalid, the selection should be terminated.
+        if (!seqFile.isValidCandidate && unseqDeviceInfo.startTime <= seqDeviceInfo.endTime) {
+          return false;
+        }
+        // If the unsealed file is unclosed, the file should not be selected only when its startTime
+        // is larger than endTime of unseqFile. Or, the selection should be terminated.
+        if (seqFile.unsealed() && unseqDeviceInfo.endTime >= seqDeviceInfo.startTime) {
+          return false;
+        }
+        if (unseqDeviceInfo.endTime <= seqDeviceInfo.endTime) {
+          // When scanning the target seqFiles for unseqFile, we traverse them one by one no matter
+          // whether it is selected or not. But we only add the unselected seqFiles to next split to
+          // avoid duplication selection
+          if (!seqFile.selected) {
+            ret.add(seqFile);
+            seqFile.markAsSelected();
+          }
+          // if this condition is satisfied, all subsequent seq files is unnecessary to check
+          break;
+        } else if (unseqDeviceInfo.startTime <= seqDeviceInfo.endTime) {
+          if (!seqFile.selected) {
+            ret.add(seqFile);
+            seqFile.markAsSelected();
+          }
+        }
+      }
+    }
+    // mark candidates in next split as selected even though it may not be added to the final
+    // TaskResource
+    unseqFile.markAsSelected();
+    nextSplit = new CrossCompactionTaskResourceSplit(unseqFile, ret);
+    nextUnseqFileIndex++;
+    return true;
+  }
+
+  private List<TsFileResourceCandidate> copySeqResource(List<TsFileResource> seqFiles) {
+    List<TsFileResourceCandidate> ret = new ArrayList<>();
+    for (TsFileResource resource : seqFiles) {
+      ret.add(new TsFileResourceCandidate(resource));
+    }
+    return ret;
+  }
+
+  /**
+   * Filter the unseq files into the compaction. Unseq files should be not deleted or over ttl. To
+   * ensure that the compaction is correct, return as soon as it encounters the file being compacted
+   * or compaction candidate. Therefore, a cross space compaction can only be performed serially
+   * under a time partition in a VSG.
+   */
+  private List<TsFileResourceCandidate> filterUnseqResource(List<TsFileResource> unseqResources) {
+    List<TsFileResourceCandidate> ret = new ArrayList<>();
+    for (TsFileResource resource : unseqResources) {
+      if (resource.getStatus() != TsFileResourceStatus.CLOSED
+          || !resource.getTsFile().exists()
+          || resource.isDeleted()) {
+        break;
+      } else if (!resource.isDeleted() && resource.stillLives(ttlLowerBound)) {
+        ret.add(new TsFileResourceCandidate(resource));
+      }
+    }
+    return ret;
+  }
+
+  public List<TsFileResource> getSeqFiles() {
+    return seqFiles.stream()
+        .map(tsFileResourceCandidate -> tsFileResourceCandidate.resource)
+        .collect(Collectors.toList());
+  }
+
+  public List<TsFileResourceCandidate> getUnseqFileCandidates() {
+    return unseqFiles;
+  }
+
+  public List<TsFileResource> getUnseqFiles() {
+    return unseqFiles.stream()
+        .map(tsFileResourceCandidate -> tsFileResourceCandidate.resource)
+        .collect(Collectors.toList());
+  }
+
+  protected static class CrossCompactionTaskResourceSplit {
+    protected TsFileResourceCandidate unseqFile;
+    protected List<TsFileResourceCandidate> seqFiles;
+
+    public CrossCompactionTaskResourceSplit(
+        TsFileResourceCandidate unseqFile, List<TsFileResourceCandidate> seqFiles) {
+      this.unseqFile = unseqFile;
+      this.seqFiles = seqFiles;
+    }
+  }
+
+  protected static class TsFileResourceCandidate {
+    protected TsFileResource resource;
+    protected boolean selected;
+    protected boolean isValidCandidate;
+    private Map<String, DeviceInfo> deviceInfoMap;
+
+    protected TsFileResourceCandidate(TsFileResource tsFileResource) {
+      this.resource = tsFileResource;
+      this.selected = false;
+      // although we do the judgement here, the task should be validated before executing because
+      // the status of file may be changed after the task is submitted to queue
+      this.isValidCandidate =
+          tsFileResource.getStatus() == TsFileResourceStatus.CLOSED
+              && tsFileResource.getTsFile().exists();
+    }
+
+    /**
+     * The TsFile is unsealed means there may be more data which will be inserted into this file
+     *
+     * @return Whether the TsFile is unsealed.
+     */
+    protected boolean unsealed() {
+      return resource.getStatus() == TsFileResourceStatus.UNCLOSED;
+    }
+
+    private void prepareDeviceInfos() throws IOException {
+      if (deviceInfoMap != null) {
+        return;
+      }
+      deviceInfoMap = new LinkedHashMap<>();
+      if (resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE) {
+        DeviceTimeIndex timeIndex = resource.buildDeviceTimeIndex();
+        for (String deviceId : timeIndex.getDevices()) {
+          deviceInfoMap.put(
+              deviceId,
+              new DeviceInfo(
+                  deviceId, timeIndex.getStartTime(deviceId), timeIndex.getEndTime(deviceId)));
+        }
+      } else {
+        for (String deviceId : resource.getDevices()) {
+          deviceInfoMap.put(
+              deviceId,
+              new DeviceInfo(
+                  deviceId, resource.getStartTime(deviceId), resource.getEndTime(deviceId)));
+        }
+      }
+    }
+
+    protected void markAsSelected() {
+      this.selected = true;
+    }
+
+    protected List<DeviceInfo> getDevices() throws IOException {
+      prepareDeviceInfos();
+      return new ArrayList<>(deviceInfoMap.values());
+    }
+
+    protected DeviceInfo getDeviceInfoById(String deviceId) throws IOException {
+      prepareDeviceInfos();
+      return deviceInfoMap.get(deviceId);
+    }
+
+    protected boolean containsDevice(String deviceId) throws IOException {
+      prepareDeviceInfos();
+      return deviceInfoMap.containsKey(deviceId);
+    }
+  }
+
+  protected static class DeviceInfo {
+    protected String deviceId;
+    protected long startTime;
+    protected long endTime;
+
+    public DeviceInfo(String deviceId, long startTime, long endTime) {
+      this.deviceId = deviceId;
+      this.startTime = startTime;
+      this.endTime = endTime;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossSpaceCompactionResource.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossSpaceCompactionResource.java
deleted file mode 100644
index 34d270a026..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossSpaceCompactionResource.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.cross.rewrite;
-
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * CrossSpaceCompactionResource manages files and caches of readers to avoid unnecessary object
- * creations and file openings.
- */
-public class CrossSpaceCompactionResource {
-  private List<TsFileResource> seqFiles;
-  private List<TsFileResource> unseqFiles = new ArrayList<>();
-
-  private long ttlLowerBound = Long.MIN_VALUE;
-
-  public CrossSpaceCompactionResource(
-      List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
-    this.seqFiles = seqFiles;
-    filterUnseqResource(unseqFiles);
-  }
-
-  /**
-   * Filter the unseq files into the compaction. Unseq files should be not deleted or over ttl. To
-   * ensure that the compaction is correct, return as soon as it encounters the file being compacted
-   * or compaction candidate. Therefore, a cross space compaction can only be performed serially
-   * under a time partition in a VSG.
-   */
-  private void filterUnseqResource(List<TsFileResource> unseqResources) {
-    for (TsFileResource resource : unseqResources) {
-      if (resource.getStatus() != TsFileResourceStatus.CLOSED) {
-        return;
-      } else if (!resource.isDeleted() && resource.stillLives(ttlLowerBound)) {
-        this.unseqFiles.add(resource);
-      }
-    }
-  }
-
-  public CrossSpaceCompactionResource(
-      List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles, long ttlLowerBound) {
-    this.ttlLowerBound = ttlLowerBound;
-    this.seqFiles = seqFiles;
-    filterUnseqResource(unseqFiles);
-  }
-
-  public List<TsFileResource> getSeqFiles() {
-    return seqFiles;
-  }
-
-  public List<TsFileResource> getUnseqFiles() {
-    return unseqFiles;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionSelector.java
index 0aa0de10c5..5a6e248acc 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionSelector.java
@@ -23,53 +23,38 @@ import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
 import org.apache.iotdb.db.engine.compaction.cross.ICrossSpaceSelector;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate.CrossCompactionTaskResourceSplit;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate.TsFileResourceCandidate;
 import org.apache.iotdb.db.engine.compaction.cross.utils.AbstractCompactionEstimator;
 import org.apache.iotdb.db.engine.compaction.task.ICompactionSelector;
 import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
 import org.apache.iotdb.db.exception.MergeException;
 import org.apache.iotdb.db.rescon.SystemInfo;
-import org.apache.iotdb.tsfile.utils.Pair;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
+import java.util.stream.Collectors;
 
 public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector {
   private static final Logger LOGGER =
       LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
   private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
 
-  private static boolean hasPrintLog = false;
   private final int SELECT_WARN_THRESHOLD = 10;
   protected String logicalStorageGroupName;
   protected String dataRegionId;
   protected long timePartition;
   protected TsFileManager tsFileManager;
 
-  private CrossSpaceCompactionResource resource;
-
   private long totalCost;
-  private long totalSize;
   private final long memoryBudget;
   private final int maxCrossCompactionFileNum;
   private final long maxCrossCompactionFileSize;
-
-  private List<TsFileResource> selectedUnseqFiles;
-  private List<TsFileResource> selectedSeqFiles;
-
-  private Collection<Integer> tmpSelectedSeqFiles;
-
-  private boolean[] seqSelected;
   private int seqSelectedNum;
 
   private AbstractCompactionEstimator compactionEstimator;
@@ -121,18 +106,24 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector
    * @return two lists of TsFileResource, the former is selected seqFiles and the latter is selected
    *     unseqFiles or an empty array if there are no proper candidates by the budget.
    */
-  private List[] select() throws MergeException {
+  private CrossCompactionTaskResource selectOneTaskResources(
+      CrossSpaceCompactionCandidate candidate) throws MergeException {
     long startTime = System.currentTimeMillis();
     try {
       LOGGER.debug(
-          "Selecting merge candidates from {} seqFile, {} unseqFiles",
-          resource.getSeqFiles().size(),
-          resource.getUnseqFiles().size());
-      selectSourceFiles();
-      if (selectedUnseqFiles.isEmpty()) {
-        LOGGER.debug("No merge candidates are found");
-        return new List[0];
-      }
+          "Selecting cross compaction task resources from {} seqFile, {} unseqFiles",
+          candidate.getSeqFiles().size(),
+          candidate.getUnseqFiles().size());
+      CrossCompactionTaskResource taskResource = executeTaskResourceSelection(candidate);
+      LOGGER.info(
+          "selected one cross compaction task resource. is valid: {}, {} seqFiles, {} unseqFiles, total memory cost {}, "
+              + "time consumption {}ms",
+          taskResource.isValid(),
+          taskResource.getSeqFiles().size(),
+          taskResource.getUnseqFiles().size(),
+          taskResource.getTotalMemoryCost(),
+          System.currentTimeMillis() - startTime);
+      return taskResource;
     } catch (IOException e) {
       throw new MergeException(e);
     } finally {
@@ -142,15 +133,15 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector
         throw new MergeException(e);
       }
     }
-    LOGGER.info(
-        "Selected merge candidates, {} seqFiles, {} unseqFiles, total memory cost {}, "
-            + "time consumption {}ms",
-        selectedSeqFiles.size(),
-        selectedUnseqFiles.size(),
-        totalCost,
-        System.currentTimeMillis() - startTime);
+  }
 
-    return new List[] {selectedSeqFiles, selectedUnseqFiles};
+  private boolean isAllFileCandidateValid(List<TsFileResourceCandidate> tsFileResourceCandidates) {
+    for (TsFileResourceCandidate candidate : tsFileResourceCandidates) {
+      if (!candidate.isValidCandidate) {
+        return false;
+      }
+    }
+    return true;
   }
 
   /**
@@ -161,199 +152,60 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector
    * exceed the memory overhead preset by the system for the compaction thread, put them into the
    * selectedSeqFiles and selectedUnseqFiles.
    */
-  void selectSourceFiles() throws IOException {
-    tmpSelectedSeqFiles = new HashSet<>();
-    seqSelected = new boolean[resource.getSeqFiles().size()];
-    seqSelectedNum = 0;
-    selectedSeqFiles = new ArrayList<>();
-    selectedUnseqFiles = new ArrayList<>();
-
-    totalCost = 0;
-    totalSize = 0;
-
-    int unseqIndex = 0;
-    long startTime = System.currentTimeMillis();
-    long timeConsumption = 0;
-    long timeLimit =
-        IoTDBDescriptor.getInstance().getConfig().getCrossCompactionFileSelectionTimeBudget();
-    if (timeLimit < 0) {
-      timeLimit = Long.MAX_VALUE;
-    }
-    while (unseqIndex < resource.getUnseqFiles().size() && timeConsumption < timeLimit) {
-      // select next unseq files
-      TsFileResource unseqFile = resource.getUnseqFiles().get(unseqIndex);
-      if (!unseqFile.getTsFile().exists() || unseqFile.isDeleted()) {
-        break;
-      }
-
-      if (seqSelectedNum != resource.getSeqFiles().size()) {
-        selectOverlappedSeqFiles(unseqFile);
-      }
-      boolean isSeqFilesValid = checkIsSeqFilesValid();
-      if (!isSeqFilesValid) {
-        tmpSelectedSeqFiles.clear();
+  private CrossCompactionTaskResource executeTaskResourceSelection(
+      CrossSpaceCompactionCandidate candidate) throws IOException {
+    CrossCompactionTaskResource taskResource = new CrossCompactionTaskResource();
+
+    while (candidate.hasNextSplit()) {
+      CrossCompactionTaskResourceSplit split = candidate.nextSplit();
+      TsFileResource unseqFile = split.unseqFile.resource;
+      List<TsFileResource> targetSeqFiles =
+          split.seqFiles.stream().map(c -> c.resource).collect(Collectors.toList());
+      long memoryCost =
+          compactionEstimator.estimateCrossCompactionMemory(targetSeqFiles, unseqFile);
+      if (!canAddToTaskResource(taskResource, unseqFile, targetSeqFiles, memoryCost)) {
         break;
       }
-
-      // Filter out the selected seq files
-      for (int i = 0; i < seqSelected.length; i++) {
-        if (seqSelected[i]) {
-          tmpSelectedSeqFiles.remove(i);
-        }
-      }
-
-      List<TsFileResource> tmpSelectedSeqFileResources = new ArrayList<>();
-      for (int seqIndex : tmpSelectedSeqFiles) {
-        TsFileResource tsFileResource = resource.getSeqFiles().get(seqIndex);
-        tmpSelectedSeqFileResources.add(tsFileResource);
-        totalSize += tsFileResource.getTsFileSize();
-      }
-      totalSize += unseqFile.getTsFileSize();
-
-      long newCost =
-          compactionEstimator.estimateCrossCompactionMemory(tmpSelectedSeqFileResources, unseqFile);
-      if (!updateSelectedFiles(newCost, unseqFile)) {
-        // older unseq files must be merged before newer ones
-        break;
-      }
-
-      tmpSelectedSeqFiles.clear();
-      unseqIndex++;
-      timeConsumption = System.currentTimeMillis() - startTime;
-    }
-    for (int i = 0; i < seqSelected.length; i++) {
-      if (seqSelected[i]) {
-        selectedSeqFiles.add(resource.getSeqFiles().get(i));
-      }
-    }
-  }
-
-  private boolean updateSelectedFiles(long newCost, TsFileResource unseqFile) {
-    if (selectedUnseqFiles.size() == 0
-        || (seqSelectedNum + selectedUnseqFiles.size() + 1 + tmpSelectedSeqFiles.size()
-                <= maxCrossCompactionFileNum
-            && totalSize <= maxCrossCompactionFileSize
-            && totalCost + newCost < memoryBudget)) {
-      selectedUnseqFiles.add(unseqFile);
-
-      for (Integer seqIdx : tmpSelectedSeqFiles) {
-        if (!seqSelected[seqIdx]) {
-          seqSelectedNum++;
-          seqSelected[seqIdx] = true;
-        }
-      }
-      totalCost += newCost;
+      taskResource.putResources(unseqFile, targetSeqFiles, memoryCost);
       LOGGER.debug(
-          "Adding a new unseqFile {} and seqFiles {} as candidates, new cost {}, total"
-              + " cost {}",
+          "Adding a new unseqFile {} and seqFiles {} as candidates, new cost {}, total cost {}",
           unseqFile,
-          tmpSelectedSeqFiles,
-          newCost,
+          targetSeqFiles,
+          memoryCost,
           totalCost);
-      return true;
     }
-    return false;
+    taskResource.sortSeqFiles(candidate.getSeqFiles());
+    return taskResource;
   }
 
-  /**
-   * To avoid redundant data in seq files, cross space compaction should select all the seq files
-   * which have overlap with unseq files whether they are compacting or not. Therefore, before
-   * adding task into the queue, cross space compaction task should check whether source seq files
-   * are being compacted or not to speed up compaction.
-   */
-  private boolean checkIsSeqFilesValid() {
-    for (Integer seqIdx : tmpSelectedSeqFiles) {
-      if (resource.getSeqFiles().get(seqIdx).getStatus() != TsFileResourceStatus.CLOSED
-          || !resource.getSeqFiles().get(seqIdx).getTsFile().exists()) {
-        return false;
-      }
+  // TODO: (xingtanzjr) need to confirm whether we should strictly guarantee the conditions
+  // If we guarantee the condition strictly, the smallest collection of cross task resource may not
+  // satisfied
+  private boolean canAddToTaskResource(
+      CrossCompactionTaskResource taskResource,
+      TsFileResource unseqFile,
+      List<TsFileResource> seqFiles,
+      long memoryCost) {
+    // currently, we must allow at least one unseqFile be selected to handle the situation that
+    // an unseqFile has huge time range but few data points.
+    if (taskResource.getUnseqFiles().size() == 0) {
+      return true;
     }
-    return true;
+    long totalFileSize = unseqFile.getTsFileSize();
+    for (TsFileResource f : seqFiles) {
+      totalFileSize += f.getTsFileSize();
+    }
+    if (taskResource.getTotalFileNums() + 1 + seqFiles.size() <= maxCrossCompactionFileNum
+        && taskResource.getTotalFileSize() + totalFileSize <= maxCrossCompactionFileSize
+        && taskResource.getTotalMemoryCost() + memoryCost < memoryBudget) {
+      return true;
+    }
+    return false;
   }
 
-  /**
-   * Put the index of the seqFile that has an overlap with the specific unseqFile and has not been
-   * selected by the file selector of the compaction task into the tmpSelectedSeqFiles list. To
-   * determine whether overlap exists is to traverse each device ChunkGroup in unseqFiles, and
-   * determine whether it overlaps with the same device ChunkGroup of each seqFile that are not
-   * selected by the compaction task, if so, select this seqFile.
-   *
-   * @param unseqFile the tsFileResource of unseqFile to be compacted
-   */
-  private void selectOverlappedSeqFiles(TsFileResource unseqFile) {
-    for (String deviceId : unseqFile.getDevices()) {
-      long unseqStartTime = unseqFile.getStartTime(deviceId);
-      long unseqEndTime = unseqFile.getEndTime(deviceId);
-
-      boolean noMoreOverlap = false;
-      for (int i = 0; i < resource.getSeqFiles().size() && !noMoreOverlap; i++) {
-        TsFileResource seqFile = resource.getSeqFiles().get(i);
-        if (!seqFile.mayContainsDevice(deviceId)) {
-          continue;
-        }
-
-        int crossSpaceCompactionTimes = 0;
-        try {
-          TsFileNameGenerator.TsFileName tsFileName =
-              TsFileNameGenerator.getTsFileName(seqFile.getTsFile().getName());
-          crossSpaceCompactionTimes = tsFileName.getCrossCompactionCnt();
-        } catch (IOException e) {
-          LOGGER.warn("Meets IOException when selecting files for cross space compaction", e);
-        }
-
-        long seqEndTime = seqFile.getEndTime(deviceId);
-        long seqStartTime = seqFile.getStartTime(deviceId);
-        if (!seqFile.isClosed()) {
-          // for unclosed file, only select those that overlap with the unseq file
-          if (unseqEndTime >= seqStartTime) {
-            tmpSelectedSeqFiles.add(i);
-            if (crossSpaceCompactionTimes >= SELECT_WARN_THRESHOLD) {
-              LOGGER.warn(
-                  "{} is selected for cross space compaction, it is overlapped with {} in device {}. Sequence file time range:[{},{}], Unsequence file time range:[{},{}]",
-                  seqFile.getTsFile().getAbsolutePath(),
-                  unseqFile.getTsFile().getAbsolutePath(),
-                  deviceId,
-                  seqStartTime,
-                  seqEndTime,
-                  unseqStartTime,
-                  unseqEndTime);
-            }
-          }
-        } else if (unseqEndTime <= seqEndTime) {
-          // if time range in unseq file is 10-20, seq file is 30-40, or
-          // time range in unseq file is 10-20, seq file is 15-25, then select this seq file and
-          // there is no more overlap later.
-          tmpSelectedSeqFiles.add(i);
-          noMoreOverlap = true;
-          if (crossSpaceCompactionTimes >= SELECT_WARN_THRESHOLD) {
-            LOGGER.warn(
-                "{} is selected for cross space compaction, it is overlapped with {} in device {}. Sequence file time range:[{},{}], Unsequence file time range:[{},{}]",
-                seqFile.getTsFile().getAbsolutePath(),
-                unseqFile.getTsFile().getAbsolutePath(),
-                deviceId,
-                seqStartTime,
-                seqEndTime,
-                unseqStartTime,
-                unseqEndTime);
-          }
-        } else if (unseqStartTime <= seqEndTime) {
-          // if time range in unseq file is 10-20, seq file is 0-15, then select this seq file and
-          // there may be overlap later.
-          tmpSelectedSeqFiles.add(i);
-          if (crossSpaceCompactionTimes >= SELECT_WARN_THRESHOLD) {
-            LOGGER.warn(
-                "{} is selected for cross space compaction, it is overlapped with {} in device {}. Sequence file time range:[{},{}], Unsequence file time range:[{},{}]",
-                seqFile.getTsFile().getAbsolutePath(),
-                unseqFile.getTsFile().getAbsolutePath(),
-                deviceId,
-                seqStartTime,
-                seqEndTime,
-                unseqStartTime,
-                unseqEndTime);
-          }
-        }
-      }
-    }
+  private boolean canSubmitCrossTask() {
+    return config.isEnableCrossSpaceCompaction()
+        && (CompactionTaskManager.currentTaskNum.get() < config.getCompactionThreadCount());
   }
 
   /**
@@ -365,49 +217,35 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector
    * @return Returns whether the file was found and submits the merge task
    */
   @Override
-  public List selectCrossSpaceTask(
+  public List<CrossCompactionTaskResource> selectCrossSpaceTask(
       List<TsFileResource> sequenceFileList, List<TsFileResource> unsequenceFileList) {
-    if ((CompactionTaskManager.currentTaskNum.get() >= config.getCompactionThreadCount())
-        || (!config.isEnableCrossSpaceCompaction())) {
+    if (!canSubmitCrossTask()) {
       return Collections.emptyList();
     }
-    Iterator<TsFileResource> seqIterator = sequenceFileList.iterator();
-    Iterator<TsFileResource> unSeqIterator = unsequenceFileList.iterator();
-    List<TsFileResource> seqFileList = new ArrayList<>();
-    List<TsFileResource> unSeqFileList = new ArrayList<>();
-    while (seqIterator.hasNext()) {
-      seqFileList.add(seqIterator.next());
-    }
-    while (unSeqIterator.hasNext()) {
-      unSeqFileList.add(unSeqIterator.next());
-    }
-    if (seqFileList.isEmpty() || unSeqFileList.isEmpty()) {
+    if (sequenceFileList.isEmpty() || unsequenceFileList.isEmpty()) {
       return Collections.emptyList();
     }
-    long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
-    this.resource = new CrossSpaceCompactionResource(seqFileList, unSeqFileList, timeLowerBound);
-
+    // TODO: (xingtanzjr) need to confirm what this ttl is used for
+    long ttlLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
+    // we record the variable `candidate` here is used for selecting more than one
+    // CrossCompactionTaskResources in this method
+    CrossSpaceCompactionCandidate candidate =
+        new CrossSpaceCompactionCandidate(sequenceFileList, unsequenceFileList, ttlLowerBound);
     try {
-      List[] mergeFiles = select();
-      if (mergeFiles.length == 0) {
-        if (!hasPrintLog) {
-          LOGGER.info(
-              "{} [Compaction] Cannot select any files, because source files may be occupied by other compaction threads.",
-              logicalStorageGroupName + "-" + dataRegionId);
-          hasPrintLog = true;
-        }
+      CrossCompactionTaskResource taskResources = selectOneTaskResources(candidate);
+      if (!taskResources.isValid()) {
+        LOGGER.info(
+            "{} [Compaction] Cannot select any files, because source files may be occupied by other compaction threads.",
+            logicalStorageGroupName + "-" + dataRegionId);
         return Collections.emptyList();
       }
-      hasPrintLog = false;
 
-      if (mergeFiles[0].size() > 0 && mergeFiles[1].size() > 0) {
-        LOGGER.info(
-            "{} [Compaction] submit a task with {} sequence file and {} unseq files",
-            logicalStorageGroupName + "-" + dataRegionId,
-            mergeFiles[0].size(),
-            mergeFiles[1].size());
-        return Collections.singletonList(new Pair<>(mergeFiles[0], mergeFiles[1]));
-      }
+      LOGGER.info(
+          "{} [Compaction] submit a task with {} sequence file and {} unseq files",
+          logicalStorageGroupName + "-" + dataRegionId,
+          taskResources.getSeqFiles().size(),
+          taskResources.getUnseqFiles().size());
+      return Collections.singletonList(taskResources);
 
     } catch (MergeException e) {
       LOGGER.error("{} cannot select file for cross space compaction", logicalStorageGroupName, e);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/ICompactionSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/ICompactionSelector.java
index e198acda6a..34fd13c664 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/ICompactionSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/ICompactionSelector.java
@@ -19,10 +19,10 @@
 package org.apache.iotdb.db.engine.compaction.task;
 
 import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
 import org.apache.iotdb.db.engine.compaction.cross.utils.AbstractCompactionEstimator;
 import org.apache.iotdb.db.engine.compaction.cross.utils.ReadPointCrossCompactionEstimator;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.tsfile.utils.Pair;
 
 import java.util.Collections;
 import java.util.List;
@@ -49,7 +49,7 @@ public interface ICompactionSelector {
    * list contains two list: the left one is the selected sequence files, the right one is the selected
    * unsequence files. Each pair is corresponding to a cross space compaction task.
    */
-  default List<Pair<List<TsFileResource>, List<TsFileResource>>> selectCrossSpaceTask(
+  default List<CrossCompactionTaskResource> selectCrossSpaceTask(
       List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
     {
       throw new RuntimeException("This kind of selector cannot be used to select cross space task");
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
index b3db6133eb..6a27ff1d54 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
@@ -417,6 +417,25 @@ public class TsFileResource {
     return timeIndex.getDevices(file.getPath(), this);
   }
 
+  public DeviceTimeIndex buildDeviceTimeIndex() throws IOException {
+    readLock();
+    try (InputStream inputStream =
+        FSFactoryProducer.getFSFactory()
+            .getBufferedInputStream(file.getPath() + TsFileResource.RESOURCE_SUFFIX)) {
+      ReadWriteIOUtils.readByte(inputStream);
+      ITimeIndex timeIndexFromResourceFile = ITimeIndex.createTimeIndex(inputStream);
+      if (!(timeIndexFromResourceFile instanceof DeviceTimeIndex)) {
+        throw new IOException("cannot build DeviceTimeIndex from resource " + file.getPath());
+      }
+      return (DeviceTimeIndex) timeIndexFromResourceFile;
+    } catch (Exception e) {
+      throw new IOException(
+          "Can't read file " + file.getPath() + TsFileResource.RESOURCE_SUFFIX + " from disk", e);
+    } finally {
+      readUnlock();
+    }
+  }
+
   /**
    * Whether this TsFileResource contains this device, if false, it must not contain this device, if
    * true, it may or may not contain this device
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
index 1c42ecd31a..716bb9986e 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
@@ -150,6 +150,10 @@ public class DeviceTimeIndex implements ITimeIndex {
     endTimes = Arrays.copyOfRange(endTimes, 0, deviceToIndex.size());
   }
 
+  public Set<String> getDevices() {
+    return deviceToIndex.keySet();
+  }
+
   @Override
   public Set<String> getDevices(String tsFilePath, TsFileResource tsFileResource) {
     return deviceToIndex.keySet();
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/MergeException.java b/server/src/main/java/org/apache/iotdb/db/exception/MergeException.java
index 4b0f7ddf9d..60395faa0c 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/MergeException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/MergeException.java
@@ -27,6 +27,6 @@ public class MergeException extends IoTDBException {
   private static final long serialVersionUID = 5445240999141374140L;
 
   public MergeException(Throwable cause) {
-    super(cause.getMessage(), TSStatusCode.COMPACTION_ERROR.getStatusCode());
+    super(cause.getMessage(), cause, TSStatusCode.COMPACTION_ERROR.getStatusCode());
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithFastPerformerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithFastPerformerTest.java
index da850a691a..571d3167ee 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithFastPerformerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithFastPerformerTest.java
@@ -24,7 +24,8 @@ import org.apache.iotdb.db.engine.cache.BloomFilterCache;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
 import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate;
 import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionSelector;
 import org.apache.iotdb.db.engine.compaction.performer.impl.FastCompactionPerformer;
 import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
@@ -403,12 +404,12 @@ public class CrossSpaceCompactionWithFastPerformerTest {
           TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
           unseqTsFileResourceList.addAll(unseqResources);
           long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
-          CrossSpaceCompactionResource mergeResource =
-              new CrossSpaceCompactionResource(
+          CrossSpaceCompactionCandidate mergeResource =
+              new CrossSpaceCompactionCandidate(
                   seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
           RewriteCrossSpaceCompactionSelector selector =
               new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-          List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+          List<CrossCompactionTaskResource> selected =
               selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList);
           index++;
           if (selected.size() > 0) {
@@ -708,12 +709,12 @@ public class CrossSpaceCompactionWithFastPerformerTest {
           TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
           unseqTsFileResourceList.addAll(unseqResources);
           long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
-          CrossSpaceCompactionResource mergeResource =
-              new CrossSpaceCompactionResource(
+          CrossSpaceCompactionCandidate mergeResource =
+              new CrossSpaceCompactionCandidate(
                   seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
           RewriteCrossSpaceCompactionSelector selector =
               new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-          List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+          List<CrossCompactionTaskResource> selected =
               selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList);
           if (selected.size() > 0) {
             AbstractCompactionTask compactionTask =
@@ -1011,12 +1012,12 @@ public class CrossSpaceCompactionWithFastPerformerTest {
           TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
           unseqTsFileResourceList.addAll(unseqResources);
           long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
-          CrossSpaceCompactionResource mergeResource =
-              new CrossSpaceCompactionResource(
+          CrossSpaceCompactionCandidate mergeResource =
+              new CrossSpaceCompactionCandidate(
                   seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
           RewriteCrossSpaceCompactionSelector selector =
               new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-          List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+          List<CrossCompactionTaskResource> selected =
               selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList);
           if (selected.size() > 0) {
             AbstractCompactionTask compactionTask =
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java
index d08ee08719..c460be82bd 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java
@@ -23,7 +23,8 @@ import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
 import org.apache.iotdb.db.engine.compaction.CompactionUtils;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate;
 import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionSelector;
 import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
 import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
@@ -40,7 +41,6 @@ import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.db.tools.validate.TsFileValidationTool;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
-import org.apache.iotdb.tsfile.utils.Pair;
 
 import org.junit.After;
 import org.junit.Assert;
@@ -98,20 +98,20 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
 
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
-    Assert.assertEquals(1, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
+    Assert.assertEquals(1, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -137,26 +137,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(1, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(1, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -182,26 +182,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -233,28 +233,28 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(4, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(4, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
-    Assert.assertEquals(selected.get(0).right.get(2), unseqResources.get(2));
-    Assert.assertEquals(selected.get(0).right.get(3), unseqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(2), unseqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(3), unseqResources.get(3));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -282,29 +282,29 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(5));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(5));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -332,27 +332,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(1, selected.get(0).left.size());
-    Assert.assertEquals(3, selected.get(0).right.size());
-    for (TsFileResource selectedResource : (List<TsFileResource>) selected.get(0).left) {
+    Assert.assertEquals(1, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(3, selected.get(0).getUnseqFiles().size());
+    for (TsFileResource selectedResource : (List<TsFileResource>) selected.get(0).getSeqFiles()) {
       Assert.assertEquals(selectedResource, seqResources.get(1));
     }
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
-    Assert.assertEquals(selected.get(0).right.get(2), unseqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(2), unseqResources.get(2));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -381,29 +381,29 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(4, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(4, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
-    Assert.assertEquals(selected.get(0).right.get(2), unseqResources.get(2));
-    Assert.assertEquals(selected.get(0).right.get(3), unseqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(2), unseqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(3), unseqResources.get(3));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -431,29 +431,29 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(4, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(4, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
-    Assert.assertEquals(selected.get(0).right.get(2), unseqResources.get(2));
-    Assert.assertEquals(selected.get(0).right.get(3), unseqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(2), unseqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(3), unseqResources.get(3));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -484,26 +484,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -535,26 +535,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -587,25 +587,25 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -637,26 +637,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -689,26 +689,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -742,26 +742,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -794,27 +794,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -848,27 +848,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -903,27 +903,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -957,28 +957,28 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1012,28 +1012,28 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1067,28 +1067,28 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(6, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
-    Assert.assertEquals(selected.get(0).left.get(5), seqResources.get(7));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(6, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(5), seqResources.get(7));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1122,27 +1122,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(7));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(7));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1173,26 +1173,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1224,26 +1224,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1276,25 +1276,25 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1326,26 +1326,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1378,26 +1378,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1431,26 +1431,26 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1483,27 +1483,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1537,27 +1537,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1592,27 +1592,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1646,28 +1646,28 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1701,28 +1701,28 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1756,28 +1756,28 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(6, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
-    Assert.assertEquals(selected.get(0).left.get(5), seqResources.get(7));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(6, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(5), seqResources.get(7));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1811,27 +1811,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(7));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(7));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1866,24 +1866,24 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1920,18 +1920,18 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
     // Assert.assertEquals(0, result.length);
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1967,24 +1967,24 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -2020,25 +2020,25 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -2075,24 +2075,24 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -2143,7 +2143,7 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
             .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager);
     // In the process of getting the file list and starting to select files, the file list is
     // updated (the file is deleted or the status is updated)
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
     Assert.assertEquals(0, selected.size());
@@ -2219,25 +2219,27 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
             .getConfig()
             .getCrossCompactionSelector()
             .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager);
-    Pair<List<TsFileResource>, List<TsFileResource>> sourceFiles =
+    CrossCompactionTaskResource sourceFiles =
         crossSpaceCompactionSelector
             .selectCrossSpaceTask(
                 tsFileManager.getSequenceListByTimePartition(0),
                 tsFileManager.getUnsequenceListByTimePartition(0))
             .get(0);
-    Assert.assertEquals(2, sourceFiles.left.size());
-    Assert.assertEquals(1, sourceFiles.right.size());
+    Assert.assertEquals(2, sourceFiles.getSeqFiles().size());
+    Assert.assertEquals(1, sourceFiles.getUnseqFiles().size());
     List<TsFileResource> targetResources =
-        CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(sourceFiles.left);
-    performer.setSourceFiles(sourceFiles.left, sourceFiles.right);
+        CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(
+            sourceFiles.getSeqFiles());
+    performer.setSourceFiles(sourceFiles.getSeqFiles(), sourceFiles.getUnseqFiles());
     performer.setTargetFiles(targetResources);
     performer.setSummary(new CompactionTaskSummary());
     performer.perform();
 
     CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG + "-" + "0");
     CompactionUtils.combineModsInCrossCompaction(
-        sourceFiles.left, sourceFiles.right, targetResources);
-    tsFileManager.replace(sourceFiles.left, sourceFiles.right, targetResources, 0, true);
+        sourceFiles.getSeqFiles(), sourceFiles.getUnseqFiles(), targetResources);
+    tsFileManager.replace(
+        sourceFiles.getSeqFiles(), sourceFiles.getUnseqFiles(), targetResources, 0, true);
 
     // Suppose the read lock of the source file is occupied by other threads, causing the first task
     // to get stuck.
@@ -2265,16 +2267,19 @@ public class CrossSpaceCompactionWithFastPerformerValidationTest extends Abstrac
 
     // target file of first compaction task can be selected to participate in another cross
     // compaction task
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> pairs =
+    List<CrossCompactionTaskResource> pairs =
         crossSpaceCompactionSelector.selectCrossSpaceTask(
             tsFileManager.getSequenceListByTimePartition(0),
             tsFileManager.getUnsequenceListByTimePartition(0));
     Assert.assertEquals(1, pairs.size());
-    Assert.assertEquals(2, pairs.get(0).left.size());
-    Assert.assertEquals(1, pairs.get(0).right.size());
-    Assert.assertEquals(tsFileManager.getTsFileList(true).get(4), pairs.get(0).left.get(0));
-    Assert.assertEquals(tsFileManager.getTsFileList(true).get(5), pairs.get(0).left.get(1));
-    Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), pairs.get(0).right.get(0));
+    Assert.assertEquals(2, pairs.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, pairs.get(0).getUnseqFiles().size());
+    Assert.assertEquals(
+        tsFileManager.getTsFileList(true).get(4), pairs.get(0).getSeqFiles().get(0));
+    Assert.assertEquals(
+        tsFileManager.getTsFileList(true).get(5), pairs.get(0).getSeqFiles().get(1));
+    Assert.assertEquals(
+        tsFileManager.getTsFileList(false).get(0), pairs.get(0).getUnseqFiles().get(0));
 
     // target file of first compaction task can be selected to participate in another inner
     // compaction task
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithReadPointPerformerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithReadPointPerformerTest.java
index 594c312f6e..603ad2250e 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithReadPointPerformerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithReadPointPerformerTest.java
@@ -24,7 +24,8 @@ import org.apache.iotdb.db.engine.cache.BloomFilterCache;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
 import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate;
 import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionSelector;
 import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
 import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
@@ -403,12 +404,12 @@ public class CrossSpaceCompactionWithReadPointPerformerTest {
           TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
           unseqTsFileResourceList.addAll(unseqResources);
           long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
-          CrossSpaceCompactionResource mergeResource =
-              new CrossSpaceCompactionResource(
+          CrossSpaceCompactionCandidate mergeResource =
+              new CrossSpaceCompactionCandidate(
                   seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
           RewriteCrossSpaceCompactionSelector selector =
               new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-          List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+          List<CrossCompactionTaskResource> selected =
               selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList);
           index++;
           if (selected.size() > 0) {
@@ -708,12 +709,12 @@ public class CrossSpaceCompactionWithReadPointPerformerTest {
           TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
           unseqTsFileResourceList.addAll(unseqResources);
           long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
-          CrossSpaceCompactionResource mergeResource =
-              new CrossSpaceCompactionResource(
+          CrossSpaceCompactionCandidate mergeResource =
+              new CrossSpaceCompactionCandidate(
                   seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
           RewriteCrossSpaceCompactionSelector selector =
               new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-          List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+          List<CrossCompactionTaskResource> selected =
               selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList);
           if (selected.size() > 0) {
             AbstractCompactionTask compactionTask =
@@ -1011,12 +1012,12 @@ public class CrossSpaceCompactionWithReadPointPerformerTest {
           TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
           unseqTsFileResourceList.addAll(unseqResources);
           long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
-          CrossSpaceCompactionResource mergeResource =
-              new CrossSpaceCompactionResource(
+          CrossSpaceCompactionCandidate mergeResource =
+              new CrossSpaceCompactionCandidate(
                   seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
           RewriteCrossSpaceCompactionSelector selector =
               new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-          List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+          List<CrossCompactionTaskResource> selected =
               selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList);
           if (selected.size() > 0) {
             AbstractCompactionTask compactionTask =
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithReadPointPerformerValidationTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithReadPointPerformerValidationTest.java
index 0ab3b3f411..d4d15657e6 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithReadPointPerformerValidationTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionWithReadPointPerformerValidationTest.java
@@ -23,7 +23,8 @@ import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
 import org.apache.iotdb.db.engine.compaction.CompactionUtils;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate;
 import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionSelector;
 import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
 import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
@@ -39,7 +40,6 @@ import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.db.tools.validate.TsFileValidationTool;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
-import org.apache.iotdb.tsfile.utils.Pair;
 
 import org.junit.After;
 import org.junit.Assert;
@@ -96,20 +96,20 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
 
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
-    Assert.assertEquals(1, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
+    Assert.assertEquals(1, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -135,26 +135,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(1, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(1, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -180,26 +180,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -231,28 +231,28 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(4, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(4, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
-    Assert.assertEquals(selected.get(0).right.get(2), unseqResources.get(2));
-    Assert.assertEquals(selected.get(0).right.get(3), unseqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(2), unseqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(3), unseqResources.get(3));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -280,29 +280,29 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(5));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(5));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -330,27 +330,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(1, selected.get(0).left.size());
-    Assert.assertEquals(3, selected.get(0).right.size());
-    for (TsFileResource selectedResource : (List<TsFileResource>) selected.get(0).left) {
+    Assert.assertEquals(1, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(3, selected.get(0).getUnseqFiles().size());
+    for (TsFileResource selectedResource : (List<TsFileResource>) selected.get(0).getSeqFiles()) {
       Assert.assertEquals(selectedResource, seqResources.get(1));
     }
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
-    Assert.assertEquals(selected.get(0).right.get(2), unseqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(2), unseqResources.get(2));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -379,29 +379,29 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(4, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(4, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
-    Assert.assertEquals(selected.get(0).right.get(2), unseqResources.get(2));
-    Assert.assertEquals(selected.get(0).right.get(3), unseqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(2), unseqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(3), unseqResources.get(3));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -429,29 +429,29 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(4, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(4, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
-    Assert.assertEquals(selected.get(0).right.get(2), unseqResources.get(2));
-    Assert.assertEquals(selected.get(0).right.get(3), unseqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(2), unseqResources.get(2));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(3), unseqResources.get(3));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -482,26 +482,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -533,26 +533,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -585,25 +585,25 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -635,26 +635,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -687,26 +687,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -740,26 +740,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -792,27 +792,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -846,27 +846,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -901,27 +901,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -955,28 +955,28 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1010,28 +1010,28 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1065,28 +1065,28 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(6, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
-    Assert.assertEquals(selected.get(0).left.get(5), seqResources.get(7));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(6, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(5), seqResources.get(7));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1120,27 +1120,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(7));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(7));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1171,26 +1171,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1222,26 +1222,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1274,25 +1274,25 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1324,26 +1324,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1376,26 +1376,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1429,26 +1429,26 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1481,27 +1481,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1535,27 +1535,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1590,27 +1590,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(3, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
+    Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
 
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(1));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(1));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1644,28 +1644,28 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1699,28 +1699,28 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
 
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1754,28 +1754,28 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(6, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(6));
-    Assert.assertEquals(selected.get(0).left.get(5), seqResources.get(7));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(6, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(6));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(5), seqResources.get(7));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1809,27 +1809,27 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(5, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).left.get(2), seqResources.get(4));
-    Assert.assertEquals(selected.get(0).left.get(3), seqResources.get(5));
-    Assert.assertEquals(selected.get(0).left.get(4), seqResources.get(7));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(2), seqResources.get(4));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(3), seqResources.get(5));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(4), seqResources.get(7));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1864,24 +1864,24 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1918,18 +1918,18 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
     // Assert.assertEquals(0, result.length);
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -1965,24 +1965,24 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -2018,25 +2018,25 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(2, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
-    Assert.assertEquals(selected.get(0).right.get(1), unseqResources.get(1));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(1), unseqResources.get(1));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -2073,24 +2073,24 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
     tsFileManager.addAll(seqResources, true);
     tsFileManager.addAll(unseqResources, false);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
-    Assert.assertEquals(2, selected.get(0).left.size());
-    Assert.assertEquals(1, selected.get(0).right.size());
-    Assert.assertEquals(selected.get(0).left.get(0), seqResources.get(2));
-    Assert.assertEquals(selected.get(0).left.get(1), seqResources.get(3));
-    Assert.assertEquals(selected.get(0).right.get(0), unseqResources.get(0));
+    Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+    Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(0), seqResources.get(2));
+    Assert.assertEquals(selected.get(0).getSeqFiles().get(1), seqResources.get(3));
+    Assert.assertEquals(selected.get(0).getUnseqFiles().get(0), unseqResources.get(0));
 
     new CrossSpaceCompactionTask(
             0,
             tsFileManager,
-            selected.get(0).left,
-            selected.get(0).right,
+            selected.get(0).getSeqFiles(),
+            selected.get(0).getUnseqFiles(),
             performer,
             new AtomicInteger(0),
             0,
@@ -2139,7 +2139,7 @@ public class CrossSpaceCompactionWithReadPointPerformerValidationTest
             .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager);
     // In the process of getting the file list and starting to select files, the file list is
     // updated (the file is deleted or the status is updated)
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
 
     Assert.assertEquals(0, selected.size());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/MergeUpgradeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/MergeUpgradeTest.java
index 559abd7a12..8a4ea7bc48 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/MergeUpgradeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/MergeUpgradeTest.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.engine.compaction.cross;
 
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.db.constant.TestConstant;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
 import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionSelector;
 import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
 import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
@@ -33,7 +34,6 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.TsFileWriter;
 import org.apache.iotdb.tsfile.write.record.TSRecord;
 import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
@@ -85,7 +85,7 @@ public class MergeUpgradeTest {
     tsFileManager.addAll(unseqResources, true);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, tsFileManager);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
     assertEquals(0, selected.size());
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCompactionFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCompactionFileSelectorTest.java
index e6417aca11..6d58b093a2 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCompactionFileSelectorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCompactionFileSelectorTest.java
@@ -22,7 +22,8 @@ package org.apache.iotdb.db.engine.compaction.cross;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossCompactionTaskResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate;
 import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionSelector;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
@@ -31,7 +32,6 @@ import org.apache.iotdb.db.exception.MergeException;
 import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.TsFileWriter;
 import org.apache.iotdb.tsfile.write.record.TSRecord;
 import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
@@ -61,49 +61,49 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
   public void testFullSelection() throws MergeException, IOException {
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
-    List<TsFileResource> seqSelected = selected.get(0).left;
-    List<TsFileResource> unseqSelected = selected.get(0).right;
+    List<TsFileResource> seqSelected = selected.get(0).getSeqFiles();
+    List<TsFileResource> unseqSelected = selected.get(0).getUnseqFiles();
     assertEquals(seqResources, seqSelected);
     assertEquals(unseqResources, unseqSelected);
 
     selector = new RewriteCrossSpaceCompactionSelector("", "", 0, null);
     selected = selector.selectCrossSpaceTask(seqResources.subList(0, 1), unseqResources);
-    seqSelected = selected.get(0).left;
-    unseqSelected = selected.get(0).right;
+    seqSelected = selected.get(0).getSeqFiles();
+    unseqSelected = selected.get(0).getUnseqFiles();
     assertEquals(seqResources.subList(0, 1), seqSelected);
     assertEquals(unseqResources, unseqSelected);
 
     selector = new RewriteCrossSpaceCompactionSelector("", "", 0, null);
     selected = selector.selectCrossSpaceTask(seqResources, unseqResources.subList(0, 1));
-    seqSelected = selected.get(0).left;
-    unseqSelected = selected.get(0).right;
+    seqSelected = selected.get(0).getSeqFiles();
+    unseqSelected = selected.get(0).getUnseqFiles();
     assertEquals(seqResources.subList(0, 1), seqSelected);
     assertEquals(unseqResources.subList(0, 1), unseqSelected);
   }
 
   @Test
   public void testWithFewMemoryBudgeSelection() throws MergeException, IOException {
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
     assertEquals(1, selected.size());
   }
 
   @Test
   public void testRestrictedSelection() throws MergeException, IOException {
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
-    List<TsFileResource> seqSelected = selected.get(0).left;
-    List<TsFileResource> unseqSelected = selected.get(0).right;
+    List<TsFileResource> seqSelected = selected.get(0).getSeqFiles();
+    List<TsFileResource> unseqSelected = selected.get(0).getUnseqFiles();
     assertEquals(seqResources.subList(0, 5), seqSelected);
     assertEquals(unseqResources.subList(0, 6), unseqSelected);
   }
@@ -156,7 +156,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     newUnseqResources.add(largeUnseqTsFileResource);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, newUnseqResources);
     assertEquals(0, selected.size());
   }
@@ -207,8 +207,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     newUnseqResources.add(largeUnseqTsFileResource);
 
     long ttlLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
-    CrossSpaceCompactionResource mergeResource =
-        new CrossSpaceCompactionResource(seqResources, newUnseqResources, ttlLowerBound);
+    CrossSpaceCompactionCandidate mergeResource =
+        new CrossSpaceCompactionCandidate(seqResources, newUnseqResources, ttlLowerBound);
     assertEquals(5, mergeResource.getSeqFiles().size());
     assertEquals(1, mergeResource.getUnseqFiles().size());
   }
@@ -243,13 +243,13 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     unseqResources.clear();
     unseqResources.add(largeUnseqTsFileResource);
 
-    CrossSpaceCompactionResource resource =
-        new CrossSpaceCompactionResource(seqResources, unseqResources);
+    CrossSpaceCompactionCandidate resource =
+        new CrossSpaceCompactionCandidate(seqResources, unseqResources);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
-    assertEquals(2, selected.get(0).left.size());
+    assertEquals(2, selected.get(0).getSeqFiles().size());
   }
 
   @Test
@@ -299,7 +299,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
       prepareFile(unseqList.get(1), 0, 100, 20);
       prepareFile(unseqList.get(2), 99, 1, 30);
 
-      CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+      CrossSpaceCompactionCandidate resource =
+          new CrossSpaceCompactionCandidate(seqList, unseqList);
       // the budget is enough to select unseq0 and unseq2, but not unseq1
       // the first selection should only contain seq0 and unseq0
       long originMemoryBudget = SystemInfo.getInstance().getMemorySizeForCompaction();
@@ -309,12 +310,12 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
       try {
         RewriteCrossSpaceCompactionSelector selector =
             new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-        List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+        List<CrossCompactionTaskResource> selected =
             selector.selectCrossSpaceTask(seqList, unseqList);
-        assertEquals(1, selected.get(0).left.size());
-        assertEquals(1, selected.get(0).right.size());
-        assertEquals(seqList.get(0), selected.get(0).left.get(0));
-        assertEquals(unseqList.get(0), selected.get(0).right.get(0));
+        assertEquals(1, selected.get(0).getSeqFiles().size());
+        assertEquals(1, selected.get(0).getUnseqFiles().size());
+        assertEquals(seqList.get(0), selected.get(0).getSeqFiles().get(0));
+        assertEquals(unseqList.get(0), selected.get(0).getUnseqFiles().get(0));
 
         selected =
             selector.selectCrossSpaceTask(
@@ -379,7 +380,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
       unseqList.add(fileResource);
     }
 
-    CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+    CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqList, unseqList);
     Assert.assertEquals(5, resource.getSeqFiles().size());
     Assert.assertEquals(10, resource.getUnseqFiles().size());
     long origin = SystemInfo.getInstance().getMemorySizeForCompaction();
@@ -392,11 +393,11 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     try {
       RewriteCrossSpaceCompactionSelector selector =
           new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-      List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+      List<CrossCompactionTaskResource> selected =
           selector.selectCrossSpaceTask(seqList, unseqList);
       Assert.assertEquals(1, selected.size());
-      Assert.assertEquals(1, selected.get(0).left.size());
-      Assert.assertEquals(10, selected.get(0).right.size());
+      Assert.assertEquals(1, selected.get(0).getSeqFiles().size());
+      Assert.assertEquals(10, selected.get(0).getUnseqFiles().size());
     } finally {
       SystemInfo.getInstance().setMemorySizeForCompaction(origin);
     }
@@ -452,7 +453,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
       unseqList.add(fileResource);
     }
 
-    CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+    CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqList, unseqList);
     Assert.assertEquals(5, resource.getSeqFiles().size());
     Assert.assertEquals(1, resource.getUnseqFiles().size());
     long origin = SystemInfo.getInstance().getMemorySizeForCompaction();
@@ -465,11 +466,11 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     try {
       RewriteCrossSpaceCompactionSelector selector =
           new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-      List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+      List<CrossCompactionTaskResource> selected =
           selector.selectCrossSpaceTask(seqList, unseqList);
       Assert.assertEquals(1, selected.size());
-      Assert.assertEquals(1, selected.get(0).left.size());
-      Assert.assertEquals(1, selected.get(0).right.size());
+      Assert.assertEquals(1, selected.get(0).getSeqFiles().size());
+      Assert.assertEquals(1, selected.get(0).getUnseqFiles().size());
     } finally {
       SystemInfo.getInstance().setMemorySizeForCompaction(origin);
     }
@@ -527,7 +528,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     prepareFile(unseqList.get(0), 7, 3, 7);
     prepareFile(unseqList.get(1), 10, 4, 10);
 
-    CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+    CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqList, unseqList);
     Assert.assertEquals(5, resource.getSeqFiles().size());
     Assert.assertEquals(2, resource.getUnseqFiles().size());
     long origin = SystemInfo.getInstance().getMemorySizeForCompaction();
@@ -540,11 +541,11 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     try {
       RewriteCrossSpaceCompactionSelector selector =
           new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-      List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+      List<CrossCompactionTaskResource> selected =
           selector.selectCrossSpaceTask(seqList, unseqList);
       Assert.assertEquals(1, selected.size());
-      Assert.assertEquals(3, selected.get(0).left.size());
-      Assert.assertEquals(2, selected.get(0).right.size());
+      Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+      Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
     } finally {
       SystemInfo.getInstance().setMemorySizeForCompaction(origin);
     }
@@ -603,7 +604,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     prepareFile(unseqList.get(2), 14, 3, 14);
     prepareFile(unseqList.get(3), 17, 2, 17);
 
-    CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+    CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqList, unseqList);
     Assert.assertEquals(5, resource.getSeqFiles().size());
     Assert.assertEquals(4, resource.getUnseqFiles().size());
     long origin = SystemInfo.getInstance().getMemorySizeForCompaction();
@@ -616,11 +617,11 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     try {
       RewriteCrossSpaceCompactionSelector selector =
           new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-      List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+      List<CrossCompactionTaskResource> selected =
           selector.selectCrossSpaceTask(seqList, unseqList);
       Assert.assertEquals(1, selected.size());
-      Assert.assertEquals(5, selected.get(0).left.size());
-      Assert.assertEquals(4, selected.get(0).right.size());
+      Assert.assertEquals(5, selected.get(0).getSeqFiles().size());
+      Assert.assertEquals(4, selected.get(0).getUnseqFiles().size());
     } finally {
       SystemInfo.getInstance().setMemorySizeForCompaction(origin);
     }
@@ -682,7 +683,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     prepareFile(unseqList.get(2), 14, 3, 14);
     prepareFile(unseqList.get(3), 17, 2, 17);
 
-    CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+    CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqList, unseqList);
     Assert.assertEquals(5, resource.getSeqFiles().size());
     Assert.assertEquals(4, resource.getUnseqFiles().size());
     long origin = SystemInfo.getInstance().getMemorySizeForCompaction();
@@ -695,11 +696,11 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     try {
       RewriteCrossSpaceCompactionSelector selector =
           new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-      List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+      List<CrossCompactionTaskResource> selected =
           selector.selectCrossSpaceTask(seqList, unseqList);
       Assert.assertEquals(1, selected.size());
-      Assert.assertEquals(3, selected.get(0).left.size());
-      Assert.assertEquals(2, selected.get(0).right.size());
+      Assert.assertEquals(3, selected.get(0).getSeqFiles().size());
+      Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
     } finally {
       SystemInfo.getInstance().setMemorySizeForCompaction(origin);
     }
@@ -927,11 +928,11 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     try {
       RewriteCrossSpaceCompactionSelector selector =
           new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-      List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+      List<CrossCompactionTaskResource> selected =
           selector.selectCrossSpaceTask(seqList, unseqList);
       Assert.assertEquals(1, selected.size());
-      Assert.assertEquals(2, selected.get(0).left.size());
-      Assert.assertEquals(2, selected.get(0).right.size());
+      Assert.assertEquals(2, selected.get(0).getSeqFiles().size());
+      Assert.assertEquals(2, selected.get(0).getUnseqFiles().size());
     } finally {
       SystemInfo.getInstance().setMemorySizeForCompaction(origin);
     }
@@ -944,11 +945,11 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
     IoTDBDescriptor.getInstance().getConfig().setMaxCrossCompactionCandidateFileNum(5);
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
     assertEquals(1, selected.size());
-    List<TsFileResource> seqSelected = selected.get(0).left;
-    List<TsFileResource> unseqSelected = selected.get(0).right;
+    List<TsFileResource> seqSelected = selected.get(0).getSeqFiles();
+    List<TsFileResource> unseqSelected = selected.get(0).getUnseqFiles();
     assertEquals(2, seqSelected.size());
     assertEquals(2, unseqSelected.size());
 
@@ -965,11 +966,11 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
 
     RewriteCrossSpaceCompactionSelector selector =
         new RewriteCrossSpaceCompactionSelector("", "", 0, null);
-    List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+    List<CrossCompactionTaskResource> selected =
         selector.selectCrossSpaceTask(seqResources, unseqResources);
     assertEquals(1, selected.size());
-    List<TsFileResource> seqSelected = selected.get(0).left;
-    List<TsFileResource> unseqSelected = selected.get(0).right;
+    List<TsFileResource> seqSelected = selected.get(0).getSeqFiles();
+    List<TsFileResource> unseqSelected = selected.get(0).getUnseqFiles();
     assertEquals(1, seqSelected.size());
     assertEquals(1, unseqSelected.size());
 
@@ -987,7 +988,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
         new Thread(
             () -> {
               try {
-                List<Pair<List<TsFileResource>, List<TsFileResource>>> selected =
+                List<CrossCompactionTaskResource> selected =
                     selector.selectCrossSpaceTask(seqResources, unseqResources);
               } catch (Exception e) {
                 logger.error("Exception occurs", e);