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

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

commit 779ee03e0a7cc8a7365e7dd4cc1bf1c315d9ce41
Author: Jinrui.Zhang <[email protected]>
AuthorDate: Wed Dec 21 23:44:08 2022 +0800

    complete structure of cross selection
---
 .../db/engine/compaction/CompactionScheduler.java  |   8 +-
 .../compaction/cross/ICrossSpaceSelector.java      |   4 +-
 .../cross/rewrite/CrossCompactionTaskResource.java |  98 +++++++
 .../rewrite/CrossSpaceCompactionCandidate.java     | 195 +++++++++++++
 .../rewrite/CrossSpaceCompactionResource.java      |  74 -----
 .../RewriteCrossSpaceCompactionSelector.java       | 314 +++++----------------
 .../compaction/task/ICompactionSelector.java       |   4 +-
 .../CrossSpaceCompactionWithFastPerformerTest.java |  14 +-
 ...eCompactionWithFastPerformerValidationTest.java | 154 +++++-----
 ...sSpaceCompactionWithReadPointPerformerTest.java |  14 +-
 ...actionWithReadPointPerformerValidationTest.java | 154 +++++-----
 .../cross/RewriteCompactionFileSelectorTest.java   |  30 +-
 12 files changed, 550 insertions(+), 513 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..3664ca6492
