You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2022/10/15 09:57:30 UTC

[GitHub] [iotdb] choubenson opened a new pull request, #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

choubenson opened a new pull request, #7621:
URL: https://github.com/apache/iotdb/pull/7621

   New Compaction Performer, which speeds up the compaction, including aligned and nonAligned timeseries.


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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1030072395


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/SeriesCompactionExecutor.java:
##########
@@ -0,0 +1,601 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+public abstract class SeriesCompactionExecutor {
+  protected enum ModifiedStatus {
+    ALL_DELETED,
+    PARTIAL_DELETED,
+    NONE_DELETED;
+  }
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  private final SubCompactionTaskSummary summary;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<FileElement> fileList = new ArrayList<>();;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected AbstractCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  protected String deviceId;
+
+  // Pages in this list will be sequentially judged whether there is a real overlap to choose
+  // whether to put them in the point priority reader to deserialize or directly flush to chunk
+  // writer. During the process of compacting overlapped page, there may be new overlapped pages
+  // added into this list.
+  private final List<PageElement> candidateOverlappedPages = new ArrayList<>();
+
+  public SeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.summary = summary;
+
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+  }
+
+  public abstract void excute()
+      throws PageException, IllegalPathException, IOException, WriteProcessException;
+
+  protected abstract void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException;
+
+  /** Compact chunks in chunk metadata queue. */
+  protected void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1005166751


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/FastCompactionPerformer.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.performer.impl;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.AlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.NonAlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+public class FastCompactionPerformer implements ICrossCompactionPerformer {
+  private final Logger LOGGER = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+  private List<TsFileResource> seqFiles;
+
+  private List<TsFileResource> unseqFiles;
+
+  private List<TsFileResource> sortedSourceFiles = new ArrayList<>();
+
+  private static final int subTaskNum =
+      IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
+
+  public Map<TsFileResource, TsFileSequenceReader> readerCacheMap = new ConcurrentHashMap<>();
+
+  private CompactionTaskSummary summary;
+
+  private List<TsFileResource> targetFiles;
+
+  public Map<TsFileResource, List<Modification>> modificationCache = new ConcurrentHashMap<>();
+
+  public FastCompactionPerformer(
+      List<TsFileResource> seqFiles,
+      List<TsFileResource> unseqFiles,
+      List<TsFileResource> targetFiles) {
+    this.seqFiles = seqFiles;
+    this.unseqFiles = unseqFiles;
+    this.targetFiles = targetFiles;
+  }
+
+  public FastCompactionPerformer() {}
+
+  @Override
+  public void perform()
+      throws IOException, MetadataException, StorageEngineException, InterruptedException {
+    try (MultiTsFileDeviceIterator deviceIterator =
+            new MultiTsFileDeviceIterator(seqFiles, unseqFiles, readerCacheMap);
+        FastCrossCompactionWriter compactionWriter =
+            new FastCrossCompactionWriter(targetFiles, seqFiles)) {
+      while (deviceIterator.hasNextDevice()) {
+        checkThreadInterrupted();
+        Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
+        String device = deviceInfo.left;
+        boolean isAligned = deviceInfo.right;
+
+        // sort the resources by the start time of current device from old to new, and remove
+        // resource that does not contain the current device. Notice: when the level of time index
+        // is file, there will be a false positive judgment problem, that is, the device does not
+        // actually exist but the judgment return device being existed.
+        sortedSourceFiles.addAll(seqFiles);
+        sortedSourceFiles.addAll(unseqFiles);
+        sortedSourceFiles.removeIf(x -> !x.mayContainsDevice(device));
+        sortedSourceFiles.sort(Comparator.comparingLong(x -> x.getStartTime(device)));
+
+        compactionWriter.startChunkGroup(device, isAligned);
+
+        if (isAligned) {
+          compactAlignedSeries(device, deviceIterator, compactionWriter);
+        } else {
+          compactNonAlignedSeries(device, deviceIterator, compactionWriter);
+        }
+
+        compactionWriter.endChunkGroup();
+        // update resource of the current device and check whether to flush chunk metadata or not
+        compactionWriter.checkAndMayFlushChunkMetadata();
+        sortedSourceFiles.clear();
+      }
+      compactionWriter.endFile();
+      CompactionUtils.updatePlanIndexes(targetFiles, seqFiles, unseqFiles);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    } finally {
+      // readers of source files have been closed in MultiTsFileDeviceIterator
+      // clean cache
+      sortedSourceFiles = null;
+      readerCacheMap = null;
+      modificationCache = null;
+    }
+  }
+
+  private void compactAlignedSeries(
+      String deviceId,
+      MultiTsFileDeviceIterator deviceIterator,
+      FastCrossCompactionWriter fastCrossCompactionWriter)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+    Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap =
+        new HashMap<>();
+    List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
+
+    // Get all value measurements and their schemas of the current device. Also get start offset and
+    // end offset of each timeseries metadata, in order to facilitate the reading of chunkMetadata
+    // directly by this offset later. Instead of deserializing chunk metadata later, we need to
+    // deserialize chunk metadata here to get the schemas of all value measurements, because we
+    // should get schemas of all value measurement to startMeasruement() and compaction process is
+    // to read a batch of overlapped files each time, and we cannot make sure if the first batch of
+    // overlapped tsfiles contain all the value measurements.
+    for (Map.Entry<String, Pair<MeasurementSchema, Map<TsFileResource, Pair<Long, Long>>>> entry :
+        deviceIterator.getTimeseriesSchemaAndMetadataOffsetOfCurrentDevice().entrySet()) {
+      if (!entry.getKey().equals("")) {
+        measurementSchemas.add(entry.getValue().left);
+      }
+      timeseriesMetadataOffsetMap.put(entry.getKey(), entry.getValue().right);
+    }
+
+    new AlignedFastCompactionPerformerSubTask(
+            fastCrossCompactionWriter,
+            timeseriesMetadataOffsetMap,
+            measurementSchemas,
+            readerCacheMap,
+            modificationCache,
+            sortedSourceFiles,
+            deviceId,
+            0)
+        .call();
+  }
+
+  private void compactNonAlignedSeries(
+      String deviceID,
+      MultiTsFileDeviceIterator deviceIterator,
+      FastCrossCompactionWriter fastCrossCompactionWriter)
+      throws IOException, InterruptedException {
+    // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+    // Get all measurements of the current device. Also get start offset and end offset of each
+    // timeseries metadata, in order to facilitate the reading of chunkMetadata directly by this
+    // offset later. Here we don't need to deserialize chunk metadata, we can deserialize them and
+    // get their schema later.
+    Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap =
+        deviceIterator.getTimeseriesMetadataOffsetOfCurrentDevice();
+
+    List<String> allMeasurements = new ArrayList<>(timeseriesMetadataOffsetMap.keySet());
+
+    int subTaskNums = Math.min(allMeasurements.size(), subTaskNum);
+
+    // assign all measurements to different sub tasks
+    List<String>[] measurementsForEachSubTask = new ArrayList[subTaskNums];

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004104669


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/PointElement.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+
+public class PointElement {
+  public long timestamp;
+  public int priority;
+  public Pair<Long, Object> timeValuePair;

Review Comment:
   Resolved. Change `IBatchDataIterator` to `IPointReader`.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1005211606


##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -433,7 +433,7 @@ public class IoTDBConfig {
    */
   private CrossCompactionSelector crossCompactionSelector = CrossCompactionSelector.REWRITE;

Review Comment:
   Resolved. They use the same selector currently.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004086212


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {
+        // write data point to chunk writer
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+        if (overlappedPages.size() > 0) {
+          // finish compacting the first page or there are new chunks being deserialized and find
+          // the new overlapped pages, then start compacting them
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(ChunkMetadataElement chunkMetadata) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadata.chunkMetadata.getEndTime();
+    for (ChunkMetadataElement element : chunkMetadataQueue) {
+      if (element.chunkMetadata.getStartTime() <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped files which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the current device in the file from small to
+   * large, so that each file can be compacted in order.
+   */
+  private List<FileElement> findOverlapFiles(FileElement file) {
+    List<FileElement> overlappedFiles = new ArrayList<>();
+    long endTime = file.resource.getEndTime(deviceId);
+    for (FileElement fileElement : fileList) {
+      if (fileElement.resource.getStartTime(deviceId) <= endTime) {
+        if (!fileElement.isOverlap) {
+          overlappedFiles.add(fileElement);
+          fileElement.isOverlap = true;
+        }
+      } else {
+        break;
+      }
+    }
+    return overlappedFiles;
+  }
+
+  /** Check is the page overlap with other pages later then the specific page in queue or not. */
+  private boolean isPageOverlap(PageElement pageElement) {
+    long endTime = pageElement.pageHeader.getEndTime();
+    long startTime = pageElement.startTime;
+    for (PageElement element : pageQueue) {
+      if (element.equals(pageElement)) {
+        continue;
+      }
+      // only check pages later than the specific page
+      if (element.startTime >= startTime && element.startTime <= endTime) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check whether the chunk is modified.
+   *
+   * <p>Notice: if is aligned chunk, return true if any of value chunk has data been deleted. Return
+   * false if and only if all value chunks has no data been deleted.
+   */
+  protected boolean isChunkModified(ChunkMetadataElement chunkMetadataElement) {
+    return chunkMetadataElement.chunkMetadata.isModified();
+  }
+
+  /**
+   * -1 means that no data on this page has been deleted. <br>
+   * 0 means that there is data on this page been deleted. <br>
+   * 1 means that all data on this page has been deleted.
+   *
+   * <p>Notice: If is aligned page, return 1 if and only if all value pages are deleted. Return -1
+   * if and only if no data exists on all value pages is deleted
+   */
+  protected abstract int isPageModified(PageElement pageElement);
+
+  protected int checkIsModified(long startTime, long endTime, Collection<TimeRange> deletions) {

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1005236236


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1005170286


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/FastCrossCompactionWriter.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.writer;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumn;
+import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.ValueChunkWriter;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class FastCrossCompactionWriter extends AbstractCrossCompactionWriter {
+
+  public FastCrossCompactionWriter(
+      List<TsFileResource> targetResources, List<TsFileResource> seqSourceResources)
+      throws IOException {
+    super(targetResources, seqSourceResources);
+  }
+
+  @Override
+  public void write(TimeColumn timestamps, Column[] columns, int subTaskId, int batchSize)
+      throws IOException {}
+
+  /**
+   * Flush chunk to tsfile directly. Return whether the chunk is flushed to tsfile successfully or
+   * not. Return false if the unsealed chunk is too small or the end time of chunk exceeds the end
+   * time of file, else return true. Notice: if sub-value measurement is null, then flush empty
+   * value chunk.
+   */
+  public boolean flushChunkToFileWriter(
+      IChunkMetadata iChunkMetadata, TsFileSequenceReader reader, int subTaskId)
+      throws IOException {
+    checkTimeAndMayFlushChunkToCurrentFile(iChunkMetadata.getStartTime(), subTaskId);
+    int fileIndex = seqFileIndexArray[subTaskId];
+    boolean isUnsealedChunkLargeEnough =
+        chunkWriters[subTaskId].checkIsChunkSizeOverThreshold(
+            chunkSizeLowerBoundInCompaction, chunkPointNumLowerBoundInCompaction);
+    if (!isUnsealedChunkLargeEnough
+        || (iChunkMetadata.getEndTime() > currentDeviceEndTime[fileIndex]
+            && fileIndex != targetFileWriters.size() - 1)) {
+      // if unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize the
+      // chunk
+      return false;
+    }
+
+    TsFileIOWriter tsFileIOWriter = targetFileWriters.get(fileIndex);
+
+    synchronized (tsFileIOWriter) {
+      // seal last chunk to file writer
+      chunkWriters[subTaskId].writeToFileWriter(tsFileIOWriter);
+      if (iChunkMetadata instanceof AlignedChunkMetadata) {
+        AlignedChunkMetadata alignedChunkMetadata = (AlignedChunkMetadata) iChunkMetadata;
+        // flush time chunk
+        ChunkMetadata chunkMetadata = (ChunkMetadata) alignedChunkMetadata.getTimeChunkMetadata();
+        tsFileIOWriter.writeChunk(reader.readMemChunk(chunkMetadata), chunkMetadata);
+        // flush value chunks
+        for (int i = 0; i < alignedChunkMetadata.getValueChunkMetadataList().size(); i++) {
+          IChunkMetadata valueChunkMetadata =
+              alignedChunkMetadata.getValueChunkMetadataList().get(i);
+          if (valueChunkMetadata == null) {
+            // sub sensor does not exist in current file or value chunk has been deleted completely
+            AlignedChunkWriterImpl alignedChunkWriter =
+                (AlignedChunkWriterImpl) chunkWriters[subTaskId];
+            ValueChunkWriter valueChunkWriter = alignedChunkWriter.getValueChunkWriterByIndex(i);
+            tsFileIOWriter.writeEmptyValueChunk(
+                valueChunkWriter.getMeasurementId(),
+                valueChunkWriter.getCompressionType(),
+                valueChunkWriter.getDataType(),
+                valueChunkWriter.getEncodingType(),
+                valueChunkWriter.getStatistics());
+            continue;
+          }
+          chunkMetadata = (ChunkMetadata) valueChunkMetadata;
+          tsFileIOWriter.writeChunk(reader.readMemChunk(chunkMetadata), chunkMetadata);
+        }
+      } else {
+        ChunkMetadata chunkMetadata = (ChunkMetadata) iChunkMetadata;
+        tsFileIOWriter.writeChunk(reader.readMemChunk(chunkMetadata), chunkMetadata);
+      }
+    }
+    isDeviceExistedInTargetFiles[fileIndex] = true;
+    isEmptyFile[fileIndex] = false;
+    return true;
+  }
+
+  /**
+   * Flush aligned page to tsfile directly. Return whether the page is flushed to tsfile
+   * successfully or not. Return false if the unsealed page is too small or the end time of page
+   * exceeds the end time of file, else return true. Notice: if sub-value measurement is null, then
+   * flush empty value page.
+   */
+  public boolean flushAlignedPageToChunkWriter(
+      ByteBuffer compressedTimePageData,
+      PageHeader timePageHeader,
+      List<ByteBuffer> compressedValuePageDatas,
+      List<PageHeader> valuePageHeaders,
+      int subTaskId)
+      throws IOException, PageException {
+    checkTimeAndMayFlushChunkToCurrentFile(timePageHeader.getStartTime(), subTaskId);

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004116956


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {

Review Comment:
   Resolved. It has been renamed to `candidateOverlappedPages` and put into `FastCompactionPerformerSubTask` class as a global variable. Pages in this list will be sequentially judged whether there is a real overlap to choose whether to put them in the point priority reader to deserialize or directly flush to chunk writer. During the process of compacting overlapped page, there may be new overlapped pages added into this list.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1030219217


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/NonAlignedSeriesCompactionExecutor.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedSeriesCompactionExecutor extends SeriesCompactionExecutor {
+  private boolean hasStartMeasurement = false;
+
+  // tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  private Map<TsFileResource, Pair<Long, Long>> timeseriesMetadataOffsetMap;
+
+  // it is used to initialize the fileList when compacting a new series
+  private final List<TsFileResource> sortResources;
+
+  public NonAlignedSeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    super(compactionWriter, readerCacheMap, modificationCacheMap, deviceId, subTaskId, summary);
+    this.sortResources = sortedSourceFiles;
+  }
+
+  @Override
+  public void excute()
+      throws PageException, IllegalPathException, IOException, WriteProcessException {
+    compactFiles();
+    if (hasStartMeasurement) {

Review Comment:
   rename to `setNewMeasurement`.



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

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

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


[GitHub] [iotdb] THUMarkLau commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
THUMarkLau commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1028967700


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/NonAlignedSeriesCompactionExecutor.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedSeriesCompactionExecutor extends SeriesCompactionExecutor {
+  private boolean hasStartMeasurement = false;
+
+  // tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  private Map<TsFileResource, Pair<Long, Long>> timeseriesMetadataOffsetMap;
+
+  // it is used to initialize the fileList when compacting a new series
+  private final List<TsFileResource> sortResources;
+
+  public NonAlignedSeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    super(compactionWriter, readerCacheMap, modificationCacheMap, deviceId, subTaskId, summary);
+    this.sortResources = sortedSourceFiles;
+  }
+
+  @Override
+  public void excute()

Review Comment:
   ```suggestion
     public void execute()
   ```



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/NonAlignedSeriesCompactionExecutor.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedSeriesCompactionExecutor extends SeriesCompactionExecutor {
+  private boolean hasStartMeasurement = false;
+
+  // tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  private Map<TsFileResource, Pair<Long, Long>> timeseriesMetadataOffsetMap;
+
+  // it is used to initialize the fileList when compacting a new series
+  private final List<TsFileResource> sortResources;
+
+  public NonAlignedSeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    super(compactionWriter, readerCacheMap, modificationCacheMap, deviceId, subTaskId, summary);
+    this.sortResources = sortedSourceFiles;
+  }
+
+  @Override
+  public void excute()
+      throws PageException, IllegalPathException, IOException, WriteProcessException {
+    compactFiles();
+    if (hasStartMeasurement) {

Review Comment:
   As `startNewtMeasurement` is an individual method, there should be a method called `endMeasurement` too.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/SeriesCompactionExecutor.java:
##########
@@ -0,0 +1,601 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+public abstract class SeriesCompactionExecutor {
+  protected enum ModifiedStatus {
+    ALL_DELETED,
+    PARTIAL_DELETED,
+    NONE_DELETED;
+  }
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  private final SubCompactionTaskSummary summary;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<FileElement> fileList = new ArrayList<>();;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected AbstractCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  protected String deviceId;
+
+  // Pages in this list will be sequentially judged whether there is a real overlap to choose
+  // whether to put them in the point priority reader to deserialize or directly flush to chunk
+  // writer. During the process of compacting overlapped page, there may be new overlapped pages
+  // added into this list.
+  private final List<PageElement> candidateOverlappedPages = new ArrayList<>();
+
+  public SeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.summary = summary;
+
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+  }
+
+  public abstract void excute()
+      throws PageException, IllegalPathException, IOException, WriteProcessException;
+
+  protected abstract void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException;
+
+  /** Compact chunks in chunk metadata queue. */
+  protected void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        summary.CHUNK_OVERLAP_OR_MODIFIED += overlappedChunkMetadatas.size();
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        summary.CHUNK_NONE_OVERLAP += 1;
+        compactWithNonOverlapChunk(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      readChunk(overlappedChunkMetadata);
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunk(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    readChunk(chunkMetadataElement);
+    boolean success;
+    if (chunkMetadataElement.chunkMetadata instanceof AlignedChunkMetadata) {
+      success =
+          compactionWriter.flushAlignedChunk(
+              chunkMetadataElement.chunk,
+              ((AlignedChunkMetadata) chunkMetadataElement.chunkMetadata).getTimeChunkMetadata(),
+              chunkMetadataElement.valueChunks,
+              ((AlignedChunkMetadata) chunkMetadataElement.chunkMetadata)
+                  .getValueChunkMetadataList(),
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushNonAlignedChunk(
+              chunkMetadataElement.chunk,
+              (ChunkMetadata) chunkMetadataElement.chunkMetadata,
+              subTaskId);
+    }
+    if (success) {
+      // flush chunk successfully, then remove this chunk
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      summary.CHUNK_NONE_OVERLAP_BUT_DESERIALIZE += 1;
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  abstract void readChunk(ChunkMetadataElement chunkMetadataElement) throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      ModifiedStatus modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == ModifiedStatus.ALL_DELETED) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlapPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlapPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == ModifiedStatus.PARTIAL_DELETED) {
+        // has overlap or modified pages, then deserialize it
+        summary.PAGE_OVERLAP_OR_MODIFIED += 1;
+        pointPriorityReader.addNewPage(overlapPages.remove(0));
+        addOverlappedPagesIntoList(overlapPages);
+        compactWithOverlapPages();
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        summary.PAGE_NONE_OVERLAP += 1;
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPage(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushNonAlignedPage(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      summary.PAGE_NONE_OVERLAP_BUT_DESERIALIZE += 1;
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().getTimestamp()
+              <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will compact all 10 pages.
+   * Pages in the candidate overlapped pages list will be sequentially judged whether there is a
+   * real overlap, if so, it will be put into the point priority reader and deserialized; if not, it
+   * means that the page is located in a gap inside another pages, and it can be directly flushed to
+   * chunk writer. There will be new overlapped pages added into the list during the process of
+   * compacting overlapped pages. Notice: for a real overlap page, it will be removed from candidate
+   * list after it has been adding into point priority reader and deserializing. For a fake overlap
+   * page, it will be removed from candidate list after it has been flushing to chunk writer
+   * completely.
+   */
+  private void compactWithOverlapPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    checkAndCompactOverlappePages();
+
+    // write remaining data points, of which point.time >= the last overlapped page.startTime
+    while (pointPriorityReader.hasNext()) {
+      // write data point to chunk writer
+
+      compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+      pointPriorityReader.next();
+      if (candidateOverlappedPages.size() > 0) {
+        // finish compacting the first page or there are new chunks being deserialized and find
+        // the new overlapped pages, then start compacting them
+        checkAndCompactOverlappePages();
+      }
+    }
+  }
+
+  /**
+   * Check whether the page is true overlap or fake overlap. If a page is located in the gap of
+   * another page, then this page is fake overlap, which can be flushed to chunk writer directly.
+   * Otherwise, deserialize this page into point priority reader.
+   */
+  private void checkAndCompactOverlappePages()
+      throws IllegalPathException, IOException, WriteProcessException, PageException {
+    // write point.time < the last overlapped page.startTime
+    while (candidateOverlappedPages.size() > 0) {
+      PageElement nextPageElement = candidateOverlappedPages.get(0);
+
+      int oldSize = candidateOverlappedPages.size();
+      // write currentPage.point.time < nextPage.startTime to chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().getTimestamp() < nextPageElement.startTime) {
+        // write data point to chunk writer
+        compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+        pointPriorityReader.next();
+        if (candidateOverlappedPages.size() > oldSize) {
+          // during the process of writing overlapped points, if the first page is compacted
+          // completely or a new chunk is deserialized, there may be new pages overlapped with the
+          // first page in page queue which are added into the list. If so, the next overlapped
+          // page in the list may be changed, so we should re-get next overlap page here.
+          oldSize = candidateOverlappedPages.size();
+          nextPageElement = candidateOverlappedPages.get(0);
+        }
+      }
+
+      ModifiedStatus nextPageModifiedStatus = isPageModified(nextPageElement);
+
+      if (nextPageModifiedStatus == ModifiedStatus.ALL_DELETED) {
+        // all data on next page has been deleted, remove it
+        removePage(nextPageElement);
+      } else {
+        boolean isNextPageOverlap =
+            (pointPriorityReader.hasNext()
+                    && pointPriorityReader.currentPoint().getTimestamp()
+                        <= nextPageElement.pageHeader.getEndTime())
+                || isPageOverlap(nextPageElement);
+
+        if (isNextPageOverlap || nextPageModifiedStatus == ModifiedStatus.PARTIAL_DELETED) {
+          // has overlap or modified pages, then deserialize it
+          pointPriorityReader.addNewPage(nextPageElement);
+        } else {
+          // has none overlap or modified pages, flush it to chunk writer directly
+          summary.PAGE_FAKE_OVERLAP += 1;
+          compactWithNonOverlapPage(nextPageElement);
+        }
+      }
+      candidateOverlappedPages.remove(0);
+    }
+  }
+
+  /**
+   * Add the new overlapped pages into the global list and sort it according to the startTime of the
+   * page from small to large, so that each page can be compacted in order. If the page has been
+   * deleted completely, we remove it.
+   */
+  private void addOverlappedPagesIntoList(List<PageElement> newOverlappedPages) {
+    summary.PAGE_OVERLAP_OR_MODIFIED += newOverlappedPages.size();
+    int oldSize = candidateOverlappedPages.size();
+    candidateOverlappedPages.addAll(newOverlappedPages);
+    if (oldSize != 0 && candidateOverlappedPages.size() > oldSize) {
+      // if there is no pages in the overlappedPages, then we don't need to sort it after adding the
+      // new overlapped pages, because newOverlappedPages is already sorted. When there is pages in
+      // list before and there is new pages added into list, then we need to sort it again.
+      // we should ensure that the list is ordered according to the startTime of the page from small
+      // to large, so that each page can be compacted in order
+      candidateOverlappedPages.sort(Comparator.comparingLong(o -> o.startTime));
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(
+      ChunkMetadataElement chunkMetadataElement) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadataElement.chunkMetadata.getEndTime();
+    for (ChunkMetadataElement element : chunkMetadataQueue) {
+      if (element.chunkMetadata.getStartTime() <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped files which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the current device in the file from small to
+   * large, so that each file can be compacted in order.
+   */
+  protected List<FileElement> findOverlapFiles(FileElement file) {
+    List<FileElement> overlappedFiles = new ArrayList<>();
+    long endTime = file.resource.getEndTime(deviceId);
+    for (FileElement fileElement : fileList) {
+      if (fileElement.resource.getStartTime(deviceId) <= endTime) {
+        if (!fileElement.isOverlap) {
+          overlappedFiles.add(fileElement);
+          fileElement.isOverlap = true;
+        }
+      } else {
+        break;
+      }
+    }
+    return overlappedFiles;
+  }
+
+  /** Check is the page overlap with other pages later then the specific page in queue or not. */
+  private boolean isPageOverlap(PageElement pageElement) {
+    long endTime = pageElement.pageHeader.getEndTime();
+    long startTime = pageElement.startTime;
+    for (PageElement element : pageQueue) {
+      if (element.equals(pageElement)) {
+        continue;
+      }
+      // only check pages later than the specific page
+      if (element.startTime >= startTime && element.startTime <= endTime) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check whether the chunk is modified.
+   *
+   * <p>Notice: if is aligned chunk, return true if any of value chunk has data been deleted. Return
+   * false if and only if all value chunks has no data been deleted.
+   */
+  protected boolean isChunkModified(ChunkMetadataElement chunkMetadataElement) {

Review Comment:
   Change this function to a method of ChunkMetadataElement itself



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/SeriesCompactionExecutor.java:
##########
@@ -0,0 +1,601 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+public abstract class SeriesCompactionExecutor {
+  protected enum ModifiedStatus {
+    ALL_DELETED,
+    PARTIAL_DELETED,
+    NONE_DELETED;
+  }
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  private final SubCompactionTaskSummary summary;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<FileElement> fileList = new ArrayList<>();;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected AbstractCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  protected String deviceId;
+
+  // Pages in this list will be sequentially judged whether there is a real overlap to choose
+  // whether to put them in the point priority reader to deserialize or directly flush to chunk
+  // writer. During the process of compacting overlapped page, there may be new overlapped pages
+  // added into this list.
+  private final List<PageElement> candidateOverlappedPages = new ArrayList<>();
+
+  public SeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.summary = summary;
+
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+  }
+
+  public abstract void excute()
+      throws PageException, IllegalPathException, IOException, WriteProcessException;
+
+  protected abstract void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException;
+
+  /** Compact chunks in chunk metadata queue. */
+  protected void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        summary.CHUNK_OVERLAP_OR_MODIFIED += overlappedChunkMetadatas.size();
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        summary.CHUNK_NONE_OVERLAP += 1;
+        compactWithNonOverlapChunk(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      readChunk(overlappedChunkMetadata);
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunk(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    readChunk(chunkMetadataElement);
+    boolean success;
+    if (chunkMetadataElement.chunkMetadata instanceof AlignedChunkMetadata) {
+      success =
+          compactionWriter.flushAlignedChunk(
+              chunkMetadataElement.chunk,
+              ((AlignedChunkMetadata) chunkMetadataElement.chunkMetadata).getTimeChunkMetadata(),
+              chunkMetadataElement.valueChunks,
+              ((AlignedChunkMetadata) chunkMetadataElement.chunkMetadata)
+                  .getValueChunkMetadataList(),
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushNonAlignedChunk(
+              chunkMetadataElement.chunk,
+              (ChunkMetadata) chunkMetadataElement.chunkMetadata,
+              subTaskId);
+    }
+    if (success) {
+      // flush chunk successfully, then remove this chunk
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      summary.CHUNK_NONE_OVERLAP_BUT_DESERIALIZE += 1;
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  abstract void readChunk(ChunkMetadataElement chunkMetadataElement) throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      ModifiedStatus modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == ModifiedStatus.ALL_DELETED) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlapPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlapPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == ModifiedStatus.PARTIAL_DELETED) {
+        // has overlap or modified pages, then deserialize it
+        summary.PAGE_OVERLAP_OR_MODIFIED += 1;
+        pointPriorityReader.addNewPage(overlapPages.remove(0));
+        addOverlappedPagesIntoList(overlapPages);
+        compactWithOverlapPages();
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        summary.PAGE_NONE_OVERLAP += 1;
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPage(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushNonAlignedPage(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      summary.PAGE_NONE_OVERLAP_BUT_DESERIALIZE += 1;
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().getTimestamp()
+              <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will compact all 10 pages.
+   * Pages in the candidate overlapped pages list will be sequentially judged whether there is a
+   * real overlap, if so, it will be put into the point priority reader and deserialized; if not, it
+   * means that the page is located in a gap inside another pages, and it can be directly flushed to
+   * chunk writer. There will be new overlapped pages added into the list during the process of
+   * compacting overlapped pages. Notice: for a real overlap page, it will be removed from candidate
+   * list after it has been adding into point priority reader and deserializing. For a fake overlap
+   * page, it will be removed from candidate list after it has been flushing to chunk writer
+   * completely.
+   */
+  private void compactWithOverlapPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    checkAndCompactOverlappePages();
+
+    // write remaining data points, of which point.time >= the last overlapped page.startTime
+    while (pointPriorityReader.hasNext()) {
+      // write data point to chunk writer
+
+      compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+      pointPriorityReader.next();
+      if (candidateOverlappedPages.size() > 0) {
+        // finish compacting the first page or there are new chunks being deserialized and find
+        // the new overlapped pages, then start compacting them
+        checkAndCompactOverlappePages();
+      }
+    }
+  }
+
+  /**
+   * Check whether the page is true overlap or fake overlap. If a page is located in the gap of
+   * another page, then this page is fake overlap, which can be flushed to chunk writer directly.
+   * Otherwise, deserialize this page into point priority reader.
+   */
+  private void checkAndCompactOverlappePages()
+      throws IllegalPathException, IOException, WriteProcessException, PageException {
+    // write point.time < the last overlapped page.startTime
+    while (candidateOverlappedPages.size() > 0) {
+      PageElement nextPageElement = candidateOverlappedPages.get(0);
+
+      int oldSize = candidateOverlappedPages.size();
+      // write currentPage.point.time < nextPage.startTime to chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().getTimestamp() < nextPageElement.startTime) {
+        // write data point to chunk writer
+        compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+        pointPriorityReader.next();
+        if (candidateOverlappedPages.size() > oldSize) {
+          // during the process of writing overlapped points, if the first page is compacted
+          // completely or a new chunk is deserialized, there may be new pages overlapped with the
+          // first page in page queue which are added into the list. If so, the next overlapped
+          // page in the list may be changed, so we should re-get next overlap page here.
+          oldSize = candidateOverlappedPages.size();
+          nextPageElement = candidateOverlappedPages.get(0);
+        }
+      }

Review Comment:
   The iterator pattern here is not written properly. The correct iterator pattern would call `hasNext`, then `next`, and then `current`, but here it's not that order. So it's easy to think that the last data point is going to be lost here.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/FileElement.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+
+public class FileElement {
+  public TsFileResource resource;
+
+  public boolean isOverlap = false;

Review Comment:
   ```suggestion
     public boolean isSelected = false;
   ```



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/SeriesCompactionExecutor.java:
##########
@@ -0,0 +1,601 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+public abstract class SeriesCompactionExecutor {
+  protected enum ModifiedStatus {
+    ALL_DELETED,
+    PARTIAL_DELETED,
+    NONE_DELETED;
+  }
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  private final SubCompactionTaskSummary summary;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<FileElement> fileList = new ArrayList<>();;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected AbstractCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  protected String deviceId;
+
+  // Pages in this list will be sequentially judged whether there is a real overlap to choose
+  // whether to put them in the point priority reader to deserialize or directly flush to chunk
+  // writer. During the process of compacting overlapped page, there may be new overlapped pages
+  // added into this list.
+  private final List<PageElement> candidateOverlappedPages = new ArrayList<>();
+
+  public SeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.summary = summary;
+
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+  }
+
+  public abstract void excute()
+      throws PageException, IllegalPathException, IOException, WriteProcessException;
+
+  protected abstract void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException;
+
+  /** Compact chunks in chunk metadata queue. */
+  protected void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =

Review Comment:
   ```suggestion
         List<ChunkMetadataElement> overlappedChunkMetadataList =
   ```



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/ChunkMetadataElement.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+
+import java.util.List;
+
+public class ChunkMetadataElement {
+  public IChunkMetadata chunkMetadata;
+
+  public long priority;
+
+  public boolean isOverlaped = false;

Review Comment:
   ```suggestion
     public boolean isSelected= false;
   ```



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

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

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


[GitHub] [iotdb] THUMarkLau commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
THUMarkLau commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r997802429


##########
server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java:
##########
@@ -433,7 +433,7 @@ public class IoTDBConfig {
    */
   private CrossCompactionSelector crossCompactionSelector = CrossCompactionSelector.REWRITE;

Review Comment:
   If the performer is FAST, should the file selection also use FAST?



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/NonAlignedFastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedFastCompactionPerformerSubTask extends FastCompactionPerformerSubTask {
+  // measurements of the current device to be compacted, which is assigned to the current sub thread
+  private final List<String> measurements;
+
+  private String currentMeasurement;
+
+  boolean hasStartMeasurement = false;
+
+  public NonAlignedFastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      List<String> measurements,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId) {
+    super(
+        compactionWriter,
+        timeseriesMetadataOffsetMap,
+        readerCacheMap,
+        modificationCacheMap,
+        sortedSourceFiles,
+        deviceId,
+        false,
+        subTaskId);
+    this.measurements = measurements;
+  }
+
+  @Override
+  public Void call()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (String measurement : measurements) {
+      // get source files which are sorted by the startTime of current device from old to new, files
+      // that do not contain the current device have been filtered out as well.
+      sortedSourceFiles.forEach(x -> fileList.add(new FileElement(x)));
+      currentMeasurement = measurement;
+      hasStartMeasurement = false;
+
+      compactFiles();
+
+      if (hasStartMeasurement) {
+        compactionWriter.endMeasurement(subTaskId);
+      }
+    }
+
+    return null;
+  }
+
+  protected void startMeasurement() throws IOException {
+    if (!hasStartMeasurement && !chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      MeasurementSchema measurementSchema =
+          readerCacheMap
+              .get(firstChunkMetadataElement.fileElement.resource)
+              .getMeasurementSchema(
+                  Collections.singletonList(firstChunkMetadataElement.chunkMetadata));
+      compactionWriter.startMeasurement(Collections.singletonList(measurementSchema), subTaskId);
+      hasStartMeasurement = true;
+    }
+  }
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException {
+    for (FileElement fileElement : fileElements) {
+      TsFileResource resource = fileElement.resource;
+      Pair<Long, Long> timeseriesMetadataOffset =
+          timeseriesMetadataOffsetMap.get(currentMeasurement).get(resource);
+      if (timeseriesMetadataOffset == null) {
+        // tsfile does not contain this timeseries
+        removeFile(fileElement);
+        continue;

Review Comment:
   If the fileElement is the first file, the rest of the loop is meaningless, you should use a `break` or `return` here.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {

Review Comment:
   Get pointPriorityReader.currentPoint once and use it as a variable.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {
+        // write data point to chunk writer
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+        if (overlappedPages.size() > 0) {
+          // finish compacting the first page or there are new chunks being deserialized and find
+          // the new overlapped pages, then start compacting them
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(ChunkMetadataElement chunkMetadata) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadata.chunkMetadata.getEndTime();
+    for (ChunkMetadataElement element : chunkMetadataQueue) {
+      if (element.chunkMetadata.getStartTime() <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped files which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the current device in the file from small to
+   * large, so that each file can be compacted in order.
+   */
+  private List<FileElement> findOverlapFiles(FileElement file) {
+    List<FileElement> overlappedFiles = new ArrayList<>();
+    long endTime = file.resource.getEndTime(deviceId);
+    for (FileElement fileElement : fileList) {
+      if (fileElement.resource.getStartTime(deviceId) <= endTime) {
+        if (!fileElement.isOverlap) {
+          overlappedFiles.add(fileElement);
+          fileElement.isOverlap = true;
+        }
+      } else {
+        break;
+      }
+    }
+    return overlappedFiles;
+  }
+
+  /** Check is the page overlap with other pages later then the specific page in queue or not. */
+  private boolean isPageOverlap(PageElement pageElement) {
+    long endTime = pageElement.pageHeader.getEndTime();
+    long startTime = pageElement.startTime;
+    for (PageElement element : pageQueue) {
+      if (element.equals(pageElement)) {
+        continue;
+      }
+      // only check pages later than the specific page
+      if (element.startTime >= startTime && element.startTime <= endTime) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check whether the chunk is modified.
+   *
+   * <p>Notice: if is aligned chunk, return true if any of value chunk has data been deleted. Return
+   * false if and only if all value chunks has no data been deleted.
+   */
+  protected boolean isChunkModified(ChunkMetadataElement chunkMetadataElement) {
+    return chunkMetadataElement.chunkMetadata.isModified();
+  }
+
+  /**
+   * -1 means that no data on this page has been deleted. <br>
+   * 0 means that there is data on this page been deleted. <br>
+   * 1 means that all data on this page has been deleted.
+   *
+   * <p>Notice: If is aligned page, return 1 if and only if all value pages are deleted. Return -1
+   * if and only if no data exists on all value pages is deleted
+   */
+  protected abstract int isPageModified(PageElement pageElement);
+
+  protected int checkIsModified(long startTime, long endTime, Collection<TimeRange> deletions) {
+    int status = -1;
+    if (deletions != null) {
+      for (TimeRange range : deletions) {
+        if (range.contains(startTime, endTime)) {
+          // all data on this page or chunk has been deleted
+          return 1;
+        }
+        if (range.overlaps(new TimeRange(startTime, endTime))) {
+          // exist data on this page or chunk been deleted
+          status = 0;
+        }
+      }
+    }
+    return status;
+  }
+
+  /**
+   * Remove the page from page queue. If the page to be removed is the last page of chunk, it means
+   * this chunk has been compacted completely, we should remove this chunk. When removing chunks,
+   * there may be new overlapped chunks being deserialized and their pages are put into pageQueue.
+   * Therefore, when the removed page is the first page or when new chunks are deserialized and
+   * their pages are put into the queue, it is necessary to re-find new pages that overlap with the
+   * first page in the current queue, and put them put into list.
+   */
+  private void removePage(PageElement pageElement) throws IOException, IllegalPathException {
+    boolean isFirstPage = pageQueue.peek().equals(pageElement);
+    boolean hasNewOverlappedChunks = false;
+    pageQueue.remove(pageElement);
+    if (pageElement.isLastPage) {
+      // finish compacting the chunk, remove it from queue
+      hasNewOverlappedChunks = removeChunk(pageElement.chunkMetadataElement);
+    }
+
+    if ((isFirstPage || hasNewOverlappedChunks)
+        && pointPriorityReader.hasNext()
+        && pageQueue.size() != 0) {
+      // pointPriorityReader.hasNext() indicates that the new first page in page queue has not been
+      // finished compacting yet, so there may be other pages overlap with it.
+      // when deserializing new chunks into page queue or first page is removed from page queue, we
+      // should find new overlapped pages and put them into list}
+      pointPriorityReader.getNewOverlappedPages().addAll(findOverlapPages(pageQueue.peek()));
+      // we should ensure that the list is ordered according to the startTime of the page from small

Review Comment:
   By getting a variable from a object, you'd better not apply changes on it, because `getXXX` is like a read-only operation.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {
+        // write data point to chunk writer
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+        if (overlappedPages.size() > 0) {
+          // finish compacting the first page or there are new chunks being deserialized and find
+          // the new overlapped pages, then start compacting them
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(ChunkMetadataElement chunkMetadata) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadata.chunkMetadata.getEndTime();

Review Comment:
   `chunkMetadata.chunkMetadata` is not a good variable name.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {

Review Comment:
   `overlappedPages` is a local variable in this function, but it can be modified by other place in this program while the loop. I think it is weired and really hard to understand.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {

Review Comment:
   Using the same condition in nested loop is wiered.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {

Review Comment:
   Do not use magic number. Please use constant or enum.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/FastCompactionPerformer.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.performer.impl;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.AlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.NonAlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+public class FastCompactionPerformer implements ICrossCompactionPerformer {
+  private final Logger LOGGER = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+  private List<TsFileResource> seqFiles;
+
+  private List<TsFileResource> unseqFiles;
+
+  private List<TsFileResource> sortedSourceFiles = new ArrayList<>();
+
+  private static final int subTaskNum =
+      IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
+
+  public Map<TsFileResource, TsFileSequenceReader> readerCacheMap = new ConcurrentHashMap<>();
+
+  private CompactionTaskSummary summary;
+
+  private List<TsFileResource> targetFiles;
+
+  public Map<TsFileResource, List<Modification>> modificationCache = new ConcurrentHashMap<>();

Review Comment:
   The memory size of modification cache map should be control, else I think there is a risk of OOM.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {
+        // write data point to chunk writer
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+        if (overlappedPages.size() > 0) {
+          // finish compacting the first page or there are new chunks being deserialized and find
+          // the new overlapped pages, then start compacting them
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(ChunkMetadataElement chunkMetadata) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadata.chunkMetadata.getEndTime();
+    for (ChunkMetadataElement element : chunkMetadataQueue) {
+      if (element.chunkMetadata.getStartTime() <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped files which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the current device in the file from small to
+   * large, so that each file can be compacted in order.
+   */
+  private List<FileElement> findOverlapFiles(FileElement file) {
+    List<FileElement> overlappedFiles = new ArrayList<>();
+    long endTime = file.resource.getEndTime(deviceId);
+    for (FileElement fileElement : fileList) {
+      if (fileElement.resource.getStartTime(deviceId) <= endTime) {
+        if (!fileElement.isOverlap) {
+          overlappedFiles.add(fileElement);
+          fileElement.isOverlap = true;
+        }
+      } else {
+        break;
+      }
+    }
+    return overlappedFiles;
+  }
+
+  /** Check is the page overlap with other pages later then the specific page in queue or not. */
+  private boolean isPageOverlap(PageElement pageElement) {
+    long endTime = pageElement.pageHeader.getEndTime();
+    long startTime = pageElement.startTime;
+    for (PageElement element : pageQueue) {
+      if (element.equals(pageElement)) {
+        continue;
+      }
+      // only check pages later than the specific page
+      if (element.startTime >= startTime && element.startTime <= endTime) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check whether the chunk is modified.
+   *
+   * <p>Notice: if is aligned chunk, return true if any of value chunk has data been deleted. Return
+   * false if and only if all value chunks has no data been deleted.
+   */
+  protected boolean isChunkModified(ChunkMetadataElement chunkMetadataElement) {
+    return chunkMetadataElement.chunkMetadata.isModified();
+  }
+
+  /**
+   * -1 means that no data on this page has been deleted. <br>
+   * 0 means that there is data on this page been deleted. <br>
+   * 1 means that all data on this page has been deleted.
+   *
+   * <p>Notice: If is aligned page, return 1 if and only if all value pages are deleted. Return -1
+   * if and only if no data exists on all value pages is deleted
+   */
+  protected abstract int isPageModified(PageElement pageElement);
+
+  protected int checkIsModified(long startTime, long endTime, Collection<TimeRange> deletions) {
+    int status = -1;
+    if (deletions != null) {
+      for (TimeRange range : deletions) {
+        if (range.contains(startTime, endTime)) {
+          // all data on this page or chunk has been deleted
+          return 1;
+        }
+        if (range.overlaps(new TimeRange(startTime, endTime))) {
+          // exist data on this page or chunk been deleted
+          status = 0;
+        }
+      }
+    }
+    return status;
+  }
+
+  /**
+   * Remove the page from page queue. If the page to be removed is the last page of chunk, it means
+   * this chunk has been compacted completely, we should remove this chunk. When removing chunks,
+   * there may be new overlapped chunks being deserialized and their pages are put into pageQueue.
+   * Therefore, when the removed page is the first page or when new chunks are deserialized and
+   * their pages are put into the queue, it is necessary to re-find new pages that overlap with the
+   * first page in the current queue, and put them put into list.
+   */
+  private void removePage(PageElement pageElement) throws IOException, IllegalPathException {
+    boolean isFirstPage = pageQueue.peek().equals(pageElement);
+    boolean hasNewOverlappedChunks = false;
+    pageQueue.remove(pageElement);
+    if (pageElement.isLastPage) {
+      // finish compacting the chunk, remove it from queue
+      hasNewOverlappedChunks = removeChunk(pageElement.chunkMetadataElement);
+    }
+
+    if ((isFirstPage || hasNewOverlappedChunks)
+        && pointPriorityReader.hasNext()
+        && pageQueue.size() != 0) {
+      // pointPriorityReader.hasNext() indicates that the new first page in page queue has not been
+      // finished compacting yet, so there may be other pages overlap with it.
+      // when deserializing new chunks into page queue or first page is removed from page queue, we
+      // should find new overlapped pages and put them into list}

Review Comment:
   remove the final } in comment



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/NonAlignedFastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedFastCompactionPerformerSubTask extends FastCompactionPerformerSubTask {
+  // measurements of the current device to be compacted, which is assigned to the current sub thread
+  private final List<String> measurements;
+
+  private String currentMeasurement;
+
+  boolean hasStartMeasurement = false;
+
+  public NonAlignedFastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      List<String> measurements,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId) {
+    super(
+        compactionWriter,
+        timeseriesMetadataOffsetMap,
+        readerCacheMap,
+        modificationCacheMap,
+        sortedSourceFiles,
+        deviceId,
+        false,
+        subTaskId);
+    this.measurements = measurements;
+  }
+
+  @Override
+  public Void call()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (String measurement : measurements) {
+      // get source files which are sorted by the startTime of current device from old to new, files
+      // that do not contain the current device have been filtered out as well.
+      sortedSourceFiles.forEach(x -> fileList.add(new FileElement(x)));
+      currentMeasurement = measurement;
+      hasStartMeasurement = false;
+
+      compactFiles();
+
+      if (hasStartMeasurement) {
+        compactionWriter.endMeasurement(subTaskId);
+      }
+    }
+
+    return null;
+  }
+
+  protected void startMeasurement() throws IOException {
+    if (!hasStartMeasurement && !chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      MeasurementSchema measurementSchema =
+          readerCacheMap
+              .get(firstChunkMetadataElement.fileElement.resource)
+              .getMeasurementSchema(
+                  Collections.singletonList(firstChunkMetadataElement.chunkMetadata));
+      compactionWriter.startMeasurement(Collections.singletonList(measurementSchema), subTaskId);
+      hasStartMeasurement = true;
+    }
+  }
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException {
+    for (FileElement fileElement : fileElements) {
+      TsFileResource resource = fileElement.resource;
+      Pair<Long, Long> timeseriesMetadataOffset =
+          timeseriesMetadataOffsetMap.get(currentMeasurement).get(resource);
+      if (timeseriesMetadataOffset == null) {
+        // tsfile does not contain this timeseries
+        removeFile(fileElement);
+        continue;
+      }
+      List<IChunkMetadata> iChunkMetadataList =
+          readerCacheMap
+              .get(resource)
+              .getChunkMetadataListByTimeseriesMetadataOffset(
+                  timeseriesMetadataOffset.left, timeseriesMetadataOffset.right);
+
+      if (iChunkMetadataList.size() > 0) {
+        // modify chunk metadatas
+        QueryUtils.modifyChunkMetaData(
+            iChunkMetadataList,
+            getModificationsFromCache(
+                resource,
+                new PartialPath(deviceId, iChunkMetadataList.get(0).getMeasurementUid())));
+        if (iChunkMetadataList.size() == 0) {
+          // all chunks has been deleted in this file, just remove it
+          removeFile(fileElement);
+        }

Review Comment:
   Same as above.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {
+        // write data point to chunk writer
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+        if (overlappedPages.size() > 0) {
+          // finish compacting the first page or there are new chunks being deserialized and find
+          // the new overlapped pages, then start compacting them
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(ChunkMetadataElement chunkMetadata) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadata.chunkMetadata.getEndTime();
+    for (ChunkMetadataElement element : chunkMetadataQueue) {
+      if (element.chunkMetadata.getStartTime() <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped files which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the current device in the file from small to
+   * large, so that each file can be compacted in order.
+   */
+  private List<FileElement> findOverlapFiles(FileElement file) {
+    List<FileElement> overlappedFiles = new ArrayList<>();
+    long endTime = file.resource.getEndTime(deviceId);
+    for (FileElement fileElement : fileList) {
+      if (fileElement.resource.getStartTime(deviceId) <= endTime) {
+        if (!fileElement.isOverlap) {
+          overlappedFiles.add(fileElement);
+          fileElement.isOverlap = true;
+        }
+      } else {
+        break;
+      }
+    }
+    return overlappedFiles;
+  }
+
+  /** Check is the page overlap with other pages later then the specific page in queue or not. */
+  private boolean isPageOverlap(PageElement pageElement) {
+    long endTime = pageElement.pageHeader.getEndTime();
+    long startTime = pageElement.startTime;
+    for (PageElement element : pageQueue) {
+      if (element.equals(pageElement)) {
+        continue;
+      }
+      // only check pages later than the specific page
+      if (element.startTime >= startTime && element.startTime <= endTime) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check whether the chunk is modified.
+   *
+   * <p>Notice: if is aligned chunk, return true if any of value chunk has data been deleted. Return
+   * false if and only if all value chunks has no data been deleted.
+   */
+  protected boolean isChunkModified(ChunkMetadataElement chunkMetadataElement) {
+    return chunkMetadataElement.chunkMetadata.isModified();
+  }
+
+  /**
+   * -1 means that no data on this page has been deleted. <br>
+   * 0 means that there is data on this page been deleted. <br>
+   * 1 means that all data on this page has been deleted.
+   *
+   * <p>Notice: If is aligned page, return 1 if and only if all value pages are deleted. Return -1
+   * if and only if no data exists on all value pages is deleted
+   */
+  protected abstract int isPageModified(PageElement pageElement);
+
+  protected int checkIsModified(long startTime, long endTime, Collection<TimeRange> deletions) {

Review Comment:
   Using magic number as return value here is not a good coding style. Please use constant value or enum to replace it.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {

Review Comment:
   This loop is too long, please extract it as a function.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/FastCrossCompactionWriter.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.writer;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumn;
+import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.chunk.ValueChunkWriter;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class FastCrossCompactionWriter extends AbstractCrossCompactionWriter {
+
+  public FastCrossCompactionWriter(
+      List<TsFileResource> targetResources, List<TsFileResource> seqSourceResources)
+      throws IOException {
+    super(targetResources, seqSourceResources);
+  }
+
+  @Override
+  public void write(TimeColumn timestamps, Column[] columns, int subTaskId, int batchSize)
+      throws IOException {}
+
+  /**
+   * Flush chunk to tsfile directly. Return whether the chunk is flushed to tsfile successfully or
+   * not. Return false if the unsealed chunk is too small or the end time of chunk exceeds the end
+   * time of file, else return true. Notice: if sub-value measurement is null, then flush empty
+   * value chunk.
+   */
+  public boolean flushChunkToFileWriter(
+      IChunkMetadata iChunkMetadata, TsFileSequenceReader reader, int subTaskId)
+      throws IOException {
+    checkTimeAndMayFlushChunkToCurrentFile(iChunkMetadata.getStartTime(), subTaskId);
+    int fileIndex = seqFileIndexArray[subTaskId];
+    boolean isUnsealedChunkLargeEnough =
+        chunkWriters[subTaskId].checkIsChunkSizeOverThreshold(
+            chunkSizeLowerBoundInCompaction, chunkPointNumLowerBoundInCompaction);
+    if (!isUnsealedChunkLargeEnough
+        || (iChunkMetadata.getEndTime() > currentDeviceEndTime[fileIndex]
+            && fileIndex != targetFileWriters.size() - 1)) {
+      // if unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize the
+      // chunk
+      return false;
+    }
+
+    TsFileIOWriter tsFileIOWriter = targetFileWriters.get(fileIndex);
+
+    synchronized (tsFileIOWriter) {
+      // seal last chunk to file writer
+      chunkWriters[subTaskId].writeToFileWriter(tsFileIOWriter);
+      if (iChunkMetadata instanceof AlignedChunkMetadata) {
+        AlignedChunkMetadata alignedChunkMetadata = (AlignedChunkMetadata) iChunkMetadata;
+        // flush time chunk
+        ChunkMetadata chunkMetadata = (ChunkMetadata) alignedChunkMetadata.getTimeChunkMetadata();
+        tsFileIOWriter.writeChunk(reader.readMemChunk(chunkMetadata), chunkMetadata);
+        // flush value chunks
+        for (int i = 0; i < alignedChunkMetadata.getValueChunkMetadataList().size(); i++) {
+          IChunkMetadata valueChunkMetadata =
+              alignedChunkMetadata.getValueChunkMetadataList().get(i);
+          if (valueChunkMetadata == null) {
+            // sub sensor does not exist in current file or value chunk has been deleted completely
+            AlignedChunkWriterImpl alignedChunkWriter =
+                (AlignedChunkWriterImpl) chunkWriters[subTaskId];
+            ValueChunkWriter valueChunkWriter = alignedChunkWriter.getValueChunkWriterByIndex(i);
+            tsFileIOWriter.writeEmptyValueChunk(
+                valueChunkWriter.getMeasurementId(),
+                valueChunkWriter.getCompressionType(),
+                valueChunkWriter.getDataType(),
+                valueChunkWriter.getEncodingType(),
+                valueChunkWriter.getStatistics());
+            continue;
+          }
+          chunkMetadata = (ChunkMetadata) valueChunkMetadata;
+          tsFileIOWriter.writeChunk(reader.readMemChunk(chunkMetadata), chunkMetadata);
+        }
+      } else {
+        ChunkMetadata chunkMetadata = (ChunkMetadata) iChunkMetadata;
+        tsFileIOWriter.writeChunk(reader.readMemChunk(chunkMetadata), chunkMetadata);
+      }
+    }
+    isDeviceExistedInTargetFiles[fileIndex] = true;
+    isEmptyFile[fileIndex] = false;
+    return true;
+  }
+
+  /**
+   * Flush aligned page to tsfile directly. Return whether the page is flushed to tsfile
+   * successfully or not. Return false if the unsealed page is too small or the end time of page
+   * exceeds the end time of file, else return true. Notice: if sub-value measurement is null, then
+   * flush empty value page.
+   */
+  public boolean flushAlignedPageToChunkWriter(
+      ByteBuffer compressedTimePageData,
+      PageHeader timePageHeader,
+      List<ByteBuffer> compressedValuePageDatas,
+      List<PageHeader> valuePageHeaders,
+      int subTaskId)
+      throws IOException, PageException {
+    checkTimeAndMayFlushChunkToCurrentFile(timePageHeader.getStartTime(), subTaskId);

Review Comment:
   Line 126-136 is deduplicated with line 173-183. Extract them as function.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1005181242


##########
tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java:
##########
@@ -199,6 +209,63 @@ private PageReader constructPageReaderForNextPage(PageHeader pageHeader) throws
     return reader;
   }
 
+  /**
+   * Read page data without uncompressing it.
+   *
+   * @return compressed page data
+   */
+  public ByteBuffer readPageDataWithoutUncompressing(PageHeader pageHeader) throws IOException {
+    int compressedPageBodyLength = pageHeader.getCompressedSize();
+    byte[] compressedPageBody = new byte[compressedPageBodyLength];
+
+    // doesn't has a complete page body
+    if (compressedPageBodyLength > chunkDataBuffer.remaining()) {
+      throw new IOException(
+          "do not has a complete page body. Expected:"
+              + compressedPageBodyLength
+              + ". Actual:"
+              + chunkDataBuffer.remaining());
+    }
+
+    chunkDataBuffer.get(compressedPageBody);
+    return ByteBuffer.wrap(compressedPageBody);
+  }
+
+  /**
+   * Read data from compressed page data. Uncompress the page and decode it to batch data.
+   *
+   * @param compressedPageData Compressed page data
+   */
+  public TsBlock readPageData(PageHeader pageHeader, ByteBuffer compressedPageData)
+      throws IOException {
+    // uncompress page data

Review Comment:
   Yes, but if extract it as a function and put it into the `IChunkReader` interface, is it a good idea?



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004110894


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/FastCompactionPerformer.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.performer.impl;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.AlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.NonAlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+public class FastCompactionPerformer implements ICrossCompactionPerformer {
+  private final Logger LOGGER = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+  private List<TsFileResource> seqFiles;
+
+  private List<TsFileResource> unseqFiles;
+
+  private List<TsFileResource> sortedSourceFiles = new ArrayList<>();
+
+  private static final int subTaskNum =
+      IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
+
+  public Map<TsFileResource, TsFileSequenceReader> readerCacheMap = new ConcurrentHashMap<>();
+
+  private CompactionTaskSummary summary;
+
+  private List<TsFileResource> targetFiles;
+
+  public Map<TsFileResource, List<Modification>> modificationCache = new ConcurrentHashMap<>();
+
+  public FastCompactionPerformer(
+      List<TsFileResource> seqFiles,
+      List<TsFileResource> unseqFiles,
+      List<TsFileResource> targetFiles) {
+    this.seqFiles = seqFiles;
+    this.unseqFiles = unseqFiles;
+    this.targetFiles = targetFiles;
+  }
+
+  public FastCompactionPerformer() {}
+
+  @Override
+  public void perform()
+      throws IOException, MetadataException, StorageEngineException, InterruptedException {
+    try (MultiTsFileDeviceIterator deviceIterator =
+            new MultiTsFileDeviceIterator(seqFiles, unseqFiles, readerCacheMap);
+        FastCrossCompactionWriter compactionWriter =
+            new FastCrossCompactionWriter(targetFiles, seqFiles)) {
+      while (deviceIterator.hasNextDevice()) {
+        checkThreadInterrupted();
+        Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
+        String device = deviceInfo.left;
+        boolean isAligned = deviceInfo.right;
+
+        // sort the resources by the start time of current device from old to new, and remove
+        // resource that does not contain the current device. Notice: when the level of time index
+        // is file, there will be a false positive judgment problem, that is, the device does not
+        // actually exist but the judgment return device being existed.
+        sortedSourceFiles.addAll(seqFiles);
+        sortedSourceFiles.addAll(unseqFiles);
+        sortedSourceFiles.removeIf(x -> !x.mayContainsDevice(device));
+        sortedSourceFiles.sort(Comparator.comparingLong(x -> x.getStartTime(device)));
+
+        compactionWriter.startChunkGroup(device, isAligned);
+
+        if (isAligned) {
+          compactAlignedSeries(device, deviceIterator, compactionWriter);
+        } else {
+          compactNonAlignedSeries(device, deviceIterator, compactionWriter);
+        }
+
+        compactionWriter.endChunkGroup();
+        // update resource of the current device and check whether to flush chunk metadata or not
+        compactionWriter.checkAndMayFlushChunkMetadata();
+        sortedSourceFiles.clear();
+      }
+      compactionWriter.endFile();
+      CompactionUtils.updatePlanIndexes(targetFiles, seqFiles, unseqFiles);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    } finally {
+      // readers of source files have been closed in MultiTsFileDeviceIterator
+      // clean cache
+      sortedSourceFiles = null;
+      readerCacheMap = null;
+      modificationCache = null;
+    }
+  }
+
+  private void compactAlignedSeries(
+      String deviceId,
+      MultiTsFileDeviceIterator deviceIterator,
+      FastCrossCompactionWriter fastCrossCompactionWriter)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+    Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap =
+        new HashMap<>();
+    List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
+
+    // Get all value measurements and their schemas of the current device. Also get start offset and
+    // end offset of each timeseries metadata, in order to facilitate the reading of chunkMetadata
+    // directly by this offset later. Instead of deserializing chunk metadata later, we need to
+    // deserialize chunk metadata here to get the schemas of all value measurements, because we
+    // should get schemas of all value measurement to startMeasruement() and compaction process is
+    // to read a batch of overlapped files each time, and we cannot make sure if the first batch of
+    // overlapped tsfiles contain all the value measurements.
+    for (Map.Entry<String, Pair<MeasurementSchema, Map<TsFileResource, Pair<Long, Long>>>> entry :
+        deviceIterator.getTimeseriesSchemaAndMetadataOffsetOfCurrentDevice().entrySet()) {
+      if (!entry.getKey().equals("")) {

Review Comment:
   Resolved. I have added a constant named `TIME_COLUMN_ID` in TsFileConstant.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1030070815


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/NonAlignedSeriesCompactionExecutor.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedSeriesCompactionExecutor extends SeriesCompactionExecutor {
+  private boolean hasStartMeasurement = false;
+
+  // tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  private Map<TsFileResource, Pair<Long, Long>> timeseriesMetadataOffsetMap;
+
+  // it is used to initialize the fileList when compacting a new series
+  private final List<TsFileResource> sortResources;
+
+  public NonAlignedSeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    super(compactionWriter, readerCacheMap, modificationCacheMap, deviceId, subTaskId, summary);
+    this.sortResources = sortedSourceFiles;
+  }
+
+  @Override
+  public void excute()

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004118861


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {
+        // write data point to chunk writer
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+        if (overlappedPages.size() > 0) {
+          // finish compacting the first page or there are new chunks being deserialized and find
+          // the new overlapped pages, then start compacting them
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(ChunkMetadataElement chunkMetadata) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadata.chunkMetadata.getEndTime();
+    for (ChunkMetadataElement element : chunkMetadataQueue) {
+      if (element.chunkMetadata.getStartTime() <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped files which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the current device in the file from small to
+   * large, so that each file can be compacted in order.
+   */
+  private List<FileElement> findOverlapFiles(FileElement file) {
+    List<FileElement> overlappedFiles = new ArrayList<>();
+    long endTime = file.resource.getEndTime(deviceId);
+    for (FileElement fileElement : fileList) {
+      if (fileElement.resource.getStartTime(deviceId) <= endTime) {
+        if (!fileElement.isOverlap) {
+          overlappedFiles.add(fileElement);
+          fileElement.isOverlap = true;
+        }
+      } else {
+        break;
+      }
+    }
+    return overlappedFiles;
+  }
+
+  /** Check is the page overlap with other pages later then the specific page in queue or not. */
+  private boolean isPageOverlap(PageElement pageElement) {
+    long endTime = pageElement.pageHeader.getEndTime();
+    long startTime = pageElement.startTime;
+    for (PageElement element : pageQueue) {
+      if (element.equals(pageElement)) {
+        continue;
+      }
+      // only check pages later than the specific page
+      if (element.startTime >= startTime && element.startTime <= endTime) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check whether the chunk is modified.
+   *
+   * <p>Notice: if is aligned chunk, return true if any of value chunk has data been deleted. Return
+   * false if and only if all value chunks has no data been deleted.
+   */
+  protected boolean isChunkModified(ChunkMetadataElement chunkMetadataElement) {
+    return chunkMetadataElement.chunkMetadata.isModified();
+  }
+
+  /**
+   * -1 means that no data on this page has been deleted. <br>
+   * 0 means that there is data on this page been deleted. <br>
+   * 1 means that all data on this page has been deleted.
+   *
+   * <p>Notice: If is aligned page, return 1 if and only if all value pages are deleted. Return -1
+   * if and only if no data exists on all value pages is deleted
+   */
+  protected abstract int isPageModified(PageElement pageElement);
+
+  protected int checkIsModified(long startTime, long endTime, Collection<TimeRange> deletions) {
+    int status = -1;
+    if (deletions != null) {
+      for (TimeRange range : deletions) {
+        if (range.contains(startTime, endTime)) {
+          // all data on this page or chunk has been deleted
+          return 1;
+        }
+        if (range.overlaps(new TimeRange(startTime, endTime))) {
+          // exist data on this page or chunk been deleted
+          status = 0;
+        }
+      }
+    }
+    return status;
+  }
+
+  /**
+   * Remove the page from page queue. If the page to be removed is the last page of chunk, it means
+   * this chunk has been compacted completely, we should remove this chunk. When removing chunks,
+   * there may be new overlapped chunks being deserialized and their pages are put into pageQueue.
+   * Therefore, when the removed page is the first page or when new chunks are deserialized and
+   * their pages are put into the queue, it is necessary to re-find new pages that overlap with the
+   * first page in the current queue, and put them put into list.
+   */
+  private void removePage(PageElement pageElement) throws IOException, IllegalPathException {
+    boolean isFirstPage = pageQueue.peek().equals(pageElement);
+    boolean hasNewOverlappedChunks = false;
+    pageQueue.remove(pageElement);
+    if (pageElement.isLastPage) {
+      // finish compacting the chunk, remove it from queue
+      hasNewOverlappedChunks = removeChunk(pageElement.chunkMetadataElement);
+    }
+
+    if ((isFirstPage || hasNewOverlappedChunks)
+        && pointPriorityReader.hasNext()
+        && pageQueue.size() != 0) {
+      // pointPriorityReader.hasNext() indicates that the new first page in page queue has not been
+      // finished compacting yet, so there may be other pages overlap with it.
+      // when deserializing new chunks into page queue or first page is removed from page queue, we
+      // should find new overlapped pages and put them into list}
+      pointPriorityReader.getNewOverlappedPages().addAll(findOverlapPages(pageQueue.peek()));
+      // we should ensure that the list is ordered according to the startTime of the page from small

Review Comment:
   Resolved, it has been extracted as a method named `addOverlappedPagesIntoList()`.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004104669


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/PointElement.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+
+public class PointElement {
+  public long timestamp;
+  public int priority;
+  public Pair<Long, Object> timeValuePair;

Review Comment:
   The type of value in TimeValuePair is `TsPrimitiveType`, and the type returned by BatchDataIterator is `Object`. If  change this attribute to `TimeValuePair`, you need to convert `Object` type to `TsPrimitiveType` type every time you read a point, which is costly.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1005212231


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/NonAlignedFastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedFastCompactionPerformerSubTask extends FastCompactionPerformerSubTask {
+  // measurements of the current device to be compacted, which is assigned to the current sub thread
+  private final List<String> measurements;
+
+  private String currentMeasurement;
+
+  boolean hasStartMeasurement = false;
+
+  public NonAlignedFastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      List<String> measurements,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId) {
+    super(
+        compactionWriter,
+        timeseriesMetadataOffsetMap,
+        readerCacheMap,
+        modificationCacheMap,
+        sortedSourceFiles,
+        deviceId,
+        false,
+        subTaskId);
+    this.measurements = measurements;
+  }
+
+  @Override
+  public Void call()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (String measurement : measurements) {
+      // get source files which are sorted by the startTime of current device from old to new, files
+      // that do not contain the current device have been filtered out as well.
+      sortedSourceFiles.forEach(x -> fileList.add(new FileElement(x)));
+      currentMeasurement = measurement;
+      hasStartMeasurement = false;
+
+      compactFiles();
+
+      if (hasStartMeasurement) {
+        compactionWriter.endMeasurement(subTaskId);
+      }
+    }
+
+    return null;
+  }
+
+  protected void startMeasurement() throws IOException {
+    if (!hasStartMeasurement && !chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      MeasurementSchema measurementSchema =
+          readerCacheMap
+              .get(firstChunkMetadataElement.fileElement.resource)
+              .getMeasurementSchema(
+                  Collections.singletonList(firstChunkMetadataElement.chunkMetadata));
+      compactionWriter.startMeasurement(Collections.singletonList(measurementSchema), subTaskId);
+      hasStartMeasurement = true;
+    }
+  }
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException {
+    for (FileElement fileElement : fileElements) {
+      TsFileResource resource = fileElement.resource;
+      Pair<Long, Long> timeseriesMetadataOffset =
+          timeseriesMetadataOffsetMap.get(currentMeasurement).get(resource);
+      if (timeseriesMetadataOffset == null) {
+        // tsfile does not contain this timeseries
+        removeFile(fileElement);
+        continue;

Review Comment:
   Sorry I don't understand. If the first file does not contain this timeseries, it doesn't mean the remaining overlapped files do not contain this timeseries.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/NonAlignedFastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedFastCompactionPerformerSubTask extends FastCompactionPerformerSubTask {
+  // measurements of the current device to be compacted, which is assigned to the current sub thread
+  private final List<String> measurements;
+
+  private String currentMeasurement;
+
+  boolean hasStartMeasurement = false;
+
+  public NonAlignedFastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      List<String> measurements,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId) {
+    super(
+        compactionWriter,
+        timeseriesMetadataOffsetMap,
+        readerCacheMap,
+        modificationCacheMap,
+        sortedSourceFiles,
+        deviceId,
+        false,
+        subTaskId);
+    this.measurements = measurements;
+  }
+
+  @Override
+  public Void call()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (String measurement : measurements) {
+      // get source files which are sorted by the startTime of current device from old to new, files
+      // that do not contain the current device have been filtered out as well.
+      sortedSourceFiles.forEach(x -> fileList.add(new FileElement(x)));
+      currentMeasurement = measurement;
+      hasStartMeasurement = false;
+
+      compactFiles();
+
+      if (hasStartMeasurement) {
+        compactionWriter.endMeasurement(subTaskId);
+      }
+    }
+
+    return null;
+  }
+
+  protected void startMeasurement() throws IOException {
+    if (!hasStartMeasurement && !chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      MeasurementSchema measurementSchema =
+          readerCacheMap
+              .get(firstChunkMetadataElement.fileElement.resource)
+              .getMeasurementSchema(
+                  Collections.singletonList(firstChunkMetadataElement.chunkMetadata));
+      compactionWriter.startMeasurement(Collections.singletonList(measurementSchema), subTaskId);
+      hasStartMeasurement = true;
+    }
+  }
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException {
+    for (FileElement fileElement : fileElements) {
+      TsFileResource resource = fileElement.resource;
+      Pair<Long, Long> timeseriesMetadataOffset =
+          timeseriesMetadataOffsetMap.get(currentMeasurement).get(resource);
+      if (timeseriesMetadataOffset == null) {
+        // tsfile does not contain this timeseries
+        removeFile(fileElement);
+        continue;
+      }
+      List<IChunkMetadata> iChunkMetadataList =
+          readerCacheMap
+              .get(resource)
+              .getChunkMetadataListByTimeseriesMetadataOffset(
+                  timeseriesMetadataOffset.left, timeseriesMetadataOffset.right);
+
+      if (iChunkMetadataList.size() > 0) {
+        // modify chunk metadatas
+        QueryUtils.modifyChunkMetaData(
+            iChunkMetadataList,
+            getModificationsFromCache(
+                resource,
+                new PartialPath(deviceId, iChunkMetadataList.get(0).getMeasurementUid())));
+        if (iChunkMetadataList.size() == 0) {
+          // all chunks has been deleted in this file, just remove it
+          removeFile(fileElement);
+        }

Review Comment:
   Sorry I don't understand. If the first file does not contain this timeseries, it doesn't mean the remaining overlapped files do not contain this timeseries.



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

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

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


[GitHub] [iotdb] qiaojialin merged pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
qiaojialin merged PR #7621:
URL: https://github.com/apache/iotdb/pull/7621


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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004094991


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1005518163


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/NonAlignedFastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedFastCompactionPerformerSubTask extends FastCompactionPerformerSubTask {
+  // measurements of the current device to be compacted, which is assigned to the current sub thread
+  private final List<String> measurements;
+
+  private String currentMeasurement;
+
+  boolean hasStartMeasurement = false;
+
+  public NonAlignedFastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      List<String> measurements,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId) {
+    super(
+        compactionWriter,
+        timeseriesMetadataOffsetMap,
+        readerCacheMap,
+        modificationCacheMap,
+        sortedSourceFiles,
+        deviceId,
+        false,
+        subTaskId);
+    this.measurements = measurements;
+  }
+
+  @Override
+  public Void call()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (String measurement : measurements) {

Review Comment:
   Resolved. I have created a new class named `SeriesCompactionExecutor`.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1006510455


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/FastCompactionPerformer.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.performer.impl;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.AlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.NonAlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+public class FastCompactionPerformer implements ICrossCompactionPerformer {
+  private final Logger LOGGER = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+  private List<TsFileResource> seqFiles;
+
+  private List<TsFileResource> unseqFiles;
+
+  private List<TsFileResource> sortedSourceFiles = new ArrayList<>();
+
+  private static final int subTaskNum =
+      IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
+
+  public Map<TsFileResource, TsFileSequenceReader> readerCacheMap = new ConcurrentHashMap<>();
+
+  private CompactionTaskSummary summary;
+
+  private List<TsFileResource> targetFiles;
+
+  public Map<TsFileResource, List<Modification>> modificationCache = new ConcurrentHashMap<>();

Review Comment:
   It can be put into the next pr to do.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1030071386


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/FileElement.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+
+public class FileElement {
+  public TsFileResource resource;
+
+  public boolean isOverlap = false;

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004094680


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {

Review Comment:
   Resolved.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004122233


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/reader/PointPriorityReader.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.reader;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.FastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PointElement;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+/**
+ * This reader is used to deduplicate and organize overlapping pages, and read out points in order.
+ * It is used for compaction.
+ */
+public class PointPriorityReader {
+  private long lastTime;
+
+  private final PriorityQueue<PointElement> pointQueue;
+
+  private final FastCompactionPerformerSubTask.RemovePage removePage;
+
+  private Pair<Long, Object> currentPoint;
+
+  private boolean isNewPoint = true;

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004085989


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {
+        // write data point to chunk writer
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+        if (overlappedPages.size() > 0) {
+          // finish compacting the first page or there are new chunks being deserialized and find
+          // the new overlapped pages, then start compacting them
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(ChunkMetadataElement chunkMetadata) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadata.chunkMetadata.getEndTime();

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004112519


##########
server/src/main/java/org/apache/iotdb/db/tools/validate/TsFileValidationTool.java:
##########
@@ -72,7 +72,7 @@
  */
 public class TsFileValidationTool {
   // print detail type of overlap or not
-  private static boolean printDetails = false;
+  private static boolean printDetails = true;

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004110894


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/FastCompactionPerformer.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.performer.impl;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.AlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.NonAlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+public class FastCompactionPerformer implements ICrossCompactionPerformer {
+  private final Logger LOGGER = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+  private List<TsFileResource> seqFiles;
+
+  private List<TsFileResource> unseqFiles;
+
+  private List<TsFileResource> sortedSourceFiles = new ArrayList<>();
+
+  private static final int subTaskNum =
+      IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
+
+  public Map<TsFileResource, TsFileSequenceReader> readerCacheMap = new ConcurrentHashMap<>();
+
+  private CompactionTaskSummary summary;
+
+  private List<TsFileResource> targetFiles;
+
+  public Map<TsFileResource, List<Modification>> modificationCache = new ConcurrentHashMap<>();
+
+  public FastCompactionPerformer(
+      List<TsFileResource> seqFiles,
+      List<TsFileResource> unseqFiles,
+      List<TsFileResource> targetFiles) {
+    this.seqFiles = seqFiles;
+    this.unseqFiles = unseqFiles;
+    this.targetFiles = targetFiles;
+  }
+
+  public FastCompactionPerformer() {}
+
+  @Override
+  public void perform()
+      throws IOException, MetadataException, StorageEngineException, InterruptedException {
+    try (MultiTsFileDeviceIterator deviceIterator =
+            new MultiTsFileDeviceIterator(seqFiles, unseqFiles, readerCacheMap);
+        FastCrossCompactionWriter compactionWriter =
+            new FastCrossCompactionWriter(targetFiles, seqFiles)) {
+      while (deviceIterator.hasNextDevice()) {
+        checkThreadInterrupted();
+        Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
+        String device = deviceInfo.left;
+        boolean isAligned = deviceInfo.right;
+
+        // sort the resources by the start time of current device from old to new, and remove
+        // resource that does not contain the current device. Notice: when the level of time index
+        // is file, there will be a false positive judgment problem, that is, the device does not
+        // actually exist but the judgment return device being existed.
+        sortedSourceFiles.addAll(seqFiles);
+        sortedSourceFiles.addAll(unseqFiles);
+        sortedSourceFiles.removeIf(x -> !x.mayContainsDevice(device));
+        sortedSourceFiles.sort(Comparator.comparingLong(x -> x.getStartTime(device)));
+
+        compactionWriter.startChunkGroup(device, isAligned);
+
+        if (isAligned) {
+          compactAlignedSeries(device, deviceIterator, compactionWriter);
+        } else {
+          compactNonAlignedSeries(device, deviceIterator, compactionWriter);
+        }
+
+        compactionWriter.endChunkGroup();
+        // update resource of the current device and check whether to flush chunk metadata or not
+        compactionWriter.checkAndMayFlushChunkMetadata();
+        sortedSourceFiles.clear();
+      }
+      compactionWriter.endFile();
+      CompactionUtils.updatePlanIndexes(targetFiles, seqFiles, unseqFiles);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    } finally {
+      // readers of source files have been closed in MultiTsFileDeviceIterator
+      // clean cache
+      sortedSourceFiles = null;
+      readerCacheMap = null;
+      modificationCache = null;
+    }
+  }
+
+  private void compactAlignedSeries(
+      String deviceId,
+      MultiTsFileDeviceIterator deviceIterator,
+      FastCrossCompactionWriter fastCrossCompactionWriter)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+    Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap =
+        new HashMap<>();
+    List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
+
+    // Get all value measurements and their schemas of the current device. Also get start offset and
+    // end offset of each timeseries metadata, in order to facilitate the reading of chunkMetadata
+    // directly by this offset later. Instead of deserializing chunk metadata later, we need to
+    // deserialize chunk metadata here to get the schemas of all value measurements, because we
+    // should get schemas of all value measurement to startMeasruement() and compaction process is
+    // to read a batch of overlapped files each time, and we cannot make sure if the first batch of
+    // overlapped tsfiles contain all the value measurements.
+    for (Map.Entry<String, Pair<MeasurementSchema, Map<TsFileResource, Pair<Long, Long>>>> entry :
+        deviceIterator.getTimeseriesSchemaAndMetadataOffsetOfCurrentDevice().entrySet()) {
+      if (!entry.getKey().equals("")) {

Review Comment:
   Resolved. I have add a constant named `TIME_COLUMN_ID` in TsFileConstant.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1005156599


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCrossCompactionWriter.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.writer;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.rescon.SystemInfo;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumn;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public abstract class AbstractCrossCompactionWriter extends AbstractCompactionWriter {
+
+  // target fileIOWriters
+  protected List<TsFileIOWriter> targetFileWriters = new ArrayList<>();
+
+  // source tsfiles
+  private List<TsFileResource> seqTsFileResources;
+
+  // Each sub task has its corresponding seq file index.
+  // The index of the array corresponds to subTaskId.
+  protected int[] seqFileIndexArray = new int[subTaskNum];
+
+  // device end time in each source seq file
+  protected final long[] currentDeviceEndTime;
+
+  // whether each target file is empty or not
+  protected final boolean[] isEmptyFile;
+
+  // whether each target file has device data or not
+  protected final boolean[] isDeviceExistedInTargetFiles;
+
+  // current chunk group header size
+  private int chunkGroupHeaderSize;
+
+  protected List<TsFileResource> targetResources;
+
+  public AbstractCrossCompactionWriter(
+      List<TsFileResource> targetResources, List<TsFileResource> seqFileResources)
+      throws IOException {
+    currentDeviceEndTime = new long[seqFileResources.size()];
+    isEmptyFile = new boolean[seqFileResources.size()];
+    isDeviceExistedInTargetFiles = new boolean[targetResources.size()];
+    long memorySizeForEachWriter =
+        (long)
+            (SystemInfo.getInstance().getMemorySizeForCompaction()
+                / IoTDBDescriptor.getInstance().getConfig().getConcurrentCompactionThread()
+                * IoTDBDescriptor.getInstance().getConfig().getChunkMetadataSizeProportion()
+                / targetResources.size());
+    for (int i = 0; i < targetResources.size(); i++) {
+      this.targetFileWriters.add(
+          new TsFileIOWriter(targetResources.get(i).getTsFile(), true, memorySizeForEachWriter));
+      isEmptyFile[i] = true;
+    }
+    this.seqTsFileResources = seqFileResources;
+    this.targetResources = targetResources;
+  }
+
+  @Override
+  public void startChunkGroup(String deviceId, boolean isAlign) throws IOException {
+    this.deviceId = deviceId;
+    this.isAlign = isAlign;
+    this.seqFileIndexArray = new int[subTaskNum];
+    checkIsDeviceExistAndGetDeviceEndTime();
+    for (int i = 0; i < targetFileWriters.size(); i++) {
+      chunkGroupHeaderSize = targetFileWriters.get(i).startChunkGroup(deviceId);
+    }
+  }
+
+  @Override
+  public void endChunkGroup() throws IOException {
+    for (int i = 0; i < seqTsFileResources.size(); i++) {
+      TsFileIOWriter targetFileWriter = targetFileWriters.get(i);
+      if (isDeviceExistedInTargetFiles[i]) {
+        targetFileWriter.endChunkGroup();
+      } else {
+        targetFileWriter.truncate(targetFileWriter.getPos() - chunkGroupHeaderSize);
+      }
+      isDeviceExistedInTargetFiles[i] = false;
+    }
+    seqFileIndexArray = null;
+  }
+
+  @Override
+  public void endMeasurement(int subTaskId) throws IOException {
+    flushChunkToFileWriter(
+        targetFileWriters.get(seqFileIndexArray[subTaskId]), chunkWriters[subTaskId]);
+    seqFileIndexArray[subTaskId] = 0;
+  }
+
+  @Override
+  public void write(long timestamp, Object value, int subTaskId) throws IOException {
+    checkTimeAndMayFlushChunkToCurrentFile(timestamp, subTaskId);
+    int fileIndex = seqFileIndexArray[subTaskId];
+    writeDataPoint(timestamp, value, chunkWriters[subTaskId]);
+    chunkPointNumArray[subTaskId]++;
+    checkChunkSizeAndMayOpenANewChunk(
+        targetFileWriters.get(fileIndex), chunkWriters[subTaskId], subTaskId, true);
+    isDeviceExistedInTargetFiles[fileIndex] = true;
+    isEmptyFile[fileIndex] = false;
+  }
+
+  /** Write data in batch, only used for aligned device. */
+  @Override
+  public abstract void write(TimeColumn timestamps, Column[] columns, int subTaskId, int batchSize)
+      throws IOException;
+
+  @Override
+  public void endFile() throws IOException {
+    for (int i = 0; i < isEmptyFile.length; i++) {
+      targetFileWriters.get(i).endFile();
+      // delete empty target file
+      if (isEmptyFile[i]) {
+        targetFileWriters.get(i).getFile().delete();
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    for (TsFileIOWriter targetWriter : targetFileWriters) {
+      if (targetWriter != null && targetWriter.canWrite()) {
+        targetWriter.close();
+      }
+    }
+    targetFileWriters = null;
+    seqTsFileResources = null;
+  }
+
+  @Override
+  public void checkAndMayFlushChunkMetadata() throws IOException {
+    for (int i = 0; i < targetFileWriters.size(); i++) {
+      TsFileIOWriter fileIOWriter = targetFileWriters.get(i);
+      TsFileResource resource = targetResources.get(i);
+      // Before flushing chunk metadatas, we use chunk metadatas in tsfile io writer to update start
+      // time and end time in resource.
+      List<TimeseriesMetadata> timeseriesMetadatasOfCurrentDevice =
+          fileIOWriter.getDeviceTimeseriesMetadataMap().get(deviceId);

Review Comment:
   Resolved. I have added a new method named `getChunkMetadatasOfDeviceInMemory()` in `TsFileIOWriter`.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractInnerCompactionWriter.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.writer;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.rescon.SystemInfo;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumn;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import java.io.IOException;
+import java.util.List;
+
+public abstract class AbstractInnerCompactionWriter extends AbstractCompactionWriter {
+  protected TsFileIOWriter fileWriter;
+
+  protected boolean isEmptyFile;
+
+  protected TsFileResource targetResource;
+
+  public AbstractInnerCompactionWriter(TsFileResource targetFileResource) throws IOException {
+    long sizeForFileWriter =
+        (long)
+            (SystemInfo.getInstance().getMemorySizeForCompaction()
+                / IoTDBDescriptor.getInstance().getConfig().getConcurrentCompactionThread()
+                * IoTDBDescriptor.getInstance().getConfig().getChunkMetadataSizeProportion());
+    this.fileWriter = new TsFileIOWriter(targetFileResource.getTsFile(), true, sizeForFileWriter);
+    this.targetResource = targetFileResource;
+    isEmptyFile = true;
+  }
+
+  @Override
+  public void startChunkGroup(String deviceId, boolean isAlign) throws IOException {
+    fileWriter.startChunkGroup(deviceId);
+    this.isAlign = isAlign;
+    this.deviceId = deviceId;
+  }
+
+  @Override
+  public void endChunkGroup() throws IOException {
+    fileWriter.endChunkGroup();
+  }
+
+  @Override
+  public void endMeasurement(int subTaskId) throws IOException {
+    flushChunkToFileWriter(fileWriter, chunkWriters[subTaskId]);
+  }
+
+  @Override
+  public abstract void write(long timestamp, Object value, int subTaskId) throws IOException;
+
+  @Override
+  public abstract void write(TimeColumn timestamps, Column[] columns, int subTaskId, int batchSize)
+      throws IOException;
+
+  @Override
+  public void endFile() throws IOException {
+    fileWriter.endFile();
+    if (isEmptyFile) {
+      fileWriter.getFile().delete();
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (fileWriter != null && fileWriter.canWrite()) {
+      fileWriter.close();
+    }
+    fileWriter = null;
+  }
+
+  @Override
+  public void checkAndMayFlushChunkMetadata() throws IOException {
+    // Before flushing chunk metadatas, we use chunk metadatas in tsfile io writer to update start
+    // time and end time in resource.
+    List<TimeseriesMetadata> timeseriesMetadatasOfCurrentDevice =
+        fileWriter.getDeviceTimeseriesMetadataMap().get(deviceId);

Review Comment:
   Resolved. I have added a new method named `getChunkMetadatasOfDeviceInMemory()` in `TsFileIOWriter`.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004086636


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/FastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+
+public abstract class FastCompactionPerformerSubTask implements Callable<Void> {
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  // sorted source files by the start time of device
+  protected List<FileElement> fileList;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected FastCrossCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+  // used to get the chunk metadatas from tsfile directly according to timeseries metadata offset.
+  protected Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<TsFileResource> sortedSourceFiles;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  private final boolean isAligned;
+
+  protected String deviceId;
+
+  public FastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      boolean isAligned,
+      int subTaskId) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
+    this.isAligned = isAligned;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.sortedSourceFiles = sortedSourceFiles;
+
+    this.fileList = new ArrayList<>();
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Integer.compare(o2.priority, o1.priority);
+            });
+  }
+
+  protected void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    while (!fileList.isEmpty()) {
+      List<FileElement> overlappedFiles = findOverlapFiles(fileList.get(0));
+
+      // read chunk metadatas from files and put them into chunk metadata queue
+      deserializeFileIntoQueue(overlappedFiles);
+
+      if (!isAligned) {
+        // for nonAligned sensors, only after getting chunkMetadatas can we create schema to start
+        // measurement; for aligned sensors, we get all schemas of value sensors and
+        // startMeasurement() in the previous process, because we need to get all chunk metadatas of
+        // sensors and their schemas under the current device, but since the compaction process is
+        // to read a batch of overlapped files each time, which may not contain all the sensors.
+        startMeasurement();
+      }
+
+      compactChunks();
+    }
+  }
+
+  protected abstract void startMeasurement() throws IOException;
+
+  /**
+   * Compact chunks in chunk metadata queue.
+   *
+   * @throws IOException
+   * @throws PageException
+   */
+  private void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        compactWithNonOverlapChunks(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunks(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    if (compactionWriter.flushChunkToFileWriter(
+        chunkMetadataElement.chunkMetadata,
+        readerCacheMap.get(chunkMetadataElement.fileElement.resource),
+        subTaskId)) {
+      // flush chunk successfully
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      int modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == 1) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlappedPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlappedPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == 0) {
+        // has overlap or modified pages, then deserialize it
+        compactWithOverlapPages(overlappedPages);
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPageToChunkWriter(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushPageToChunkWriter(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().left <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will merge all 10 pages.
+   */
+  private void compactWithOverlapPages(List<PageElement> overlappedPages)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    pointPriorityReader.addNewPage(overlappedPages.remove(0));
+    pointPriorityReader.updateNewOverlappedPages(overlappedPages);
+    while (pointPriorityReader.hasNext()) {
+      // write point.time < the last overlapped page.startTime
+      while (overlappedPages.size() > 0) {
+        PageElement nextPageElement = overlappedPages.get(0);
+
+        int oldSize = overlappedPages.size();
+        // write currentPage.point.time < nextPage.startTime to chunk writer
+        while (pointPriorityReader.currentPoint().left < nextPageElement.startTime) {
+          // write data point to chunk writer
+          compactionWriter.write(
+              pointPriorityReader.currentPoint().left,
+              pointPriorityReader.currentPoint().right,
+              subTaskId);
+          pointPriorityReader.next();
+          if (overlappedPages.size() > oldSize) {
+            // during the process of writing overlapped points, if the first page is compacted
+            // completely or a new chunk is deserialized, there may be new pages overlapped with the
+            // first page in page queue which are added into the list. If so, the next overlapped
+            // page in the list may be changed, so we should re-get next overlap page here.
+            oldSize = overlappedPages.size();
+            nextPageElement = overlappedPages.get(0);
+          }
+        }
+
+        int nextPageModifiedStatus = isPageModified(nextPageElement);
+
+        if (nextPageModifiedStatus == 1) {
+          // all data on next page has been deleted, remove it
+          removePage(nextPageElement);
+        } else {
+          boolean isNextPageOverlap =
+              pointPriorityReader.currentPoint().left <= nextPageElement.pageHeader.getEndTime()
+                  || isPageOverlap(nextPageElement);
+
+          if (isNextPageOverlap || nextPageModifiedStatus == 0) {
+            // has overlap or modified pages, then deserialize it
+            pointPriorityReader.addNewPage(nextPageElement);
+          } else {
+            // has none overlap or modified pages, flush it to chunk writer directly
+            compactWithNonOverlapPage(nextPageElement);
+          }
+        }
+        overlappedPages.remove(0);
+      }
+
+      // write remaining data points, of which point.time >= the last overlapped page.startTime
+      while (pointPriorityReader.hasNext()) {
+        // write data point to chunk writer
+        compactionWriter.write(
+            pointPriorityReader.currentPoint().left,
+            pointPriorityReader.currentPoint().right,
+            subTaskId);
+        pointPriorityReader.next();
+        if (overlappedPages.size() > 0) {
+          // finish compacting the first page or there are new chunks being deserialized and find
+          // the new overlapped pages, then start compacting them
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(ChunkMetadataElement chunkMetadata) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadata.chunkMetadata.getEndTime();
+    for (ChunkMetadataElement element : chunkMetadataQueue) {
+      if (element.chunkMetadata.getStartTime() <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped files which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the current device in the file from small to
+   * large, so that each file can be compacted in order.
+   */
+  private List<FileElement> findOverlapFiles(FileElement file) {
+    List<FileElement> overlappedFiles = new ArrayList<>();
+    long endTime = file.resource.getEndTime(deviceId);
+    for (FileElement fileElement : fileList) {
+      if (fileElement.resource.getStartTime(deviceId) <= endTime) {
+        if (!fileElement.isOverlap) {
+          overlappedFiles.add(fileElement);
+          fileElement.isOverlap = true;
+        }
+      } else {
+        break;
+      }
+    }
+    return overlappedFiles;
+  }
+
+  /** Check is the page overlap with other pages later then the specific page in queue or not. */
+  private boolean isPageOverlap(PageElement pageElement) {
+    long endTime = pageElement.pageHeader.getEndTime();
+    long startTime = pageElement.startTime;
+    for (PageElement element : pageQueue) {
+      if (element.equals(pageElement)) {
+        continue;
+      }
+      // only check pages later than the specific page
+      if (element.startTime >= startTime && element.startTime <= endTime) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check whether the chunk is modified.
+   *
+   * <p>Notice: if is aligned chunk, return true if any of value chunk has data been deleted. Return
+   * false if and only if all value chunks has no data been deleted.
+   */
+  protected boolean isChunkModified(ChunkMetadataElement chunkMetadataElement) {
+    return chunkMetadataElement.chunkMetadata.isModified();
+  }
+
+  /**
+   * -1 means that no data on this page has been deleted. <br>
+   * 0 means that there is data on this page been deleted. <br>
+   * 1 means that all data on this page has been deleted.
+   *
+   * <p>Notice: If is aligned page, return 1 if and only if all value pages are deleted. Return -1
+   * if and only if no data exists on all value pages is deleted
+   */
+  protected abstract int isPageModified(PageElement pageElement);
+
+  protected int checkIsModified(long startTime, long endTime, Collection<TimeRange> deletions) {
+    int status = -1;
+    if (deletions != null) {
+      for (TimeRange range : deletions) {
+        if (range.contains(startTime, endTime)) {
+          // all data on this page or chunk has been deleted
+          return 1;
+        }
+        if (range.overlaps(new TimeRange(startTime, endTime))) {
+          // exist data on this page or chunk been deleted
+          status = 0;
+        }
+      }
+    }
+    return status;
+  }
+
+  /**
+   * Remove the page from page queue. If the page to be removed is the last page of chunk, it means
+   * this chunk has been compacted completely, we should remove this chunk. When removing chunks,
+   * there may be new overlapped chunks being deserialized and their pages are put into pageQueue.
+   * Therefore, when the removed page is the first page or when new chunks are deserialized and
+   * their pages are put into the queue, it is necessary to re-find new pages that overlap with the
+   * first page in the current queue, and put them put into list.
+   */
+  private void removePage(PageElement pageElement) throws IOException, IllegalPathException {
+    boolean isFirstPage = pageQueue.peek().equals(pageElement);
+    boolean hasNewOverlappedChunks = false;
+    pageQueue.remove(pageElement);
+    if (pageElement.isLastPage) {
+      // finish compacting the chunk, remove it from queue
+      hasNewOverlappedChunks = removeChunk(pageElement.chunkMetadataElement);
+    }
+
+    if ((isFirstPage || hasNewOverlappedChunks)
+        && pointPriorityReader.hasNext()
+        && pageQueue.size() != 0) {
+      // pointPriorityReader.hasNext() indicates that the new first page in page queue has not been
+      // finished compacting yet, so there may be other pages overlap with it.
+      // when deserializing new chunks into page queue or first page is removed from page queue, we
+      // should find new overlapped pages and put them into list}

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1004120382


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/reader/PointPriorityReader.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.reader;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.FastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PointElement;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+/**
+ * This reader is used to deduplicate and organize overlapping pages, and read out points in order.
+ * It is used for compaction.
+ */
+public class PointPriorityReader {
+  private long lastTime;
+
+  private final PriorityQueue<PointElement> pointQueue;
+
+  private final FastCompactionPerformerSubTask.RemovePage removePage;
+
+  private Pair<Long, Object> currentPoint;
+
+  private boolean isNewPoint = true;
+
+  private List<PageElement> newOverlappedPages;

Review Comment:
   Resolved. It has been renamed to `candidateOverlappedPages` and put into `FastCompactionPerformerSubTask` class as a global variable. Pages in this list will be sequentially judged whether there is a real overlap to choose whether to put them in the point priority reader to deserialize or directly flush to chunk writer. During the process of compacting overlapped page, there may be new overlapped pages added into this list.



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

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

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


[GitHub] [iotdb] THUMarkLau commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
THUMarkLau commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r998925243


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/FastCompactionPerformer.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.performer.impl;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.AlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.NonAlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+public class FastCompactionPerformer implements ICrossCompactionPerformer {
+  private final Logger LOGGER = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+  private List<TsFileResource> seqFiles;
+
+  private List<TsFileResource> unseqFiles;
+
+  private List<TsFileResource> sortedSourceFiles = new ArrayList<>();
+
+  private static final int subTaskNum =
+      IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
+
+  public Map<TsFileResource, TsFileSequenceReader> readerCacheMap = new ConcurrentHashMap<>();
+
+  private CompactionTaskSummary summary;
+
+  private List<TsFileResource> targetFiles;
+
+  public Map<TsFileResource, List<Modification>> modificationCache = new ConcurrentHashMap<>();
+
+  public FastCompactionPerformer(
+      List<TsFileResource> seqFiles,
+      List<TsFileResource> unseqFiles,
+      List<TsFileResource> targetFiles) {
+    this.seqFiles = seqFiles;
+    this.unseqFiles = unseqFiles;
+    this.targetFiles = targetFiles;
+  }
+
+  public FastCompactionPerformer() {}
+
+  @Override
+  public void perform()
+      throws IOException, MetadataException, StorageEngineException, InterruptedException {
+    try (MultiTsFileDeviceIterator deviceIterator =
+            new MultiTsFileDeviceIterator(seqFiles, unseqFiles, readerCacheMap);
+        FastCrossCompactionWriter compactionWriter =
+            new FastCrossCompactionWriter(targetFiles, seqFiles)) {
+      while (deviceIterator.hasNextDevice()) {
+        checkThreadInterrupted();
+        Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
+        String device = deviceInfo.left;
+        boolean isAligned = deviceInfo.right;
+
+        // sort the resources by the start time of current device from old to new, and remove
+        // resource that does not contain the current device. Notice: when the level of time index
+        // is file, there will be a false positive judgment problem, that is, the device does not
+        // actually exist but the judgment return device being existed.
+        sortedSourceFiles.addAll(seqFiles);
+        sortedSourceFiles.addAll(unseqFiles);
+        sortedSourceFiles.removeIf(x -> !x.mayContainsDevice(device));
+        sortedSourceFiles.sort(Comparator.comparingLong(x -> x.getStartTime(device)));
+
+        compactionWriter.startChunkGroup(device, isAligned);
+
+        if (isAligned) {
+          compactAlignedSeries(device, deviceIterator, compactionWriter);
+        } else {
+          compactNonAlignedSeries(device, deviceIterator, compactionWriter);
+        }
+
+        compactionWriter.endChunkGroup();
+        // update resource of the current device and check whether to flush chunk metadata or not
+        compactionWriter.checkAndMayFlushChunkMetadata();
+        sortedSourceFiles.clear();
+      }
+      compactionWriter.endFile();
+      CompactionUtils.updatePlanIndexes(targetFiles, seqFiles, unseqFiles);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    } finally {
+      // readers of source files have been closed in MultiTsFileDeviceIterator
+      // clean cache
+      sortedSourceFiles = null;
+      readerCacheMap = null;
+      modificationCache = null;
+    }
+  }
+
+  private void compactAlignedSeries(
+      String deviceId,
+      MultiTsFileDeviceIterator deviceIterator,
+      FastCrossCompactionWriter fastCrossCompactionWriter)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+    Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap =
+        new HashMap<>();
+    List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
+
+    // Get all value measurements and their schemas of the current device. Also get start offset and
+    // end offset of each timeseries metadata, in order to facilitate the reading of chunkMetadata
+    // directly by this offset later. Instead of deserializing chunk metadata later, we need to
+    // deserialize chunk metadata here to get the schemas of all value measurements, because we
+    // should get schemas of all value measurement to startMeasruement() and compaction process is
+    // to read a batch of overlapped files each time, and we cannot make sure if the first batch of
+    // overlapped tsfiles contain all the value measurements.
+    for (Map.Entry<String, Pair<MeasurementSchema, Map<TsFileResource, Pair<Long, Long>>>> entry :
+        deviceIterator.getTimeseriesSchemaAndMetadataOffsetOfCurrentDevice().entrySet()) {
+      if (!entry.getKey().equals("")) {

Review Comment:
   Do use a string here, put it into IoTDBConstant and use a constant.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/FastCompactionPerformer.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.performer.impl;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.AlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.NonAlignedFastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+public class FastCompactionPerformer implements ICrossCompactionPerformer {
+  private final Logger LOGGER = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+  private List<TsFileResource> seqFiles;
+
+  private List<TsFileResource> unseqFiles;
+
+  private List<TsFileResource> sortedSourceFiles = new ArrayList<>();
+
+  private static final int subTaskNum =
+      IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
+
+  public Map<TsFileResource, TsFileSequenceReader> readerCacheMap = new ConcurrentHashMap<>();
+
+  private CompactionTaskSummary summary;
+
+  private List<TsFileResource> targetFiles;
+
+  public Map<TsFileResource, List<Modification>> modificationCache = new ConcurrentHashMap<>();
+
+  public FastCompactionPerformer(
+      List<TsFileResource> seqFiles,
+      List<TsFileResource> unseqFiles,
+      List<TsFileResource> targetFiles) {
+    this.seqFiles = seqFiles;
+    this.unseqFiles = unseqFiles;
+    this.targetFiles = targetFiles;
+  }
+
+  public FastCompactionPerformer() {}
+
+  @Override
+  public void perform()
+      throws IOException, MetadataException, StorageEngineException, InterruptedException {
+    try (MultiTsFileDeviceIterator deviceIterator =
+            new MultiTsFileDeviceIterator(seqFiles, unseqFiles, readerCacheMap);
+        FastCrossCompactionWriter compactionWriter =
+            new FastCrossCompactionWriter(targetFiles, seqFiles)) {
+      while (deviceIterator.hasNextDevice()) {
+        checkThreadInterrupted();
+        Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
+        String device = deviceInfo.left;
+        boolean isAligned = deviceInfo.right;
+
+        // sort the resources by the start time of current device from old to new, and remove
+        // resource that does not contain the current device. Notice: when the level of time index
+        // is file, there will be a false positive judgment problem, that is, the device does not
+        // actually exist but the judgment return device being existed.
+        sortedSourceFiles.addAll(seqFiles);
+        sortedSourceFiles.addAll(unseqFiles);
+        sortedSourceFiles.removeIf(x -> !x.mayContainsDevice(device));
+        sortedSourceFiles.sort(Comparator.comparingLong(x -> x.getStartTime(device)));
+
+        compactionWriter.startChunkGroup(device, isAligned);
+
+        if (isAligned) {
+          compactAlignedSeries(device, deviceIterator, compactionWriter);
+        } else {
+          compactNonAlignedSeries(device, deviceIterator, compactionWriter);
+        }
+
+        compactionWriter.endChunkGroup();
+        // update resource of the current device and check whether to flush chunk metadata or not
+        compactionWriter.checkAndMayFlushChunkMetadata();
+        sortedSourceFiles.clear();
+      }
+      compactionWriter.endFile();
+      CompactionUtils.updatePlanIndexes(targetFiles, seqFiles, unseqFiles);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    } finally {
+      // readers of source files have been closed in MultiTsFileDeviceIterator
+      // clean cache
+      sortedSourceFiles = null;
+      readerCacheMap = null;
+      modificationCache = null;
+    }
+  }
+
+  private void compactAlignedSeries(
+      String deviceId,
+      MultiTsFileDeviceIterator deviceIterator,
+      FastCrossCompactionWriter fastCrossCompactionWriter)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+    Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap =
+        new HashMap<>();
+    List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
+
+    // Get all value measurements and their schemas of the current device. Also get start offset and
+    // end offset of each timeseries metadata, in order to facilitate the reading of chunkMetadata
+    // directly by this offset later. Instead of deserializing chunk metadata later, we need to
+    // deserialize chunk metadata here to get the schemas of all value measurements, because we
+    // should get schemas of all value measurement to startMeasruement() and compaction process is
+    // to read a batch of overlapped files each time, and we cannot make sure if the first batch of
+    // overlapped tsfiles contain all the value measurements.
+    for (Map.Entry<String, Pair<MeasurementSchema, Map<TsFileResource, Pair<Long, Long>>>> entry :
+        deviceIterator.getTimeseriesSchemaAndMetadataOffsetOfCurrentDevice().entrySet()) {
+      if (!entry.getKey().equals("")) {
+        measurementSchemas.add(entry.getValue().left);
+      }
+      timeseriesMetadataOffsetMap.put(entry.getKey(), entry.getValue().right);
+    }
+
+    new AlignedFastCompactionPerformerSubTask(
+            fastCrossCompactionWriter,
+            timeseriesMetadataOffsetMap,
+            measurementSchemas,
+            readerCacheMap,
+            modificationCache,
+            sortedSourceFiles,
+            deviceId,
+            0)
+        .call();
+  }
+
+  private void compactNonAlignedSeries(
+      String deviceID,
+      MultiTsFileDeviceIterator deviceIterator,
+      FastCrossCompactionWriter fastCrossCompactionWriter)
+      throws IOException, InterruptedException {
+    // measurement -> tsfile resource -> timeseries metadata <startOffset, endOffset>
+    // Get all measurements of the current device. Also get start offset and end offset of each
+    // timeseries metadata, in order to facilitate the reading of chunkMetadata directly by this
+    // offset later. Here we don't need to deserialize chunk metadata, we can deserialize them and
+    // get their schema later.
+    Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap =
+        deviceIterator.getTimeseriesMetadataOffsetOfCurrentDevice();
+
+    List<String> allMeasurements = new ArrayList<>(timeseriesMetadataOffsetMap.keySet());
+
+    int subTaskNums = Math.min(allMeasurements.size(), subTaskNum);
+
+    // assign all measurements to different sub tasks
+    List<String>[] measurementsForEachSubTask = new ArrayList[subTaskNums];

Review Comment:
   Since the list is only used to traverse sequentially, LinkedList may be better than ArrayList.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/reader/PointPriorityReader.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.reader;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.FastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PointElement;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+/**
+ * This reader is used to deduplicate and organize overlapping pages, and read out points in order.
+ * It is used for compaction.
+ */
+public class PointPriorityReader {
+  private long lastTime;
+
+  private final PriorityQueue<PointElement> pointQueue;
+
+  private final FastCompactionPerformerSubTask.RemovePage removePage;
+
+  private Pair<Long, Object> currentPoint;
+
+  private boolean isNewPoint = true;
+
+  private List<PageElement> newOverlappedPages;

Review Comment:
   The usage and update of this variable is outside this class, it should be moved outside.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractInnerCompactionWriter.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.writer;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.rescon.SystemInfo;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumn;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import java.io.IOException;
+import java.util.List;
+
+public abstract class AbstractInnerCompactionWriter extends AbstractCompactionWriter {
+  protected TsFileIOWriter fileWriter;
+
+  protected boolean isEmptyFile;
+
+  protected TsFileResource targetResource;
+
+  public AbstractInnerCompactionWriter(TsFileResource targetFileResource) throws IOException {
+    long sizeForFileWriter =
+        (long)
+            (SystemInfo.getInstance().getMemorySizeForCompaction()
+                / IoTDBDescriptor.getInstance().getConfig().getConcurrentCompactionThread()
+                * IoTDBDescriptor.getInstance().getConfig().getChunkMetadataSizeProportion());
+    this.fileWriter = new TsFileIOWriter(targetFileResource.getTsFile(), true, sizeForFileWriter);
+    this.targetResource = targetFileResource;
+    isEmptyFile = true;
+  }
+
+  @Override
+  public void startChunkGroup(String deviceId, boolean isAlign) throws IOException {
+    fileWriter.startChunkGroup(deviceId);
+    this.isAlign = isAlign;
+    this.deviceId = deviceId;
+  }
+
+  @Override
+  public void endChunkGroup() throws IOException {
+    fileWriter.endChunkGroup();
+  }
+
+  @Override
+  public void endMeasurement(int subTaskId) throws IOException {
+    flushChunkToFileWriter(fileWriter, chunkWriters[subTaskId]);
+  }
+
+  @Override
+  public abstract void write(long timestamp, Object value, int subTaskId) throws IOException;
+
+  @Override
+  public abstract void write(TimeColumn timestamps, Column[] columns, int subTaskId, int batchSize)
+      throws IOException;
+
+  @Override
+  public void endFile() throws IOException {
+    fileWriter.endFile();
+    if (isEmptyFile) {
+      fileWriter.getFile().delete();
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (fileWriter != null && fileWriter.canWrite()) {
+      fileWriter.close();
+    }
+    fileWriter = null;
+  }
+
+  @Override
+  public void checkAndMayFlushChunkMetadata() throws IOException {
+    // Before flushing chunk metadatas, we use chunk metadatas in tsfile io writer to update start
+    // time and end time in resource.
+    List<TimeseriesMetadata> timeseriesMetadatasOfCurrentDevice =
+        fileWriter.getDeviceTimeseriesMetadataMap().get(deviceId);

Review Comment:
   Using this function is not a good idea, it may read all the chunk metadata from .meta file, which is costly.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/NonAlignedFastCompactionPerformerSubTask.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.task;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.cross.utils.ChunkMetadataElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.FileElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.writer.FastCrossCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class NonAlignedFastCompactionPerformerSubTask extends FastCompactionPerformerSubTask {
+  // measurements of the current device to be compacted, which is assigned to the current sub thread
+  private final List<String> measurements;
+
+  private String currentMeasurement;
+
+  boolean hasStartMeasurement = false;
+
+  public NonAlignedFastCompactionPerformerSubTask(
+      FastCrossCompactionWriter compactionWriter,
+      Map<String, Map<TsFileResource, Pair<Long, Long>>> timeseriesMetadataOffsetMap,
+      List<String> measurements,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      List<TsFileResource> sortedSourceFiles,
+      String deviceId,
+      int subTaskId) {
+    super(
+        compactionWriter,
+        timeseriesMetadataOffsetMap,
+        readerCacheMap,
+        modificationCacheMap,
+        sortedSourceFiles,
+        deviceId,
+        false,
+        subTaskId);
+    this.measurements = measurements;
+  }
+
+  @Override
+  public Void call()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (String measurement : measurements) {

Review Comment:
   SubTask is responsible for multi-series compaction, but some member variable in this class is used to execute single series compaction. I think the responsibilities of this class are a bit confusing, so we can extract the parts that perform a single series compaction into a class and put the associated variables into that class.



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/PointElement.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+
+public class PointElement {
+  public long timestamp;
+  public int priority;
+  public Pair<Long, Object> timeValuePair;

Review Comment:
   Use TimeValuePair instead of Pair



##########
tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java:
##########
@@ -199,6 +209,63 @@ private PageReader constructPageReaderForNextPage(PageHeader pageHeader) throws
     return reader;
   }
 
+  /**
+   * Read page data without uncompressing it.
+   *
+   * @return compressed page data
+   */
+  public ByteBuffer readPageDataWithoutUncompressing(PageHeader pageHeader) throws IOException {
+    int compressedPageBodyLength = pageHeader.getCompressedSize();
+    byte[] compressedPageBody = new byte[compressedPageBodyLength];
+
+    // doesn't has a complete page body
+    if (compressedPageBodyLength > chunkDataBuffer.remaining()) {
+      throw new IOException(
+          "do not has a complete page body. Expected:"
+              + compressedPageBodyLength
+              + ". Actual:"
+              + chunkDataBuffer.remaining());
+    }
+
+    chunkDataBuffer.get(compressedPageBody);
+    return ByteBuffer.wrap(compressedPageBody);
+  }
+
+  /**
+   * Read data from compressed page data. Uncompress the page and decode it to batch data.
+   *
+   * @param compressedPageData Compressed page data
+   */
+  public TsBlock readPageData(PageHeader pageHeader, ByteBuffer compressedPageData)
+      throws IOException {
+    // uncompress page data

Review Comment:
   Line 241-258 is duplicated with AlignedChunkReader::uncompressPageData



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/reader/PointPriorityReader.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.reader;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.FastCompactionPerformerSubTask;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PageElement;
+import org.apache.iotdb.db.engine.compaction.cross.utils.PointElement;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.read.common.IBatchDataIterator;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+/**
+ * This reader is used to deduplicate and organize overlapping pages, and read out points in order.
+ * It is used for compaction.
+ */
+public class PointPriorityReader {
+  private long lastTime;
+
+  private final PriorityQueue<PointElement> pointQueue;
+
+  private final FastCompactionPerformerSubTask.RemovePage removePage;
+
+  private Pair<Long, Object> currentPoint;
+
+  private boolean isNewPoint = true;

Review Comment:
   ```suggestion
     private boolean shouldReadNextPoint = true;
   ```



##########
server/src/main/java/org/apache/iotdb/db/tools/validate/TsFileValidationTool.java:
##########
@@ -72,7 +72,7 @@
  */
 public class TsFileValidationTool {
   // print detail type of overlap or not
-  private static boolean printDetails = false;
+  private static boolean printDetails = true;

Review Comment:
   set to false



##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCrossCompactionWriter.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.writer;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.rescon.SystemInfo;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumn;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public abstract class AbstractCrossCompactionWriter extends AbstractCompactionWriter {
+
+  // target fileIOWriters
+  protected List<TsFileIOWriter> targetFileWriters = new ArrayList<>();
+
+  // source tsfiles
+  private List<TsFileResource> seqTsFileResources;
+
+  // Each sub task has its corresponding seq file index.
+  // The index of the array corresponds to subTaskId.
+  protected int[] seqFileIndexArray = new int[subTaskNum];
+
+  // device end time in each source seq file
+  protected final long[] currentDeviceEndTime;
+
+  // whether each target file is empty or not
+  protected final boolean[] isEmptyFile;
+
+  // whether each target file has device data or not
+  protected final boolean[] isDeviceExistedInTargetFiles;
+
+  // current chunk group header size
+  private int chunkGroupHeaderSize;
+
+  protected List<TsFileResource> targetResources;
+
+  public AbstractCrossCompactionWriter(
+      List<TsFileResource> targetResources, List<TsFileResource> seqFileResources)
+      throws IOException {
+    currentDeviceEndTime = new long[seqFileResources.size()];
+    isEmptyFile = new boolean[seqFileResources.size()];
+    isDeviceExistedInTargetFiles = new boolean[targetResources.size()];
+    long memorySizeForEachWriter =
+        (long)
+            (SystemInfo.getInstance().getMemorySizeForCompaction()
+                / IoTDBDescriptor.getInstance().getConfig().getConcurrentCompactionThread()
+                * IoTDBDescriptor.getInstance().getConfig().getChunkMetadataSizeProportion()
+                / targetResources.size());
+    for (int i = 0; i < targetResources.size(); i++) {
+      this.targetFileWriters.add(
+          new TsFileIOWriter(targetResources.get(i).getTsFile(), true, memorySizeForEachWriter));
+      isEmptyFile[i] = true;
+    }
+    this.seqTsFileResources = seqFileResources;
+    this.targetResources = targetResources;
+  }
+
+  @Override
+  public void startChunkGroup(String deviceId, boolean isAlign) throws IOException {
+    this.deviceId = deviceId;
+    this.isAlign = isAlign;
+    this.seqFileIndexArray = new int[subTaskNum];
+    checkIsDeviceExistAndGetDeviceEndTime();
+    for (int i = 0; i < targetFileWriters.size(); i++) {
+      chunkGroupHeaderSize = targetFileWriters.get(i).startChunkGroup(deviceId);
+    }
+  }
+
+  @Override
+  public void endChunkGroup() throws IOException {
+    for (int i = 0; i < seqTsFileResources.size(); i++) {
+      TsFileIOWriter targetFileWriter = targetFileWriters.get(i);
+      if (isDeviceExistedInTargetFiles[i]) {
+        targetFileWriter.endChunkGroup();
+      } else {
+        targetFileWriter.truncate(targetFileWriter.getPos() - chunkGroupHeaderSize);
+      }
+      isDeviceExistedInTargetFiles[i] = false;
+    }
+    seqFileIndexArray = null;
+  }
+
+  @Override
+  public void endMeasurement(int subTaskId) throws IOException {
+    flushChunkToFileWriter(
+        targetFileWriters.get(seqFileIndexArray[subTaskId]), chunkWriters[subTaskId]);
+    seqFileIndexArray[subTaskId] = 0;
+  }
+
+  @Override
+  public void write(long timestamp, Object value, int subTaskId) throws IOException {
+    checkTimeAndMayFlushChunkToCurrentFile(timestamp, subTaskId);
+    int fileIndex = seqFileIndexArray[subTaskId];
+    writeDataPoint(timestamp, value, chunkWriters[subTaskId]);
+    chunkPointNumArray[subTaskId]++;
+    checkChunkSizeAndMayOpenANewChunk(
+        targetFileWriters.get(fileIndex), chunkWriters[subTaskId], subTaskId, true);
+    isDeviceExistedInTargetFiles[fileIndex] = true;
+    isEmptyFile[fileIndex] = false;
+  }
+
+  /** Write data in batch, only used for aligned device. */
+  @Override
+  public abstract void write(TimeColumn timestamps, Column[] columns, int subTaskId, int batchSize)
+      throws IOException;
+
+  @Override
+  public void endFile() throws IOException {
+    for (int i = 0; i < isEmptyFile.length; i++) {
+      targetFileWriters.get(i).endFile();
+      // delete empty target file
+      if (isEmptyFile[i]) {
+        targetFileWriters.get(i).getFile().delete();
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    for (TsFileIOWriter targetWriter : targetFileWriters) {
+      if (targetWriter != null && targetWriter.canWrite()) {
+        targetWriter.close();
+      }
+    }
+    targetFileWriters = null;
+    seqTsFileResources = null;
+  }
+
+  @Override
+  public void checkAndMayFlushChunkMetadata() throws IOException {
+    for (int i = 0; i < targetFileWriters.size(); i++) {
+      TsFileIOWriter fileIOWriter = targetFileWriters.get(i);
+      TsFileResource resource = targetResources.get(i);
+      // Before flushing chunk metadatas, we use chunk metadatas in tsfile io writer to update start
+      // time and end time in resource.
+      List<TimeseriesMetadata> timeseriesMetadatasOfCurrentDevice =
+          fileIOWriter.getDeviceTimeseriesMetadataMap().get(deviceId);

Review Comment:
   Calling this function is costly.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1030071871


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/SeriesCompactionExecutor.java:
##########
@@ -0,0 +1,601 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.compaction.reader.PointPriorityReader;
+import org.apache.iotdb.db.engine.compaction.task.SubCompactionTaskSummary;
+import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.tsfile.exception.write.PageException;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+public abstract class SeriesCompactionExecutor {
+  protected enum ModifiedStatus {
+    ALL_DELETED,
+    PARTIAL_DELETED,
+    NONE_DELETED;
+  }
+
+  @FunctionalInterface
+  public interface RemovePage {
+    void call(PageElement pageElement)
+        throws WriteProcessException, IOException, IllegalPathException;
+  }
+
+  private final SubCompactionTaskSummary summary;
+
+  // source files which are sorted by the start time of current device from old to new. Notice: If
+  // the type of timeIndex is FileTimeIndex, it may contain resources in which the current device
+  // does not exist.
+  protected List<FileElement> fileList = new ArrayList<>();;
+
+  protected final PriorityQueue<ChunkMetadataElement> chunkMetadataQueue;
+
+  protected final PriorityQueue<PageElement> pageQueue;
+
+  protected AbstractCompactionWriter compactionWriter;
+
+  protected int subTaskId;
+
+  protected Map<TsFileResource, TsFileSequenceReader> readerCacheMap;
+
+  private final Map<TsFileResource, List<Modification>> modificationCacheMap;
+
+  private final PointPriorityReader pointPriorityReader = new PointPriorityReader(this::removePage);
+
+  protected String deviceId;
+
+  // Pages in this list will be sequentially judged whether there is a real overlap to choose
+  // whether to put them in the point priority reader to deserialize or directly flush to chunk
+  // writer. During the process of compacting overlapped page, there may be new overlapped pages
+  // added into this list.
+  private final List<PageElement> candidateOverlappedPages = new ArrayList<>();
+
+  public SeriesCompactionExecutor(
+      AbstractCompactionWriter compactionWriter,
+      Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
+      Map<TsFileResource, List<Modification>> modificationCacheMap,
+      String deviceId,
+      int subTaskId,
+      SubCompactionTaskSummary summary) {
+    this.compactionWriter = compactionWriter;
+    this.subTaskId = subTaskId;
+    this.deviceId = deviceId;
+    this.readerCacheMap = readerCacheMap;
+    this.modificationCacheMap = modificationCacheMap;
+    this.summary = summary;
+
+    chunkMetadataQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+
+    pageQueue =
+        new PriorityQueue<>(
+            (o1, o2) -> {
+              int timeCompare = Long.compare(o1.startTime, o2.startTime);
+              return timeCompare != 0 ? timeCompare : Long.compare(o2.priority, o1.priority);
+            });
+  }
+
+  public abstract void excute()
+      throws PageException, IllegalPathException, IOException, WriteProcessException;
+
+  protected abstract void compactFiles()
+      throws PageException, IOException, WriteProcessException, IllegalPathException;
+
+  /** Compact chunks in chunk metadata queue. */
+  protected void compactChunks()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!chunkMetadataQueue.isEmpty()) {
+      ChunkMetadataElement firstChunkMetadataElement = chunkMetadataQueue.peek();
+      List<ChunkMetadataElement> overlappedChunkMetadatas =
+          findOverlapChunkMetadatas(firstChunkMetadataElement);
+      boolean isChunkOverlap = overlappedChunkMetadatas.size() > 1;
+      boolean isModified = isChunkModified(firstChunkMetadataElement);
+
+      if (isChunkOverlap || isModified) {
+        // has overlap or modified chunk, then deserialize it
+        summary.CHUNK_OVERLAP_OR_MODIFIED += overlappedChunkMetadatas.size();
+        compactWithOverlapChunks(overlappedChunkMetadatas);
+      } else {
+        // has none overlap or modified chunk, flush it to file writer directly
+        summary.CHUNK_NONE_OVERLAP += 1;
+        compactWithNonOverlapChunk(firstChunkMetadataElement);
+      }
+    }
+  }
+
+  /**
+   * Deserialize chunks and start compacting pages. Compact a series of chunks that overlap with
+   * each other. Eg: The parameters are chunk 1 and chunk 2, that is, chunk 1 only overlaps with
+   * chunk 2, while chunk 2 overlap with chunk 3, chunk 3 overlap with chunk 4,and so on, there are
+   * 10 chunks in total. This method will merge all 10 chunks.
+   */
+  private void compactWithOverlapChunks(List<ChunkMetadataElement> overlappedChunkMetadatas)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    for (ChunkMetadataElement overlappedChunkMetadata : overlappedChunkMetadatas) {
+      readChunk(overlappedChunkMetadata);
+      deserializeChunkIntoQueue(overlappedChunkMetadata);
+    }
+    compactPages();
+  }
+
+  /**
+   * Flush chunk to target file directly. If the end time of chunk exceeds the end time of file or
+   * the unsealed chunk is too small, then deserialize it.
+   */
+  private void compactWithNonOverlapChunk(ChunkMetadataElement chunkMetadataElement)
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    readChunk(chunkMetadataElement);
+    boolean success;
+    if (chunkMetadataElement.chunkMetadata instanceof AlignedChunkMetadata) {
+      success =
+          compactionWriter.flushAlignedChunk(
+              chunkMetadataElement.chunk,
+              ((AlignedChunkMetadata) chunkMetadataElement.chunkMetadata).getTimeChunkMetadata(),
+              chunkMetadataElement.valueChunks,
+              ((AlignedChunkMetadata) chunkMetadataElement.chunkMetadata)
+                  .getValueChunkMetadataList(),
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushNonAlignedChunk(
+              chunkMetadataElement.chunk,
+              (ChunkMetadata) chunkMetadataElement.chunkMetadata,
+              subTaskId);
+    }
+    if (success) {
+      // flush chunk successfully, then remove this chunk
+      removeChunk(chunkMetadataQueue.peek());
+    } else {
+      // unsealed chunk is not large enough or chunk.endTime > file.endTime, then deserialize chunk
+      summary.CHUNK_NONE_OVERLAP_BUT_DESERIALIZE += 1;
+      deserializeChunkIntoQueue(chunkMetadataElement);
+      compactPages();
+    }
+  }
+
+  abstract void deserializeChunkIntoQueue(ChunkMetadataElement chunkMetadataElement)
+      throws IOException;
+
+  abstract void readChunk(ChunkMetadataElement chunkMetadataElement) throws IOException;
+
+  /** Deserialize files into chunk metadatas and put them into the chunk metadata queue. */
+  abstract void deserializeFileIntoQueue(List<FileElement> fileElements)
+      throws IOException, IllegalPathException;
+
+  /** Compact pages in page queue. */
+  private void compactPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    while (!pageQueue.isEmpty()) {
+      PageElement firstPageElement = pageQueue.peek();
+      ModifiedStatus modifiedStatus = isPageModified(firstPageElement);
+
+      if (modifiedStatus == ModifiedStatus.ALL_DELETED) {
+        // all data on this page has been deleted, remove it
+        removePage(firstPageElement);
+        continue;
+      }
+
+      List<PageElement> overlapPages = findOverlapPages(firstPageElement);
+      boolean isPageOverlap = overlapPages.size() > 1;
+
+      if (isPageOverlap || modifiedStatus == ModifiedStatus.PARTIAL_DELETED) {
+        // has overlap or modified pages, then deserialize it
+        summary.PAGE_OVERLAP_OR_MODIFIED += 1;
+        pointPriorityReader.addNewPage(overlapPages.remove(0));
+        addOverlappedPagesIntoList(overlapPages);
+        compactWithOverlapPages();
+      } else {
+        // has none overlap or modified pages, flush it to chunk writer directly
+        summary.PAGE_NONE_OVERLAP += 1;
+        compactWithNonOverlapPage(firstPageElement);
+      }
+    }
+  }
+
+  private void compactWithNonOverlapPage(PageElement pageElement)
+      throws PageException, IOException, WriteProcessException, IllegalPathException {
+    boolean success;
+    if (pageElement.iChunkReader instanceof AlignedChunkReader) {
+      success =
+          compactionWriter.flushAlignedPage(
+              pageElement.pageData,
+              pageElement.pageHeader,
+              pageElement.valuePageDatas,
+              pageElement.valuePageHeaders,
+              subTaskId);
+    } else {
+      success =
+          compactionWriter.flushNonAlignedPage(
+              pageElement.pageData, pageElement.pageHeader, subTaskId);
+    }
+    if (success) {
+      // flush the page successfully, then remove this page
+      removePage(pageElement);
+    } else {
+      // unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
+      summary.PAGE_NONE_OVERLAP_BUT_DESERIALIZE += 1;
+      pointPriorityReader.addNewPage(pageElement);
+
+      // write data points of the current page into chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().getTimestamp()
+              <= pageElement.pageHeader.getEndTime()) {
+        compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+        pointPriorityReader.next();
+      }
+    }
+  }
+
+  /**
+   * Compact a series of pages that overlap with each other. Eg: The parameters are page 1 and page
+   * 2, that is, page 1 only overlaps with page 2, while page 2 overlap with page 3, page 3 overlap
+   * with page 4,and so on, there are 10 pages in total. This method will compact all 10 pages.
+   * Pages in the candidate overlapped pages list will be sequentially judged whether there is a
+   * real overlap, if so, it will be put into the point priority reader and deserialized; if not, it
+   * means that the page is located in a gap inside another pages, and it can be directly flushed to
+   * chunk writer. There will be new overlapped pages added into the list during the process of
+   * compacting overlapped pages. Notice: for a real overlap page, it will be removed from candidate
+   * list after it has been adding into point priority reader and deserializing. For a fake overlap
+   * page, it will be removed from candidate list after it has been flushing to chunk writer
+   * completely.
+   */
+  private void compactWithOverlapPages()
+      throws IOException, PageException, WriteProcessException, IllegalPathException {
+    checkAndCompactOverlappePages();
+
+    // write remaining data points, of which point.time >= the last overlapped page.startTime
+    while (pointPriorityReader.hasNext()) {
+      // write data point to chunk writer
+
+      compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+      pointPriorityReader.next();
+      if (candidateOverlappedPages.size() > 0) {
+        // finish compacting the first page or there are new chunks being deserialized and find
+        // the new overlapped pages, then start compacting them
+        checkAndCompactOverlappePages();
+      }
+    }
+  }
+
+  /**
+   * Check whether the page is true overlap or fake overlap. If a page is located in the gap of
+   * another page, then this page is fake overlap, which can be flushed to chunk writer directly.
+   * Otherwise, deserialize this page into point priority reader.
+   */
+  private void checkAndCompactOverlappePages()
+      throws IllegalPathException, IOException, WriteProcessException, PageException {
+    // write point.time < the last overlapped page.startTime
+    while (candidateOverlappedPages.size() > 0) {
+      PageElement nextPageElement = candidateOverlappedPages.get(0);
+
+      int oldSize = candidateOverlappedPages.size();
+      // write currentPage.point.time < nextPage.startTime to chunk writer
+      while (pointPriorityReader.hasNext()
+          && pointPriorityReader.currentPoint().getTimestamp() < nextPageElement.startTime) {
+        // write data point to chunk writer
+        compactionWriter.write(pointPriorityReader.currentPoint(), subTaskId);
+        pointPriorityReader.next();
+        if (candidateOverlappedPages.size() > oldSize) {
+          // during the process of writing overlapped points, if the first page is compacted
+          // completely or a new chunk is deserialized, there may be new pages overlapped with the
+          // first page in page queue which are added into the list. If so, the next overlapped
+          // page in the list may be changed, so we should re-get next overlap page here.
+          oldSize = candidateOverlappedPages.size();
+          nextPageElement = candidateOverlappedPages.get(0);
+        }
+      }
+
+      ModifiedStatus nextPageModifiedStatus = isPageModified(nextPageElement);
+
+      if (nextPageModifiedStatus == ModifiedStatus.ALL_DELETED) {
+        // all data on next page has been deleted, remove it
+        removePage(nextPageElement);
+      } else {
+        boolean isNextPageOverlap =
+            (pointPriorityReader.hasNext()
+                    && pointPriorityReader.currentPoint().getTimestamp()
+                        <= nextPageElement.pageHeader.getEndTime())
+                || isPageOverlap(nextPageElement);
+
+        if (isNextPageOverlap || nextPageModifiedStatus == ModifiedStatus.PARTIAL_DELETED) {
+          // has overlap or modified pages, then deserialize it
+          pointPriorityReader.addNewPage(nextPageElement);
+        } else {
+          // has none overlap or modified pages, flush it to chunk writer directly
+          summary.PAGE_FAKE_OVERLAP += 1;
+          compactWithNonOverlapPage(nextPageElement);
+        }
+      }
+      candidateOverlappedPages.remove(0);
+    }
+  }
+
+  /**
+   * Add the new overlapped pages into the global list and sort it according to the startTime of the
+   * page from small to large, so that each page can be compacted in order. If the page has been
+   * deleted completely, we remove it.
+   */
+  private void addOverlappedPagesIntoList(List<PageElement> newOverlappedPages) {
+    summary.PAGE_OVERLAP_OR_MODIFIED += newOverlappedPages.size();
+    int oldSize = candidateOverlappedPages.size();
+    candidateOverlappedPages.addAll(newOverlappedPages);
+    if (oldSize != 0 && candidateOverlappedPages.size() > oldSize) {
+      // if there is no pages in the overlappedPages, then we don't need to sort it after adding the
+      // new overlapped pages, because newOverlappedPages is already sorted. When there is pages in
+      // list before and there is new pages added into list, then we need to sort it again.
+      // we should ensure that the list is ordered according to the startTime of the page from small
+      // to large, so that each page can be compacted in order
+      candidateOverlappedPages.sort(Comparator.comparingLong(o -> o.startTime));
+    }
+  }
+
+  /**
+   * Find overlaped pages which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the page from small to large, so that each page
+   * can be compacted in order.
+   */
+  private List<PageElement> findOverlapPages(PageElement page) {
+    List<PageElement> elements = new ArrayList<>();
+    long endTime = page.pageHeader.getEndTime();
+    for (PageElement element : pageQueue) {
+      if (element.startTime <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped chunks which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the chunk from small to large, so that each chunk
+   * can be compacted in order.
+   */
+  private List<ChunkMetadataElement> findOverlapChunkMetadatas(
+      ChunkMetadataElement chunkMetadataElement) {
+    List<ChunkMetadataElement> elements = new ArrayList<>();
+    long endTime = chunkMetadataElement.chunkMetadata.getEndTime();
+    for (ChunkMetadataElement element : chunkMetadataQueue) {
+      if (element.chunkMetadata.getStartTime() <= endTime) {
+        if (!element.isOverlaped) {
+          elements.add(element);
+          element.isOverlaped = true;
+        }
+      }
+    }
+    elements.sort(Comparator.comparingLong(o -> o.startTime));
+    return elements;
+  }
+
+  /**
+   * Find overlapped files which have not been selected. Notice: We must ensure that the returned
+   * list is ordered according to the startTime of the current device in the file from small to
+   * large, so that each file can be compacted in order.
+   */
+  protected List<FileElement> findOverlapFiles(FileElement file) {
+    List<FileElement> overlappedFiles = new ArrayList<>();
+    long endTime = file.resource.getEndTime(deviceId);
+    for (FileElement fileElement : fileList) {
+      if (fileElement.resource.getStartTime(deviceId) <= endTime) {
+        if (!fileElement.isOverlap) {
+          overlappedFiles.add(fileElement);
+          fileElement.isOverlap = true;
+        }
+      } else {
+        break;
+      }
+    }
+    return overlappedFiles;
+  }
+
+  /** Check is the page overlap with other pages later then the specific page in queue or not. */
+  private boolean isPageOverlap(PageElement pageElement) {
+    long endTime = pageElement.pageHeader.getEndTime();
+    long startTime = pageElement.startTime;
+    for (PageElement element : pageQueue) {
+      if (element.equals(pageElement)) {
+        continue;
+      }
+      // only check pages later than the specific page
+      if (element.startTime >= startTime && element.startTime <= endTime) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check whether the chunk is modified.
+   *
+   * <p>Notice: if is aligned chunk, return true if any of value chunk has data been deleted. Return
+   * false if and only if all value chunks has no data been deleted.
+   */
+  protected boolean isChunkModified(ChunkMetadataElement chunkMetadataElement) {

Review Comment:
   Resolved.



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

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

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


[GitHub] [iotdb] choubenson commented on a diff in pull request #7621: [IOTDB-3928][IOTDB-4097]New Compaction Performer —— Fast Compaction

Posted by GitBox <gi...@apache.org>.
choubenson commented on code in PR #7621:
URL: https://github.com/apache/iotdb/pull/7621#discussion_r1030071487


##########
server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/utils/ChunkMetadataElement.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+
+import java.util.List;
+
+public class ChunkMetadataElement {
+  public IChunkMetadata chunkMetadata;
+
+  public long priority;
+
+  public boolean isOverlaped = false;

Review Comment:
   Resolved.



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

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

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