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

jiangtian pushed a commit to branch force_ci/support_schema_evolution
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 3a059a5caec624e2e4cbd5fc73bacc3ca57e0928
Author: Tian Jiang <[email protected]>
AuthorDate: Tue Jan 13 18:02:05 2026 +0800

    fix compilation
---
 .../operator/source/AlignedSeriesScanUtil.java     |  12 +-
 .../execution/operator/source/FileLoaderUtils.java |  23 ++-
 .../execution/operator/source/SeriesScanUtil.java  |  45 +++-
 .../plan/planner/plan/node/PlanNodeType.java       |   2 +-
 .../db/storageengine/dataregion/DataRegion.java    |  78 ++++---
 .../performer/impl/FastCompactionPerformer.java    |  16 +-
 .../impl/ReadChunkCompactionPerformer.java         |   5 +-
 .../impl/ReadPointCompactionPerformer.java         |  10 +-
 .../subtask/FastCompactionPerformerSubTask.java    |  12 +-
 .../task/subtask/ReadPointPerformerSubTask.java    |   3 +-
 .../execute/utils/MultiTsFileDeviceIterator.java   |  34 ++--
 .../fast/FastAlignedSeriesCompactionExecutor.java  |  23 ++-
 .../executor/fast/SeriesCompactionExecutor.java    |   8 +-
 .../writer/AbstractInnerCompactionWriter.java      |   5 +-
 .../compaction/io/CompactionTsFileWriter.java      |  12 +-
 .../dataregion/modification/DeletionPredicate.java |   9 +-
 .../modification/TableDeletionEntry.java           |   6 +-
 .../dataregion/modification/TagPredicate.java      |  42 ++--
 .../dataregion/read/QueryDataSource.java           |  34 +++-
 .../dataregion/tsfile/TsFileResource.java          |  13 +-
 .../dataregion/tsfile/evolution/EvolvedSchema.java |  17 +-
 .../tsfile/evolution/EvolvedSchemaCache.java       |  30 +--
 .../dataregion/tsfile/fileset/TsFileSet.java       |  19 +-
 .../compaction/CompactionWithSevoTest.java         | 226 +++++++++++++--------
 .../FastInnerCompactionPerformerTest.java          |  21 --
 .../compaction/ReadChunkInnerCompactionTest.java   |  23 ---
 .../ReadPointCompactionPerformerTest.java          |  41 ++--
 27 files changed, 442 insertions(+), 327 deletions(-)

diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java
index c9d7dfeabbc..f17c01d2820 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java
@@ -78,7 +78,14 @@ public class AlignedSeriesScanUtil extends SeriesScanUtil {
       FragmentInstanceContext context,
       boolean queryAllSensors,
       List<TSDataType> givenDataTypes) {
-    this(seriesPath, scanOrder, scanOptions, context, queryAllSensors, 
givenDataTypes, Long.MAX_VALUE);
+    this(
+        seriesPath,
+        scanOrder,
+        scanOptions,
+        context,
+        queryAllSensors,
+        givenDataTypes,
+        Long.MAX_VALUE);
   }
 
   public AlignedSeriesScanUtil(
@@ -120,7 +127,8 @@ public class AlignedSeriesScanUtil extends SeriesScanUtil {
         context,
         scanOptions.getGlobalTimeFilter(),
         isSeq,
-        ignoreAllNullRows, maxTsFileSetEndVersion);
+        ignoreAllNullRows,
+        maxTsFileSetEndVersion);
   }
 
   @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java
index 2fe30a0fcae..ec8b73d1c4f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.db.queryengine.execution.operator.source;
 
-import java.util.stream.Collectors;
 import org.apache.iotdb.commons.path.AlignedFullPath;
 import org.apache.iotdb.commons.path.NonAlignedFullPath;
 import org.apache.iotdb.db.exception.ChunkTypeInconsistentException;
@@ -61,7 +60,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
-import org.apache.tsfile.write.schema.IMeasurementSchema;
 import java.util.stream.Collectors;
 
 import static com.google.common.base.Preconditions.checkArgument;