--- /dev/null
+++ 
b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossCompactionTaskResource.java
@@ -0,0 +1,98 @@
+/*
+ * 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.List;
+
+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;
+  }
+
+  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..285de9430a
--- /dev/null
+++ 
b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/CrossSpaceCompactionCandidate.java
@@ -0,0 +1,195 @@
+/*
+ * 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;
+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() {
+    if (nextUnseqFileIndex >= unseqFiles.size()) {
+      return false;
+    }
+    return prepareNextSplit();
+  }
+
+  public CrossCompactionTaskResourceSplit nextSplit() {
+    return nextSplit;
+  }
+
+  private boolean prepareNextSplit() {
+    TsFileResourceCandidate unseqFile = unseqFiles.get(nextUnseqFileIndex);
+    List<TsFileResourceCandidate> ret = new ArrayList<>();
+    boolean lastTargetSeqFileSelected = false;
+    for (TsFileResourceCandidate seqFile : seqFiles) {
+      if (lastTargetSeqFileSelected) {
+        break;
+      }
+      for (DeviceInfo unseqDeviceInfo : unseqFile.getDevices()) {
+        if (!seqFile.containsDevice(unseqDeviceInfo.deviceId)) {
+          continue;
+        }
+        DeviceInfo seqDeviceInfo = 
seqFile.getDeviceInfoById(unseqDeviceInfo.deviceId);
+        if (!seqFile.isValidCandidate) {
+          // If the unclosed seqFile should be selected, the whole selection 
should be terminated
+          if (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);
+          }
+          // prepare
+          lastTargetSeqFileSelected = true;
+          break;
+        } else if (unseqDeviceInfo.startTime <= seqDeviceInfo.endTime) {
+          if (!seqFile.selected) {
+            ret.add(seqFile);
+          }
+          break;
+        }
+      }
+    }
+    // mark candidates in next split as selected even though it may not be 
added to the final TaskResource
+    unseqFile.markAsSelected();
+    for (TsFileResourceCandidate fileCandidate : ret) {
+      fileCandidate.markAsSelected();
+    }
+    nextSplit = new CrossCompactionTaskResourceSplit(unseqFile, ret);
+    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;
+    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.isClosed() && 
tsFileResource.getTsFile().exists();
+    }
+
+    protected void markAsSelected() {
+      this.selected = true;
+    }
+
+    protected List<DeviceInfo> getDevices() {
+      return null;
+    }
+
+    protected DeviceInfo getDeviceInfoById(String deviceId) {
+      return null;
+    }
+
+    protected boolean containsDevice(String deviceId) {
+      return false;
+    }
+  }
+
+  protected static class DeviceInfo {
+    protected String deviceId;
+    protected long startTime;
+    protected long endTime;
+  }
+}
\ No newline at end of file
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..bf55a199dc 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
@@ -28,48 +28,35 @@ 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.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate.TsFileResourceCandidate;
+import 
org.apache.iotdb.db.engine.compaction.cross.rewrite.CrossSpaceCompactionCandidate.CrossCompactionTaskResourceSplit;
 
 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 +108,23 @@ 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 +134,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 +153,40 @@ 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()) {
+  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;
       }
-
-      if (seqSelectedNum != resource.getSeqFiles().size()) {
-        selectOverlappedSeqFiles(unseqFile);
-      }
-      boolean isSeqFilesValid = checkIsSeqFilesValid();
-      if (!isSeqFilesValid) {
-        tmpSelectedSeqFiles.clear();
-        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));
-      }
+      taskResource.putResources(unseqFile, targetSeqFiles, memoryCost);
+      LOGGER.debug("Adding a new unseqFile {} and seqFiles {} as candidates, 
new cost {}, total cost {}", unseqFile, targetSeqFiles, memoryCost, totalCost);
     }
+    return taskResource;
   }
 
-  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;
-      LOGGER.debug(
-          "Adding a new unseqFile {} and seqFiles {} as candidates, new cost 
{}, total"
-              + " cost {}",
-          unseqFile,
-          tmpSelectedSeqFiles,
-          newCost,
-          totalCost);
+  // 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) {
+    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;
   }
 
-  /**
-   * 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;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * 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 +198,34 @@ 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(
+      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);
-          hasPrintLog = true;
-        }
         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/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..8de6fd3118 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,7 @@ 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.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,8 +403,8 @@ 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);
@@ -708,8 +708,8 @@ 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);
@@ -1011,8 +1011,8 @@ 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);
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 4a939da189..d1171d86e4 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,7 @@ 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.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;
@@ -136,8 +136,8 @@ 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 =
@@ -181,8 +181,8 @@ 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 =
@@ -232,8 +232,8 @@ 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 =
@@ -281,8 +281,8 @@ 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 =
@@ -331,8 +331,8 @@ 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 =
@@ -380,8 +380,8 @@ 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 =
@@ -430,8 +430,8 @@ 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 =
@@ -483,8 +483,8 @@ 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 =
@@ -534,8 +534,8 @@ 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 =
@@ -586,8 +586,8 @@ 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 =
@@ -636,8 +636,8 @@ 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 =
@@ -688,8 +688,8 @@ 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 =
@@ -741,8 +741,8 @@ 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 =
@@ -793,8 +793,8 @@ 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 =
@@ -847,8 +847,8 @@ 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 =
@@ -902,8 +902,8 @@ 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 =
@@ -956,8 +956,8 @@ 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 =
@@ -1011,8 +1011,8 @@ 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 =
@@ -1066,8 +1066,8 @@ 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 =
@@ -1121,8 +1121,8 @@ 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 =
@@ -1172,8 +1172,8 @@ 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 =
@@ -1223,8 +1223,8 @@ 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 =
@@ -1275,8 +1275,8 @@ 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 =
@@ -1325,8 +1325,8 @@ 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 =
@@ -1377,8 +1377,8 @@ 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 =
@@ -1430,8 +1430,8 @@ 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 =
@@ -1482,8 +1482,8 @@ 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 =
@@ -1536,8 +1536,8 @@ 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 =
@@ -1591,8 +1591,8 @@ 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 =
@@ -1645,8 +1645,8 @@ 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 =
@@ -1700,8 +1700,8 @@ 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 =
@@ -1755,8 +1755,8 @@ 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 =
@@ -1810,8 +1810,8 @@ 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 =
@@ -1865,8 +1865,8 @@ 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 =
@@ -1919,8 +1919,8 @@ 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 =
@@ -1966,8 +1966,8 @@ 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 =
@@ -2019,8 +2019,8 @@ 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 =
@@ -2074,8 +2074,8 @@ 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 =
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..846484f7d9 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,7 @@ 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.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,8 +403,8 @@ 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);
@@ -708,8 +708,8 @@ 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);
@@ -1011,8 +1011,8 @@ 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);
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..93ed0ddbe3 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,7 @@ 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.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;
@@ -135,8 +135,8 @@ 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 =
@@ -180,8 +180,8 @@ 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 =
@@ -231,8 +231,8 @@ 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 =
@@ -280,8 +280,8 @@ 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 =
@@ -330,8 +330,8 @@ 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 =
@@ -379,8 +379,8 @@ 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 =
@@ -429,8 +429,8 @@ 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 =
@@ -482,8 +482,8 @@ 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 =
@@ -533,8 +533,8 @@ 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 =
@@ -585,8 +585,8 @@ 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 =
@@ -635,8 +635,8 @@ 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 =
@@ -687,8 +687,8 @@ 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 =
@@ -740,8 +740,8 @@ 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 =
@@ -792,8 +792,8 @@ 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 =
@@ -846,8 +846,8 @@ 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 =
@@ -901,8 +901,8 @@ 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 =
@@ -955,8 +955,8 @@ 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 =
@@ -1010,8 +1010,8 @@ 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 =
@@ -1065,8 +1065,8 @@ 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 =
@@ -1120,8 +1120,8 @@ 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 =
@@ -1171,8 +1171,8 @@ 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 =
@@ -1222,8 +1222,8 @@ 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 =
@@ -1274,8 +1274,8 @@ 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 =
@@ -1324,8 +1324,8 @@ 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 =
@@ -1376,8 +1376,8 @@ 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 =
@@ -1429,8 +1429,8 @@ 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 =
@@ -1481,8 +1481,8 @@ 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 =
@@ -1535,8 +1535,8 @@ 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 =
@@ -1590,8 +1590,8 @@ 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 =
@@ -1644,8 +1644,8 @@ 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 =
@@ -1699,8 +1699,8 @@ 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 =
@@ -1754,8 +1754,8 @@ 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 =
@@ -1809,8 +1809,8 @@ 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 =
@@ -1864,8 +1864,8 @@ 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 =
@@ -1918,8 +1918,8 @@ 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 =
@@ -1965,8 +1965,8 @@ 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 =
@@ -2018,8 +2018,8 @@ 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 =
@@ -2073,8 +2073,8 @@ 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 =
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..c8683bfa53 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,7 @@ 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.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;
@@ -85,8 +85,8 @@ public class RewriteCompactionFileSelectorTest extends 
MergeTest {
 
   @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 =
@@ -96,8 +96,8 @@ public class RewriteCompactionFileSelectorTest extends 
MergeTest {
 
   @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 =
@@ -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,8 +243,8 @@ 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 =
@@ -299,7 +299,7 @@ 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();
@@ -379,7 +379,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();
@@ -452,7 +452,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();
@@ -527,7 +527,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();
@@ -603,7 +603,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();
@@ -682,7 +682,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();

Reply via email to