@@ -217,7 +215,12 @@ public class FileLoaderUtils {
       if (resource.isClosed()) {
         alignedTimeSeriesMetadata =
             loadAlignedTimeSeriesMetadataFromDisk(
-                resource, alignedPath, context, globalTimeFilter, 
ignoreAllNullRows, maxTsFileSetEndVersion);
+                resource,
+                alignedPath,
+                context,
+                globalTimeFilter,
+                ignoreAllNullRows,
+                maxTsFileSetEndVersion);
         SchemaUtils.changeAlignedMetadataModified(alignedTimeSeriesMetadata, 
targetDataTypeList);
       } else { // if the tsfile is unclosed, we just get it directly from 
TsFileResource
         loadFromMem = true;
@@ -303,9 +306,14 @@ public class FileLoaderUtils {
     EvolvedSchema evolvedSchema = 
resource.getMergedEvolvedSchema(maxTsFileSetEndVersion);
     if (evolvedSchema != null) {
       IDeviceID finalDeviceId = deviceId;
-      valueMeasurementList = valueMeasurementList.stream().map(m -> 
evolvedSchema.getOriginalColumnName(finalDeviceId.getTableName(), m)).collect(
-          Collectors.toList());
-      allSensors = allSensors.stream().map(m -> 
evolvedSchema.getOriginalColumnName(finalDeviceId.getTableName(), 
m)).collect(Collectors.toSet());
+      valueMeasurementList =
+          valueMeasurementList.stream()
+              .map(m -> 
evolvedSchema.getOriginalColumnName(finalDeviceId.getTableName(), m))
+              .collect(Collectors.toList());
+      allSensors =
+          allSensors.stream()
+              .map(m -> 
evolvedSchema.getOriginalColumnName(finalDeviceId.getTableName(), m))
+              .collect(Collectors.toSet());
       deviceId = evolvedSchema.rewriteToOriginal(deviceId);
     }
 
@@ -387,8 +395,7 @@ public class FileLoaderUtils {
 
     // deal with time column
     List<ModEntry> timeModifications =
-        context.getPathModifications(
-            resource, deviceID, timeColumnMetadata.getMeasurementId());
+        context.getPathModifications(resource, deviceID, 
timeColumnMetadata.getMeasurementId());
     // all rows are deleted, just return null to skip device data in this file
     if (ModificationUtils.isAllDeletedByMods(
         timeModifications,
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java
index 8bb2e70ef1d..fbf85e5cac6 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java
@@ -2401,26 +2401,38 @@ public class SeriesScanUtil implements Accountable {
 
     @Override
     public boolean hasNextSeqResource() {
-      while (dataSource.hasNextSeqResource(curSeqFileIndex, false, deviceID, 
maxTsFileSetEndVersion)) {
+      while (dataSource.hasNextSeqResource(
+          curSeqFileIndex, false, deviceID, maxTsFileSetEndVersion)) {
         if (dataSource.isSeqSatisfied(
-            deviceID, curSeqFileIndex, scanOptions.getGlobalTimeFilter(), 
false, maxTsFileSetEndVersion)) {
+            deviceID,
+            curSeqFileIndex,
+            scanOptions.getGlobalTimeFilter(),
+            false,
+            maxTsFileSetEndVersion)) {
           break;
         }
         curSeqFileIndex--;
       }
-      return dataSource.hasNextSeqResource(curSeqFileIndex, false, deviceID, 
maxTsFileSetEndVersion);
+      return dataSource.hasNextSeqResource(
+          curSeqFileIndex, false, deviceID, maxTsFileSetEndVersion);
     }
 
     @Override
     public boolean hasNextUnseqResource() {
-      while (dataSource.hasNextUnseqResource(curUnseqFileIndex, false, 
deviceID, maxTsFileSetEndVersion)) {
+      while (dataSource.hasNextUnseqResource(
+          curUnseqFileIndex, false, deviceID, maxTsFileSetEndVersion)) {
         if (dataSource.isUnSeqSatisfied(
-            deviceID, curUnseqFileIndex, scanOptions.getGlobalTimeFilter(), 
false, maxTsFileSetEndVersion)) {
+            deviceID,
+            curUnseqFileIndex,
+            scanOptions.getGlobalTimeFilter(),
+            false,
+            maxTsFileSetEndVersion)) {
           break;
         }
         curUnseqFileIndex++;
       }
-      return dataSource.hasNextUnseqResource(curUnseqFileIndex, false, 
deviceID, maxTsFileSetEndVersion);
+      return dataSource.hasNextUnseqResource(
+          curUnseqFileIndex, false, deviceID, maxTsFileSetEndVersion);
     }
 
     @Override
@@ -2530,9 +2542,14 @@ public class SeriesScanUtil implements Accountable {
 
     @Override
     public boolean hasNextSeqResource() {
-      while (dataSource.hasNextSeqResource(curSeqFileIndex, true, deviceID, 
maxTsFileSetEndVersion)) {
+      while (dataSource.hasNextSeqResource(
+          curSeqFileIndex, true, deviceID, maxTsFileSetEndVersion)) {
         if (dataSource.isSeqSatisfied(
-            deviceID, curSeqFileIndex, scanOptions.getGlobalTimeFilter(), 
false, maxTsFileSetEndVersion)) {
+            deviceID,
+            curSeqFileIndex,
+            scanOptions.getGlobalTimeFilter(),
+            false,
+            maxTsFileSetEndVersion)) {
           break;
         }
         curSeqFileIndex++;
@@ -2542,14 +2559,20 @@ public class SeriesScanUtil implements Accountable {
 
     @Override
     public boolean hasNextUnseqResource() {
-      while (dataSource.hasNextUnseqResource(curUnseqFileIndex, true, 
deviceID, maxTsFileSetEndVersion)) {
+      while (dataSource.hasNextUnseqResource(
+          curUnseqFileIndex, true, deviceID, maxTsFileSetEndVersion)) {
         if (dataSource.isUnSeqSatisfied(
-            deviceID, curUnseqFileIndex, scanOptions.getGlobalTimeFilter(), 
false, maxTsFileSetEndVersion)) {
+            deviceID,
+            curUnseqFileIndex,
+            scanOptions.getGlobalTimeFilter(),
+            false,
+            maxTsFileSetEndVersion)) {
           break;
         }
         curUnseqFileIndex++;
       }
-      return dataSource.hasNextUnseqResource(curUnseqFileIndex, true, 
deviceID, maxTsFileSetEndVersion);
+      return dataSource.hasNextUnseqResource(
+          curUnseqFileIndex, true, deviceID, maxTsFileSetEndVersion);
     }
 
     @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java
index bc8a992cbdd..f46b1f4b79f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java
@@ -372,7 +372,7 @@ public enum PlanNodeType {
         return RelationalDeleteDataNode.deserializeFromWAL(stream);
       case 2004:
         return ObjectNode.deserializeFromWAL(stream);
-      case 2005
+      case 2005:
         return EvolveSchemaNode.deserializeFromWAL(stream);
       case 2006:
         return PipeEnrichedEvolveSchemaNode.deserializeFromWAL(stream);
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java
index d57e7b2a904..4f6ddb7f46d 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java
@@ -116,11 +116,11 @@ import 
org.apache.iotdb.db.storageengine.dataregion.flush.TsFileFlushPolicy;
 import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable;
 import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor;
 import 
org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessorInfo;
-import org.apache.iotdb.db.storageengine.dataregion.modification.IDPredicate;
 import org.apache.iotdb.db.storageengine.dataregion.modification.ModEntry;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.ModEntry.ModType;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.TableDeletionEntry;
+import 
org.apache.iotdb.db.storageengine.dataregion.modification.TagPredicate.TagPredicateType;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.TreeDeletionEntry;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1;
 import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource;
@@ -3496,14 +3496,6 @@ public class DataRegion implements IDataRegionForQuery {
       throws IOException {
     Set<Pair<ModificationFile, ModEntry>> involvedModificationFiles = new 
HashSet<>();
     List<TsFileResource> deletedByFiles = new ArrayList<>();
-    boolean isDropMeasurementExist = false;
-    IDPredicate.IDPredicateType idPredicateType = null;
-
-    if (deletion instanceof TableDeletionEntry) {
-      TableDeletionEntry tableDeletionEntry = (TableDeletionEntry) deletion;
-      isDropMeasurementExist = 
!tableDeletionEntry.getPredicate().getMeasurementNames().isEmpty();
-      idPredicateType = tableDeletionEntry.getPredicate().getIdPredicateType();
-    }
 
     for (TsFileResource sealedTsFile : sealedTsFiles) {
       if (canSkipDelete(sealedTsFile, deletion)) {
@@ -3517,45 +3509,46 @@ public class DataRegion implements IDataRegionForQuery {
           && (deletion.getType() == ModType.TABLE_DELETION)) {
         ArrayDeviceTimeIndex deviceTimeIndex = (ArrayDeviceTimeIndex) 
timeIndex;
 
+        Set<IDeviceID> devicesInFile = deviceTimeIndex.getDevices();
+        boolean onlyOneTable = false;
+
         TableDeletionEntry tableDeletionEntry = (TableDeletionEntry) deletion;
         tableDeletionEntry =
             evolvedSchema != null
                 ? evolvedSchema.rewriteToOriginal(tableDeletionEntry)
                 : tableDeletionEntry;
-
-        Set<IDeviceID> devicesInFile = deviceTimeIndex.getDevices();
-        boolean onlyOneTable = false;
-
-        if (deletion instanceof TableDeletionEntry) {
-          TableDeletionEntry tableDeletionEntry = (TableDeletionEntry) 
deletion;
-          String tableName = tableDeletionEntry.getTableName();
-          long matchSize =
-              devicesInFile.stream()
-                  .filter(
-                      device -> {
-                        if (logger.isDebugEnabled()) {
-                          logger.debug(
-                              "device is {}, deviceTable is {}, 
tableDeletionEntry.getPredicate().matches(device) is {}",
-                              device,
-                              device.getTableName(),
-                              
tableDeletionEntry.getPredicate().matches(device));
-                        }
-                        return tableName.equals(device.getTableName())
-                            && 
tableDeletionEntry.getPredicate().matches(device);
-                      })
-                  .count();
-          onlyOneTable = matchSize == devicesInFile.size();
-          if (logger.isDebugEnabled()) {
-            logger.debug(
-                "tableName is {}, matchSize is {}, onlyOneTable is {}",
-                tableName,
-                matchSize,
-                onlyOneTable);
-          }
+        boolean isDropMeasurementExist =
+            !tableDeletionEntry.getPredicate().getMeasurementNames().isEmpty();
+        TagPredicateType tagPredicateType = 
tableDeletionEntry.getPredicate().getTagPredicateType();
+
+        String tableName = tableDeletionEntry.getTableName();
+        TableDeletionEntry finalTableDeletionEntry = tableDeletionEntry;
+        long matchSize =
+            devicesInFile.stream()
+                .filter(
+                    device -> {
+                      if (logger.isDebugEnabled()) {
+                        logger.debug(
+                            "device is {}, deviceTable is {}, 
tableDeletionEntry.getPredicate().matches(device) is {}",
+                            device,
+                            device.getTableName(),
+                            
finalTableDeletionEntry.getPredicate().matches(device));
+                      }
+                      return tableName.equals(device.getTableName())
+                          && 
finalTableDeletionEntry.getPredicate().matches(device);
+                    })
+                .count();
+        onlyOneTable = matchSize == devicesInFile.size();
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "tableName is {}, matchSize is {}, onlyOneTable is {}",
+              tableName,
+              matchSize,
+              onlyOneTable);
         }
 
         if (onlyOneTable) {
-          int matchSize = 0;
+          matchSize = 0;
           for (IDeviceID device : devicesInFile) {
             Optional<Long> optStart = deviceTimeIndex.getStartTime(device);
             Optional<Long> optEnd = deviceTimeIndex.getEndTime(device);
@@ -3577,11 +3570,12 @@ public class DataRegion implements IDataRegionForQuery {
                   fileEndTime);
             }
             if (isFileFullyMatchedByTime(deletion, fileStartTime, fileEndTime)
-                && idPredicateType.equals(IDPredicate.IDPredicateType.NOP)
+                && tagPredicateType.equals(TagPredicateType.NOP)
                 && !isDropMeasurementExist) {
               ++matchSize;
             } else {
-              deletedByMods.add(sealedTsFile);
+              involvedModificationFiles.add(
+                  new Pair<>(sealedTsFile.getModFileForWrite(), 
tableDeletionEntry));
               break;
             }
           }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java
index b531a6460d0..1390fa2bdcf 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java
@@ -212,8 +212,8 @@ public class FastCompactionPerformer
         long ttl = deviceIterator.getTTLForCurrentDevice();
         sortedSourceFiles.removeIf(
             x -> {
-              EvolvedSchema evolvedSchema = x.getMergedEvolvedSchema(
-                  maxTsFileSetEndVersionAndMinResource.left);
+              EvolvedSchema evolvedSchema =
+                  
x.getMergedEvolvedSchema(maxTsFileSetEndVersionAndMinResource.left);
               IDeviceID originalDevice = device;
               if (evolvedSchema != null) {
                 originalDevice = evolvedSchema.rewriteToOriginal(device);
@@ -221,10 +221,12 @@ public class FastCompactionPerformer
               return x.definitelyNotContains(originalDevice);
             });
         // checked above
-        sortedSourceFiles.sort(Comparator.comparingLong(x -> {
-          //noinspection OptionalGetWithoutIsPresent
-          return x.getStartTime(device, 
maxTsFileSetEndVersionAndMinResource.left).get();
-        }));
+        sortedSourceFiles.sort(
+            Comparator.comparingLong(
+                x -> {
+                  //noinspection OptionalGetWithoutIsPresent
+                  return x.getStartTime(device, 
maxTsFileSetEndVersionAndMinResource.left).get();
+                }));
         ModEntry ttlDeletion = null;
         if (ttl != Long.MAX_VALUE) {
           ttlDeletion =
@@ -311,7 +313,7 @@ public class FastCompactionPerformer
             deviceId,
             taskSummary,
             ignoreAllNullRows,
-        maxTsFileSetEndVersionAndMinResource)
+            maxTsFileSetEndVersionAndMinResource)
         .call();
     subTaskSummary.increase(taskSummary);
   }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadChunkCompactionPerformer.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadChunkCompactionPerformer.java
index 0d9a4f09481..3495b1d7d17 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadChunkCompactionPerformer.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadChunkCompactionPerformer.java
@@ -229,7 +229,10 @@ public class ReadChunkCompactionPerformer implements 
ISeqCompactionPerformer {
     tsFileResource.setTsFileManager(minVersionResource.getTsFileManager());
     EvolvedSchema evolvedSchema =
         
tsFileResource.getMergedEvolvedSchema(maxTsFileSetEndVersionAndMinResource.getLeft());
-    currentWriter.setSchema(evolvedSchema != null ? 
evolvedSchema.rewriteToOriginal(schema, CompactionTableSchema::new) : schema);
+    currentWriter.setSchema(
+        evolvedSchema != null
+            ? evolvedSchema.rewriteToOriginal(schema, 
CompactionTableSchema::new)
+            : schema);
   }
 
   @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadPointCompactionPerformer.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadPointCompactionPerformer.java
index b7c143808c6..00cbfddfd79 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadPointCompactionPerformer.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadPointCompactionPerformer.java
@@ -309,7 +309,8 @@ public class ReadPointCompactionPerformer
                         new QueryDataSource(queryDataSource),
                         compactionWriter,
                         schemaMap,
-                        i, maxTsFileSetEndVersionAndMinResource.left)));
+                        i,
+                        maxTsFileSetEndVersionAndMinResource.left)));
       }
       for (Future<Void> future : futures) {
         future.get();
@@ -343,7 +344,12 @@ public class ReadPointCompactionPerformer
     }
 
     return new SeriesDataBlockReader(
-        seriesPath, new HashSet<>(allSensors), fragmentInstanceContext, 
queryDataSource, true, maxTsFileSetEndVersion);
+        seriesPath,
+        new HashSet<>(allSensors),
+        fragmentInstanceContext,
+        queryDataSource,
+        true,
+        maxTsFileSetEndVersion);
   }
 
   @SuppressWarnings("squid:S1172")
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/subtask/FastCompactionPerformerSubTask.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/subtask/FastCompactionPerformerSubTask.java
index 3de4656b66c..c8bd778c0ff 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/subtask/FastCompactionPerformerSubTask.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/subtask/FastCompactionPerformerSubTask.java
@@ -31,8 +31,8 @@ import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.wri
 import org.apache.iotdb.db.storageengine.dataregion.modification.ModEntry;
 import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
 import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory;
-
 import 
org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory.ModsSerializer;
+
 import org.apache.tsfile.exception.write.PageException;
 import org.apache.tsfile.file.metadata.IDeviceID;
 import org.apache.tsfile.read.TsFileSequenceReader;
@@ -119,7 +119,8 @@ public class FastCompactionPerformerSubTask implements 
Callable<Void> {
       List<String> measurements,
       IDeviceID deviceId,
       FastCompactionTaskSummary summary,
-      int subTaskId) {
+      int subTaskId,
+      Pair<Long, TsFileResource> maxTsFileSetEndVersionAndMinResource) {
     this.compactionWriter = compactionWriter;
     this.subTaskId = subTaskId;
     this.compactionSeriesContextMap = compactionSeriesContextMap;
@@ -132,6 +133,7 @@ public class FastCompactionPerformerSubTask implements 
Callable<Void> {
     this.measurements = measurements;
     this.summary = summary;
     this.ignoreAllNullRows = true;
+    this.maxTsFileSetEndVersionAndMinResource = 
maxTsFileSetEndVersionAndMinResource;
   }
 
   /** Used for aligned timeseries. */
@@ -139,13 +141,13 @@ public class FastCompactionPerformerSubTask implements 
Callable<Void> {
       AbstractCompactionWriter compactionWriter,
       Map<String, Map<TsFileResource, Pair<Long, Long>>> 
timeseriesMetadataOffsetMap,
       Map<TsFileResource, TsFileSequenceReader> readerCacheMap,
-      Map<String, PatternTreeMap<ModEntry, ModsSerializer>>
-          modificationCacheMap,
+      Map<String, PatternTreeMap<ModEntry, ModsSerializer>> 
modificationCacheMap,
       List<TsFileResource> sortedSourceFiles,
       List<IMeasurementSchema> measurementSchemas,
       IDeviceID deviceId,
       FastCompactionTaskSummary summary,
-      boolean ignoreAllNullRows, Pair<Long, TsFileResource> 
maxTsFileSetEndVersionAndMinResource) {
+      boolean ignoreAllNullRows,
+      Pair<Long, TsFileResource> maxTsFileSetEndVersionAndMinResource) {
     this.compactionWriter = compactionWriter;
     this.subTaskId = 0;
     this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/subtask/ReadPointPerformerSubTask.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/subtask/ReadPointPerformerSubTask.java
index ed967030b52..741a6f314cd 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/subtask/ReadPointPerformerSubTask.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/subtask/ReadPointPerformerSubTask.java
@@ -66,7 +66,8 @@ public class ReadPointPerformerSubTask implements 
Callable<Void> {
       QueryDataSource queryDataSource,
       AbstractCompactionWriter compactionWriter,
       Map<String, MeasurementSchema> schemaMap,
-      int taskId, long maxTsFileSetEndVersion) {
+      int taskId,
+      long maxTsFileSetEndVersion) {
     this.device = device;
     this.measurementList = measurementList;
     this.fragmentInstanceContext = fragmentInstanceContext;
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/MultiTsFileDeviceIterator.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/MultiTsFileDeviceIterator.java
index a3ed580ed99..ab50d6740fa 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/MultiTsFileDeviceIterator.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/MultiTsFileDeviceIterator.java
@@ -153,17 +153,18 @@ public class MultiTsFileDeviceIterator implements 
AutoCloseable {
     Collections.sort(
         this.tsFileResourcesSortedByDesc, 
TsFileResource::compareFileCreationOrderByDesc);
 
-    maxTsFileSetEndVersion = this.tsFileResourcesSortedByDesc.stream()
-        .mapToLong(
-            // max endVersion of all filesets of a TsFile
-            resource ->
-                resource.getTsFileSets().stream()
-                    .mapToLong(TsFileSet::getEndVersion)
-                    .max()
-                    .orElse(Long.MAX_VALUE))
-        // overall max endVersion
-        .max()
-        .orElse(Long.MAX_VALUE);
+    maxTsFileSetEndVersion =
+        this.tsFileResourcesSortedByDesc.stream()
+            .mapToLong(
+                // max endVersion of all filesets of a TsFile
+                resource ->
+                    resource.getTsFileSets().stream()
+                        .mapToLong(TsFileSet::getEndVersion)
+                        .max()
+                        .orElse(Long.MAX_VALUE))
+            // overall max endVersion
+            .max()
+            .orElse(Long.MAX_VALUE);
 
     for (TsFileResource tsFileResource : tsFileResourcesSortedByDesc) {
       TsFileSequenceReader reader =
@@ -518,8 +519,8 @@ public class MultiTsFileDeviceIterator implements 
AutoCloseable {
         String measurementId = entrySet.getKey();
         EvolvedSchema evolvedSchema = 
resource.getMergedEvolvedSchema(maxTsFileSetEndVersion);
         if (evolvedSchema != null) {
-          String originalTableName = evolvedSchema.getOriginalTableName(
-              currentDevice.left.getTableName());
+          String originalTableName =
+              
evolvedSchema.getOriginalTableName(currentDevice.left.getTableName());
           measurementId = evolvedSchema.getFinalColumnName(originalTableName, 
measurementId);
         }
         if (!timeseriesMetadataOffsetMap.containsKey(measurementId)) {
@@ -596,9 +597,12 @@ public class MultiTsFileDeviceIterator implements 
AutoCloseable {
       if (evolvedSchema != null) {
         // rewrite the measurementId to the final ones so that they can be 
aligned with other files
         for (AbstractAlignedChunkMetadata abstractAlignedChunkMetadata : 
alignedChunkMetadataList) {
-          for (IChunkMetadata chunkMetadata : 
abstractAlignedChunkMetadata.getValueChunkMetadataList()) {
+          for (IChunkMetadata chunkMetadata :
+              abstractAlignedChunkMetadata.getValueChunkMetadataList()) {
             if (chunkMetadata != null) {
-              
chunkMetadata.setMeasurementUid(evolvedSchema.getFinalColumnName(originalDeviceId.getTableName(),
 chunkMetadata.getMeasurementUid()));
+              chunkMetadata.setMeasurementUid(
+                  evolvedSchema.getFinalColumnName(
+                      originalDeviceId.getTableName(), 
chunkMetadata.getMeasurementUid()));
             }
           }
         }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastAlignedSeriesCompactionExecutor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastAlignedSeriesCompactionExecutor.java
index 7ab5769378e..3d03b7baf16 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastAlignedSeriesCompactionExecutor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastAlignedSeriesCompactionExecutor.java
@@ -93,7 +93,14 @@ public class FastAlignedSeriesCompactionExecutor extends 
SeriesCompactionExecuto
       boolean ignoreAllNullRows,
       Pair<Long, TsFileResource> maxTsFileSetEndVersionAndMinResource) {
     super(
-        compactionWriter, readerCacheMap, modificationCacheMap, deviceId, 
true, subTaskId, summary, maxTsFileSetEndVersionAndMinResource);
+        compactionWriter,
+        readerCacheMap,
+        modificationCacheMap,
+        deviceId,
+        true,
+        subTaskId,
+        summary,
+        maxTsFileSetEndVersionAndMinResource);
     this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap;
     this.measurementSchemas = measurementSchemas;
     this.timeColumnMeasurementSchema = measurementSchemas.get(0);
@@ -190,8 +197,8 @@ public class FastAlignedSeriesCompactionExecutor extends 
SeriesCompactionExecuto
     // read time chunk metadatas and value chunk metadatas in the current file
     List<IChunkMetadata> timeChunkMetadatas = null;
     List<List<IChunkMetadata>> valueChunkMetadatas = new ArrayList<>();
-    EvolvedSchema evolvedSchema = resource.getMergedEvolvedSchema(
-        maxTsFileSetEndVersionAndMinResource.getLeft());
+    EvolvedSchema evolvedSchema =
+        
resource.getMergedEvolvedSchema(maxTsFileSetEndVersionAndMinResource.getLeft());
 
     for (Map.Entry<String, Map<TsFileResource, Pair<Long, Long>>> entry :
         timeseriesMetadataOffsetMap.entrySet()) {
@@ -287,8 +294,7 @@ public class FastAlignedSeriesCompactionExecutor extends 
SeriesCompactionExecuto
   }
 
   private boolean isValueChunkDataTypeMatchSchema(
-      List<IChunkMetadata> chunkMetadataListOfOneValueColumn,
-      EvolvedSchema evolvedSchema) {
+      List<IChunkMetadata> chunkMetadataListOfOneValueColumn, EvolvedSchema 
evolvedSchema) {
     boolean isMatch = false;
     for (IChunkMetadata chunkMetadata : chunkMetadataListOfOneValueColumn) {
       if (chunkMetadata == null) {
@@ -394,10 +400,9 @@ public class FastAlignedSeriesCompactionExecutor extends 
SeriesCompactionExecuto
       
chunk.getHeader().setMeasurementID(valueChunkMetadata.getMeasurementUid());
 
       if (valueChunkMetadata.getNewType() != null) {
-        Chunk chunk =
-            chunk
-                .rewrite(
-                    ((ChunkMetadata) valueChunkMetadata).getNewType(), 
chunkMetadataElement.chunk);
+        chunk =
+            chunk.rewrite(
+                ((ChunkMetadata) valueChunkMetadata).getNewType(), 
chunkMetadataElement.chunk);
         valueChunks.add(chunk);
 
         ChunkMetadata chunkMetadata = (ChunkMetadata) valueChunkMetadata;
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/SeriesCompactionExecutor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/SeriesCompactionExecutor.java
index 6525132fef6..ff75a5a867b 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/SeriesCompactionExecutor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/SeriesCompactionExecutor.java
@@ -41,6 +41,7 @@ import org.apache.tsfile.file.metadata.IDeviceID;
 import org.apache.tsfile.read.TimeValuePair;
 import org.apache.tsfile.read.TsFileSequenceReader;
 import org.apache.tsfile.read.common.TimeRange;
+import org.apache.tsfile.utils.Pair;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -49,7 +50,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.PriorityQueue;
-import org.apache.tsfile.utils.Pair;
 
 public abstract class SeriesCompactionExecutor {
 
@@ -355,12 +355,14 @@ public abstract class SeriesCompactionExecutor {
    */
   protected List<FileElement> findOverlapFiles(FileElement fileToCheck) {
     List<FileElement> overlappedFiles = new ArrayList<>();
-    Optional<Long> endTimeInCheckingFile = 
fileToCheck.resource.getEndTime(deviceId, 
maxTsFileSetEndVersionAndMinResource.left);
+    Optional<Long> endTimeInCheckingFile =
+        fileToCheck.resource.getEndTime(deviceId, 
maxTsFileSetEndVersionAndMinResource.left);
     for (FileElement otherFile : fileList) {
       if (!endTimeInCheckingFile.isPresent()) {
         continue;
       }
-      Optional<Long> startTimeInOtherFile = 
otherFile.resource.getStartTime(deviceId, 
maxTsFileSetEndVersionAndMinResource.left);
+      Optional<Long> startTimeInOtherFile =
+          otherFile.resource.getStartTime(deviceId, 
maxTsFileSetEndVersionAndMinResource.left);
       if (startTimeInOtherFile.isPresent()
           && startTimeInOtherFile.get() <= endTimeInCheckingFile.get()) {
         if (!otherFile.isSelected) {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractInnerCompactionWriter.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractInnerCompactionWriter.java
index 42338a8ad1f..657b2de123d 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractInnerCompactionWriter.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractInnerCompactionWriter.java
@@ -132,7 +132,10 @@ public abstract class AbstractInnerCompactionWriter 
extends AbstractCompactionWr
     
fileWriter.getTsFileResource().setTsFileManager(minVersionResource.getTsFileManager());
     EvolvedSchema evolvedSchema =
         
fileWriter.getTsFileResource().getMergedEvolvedSchema(maxTsFileSetEndVersion);
-    fileWriter.setSchema(evolvedSchema != null ? 
evolvedSchema.rewriteToOriginal(schema, CompactionTableSchema::new) : schema);
+    fileWriter.setSchema(
+        evolvedSchema != null
+            ? evolvedSchema.rewriteToOriginal(schema, 
CompactionTableSchema::new)
+            : schema);
   }
 
   @Override
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/io/CompactionTsFileWriter.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/io/CompactionTsFileWriter.java
index cd3bbefcbf7..e03faa82026 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/io/CompactionTsFileWriter.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/io/CompactionTsFileWriter.java
@@ -114,7 +114,8 @@ public class CompactionTsFileWriter extends TsFileIOWriter {
             ? null
             : measurementName ->
                 evolvedSchema.getOriginalColumnName(
-                    
evolvedSchema.getFinalTableName(currentOriginalDeviceId.getTableName()), 
measurementName));
+                    
evolvedSchema.getFinalTableName(currentOriginalDeviceId.getTableName()),
+                    measurementName));
     long writtenDataSize = this.getPos() - beforeOffset;
     CompactionMetrics.getInstance()
         .recordWriteInfo(
@@ -130,13 +131,13 @@ public class CompactionTsFileWriter extends 
TsFileIOWriter {
       isEmptyTargetFile = false;
     }
     if (evolvedSchema != null) {
-      String finalTableName = evolvedSchema.getFinalTableName(
-          currentOriginalDeviceId.getTableName());
+      String finalTableName =
+          
evolvedSchema.getFinalTableName(currentOriginalDeviceId.getTableName());
       chunk
           .getHeader()
           .setMeasurementID(
               evolvedSchema.getOriginalColumnName(
-                 finalTableName, chunk.getHeader().getMeasurementID()));
+                  finalTableName, chunk.getHeader().getMeasurementID()));
     }
     super.writeChunk(chunk, chunkMetadata);
     long writtenDataSize = this.getPos() - beforeOffset;
@@ -157,7 +158,8 @@ public class CompactionTsFileWriter extends TsFileIOWriter {
       throws IOException {
     if (evolvedSchema != null) {
       measurementId =
-          
evolvedSchema.getOriginalColumnName(currentOriginalDeviceId.getTableName(), 
measurementId);
+          evolvedSchema.getOriginalColumnName(
+              currentOriginalDeviceId.getTableName(), measurementId);
     }
     long beforeOffset = this.getPos();
     super.writeEmptyValueChunk(
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/DeletionPredicate.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/DeletionPredicate.java
index cf5a23374e6..83eabc0fc58 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/DeletionPredicate.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/DeletionPredicate.java
@@ -19,6 +19,7 @@
 package org.apache.iotdb.db.storageengine.dataregion.modification;
 
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.TagPredicate.NOP;
+import 
org.apache.iotdb.db.storageengine.dataregion.modification.TagPredicate.TagPredicateType;
 import org.apache.iotdb.db.utils.io.BufferSerializable;
 import org.apache.iotdb.db.utils.io.StreamSerializable;
 
@@ -72,12 +73,8 @@ public class DeletionPredicate implements 
StreamSerializable, BufferSerializable
     this.tagPredicate = tagPredicate;
   }
 
-  public IDPredicate getIdPredicate() {
-    return idPredicate;
-  }
-
-  public IDPredicate.IDPredicateType getIdPredicateType() {
-    return this.idPredicate.type;
+  public TagPredicateType getTagPredicateType() {
+    return this.tagPredicate.type;
   }
 
   public String getTableName() {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/TableDeletionEntry.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/TableDeletionEntry.java
index 61ec2e0d678..60aab450bf3 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/TableDeletionEntry.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/TableDeletionEntry.java
@@ -21,7 +21,7 @@ package 
org.apache.iotdb.db.storageengine.dataregion.modification;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper;
-import 
org.apache.iotdb.db.storageengine.dataregion.modification.IDPredicate.IDPredicateType;
+import 
org.apache.iotdb.db.storageengine.dataregion.modification.TagPredicate.TagPredicateType;
 import org.apache.iotdb.db.utils.ModificationUtils;
 
 import org.apache.tsfile.file.metadata.IDeviceID;
@@ -138,8 +138,8 @@ public class TableDeletionEntry extends ModEntry {
   }
 
   public boolean isDroppingTable() {
-    IDPredicate idPredicate = predicate.getIdPredicate();
-    return idPredicate.type == IDPredicateType.NOP
+    TagPredicate tagPredicate = predicate.getTagPredicate();
+    return tagPredicate.type == TagPredicateType.NOP
         && predicate.getMeasurementNames().isEmpty()
         && timeRange.getMin() == Long.MIN_VALUE
         && timeRange.getMax() == Long.MAX_VALUE;
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/TagPredicate.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/TagPredicate.java
index a0624c8555d..9b0022aa955 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/TagPredicate.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/TagPredicate.java
@@ -47,7 +47,7 @@ public abstract class TagPredicate implements 
StreamSerializable, BufferSerializ
   }
 
   @SuppressWarnings("java:S6548")
-  public enum IDPredicateType {
+  public enum TagPredicateType {
     NOP,
     FULL_EXACT_MATCH,
     SEGMENT_EXACT_MATCH,
@@ -63,18 +63,18 @@ public abstract class TagPredicate implements 
StreamSerializable, BufferSerializ
       return 1;
     }
 
-    public static IDPredicateType deserialize(InputStream stream) throws 
IOException {
+    public static TagPredicateType deserialize(InputStream stream) throws 
IOException {
       return values()[stream.read()];
     }
 
-    public static IDPredicateType deserialize(ByteBuffer buffer) {
+    public static TagPredicateType deserialize(ByteBuffer buffer) {
       return values()[buffer.get()];
     }
   }
 
-  protected final IDPredicateType type;
+  protected final TagPredicateType type;
 
-  protected TagPredicate(IDPredicateType type) {
+  protected TagPredicate(TagPredicateType type) {
     this.type = type;
   }
 
@@ -99,15 +99,15 @@ public abstract class TagPredicate implements 
StreamSerializable, BufferSerializ
   }
 
   public static TagPredicate createFrom(ByteBuffer buffer) {
-    IDPredicateType type = IDPredicateType.deserialize(buffer);
+    TagPredicateType type = TagPredicateType.deserialize(buffer);
     TagPredicate predicate;
-    if (Objects.requireNonNull(type) == IDPredicateType.NOP) {
+    if (Objects.requireNonNull(type) == TagPredicateType.NOP) {
       predicate = new NOP();
-    } else if (Objects.requireNonNull(type) == 
IDPredicateType.FULL_EXACT_MATCH) {
+    } else if (Objects.requireNonNull(type) == 
TagPredicateType.FULL_EXACT_MATCH) {
       predicate = new FullExactMatch();
-    } else if (Objects.requireNonNull(type) == 
IDPredicateType.SEGMENT_EXACT_MATCH) {
+    } else if (Objects.requireNonNull(type) == 
TagPredicateType.SEGMENT_EXACT_MATCH) {
       predicate = new SegmentExactMatch();
-    } else if (Objects.requireNonNull(type) == IDPredicateType.AND) {
+    } else if (Objects.requireNonNull(type) == TagPredicateType.AND) {
       predicate = new And();
     } else {
       throw new IllegalArgumentException("Unrecognized predicate type: " + 
type);
@@ -117,15 +117,15 @@ public abstract class TagPredicate implements 
StreamSerializable, BufferSerializ
   }
 
   public static TagPredicate createFrom(InputStream stream) throws IOException 
{
-    IDPredicateType type = IDPredicateType.deserialize(stream);
+    TagPredicateType type = TagPredicateType.deserialize(stream);
     TagPredicate predicate;
-    if (Objects.requireNonNull(type) == IDPredicateType.NOP) {
+    if (Objects.requireNonNull(type) == TagPredicateType.NOP) {
       predicate = new NOP();
-    } else if (Objects.requireNonNull(type) == 
IDPredicateType.FULL_EXACT_MATCH) {
+    } else if (Objects.requireNonNull(type) == 
TagPredicateType.FULL_EXACT_MATCH) {
       predicate = new FullExactMatch();
-    } else if (Objects.requireNonNull(type) == 
IDPredicateType.SEGMENT_EXACT_MATCH) {
+    } else if (Objects.requireNonNull(type) == 
TagPredicateType.SEGMENT_EXACT_MATCH) {
       predicate = new SegmentExactMatch();
-    } else if (Objects.requireNonNull(type) == IDPredicateType.AND) {
+    } else if (Objects.requireNonNull(type) == TagPredicateType.AND) {
       predicate = new And();
     } else {
       throw new IllegalArgumentException("Unrecognized predicate type: " + 
type);
@@ -138,7 +138,7 @@ public abstract class TagPredicate implements 
StreamSerializable, BufferSerializ
     public static final long SHALLOW_SIZE = 
RamUsageEstimator.shallowSizeOfInstance(NOP.class);
 
     public NOP() {
-      super(IDPredicateType.NOP);
+      super(TagPredicateType.NOP);
     }
 
     @Override
@@ -184,12 +184,12 @@ public abstract class TagPredicate implements 
StreamSerializable, BufferSerializ
     private IDeviceID deviceID;
 
     public FullExactMatch(IDeviceID deviceID) {
-      super(IDPredicateType.FULL_EXACT_MATCH);
+      super(TagPredicateType.FULL_EXACT_MATCH);
       this.deviceID = deviceID;
     }
 
     public FullExactMatch() {
-      super(IDPredicateType.FULL_EXACT_MATCH);
+      super(TagPredicateType.FULL_EXACT_MATCH);
     }
 
     @Override
@@ -272,13 +272,13 @@ public abstract class TagPredicate implements 
StreamSerializable, BufferSerializ
     private int segmentIndex;
 
     public SegmentExactMatch(String pattern, int segmentIndex) {
-      super(IDPredicateType.SEGMENT_EXACT_MATCH);
+      super(TagPredicateType.SEGMENT_EXACT_MATCH);
       this.pattern = pattern;
       this.segmentIndex = segmentIndex;
     }
 
     public SegmentExactMatch() {
-      super(IDPredicateType.SEGMENT_EXACT_MATCH);
+      super(TagPredicateType.SEGMENT_EXACT_MATCH);
     }
 
     @Override
@@ -367,7 +367,7 @@ public abstract class TagPredicate implements 
StreamSerializable, BufferSerializ
     private final List<TagPredicate> predicates = new ArrayList<>();
 
     public And(TagPredicate... predicates) {
-      super(IDPredicateType.AND);
+      super(TagPredicateType.AND);
       Collections.addAll(this.predicates, predicates);
     }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java
index b1f20ab81fd..9bdbe1c4932 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java
@@ -113,18 +113,26 @@ public class QueryDataSource implements IQueryDataSource {
     return queryDataSource;
   }
 
-  public boolean hasNextSeqResource(int curIndex, boolean ascending, IDeviceID 
deviceID, long maxTsFileSetEndVersion) {
+  public boolean hasNextSeqResource(
+      int curIndex, boolean ascending, IDeviceID deviceID, long 
maxTsFileSetEndVersion) {
     boolean res = ascending ? curIndex < seqResources.size() : curIndex >= 0;
     if (res && curIndex != this.curSeqIndex) {
       this.curSeqIndex = curIndex;
-      this.curSeqOrderTime = 
seqResources.get(curIndex).getOrderTimeForSeq(deviceID, ascending, 
maxTsFileSetEndVersion);
+      this.curSeqOrderTime =
+          seqResources
+              .get(curIndex)
+              .getOrderTimeForSeq(deviceID, ascending, maxTsFileSetEndVersion);
       this.curSeqSatisfied = null;
     }
     return res;
   }
 
   public boolean isSeqSatisfied(
-      IDeviceID deviceID, int curIndex, Filter timeFilter, boolean debug, long 
maxTsFileSetEndVersion) {
+      IDeviceID deviceID,
+      int curIndex,
+      Filter timeFilter,
+      boolean debug,
+      long maxTsFileSetEndVersion) {
     if (curIndex != this.curSeqIndex) {
       throw new IllegalArgumentException(
           String.format("curIndex %d is not equal to curSeqIndex %d", 
curIndex, this.curSeqIndex));
@@ -133,7 +141,9 @@ public class QueryDataSource implements IQueryDataSource {
       TsFileResource tsFileResource = seqResources.get(curSeqIndex);
       curSeqSatisfied =
           tsFileResource != null
-              && (isSingleDevice || tsFileResource.isSatisfied(deviceID, 
timeFilter, true, debug, maxTsFileSetEndVersion));
+              && (isSingleDevice
+                  || tsFileResource.isSatisfied(
+                      deviceID, timeFilter, true, debug, 
maxTsFileSetEndVersion));
     }
 
     return curSeqSatisfied;
@@ -154,7 +164,8 @@ public class QueryDataSource implements IQueryDataSource {
     return null;
   }
 
-  public boolean hasNextUnseqResource(int curIndex, boolean ascending, 
IDeviceID deviceID, long maxTsFileSetEndVersion) {
+  public boolean hasNextUnseqResource(
+      int curIndex, boolean ascending, IDeviceID deviceID, long 
maxTsFileSetEndVersion) {
     boolean res = curIndex < unseqResources.size();
     if (res && curIndex != this.curUnSeqIndex) {
       this.curUnSeqIndex = curIndex;
@@ -168,7 +179,11 @@ public class QueryDataSource implements IQueryDataSource {
   }
 
   public boolean isUnSeqSatisfied(
-      IDeviceID deviceID, int curIndex, Filter timeFilter, boolean debug, long 
maxTsFileSetEndVersion) {
+      IDeviceID deviceID,
+      int curIndex,
+      Filter timeFilter,
+      boolean debug,
+      long maxTsFileSetEndVersion) {
     if (curIndex != this.curUnSeqIndex) {
       throw new IllegalArgumentException(
           String.format(
@@ -178,7 +193,9 @@ public class QueryDataSource implements IQueryDataSource {
       TsFileResource tsFileResource = 
unseqResources.get(unSeqFileOrderIndex[curIndex]);
       curUnSeqSatisfied =
           tsFileResource != null
-              && (isSingleDevice || tsFileResource.isSatisfied(deviceID, 
timeFilter, false, debug, maxTsFileSetEndVersion));
+              && (isSingleDevice
+                  || tsFileResource.isSatisfied(
+                      deviceID, timeFilter, false, debug, 
maxTsFileSetEndVersion));
     }
 
     return curUnSeqSatisfied;
@@ -219,7 +236,8 @@ public class QueryDataSource implements IQueryDataSource {
     for (TsFileResource resource : unseqResources) {
       orderTimeToIndexMap
           .computeIfAbsent(
-              resource.getOrderTimeForUnseq(deviceId, ascending, 
maxTsFileSetEndVersion), key -> new ArrayList<>())
+              resource.getOrderTimeForUnseq(deviceId, ascending, 
maxTsFileSetEndVersion),
+              key -> new ArrayList<>())
           .add(index++);
     }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java
index c68db6bb909..0462002f1fb 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java
@@ -665,7 +665,8 @@ public class TsFileResource implements PersistentResource, 
Cloneable {
   }
 
   // cannot use FileTimeIndex
-  public long getOrderTimeForSeq(IDeviceID deviceId, boolean ascending, long 
maxTsFileSetEndVersion) {
+  public long getOrderTimeForSeq(
+      IDeviceID deviceId, boolean ascending, long maxTsFileSetEndVersion) {
     if (timeIndex instanceof ArrayDeviceTimeIndex) {
       EvolvedSchema evolvedSchema = 
getMergedEvolvedSchema(maxTsFileSetEndVersion);
       if (evolvedSchema != null) {
@@ -680,7 +681,8 @@ public class TsFileResource implements PersistentResource, 
Cloneable {
   }
 
   // can use FileTimeIndex
-  public long getOrderTimeForUnseq(IDeviceID deviceId, boolean ascending, long 
maxTsFileSetEndVersion) {
+  public long getOrderTimeForUnseq(
+      IDeviceID deviceId, boolean ascending, long maxTsFileSetEndVersion) {
     if (timeIndex instanceof ArrayDeviceTimeIndex) {
       EvolvedSchema evolvedSchema = 
getMergedEvolvedSchema(maxTsFileSetEndVersion);
       if (evolvedSchema != null) {
@@ -1054,7 +1056,12 @@ public class TsFileResource implements 
PersistentResource, Cloneable {
    * @return true if the device is contained in the TsFile
    */
   @SuppressWarnings("OptionalGetWithoutIsPresent")
-  public boolean isSatisfied(IDeviceID deviceId, Filter timeFilter, boolean 
isSeq, boolean debug, long maxTsFileSetEndVersion) {
+  public boolean isSatisfied(
+      IDeviceID deviceId,
+      Filter timeFilter,
+      boolean isSeq,
+      boolean debug,
+      long maxTsFileSetEndVersion) {
     EvolvedSchema evolvedSchema = 
getMergedEvolvedSchema(maxTsFileSetEndVersion);
     if (evolvedSchema != null) {
       deviceId = evolvedSchema.rewriteToOriginal(deviceId);
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/evolution/EvolvedSchema.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/evolution/EvolvedSchema.java
index 1e6c22e057a..d5a18082c56 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/evolution/EvolvedSchema.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/evolution/EvolvedSchema.java
@@ -270,8 +270,7 @@ public class EvolvedSchema implements Accountable {
       columnCategories.add(tableSchema.getColumnTypes().get(i));
     }
 
-    TableSchema schema = new TableSchema(originalTableName, measurementSchemas,
-        columnCategories);
+    TableSchema schema = new TableSchema(originalTableName, 
measurementSchemas, columnCategories);
     schema.setUpdatable(tableSchema.isUpdatable());
     return schema;
   }
@@ -290,12 +289,12 @@ public class EvolvedSchema implements Accountable {
               getFinalColumnName(
                   tableSchema.getTableName(), 
measurementSchema.getMeasurementName()),
               measurementSchema.getType(),
-              measurementSchema.getEncodingType(), 
measurementSchema.getCompressor()));
+              measurementSchema.getEncodingType(),
+              measurementSchema.getCompressor()));
       columnCategories.add(tableSchema.getColumnTypes().get(i));
     }
 
-    TableSchema schema = new TableSchema(finalTableName, measurementSchemas,
-        columnCategories);
+    TableSchema schema = new TableSchema(finalTableName, measurementSchemas, 
columnCategories);
     schema.setUpdatable(tableSchema.isUpdatable());
     return schema;
   }
@@ -389,8 +388,8 @@ public class EvolvedSchema implements Accountable {
     return copySchema;
   }
 
-  public void rewriteToFinal(AbstractAlignedChunkMetadata 
abstractAlignedChunkMetadata,
-      String originalTableName) {
+  public void rewriteToFinal(
+      AbstractAlignedChunkMetadata abstractAlignedChunkMetadata, String 
originalTableName) {
     for (IChunkMetadata iChunkMetadata : 
abstractAlignedChunkMetadata.getValueChunkMetadataList()) {
       if (iChunkMetadata != null) {
         iChunkMetadata.setMeasurementUid(
@@ -403,7 +402,7 @@ public class EvolvedSchema implements Accountable {
   public long ramBytesUsed() {
     return RamUsageEstimator.sizeOfMap(this.finalToOriginalTableNames)
         + RamUsageEstimator.sizeOfMap(this.finalToOriginalColumnNames)
-        + RamUsageEstimator.sizeOfMap(this.originalToFinalTableNames) + 
RamUsageEstimator.sizeOfMap(
-        this.originalToFinalColumnNames);
+        + RamUsageEstimator.sizeOfMap(this.originalToFinalTableNames)
+        + RamUsageEstimator.sizeOfMap(this.originalToFinalColumnNames);
   }
 }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/evolution/EvolvedSchemaCache.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/evolution/EvolvedSchemaCache.java
index 7e15ea583d7..8eb4072c0af 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/evolution/EvolvedSchemaCache.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/evolution/EvolvedSchemaCache.java
@@ -1,29 +1,35 @@
 package org.apache.iotdb.db.storageengine.dataregion.tsfile.evolution;
 
+import org.apache.iotdb.db.storageengine.dataregion.tsfile.fileset.TsFileSet;
+
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.Weigher;
-import java.util.function.Supplier;
-import org.apache.iotdb.db.storageengine.dataregion.tsfile.fileset.TsFileSet;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+import java.util.function.Supplier;
+
 public class EvolvedSchemaCache {
 
   private final Cache<TsFileSet, EvolvedSchema> cache;
 
   private EvolvedSchemaCache() {
-    cache = Caffeine.newBuilder().weigher(
-        (Weigher<TsFileSet, EvolvedSchema>) (k, v) -> {
-          // TsFileSet is always in memory, do not count it
-          return (int) v.ramBytesUsed();
-        }
-    ).maximumWeight(
-        // TODO-Sevo configurable
-        128 * 1024 * 1024L
-    ).build();
+    cache =
+        Caffeine.newBuilder()
+            .weigher(
+                (Weigher<TsFileSet, EvolvedSchema>)
+                    (k, v) -> {
+                      // TsFileSet is always in memory, do not count it
+                      return (int) v.ramBytesUsed();
+                    })
+            .maximumWeight(
+                // TODO-Sevo configurable
+                128 * 1024 * 1024L)
+            .build();
   }
 
-  public @Nullable EvolvedSchema computeIfAbsent(TsFileSet tsFileSet, 
Supplier<EvolvedSchema> schemaSupplier) {
+  public @Nullable EvolvedSchema computeIfAbsent(
+      TsFileSet tsFileSet, Supplier<EvolvedSchema> schemaSupplier) {
     return cache.get(tsFileSet, k -> schemaSupplier.get());
   }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/fileset/TsFileSet.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/fileset/TsFileSet.java
index e255afb8445..676f8c4a3fd 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/fileset/TsFileSet.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/fileset/TsFileSet.java
@@ -30,8 +30,6 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.tsfile.utils.Accountable;
-import org.apache.tsfile.utils.RamUsageEstimator;
 
 /** TsFileSet represents a set of TsFiles in a time partition whose version <= 
endVersion. */
 public class TsFileSet implements Comparable<TsFileSet> {
@@ -87,13 +85,16 @@ public class TsFileSet implements Comparable<TsFileSet> {
   public EvolvedSchema readEvolvedSchema() throws IOException {
     readLock();
     try {
-      return EvolvedSchemaCache.getInstance().computeIfAbsent(this, () -> {
-        try {
-          return schemaEvolutionFile.readAsSchema();
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      });
+      return EvolvedSchemaCache.getInstance()
+          .computeIfAbsent(
+              this,
+              () -> {
+                try {
+                  return schemaEvolutionFile.readAsSchema();
+                } catch (IOException e) {
+                  throw new RuntimeException(e);
+                }
+              });
     } finally {
       readUnlock();
     }
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionWithSevoTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionWithSevoTest.java
index 9ba186ab9af..26455a3eb28 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionWithSevoTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionWithSevoTest.java
@@ -1,15 +1,5 @@
 package org.apache.iotdb.db.storageengine.dataregion.compaction;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.function.Function;
-import java.util.function.Supplier;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICompactionPerformer;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadChunkCompactionPerformer;
@@ -23,6 +13,7 @@ import 
org.apache.iotdb.db.storageengine.dataregion.tsfile.evolution.TableRename
 import org.apache.iotdb.db.storageengine.dataregion.tsfile.fileset.TsFileSet;
 import org.apache.iotdb.db.utils.EncryptDBUtils;
 import org.apache.iotdb.db.utils.constant.TestConstant;
+
 import org.apache.tsfile.enums.ColumnCategory;
 import org.apache.tsfile.enums.TSDataType;
 import org.apache.tsfile.exception.write.NoMeasurementException;
@@ -37,44 +28,86 @@ import org.apache.tsfile.write.TsFileWriter;
 import org.apache.tsfile.write.record.Tablet;
 import org.junit.Test;
 
-public class CompactionWithSevoTest extends AbstractCompactionTest{
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class CompactionWithSevoTest extends AbstractCompactionTest {
 
   @Test
   public void testReadChunkCompactionPerformer() throws Exception {
-    testInner(targets ->new ReadChunkCompactionPerformer(seqResources, 
targets, EncryptDBUtils.getDefaultFirstEncryptParam()), 
CompactionTaskSummary::new);
+    testInner(
+        targets ->
+            new ReadChunkCompactionPerformer(
+                seqResources, targets, 
EncryptDBUtils.getDefaultFirstEncryptParam()),
+        CompactionTaskSummary::new);
   }
 
   @Test
   public void testReadPointCompactionPerformerSeq() throws Exception {
-    testInner(targets ->new ReadPointCompactionPerformer(seqResources, 
Collections.emptyList(), targets), CompactionTaskSummary::new);
+    testInner(
+        targets -> new ReadPointCompactionPerformer(seqResources, 
Collections.emptyList(), targets),
+        CompactionTaskSummary::new);
   }
 
   @Test
   public void testReadPointCompactionPerformerUnseq() throws Exception {
-    testInner(targets ->new 
ReadPointCompactionPerformer(Collections.emptyList(), seqResources, targets), 
CompactionTaskSummary::new);
+    testInner(
+        targets -> new ReadPointCompactionPerformer(Collections.emptyList(), 
seqResources, targets),
+        CompactionTaskSummary::new);
   }
 
   @Test
   public void testReadPointCompactionPerformerCross() throws Exception {
-    testCross(targets ->new ReadPointCompactionPerformer(seqResources, 
unseqResources, targets), CompactionTaskSummary::new);
+    testCross(
+        targets -> new ReadPointCompactionPerformer(seqResources, 
unseqResources, targets),
+        CompactionTaskSummary::new);
   }
 
   @Test
   public void testFastCompactionPerformerSeq() throws Exception {
-    testInner(targets -> new FastCompactionPerformer(seqResources, 
Collections.emptyList(), targets, 
EncryptDBUtils.getDefaultFirstEncryptParam()), FastCompactionTaskSummary::new);
+    testInner(
+        targets ->
+            new FastCompactionPerformer(
+                seqResources,
+                Collections.emptyList(),
+                targets,
+                EncryptDBUtils.getDefaultFirstEncryptParam()),
+        FastCompactionTaskSummary::new);
   }
 
   @Test
   public void testFastCompactionPerformerUnseq() throws Exception {
-    testInner(targets -> new FastCompactionPerformer(Collections.emptyList(), 
seqResources, targets, EncryptDBUtils.getDefaultFirstEncryptParam()), 
FastCompactionTaskSummary::new);
+    testInner(
+        targets ->
+            new FastCompactionPerformer(
+                Collections.emptyList(),
+                seqResources,
+                targets,
+                EncryptDBUtils.getDefaultFirstEncryptParam()),
+        FastCompactionTaskSummary::new);
   }
 
   @Test
   public void testFastCompactionPerformerCross() throws Exception {
-    testCross(targets -> new FastCompactionPerformer(seqResources, 
unseqResources, targets, EncryptDBUtils.getDefaultFirstEncryptParam()), 
FastCompactionTaskSummary::new);
+    testCross(
+        targets ->
+            new FastCompactionPerformer(
+                seqResources,
+                unseqResources,
+                targets,
+                EncryptDBUtils.getDefaultFirstEncryptParam()),
+        FastCompactionTaskSummary::new);
   }
 
-  private void genSourceFiles() throws Exception{
+  private void genSourceFiles() throws Exception {
     String fileSetDir =
         TestConstant.BASE_OUTPUT_PATH + File.separator + 
TsFileSet.FILE_SET_DIR_NAME;
     // seq-file1:
@@ -140,10 +173,10 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
     }
     TsFileResource resource1 = new TsFileResource(seqf1);
     resource1.setTsFileManager(tsFileManager);
-    resource1.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table1"}), 0);
-    resource1.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table1"}), 0);
-    resource1.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 0);
-    resource1.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 0);
+    resource1.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table1"}), 0);
+    resource1.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table1"}), 0);
+    resource1.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 0);
+    resource1.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 0);
     resource1.close();
 
     // rename table1 -> table0
@@ -215,13 +248,12 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
     }
     TsFileResource resource2 = new TsFileResource(seqf2);
     resource2.setTsFileManager(tsFileManager);
-    resource2.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 1);
-    resource2.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 1);
-    resource2.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 1);
-    resource2.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 1);
+    resource2.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 1);
+    resource2.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 1);
+    resource2.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 1);
+    resource2.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 1);
     resource2.close();
 
-
     // rename table0.s1 -> table0.s0
     TsFileSet tsFileSet2 = new TsFileSet(2, fileSetDir, false);
     tsFileSet2.appendSchemaEvolution(
@@ -291,10 +323,10 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
     }
     TsFileResource resource3 = new TsFileResource(seqf3);
     resource3.setTsFileManager(tsFileManager);
-    resource3.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 2);
-    resource3.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 2);
-    resource3.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 2);
-    resource3.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 2);
+    resource3.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 2);
+    resource3.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 2);
+    resource3.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 2);
+    resource3.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 2);
     resource3.close();
 
     // rename table2 -> table1
@@ -370,20 +402,24 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
     }
     TsFileResource resource4 = new TsFileResource(unseqf4);
     resource4.setTsFileManager(tsFileManager);
-    resource4.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 1);
-    resource4.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 1);
-    resource4.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table1"}), 1);
-    resource4.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table1"}), 1);
+    resource4.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 1);
+    resource4.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 1);
+    resource4.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table1"}), 1);
+    resource4.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table1"}), 1);
     resource4.close();
     unseqResources.add(resource4);
   }
 
-  private void testCross(Function<List<TsFileResource>, ICompactionPerformer> 
compactionPerformerFunction, Supplier<CompactionTaskSummary> summarySupplier) 
throws Exception {
+  private void testCross(
+      Function<List<TsFileResource>, ICompactionPerformer> 
compactionPerformerFunction,
+      Supplier<CompactionTaskSummary> summarySupplier)
+      throws Exception {
     genSourceFiles();
     List<TsFileResource> targetResources;
     ICompactionPerformer performer;
 
-    targetResources = 
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
+    targetResources =
+        
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
     targetResources.forEach(s -> s.setTsFileManager(tsFileManager));
 
     performer = compactionPerformerFunction.apply(targetResources);
@@ -397,7 +433,8 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
         new 
TsFileReaderBuilder().file(targetResources.get(0).getTsFile()).build()) {
       // table0 should not exist
       try {
-        tsFileReader.query("table0", Collections.singletonList("s2"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table0", Collections.singletonList("s2"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table0 should not exist");
       } catch (NoTableException e) {
         assertEquals("Table table0 not found", e.getMessage());
@@ -405,15 +442,17 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
 
       // table1.s0 should not exist
       try {
-        tsFileReader.query("table1", Collections.singletonList("s0"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table1", Collections.singletonList("s0"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table1.s0 should not exist");
       } catch (NoMeasurementException e) {
         assertEquals("No measurement for s0", e.getMessage());
       }
 
       // check data of table1
-      ResultSet resultSet = tsFileReader.query("table1", Arrays.asList("s1", 
"s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      ResultSet resultSet =
+          tsFileReader.query(
+              "table1", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       assertTrue(resultSet.next());
       assertEquals(0, resultSet.getLong(1));
       for (int j = 0; j < 3; j++) {
@@ -421,8 +460,9 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
       }
 
       // check data of table2
-      resultSet = tsFileReader.query("table2", Arrays.asList("s1", "s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      resultSet =
+          tsFileReader.query(
+              "table2", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       assertTrue(resultSet.next());
       assertEquals(0, resultSet.getLong(1));
       for (int j = 0; j < 3; j++) {
@@ -437,7 +477,8 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
         new 
TsFileReaderBuilder().file(targetResources.get(1).getTsFile()).build()) {
       // table1 should not exist
       try {
-        tsFileReader.query("table1", Collections.singletonList("s2"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table1", Collections.singletonList("s2"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table1 should not exist");
       } catch (NoTableException e) {
         assertEquals("Table table1 not found", e.getMessage());
@@ -445,15 +486,17 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
 
       // table0.s0 should not exist
       try {
-        tsFileReader.query("table0", Collections.singletonList("s0"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table0", Collections.singletonList("s0"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table0.s0 should not exist");
       } catch (NoMeasurementException e) {
         assertEquals("No measurement for s0", e.getMessage());
       }
 
       // check data of table0
-      ResultSet resultSet = tsFileReader.query("table0", Arrays.asList("s1", 
"s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      ResultSet resultSet =
+          tsFileReader.query(
+              "table0", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       assertTrue(resultSet.next());
       assertEquals(1, resultSet.getLong(1));
       for (int j = 0; j < 3; j++) {
@@ -461,8 +504,9 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
       }
 
       // check data of table2
-      resultSet = tsFileReader.query("table2", Arrays.asList("s1", "s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      resultSet =
+          tsFileReader.query(
+              "table2", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       assertTrue(resultSet.next());
       assertEquals(1, resultSet.getLong(1));
       for (int j = 0; j < 3; j++) {
@@ -477,7 +521,8 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
         new 
TsFileReaderBuilder().file(targetResources.get(2).getTsFile()).build()) {
       // table1 should not exist
       try {
-        tsFileReader.query("table1", Collections.singletonList("s2"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table1", Collections.singletonList("s2"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table1 should not exist");
       } catch (NoTableException e) {
         assertEquals("Table table1 not found", e.getMessage());
@@ -485,15 +530,17 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
 
       // table0.s1 should not exist
       try {
-        tsFileReader.query("table0", Collections.singletonList("s1"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table0", Collections.singletonList("s1"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table0.s0 should not exist");
       } catch (NoMeasurementException e) {
         assertEquals("No measurement for s1", e.getMessage());
       }
 
       // check data of table0
-      ResultSet resultSet = tsFileReader.query("table0", Arrays.asList("s0", 
"s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      ResultSet resultSet =
+          tsFileReader.query(
+              "table0", Arrays.asList("s0", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       assertTrue(resultSet.next());
       assertEquals(2, resultSet.getLong(1));
       for (int j = 0; j < 3; j++) {
@@ -501,8 +548,9 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
       }
 
       // check data of table2
-      resultSet = tsFileReader.query("table2", Arrays.asList("s1", "s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      resultSet =
+          tsFileReader.query(
+              "table2", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       assertTrue(resultSet.next());
       assertEquals(2, resultSet.getLong(1));
       for (int j = 0; j < 3; j++) {
@@ -511,7 +559,10 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
     }
   }
 
-  private void testInner(Function<List<TsFileResource>, ICompactionPerformer> 
compactionPerformerFunction, Supplier<CompactionTaskSummary> summarySupplier) 
throws Exception {
+  private void testInner(
+      Function<List<TsFileResource>, ICompactionPerformer> 
compactionPerformerFunction,
+      Supplier<CompactionTaskSummary> summarySupplier)
+      throws Exception {
     genSourceFiles();
     List<TsFileResource> targetResources;
     ICompactionPerformer performer;
@@ -519,7 +570,8 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
     // target(version=1):
     // table1[s1, s2, s3]
     // table2[s1, s2, s3]
-    targetResources = 
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources,
 true);
+    targetResources =
+        
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources,
 true);
     targetResources.forEach(s -> s.setTsFileManager(tsFileManager));
 
     performer = compactionPerformerFunction.apply(targetResources);
@@ -530,7 +582,8 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
         new 
TsFileReaderBuilder().file(targetResources.get(0).getTsFile()).build()) {
       // table0 should not exist
       try {
-        tsFileReader.query("table0", Collections.singletonList("s2"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table0", Collections.singletonList("s2"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table0 should not exist");
       } catch (NoTableException e) {
         assertEquals("Table table0 not found", e.getMessage());
@@ -538,15 +591,17 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
 
       // table1.s0 should not exist
       try {
-        tsFileReader.query("table1", Collections.singletonList("s0"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table1", Collections.singletonList("s0"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table1.s0 should not exist");
       } catch (NoMeasurementException e) {
         assertEquals("No measurement for s0", e.getMessage());
       }
 
       // check data of table1
-      ResultSet resultSet = tsFileReader.query("table1", Arrays.asList("s1", 
"s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      ResultSet resultSet =
+          tsFileReader.query(
+              "table1", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       for (int i = 0; i < 3; i++) {
         assertTrue(resultSet.next());
         assertEquals(i, resultSet.getLong(1));
@@ -556,8 +611,9 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
       }
 
       // check data of table2
-      resultSet = tsFileReader.query("table2", Arrays.asList("s1", "s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      resultSet =
+          tsFileReader.query(
+              "table2", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       for (int i = 0; i < 3; i++) {
         assertTrue(resultSet.next());
         assertEquals(i, resultSet.getLong(1));
@@ -570,8 +626,9 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
     // target(version=2):
     // table0[s1, s2, s3]
     // table2[s1, s2, s3]
-    targetResources = 
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources.subList(1,
-        seqResources.size()), true);
+    targetResources =
+        CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(
+            seqResources.subList(1, seqResources.size()), true);
     targetResources.forEach(s -> s.setTsFileManager(tsFileManager));
 
     performer = compactionPerformerFunction.apply(targetResources);
@@ -582,7 +639,8 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
         new 
TsFileReaderBuilder().file(targetResources.get(0).getTsFile()).build()) {
       // table1 should not exist
       try {
-        tsFileReader.query("table1", Collections.singletonList("s2"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table1", Collections.singletonList("s2"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table1 should not exist");
       } catch (NoTableException e) {
         assertEquals("Table table1 not found", e.getMessage());
@@ -590,15 +648,17 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
 
       // table0.s0 should not exist
       try {
-        tsFileReader.query("table0", Collections.singletonList("s0"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table0", Collections.singletonList("s0"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table0.s0 should not exist");
       } catch (NoMeasurementException e) {
         assertEquals("No measurement for s0", e.getMessage());
       }
 
       // check data of table0
-      ResultSet resultSet = tsFileReader.query("table0", Arrays.asList("s1", 
"s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      ResultSet resultSet =
+          tsFileReader.query(
+              "table0", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       for (int i = 0; i < 3; i++) {
         assertTrue(resultSet.next());
         assertEquals(i, resultSet.getLong(1));
@@ -608,8 +668,9 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
       }
 
       // check data of table2
-      resultSet = tsFileReader.query("table2", Arrays.asList("s1", "s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      resultSet =
+          tsFileReader.query(
+              "table2", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       for (int i = 0; i < 3; i++) {
         assertTrue(resultSet.next());
         assertEquals(i, resultSet.getLong(1));
@@ -622,8 +683,9 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
     // target(version=2):
     // table0[s0, s2, s3]
     // table2[s1, s2, s3]
-    targetResources = 
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources.subList(2,
-        seqResources.size()), true);
+    targetResources =
+        CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(
+            seqResources.subList(2, seqResources.size()), true);
     targetResources.forEach(s -> s.setTsFileManager(tsFileManager));
 
     performer = compactionPerformerFunction.apply(targetResources);
@@ -634,7 +696,8 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
         new 
TsFileReaderBuilder().file(targetResources.get(0).getTsFile()).build()) {
       // table1 should not exist
       try {
-        tsFileReader.query("table1", Collections.singletonList("s2"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table1", Collections.singletonList("s2"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table1 should not exist");
       } catch (NoTableException e) {
         assertEquals("Table table1 not found", e.getMessage());
@@ -642,15 +705,17 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
 
       // table0.s1 should not exist
       try {
-        tsFileReader.query("table0", Collections.singletonList("s1"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table0", Collections.singletonList("s1"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table0.s0 should not exist");
       } catch (NoMeasurementException e) {
         assertEquals("No measurement for s1", e.getMessage());
       }
 
       // check data of table0
-      ResultSet resultSet = tsFileReader.query("table0", Arrays.asList("s0", 
"s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      ResultSet resultSet =
+          tsFileReader.query(
+              "table0", Arrays.asList("s0", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       for (int i = 0; i < 3; i++) {
         assertTrue(resultSet.next());
         assertEquals(i, resultSet.getLong(1));
@@ -660,8 +725,9 @@ public class CompactionWithSevoTest extends 
AbstractCompactionTest{
       }
 
       // check data of table2
-      resultSet = tsFileReader.query("table2", Arrays.asList("s1", "s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      resultSet =
+          tsFileReader.query(
+              "table2", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       for (int i = 0; i < 3; i++) {
         assertTrue(resultSet.next());
         assertEquals(i, resultSet.getLong(1));
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastInnerCompactionPerformerTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastInnerCompactionPerformerTest.java
index e3aae24bcf7..d75d7c4ede7 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastInnerCompactionPerformerTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastInnerCompactionPerformerTest.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iotdb.db.storageengine.dataregion.compaction;
 
-import java.io.File;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.AlignedFullPath;
@@ -27,10 +26,7 @@ import org.apache.iotdb.commons.path.NonAlignedFullPath;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import 
org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext;
-import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICompactionPerformer;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer;
-import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadChunkCompactionPerformer;
-import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CompactionTaskSummary;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.InnerSpaceCompactionTask;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.reader.IDataBlockReader;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.reader.SeriesDataBlockReader;
@@ -38,26 +34,16 @@ import 
org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionF
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionTestFileWriter;
 import 
org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager;
 import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
-import 
org.apache.iotdb.db.storageengine.dataregion.tsfile.evolution.ColumnRename;
-import 
org.apache.iotdb.db.storageengine.dataregion.tsfile.evolution.TableRename;
-import org.apache.iotdb.db.storageengine.dataregion.tsfile.fileset.TsFileSet;
-import org.apache.iotdb.db.utils.EncryptDBUtils;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 
-import org.apache.iotdb.db.utils.constant.TestConstant;
 import org.apache.tsfile.common.conf.TSFileDescriptor;
-import org.apache.tsfile.enums.ColumnCategory;
 import org.apache.tsfile.enums.TSDataType;
-import org.apache.tsfile.exception.write.NoMeasurementException;
-import org.apache.tsfile.exception.write.NoTableException;
 import org.apache.tsfile.exception.write.WriteProcessException;
 import org.apache.tsfile.file.metadata.AbstractAlignedChunkMetadata;
 import org.apache.tsfile.file.metadata.ChunkMetadata;
-import org.apache.tsfile.file.metadata.ColumnSchemaBuilder;
 import org.apache.tsfile.file.metadata.IChunkMetadata;
 import org.apache.tsfile.file.metadata.IDeviceID;
 import org.apache.tsfile.file.metadata.IDeviceID.Factory;
-import org.apache.tsfile.file.metadata.TableSchema;
 import org.apache.tsfile.file.metadata.enums.CompressionType;
 import org.apache.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.tsfile.read.TimeValuePair;
@@ -67,16 +53,11 @@ import org.apache.tsfile.read.common.Chunk;
 import org.apache.tsfile.read.common.IBatchDataIterator;
 import org.apache.tsfile.read.common.TimeRange;
 import org.apache.tsfile.read.common.block.TsBlock;
-import org.apache.tsfile.read.query.dataset.ResultSet;
 import org.apache.tsfile.read.reader.IPointReader;
 import org.apache.tsfile.read.reader.chunk.AlignedChunkReader;
-import org.apache.tsfile.read.v4.ITsFileReader;
-import org.apache.tsfile.read.v4.TsFileReaderBuilder;
 import org.apache.tsfile.utils.Pair;
 import org.apache.tsfile.utils.TsFileGeneratorUtils;
 import org.apache.tsfile.utils.TsPrimitiveType;
-import org.apache.tsfile.write.TsFileWriter;
-import org.apache.tsfile.write.record.Tablet;
 import org.apache.tsfile.write.schema.IMeasurementSchema;
 import org.apache.tsfile.write.schema.MeasurementSchema;
 import org.junit.After;
@@ -94,8 +75,6 @@ import java.util.Map;
 
 import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_SEPARATOR;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 @SuppressWarnings("OptionalGetWithoutIsPresent")
 public class FastInnerCompactionPerformerTest extends AbstractCompactionTest {
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadChunkInnerCompactionTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadChunkInnerCompactionTest.java
index e72436207a7..972db81b20a 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadChunkInnerCompactionTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadChunkInnerCompactionTest.java
@@ -26,54 +26,34 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.service.metrics.FileMetrics;
-import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICompactionPerformer;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadChunkCompactionPerformer;
-import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadPointCompactionPerformer;
-import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CompactionTaskSummary;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.InnerSpaceCompactionTask;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.log.CompactionLogger;
-import 
org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionFileGeneratorUtils;
 import 
org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionTestFileWriter;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile;
 import 
org.apache.iotdb.db.storageengine.dataregion.modification.TreeDeletionEntry;
 import 
org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager;
 import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
 
-import 
org.apache.iotdb.db.storageengine.dataregion.tsfile.evolution.ColumnRename;
-import 
org.apache.iotdb.db.storageengine.dataregion.tsfile.evolution.TableRename;
-import org.apache.iotdb.db.storageengine.dataregion.tsfile.fileset.TsFileSet;
-import org.apache.iotdb.db.utils.EncryptDBUtils;
-import org.apache.iotdb.db.utils.constant.TestConstant;
 import org.apache.tsfile.common.conf.TSFileConfig;
 import org.apache.tsfile.common.conf.TSFileDescriptor;
 import org.apache.tsfile.common.constant.TsFileConstant;
-import org.apache.tsfile.enums.ColumnCategory;
 import org.apache.tsfile.enums.TSDataType;
-import org.apache.tsfile.exception.write.NoMeasurementException;
-import org.apache.tsfile.exception.write.NoTableException;
 import org.apache.tsfile.exception.write.PageException;
 import org.apache.tsfile.exception.write.WriteProcessException;
-import org.apache.tsfile.file.metadata.ColumnSchemaBuilder;
 import org.apache.tsfile.file.metadata.IDeviceID;
-import org.apache.tsfile.file.metadata.IDeviceID.Factory;
-import org.apache.tsfile.file.metadata.TableSchema;
 import org.apache.tsfile.file.metadata.enums.CompressionType;
 import org.apache.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.tsfile.read.TimeValuePair;
 import org.apache.tsfile.read.common.TimeRange;
-import org.apache.tsfile.read.query.dataset.ResultSet;
-import org.apache.tsfile.read.v4.ITsFileReader;
-import org.apache.tsfile.read.v4.TsFileReaderBuilder;
 import org.apache.tsfile.utils.Binary;
-import org.apache.tsfile.write.TsFileWriter;
 import org.apache.tsfile.write.chunk.AlignedChunkWriterImpl;
 import org.apache.tsfile.write.chunk.ChunkWriterImpl;
 import org.apache.tsfile.write.chunk.IChunkWriter;
 import org.apache.tsfile.write.chunk.ValueChunkWriter;
 import org.apache.tsfile.write.page.TimePageWriter;
 import org.apache.tsfile.write.page.ValuePageWriter;
-import org.apache.tsfile.write.record.Tablet;
 import org.apache.tsfile.write.writer.TsFileIOWriter;
 import org.junit.After;
 import org.junit.Assert;
@@ -99,9 +79,6 @@ import static 
org.apache.iotdb.db.storageengine.dataregion.compaction.utils.TsFi
 import static 
org.apache.iotdb.db.storageengine.dataregion.compaction.utils.TsFileGeneratorUtils.writeNonAlignedChunk;
 import static 
org.apache.iotdb.db.storageengine.dataregion.compaction.utils.TsFileGeneratorUtils.writeOneAlignedPage;
 import static org.apache.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 public class ReadChunkInnerCompactionTest extends AbstractCompactionTest {
   @Before
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointCompactionPerformerTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointCompactionPerformerTest.java
index f926506e889..b74bcec8f96 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointCompactionPerformerTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointCompactionPerformerTest.java
@@ -7079,10 +7079,10 @@ public class ReadPointCompactionPerformerTest extends 
AbstractCompactionTest {
     }
     TsFileResource resource1 = new TsFileResource(f1);
     resource1.setTsFileManager(tsFileManager);
-    resource1.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table1"}), 0);
-    resource1.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table1"}), 0);
-    resource1.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 0);
-    resource1.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 0);
+    resource1.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table1"}), 0);
+    resource1.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table1"}), 0);
+    resource1.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 0);
+    resource1.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 0);
     resource1.close();
 
     // rename table1 -> table0
@@ -7154,13 +7154,12 @@ public class ReadPointCompactionPerformerTest extends 
AbstractCompactionTest {
     }
     TsFileResource resource2 = new TsFileResource(f2);
     resource2.setTsFileManager(tsFileManager);
-    resource2.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 1);
-    resource2.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 1);
-    resource2.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 1);
-    resource2.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 1);
+    resource2.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 1);
+    resource2.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 1);
+    resource2.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 1);
+    resource2.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 1);
     resource2.close();
 
-
     // rename table0.s1 -> table0.s0
     TsFileSet tsFileSet2 = new TsFileSet(2, fileSetDir, false);
     tsFileSet2.appendSchemaEvolution(
@@ -7230,10 +7229,10 @@ public class ReadPointCompactionPerformerTest extends 
AbstractCompactionTest {
     }
     TsFileResource resource3 = new TsFileResource(f3);
     resource3.setTsFileManager(tsFileManager);
-    resource3.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 2);
-    resource3.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table0"}), 2);
-    resource3.updateStartTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 2);
-    resource3.updateEndTime(Factory.DEFAULT_FACTORY.create(new 
String[]{"table2"}), 2);
+    resource3.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 2);
+    resource3.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table0"}), 2);
+    resource3.updateStartTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 2);
+    resource3.updateEndTime(Factory.DEFAULT_FACTORY.create(new String[] 
{"table2"}), 2);
     resource3.close();
 
     // rename table2 -> table1
@@ -7263,7 +7262,8 @@ public class ReadPointCompactionPerformerTest extends 
AbstractCompactionTest {
         new 
TsFileReaderBuilder().file(targetResources.get(0).getTsFile()).build()) {
       // table1 should not exist
       try {
-        tsFileReader.query("table0", Collections.singletonList("s2"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table0", Collections.singletonList("s2"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table0 should not exist");
       } catch (NoTableException e) {
         assertEquals("Table table0 not found", e.getMessage());
@@ -7271,15 +7271,17 @@ public class ReadPointCompactionPerformerTest extends 
AbstractCompactionTest {
 
       // table1.s0 should not exist
       try {
-        tsFileReader.query("table1", Collections.singletonList("s0"), 
Long.MIN_VALUE, Long.MAX_VALUE);
+        tsFileReader.query(
+            "table1", Collections.singletonList("s0"), Long.MIN_VALUE, 
Long.MAX_VALUE);
         fail("table1.s0 should not exist");
       } catch (NoMeasurementException e) {
         assertEquals("No measurement for s0", e.getMessage());
       }
 
       // check data of table1
-      ResultSet resultSet = tsFileReader.query("table1", Arrays.asList("s1", 
"s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      ResultSet resultSet =
+          tsFileReader.query(
+              "table1", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       for (int i = 0; i < 3; i++) {
         assertTrue(resultSet.next());
         assertEquals(i, resultSet.getLong(1));
@@ -7289,8 +7291,9 @@ public class ReadPointCompactionPerformerTest extends 
AbstractCompactionTest {
       }
 
       // check data of table2
-      resultSet = tsFileReader.query("table2", Arrays.asList("s1", "s2", "s3"),
-          Long.MIN_VALUE, Long.MAX_VALUE);
+      resultSet =
+          tsFileReader.query(
+              "table2", Arrays.asList("s1", "s2", "s3"), Long.MIN_VALUE, 
Long.MAX_VALUE);
       for (int i = 0; i < 3; i++) {
         assertTrue(resultSet.next());
         assertEquals(i, resultSet.getLong(1));

Reply via email to