From 598cabe238b89378c4e439628257cb819fabd9b8 Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Fri, 20 Sep 2024 17:23:54 +0800 Subject: [PATCH] Keep all null rows in table model compaction (#13523) * keep all null rows in table model compaction * add ut * adddeletion ut * add ut * revert some ut * fix ut * add context in cross space compaction task selection * fix bug * spotless * modify repair task * update tsfile version --- .../source/AlignedSeriesScanOperator.java | 3 +- .../source/AlignedSeriesScanUtil.java | 7 +- .../source/relational/TableScanOperator.java | 3 +- .../storageengine/dataregion/DataRegion.java | 8 +- .../performer/ICompactionPerformer.java | 2 + .../impl/FastCompactionPerformer.java | 13 +- .../impl/ReadChunkCompactionPerformer.java | 11 +- .../impl/ReadPointCompactionPerformer.java | 10 +- ...RepairUnsortedFileCompactionPerformer.java | 3 +- .../RepairUnsortedFileCompactionTask.java | 20 +- .../FastCompactionPerformerSubTask.java | 13 +- .../utils/MultiTsFileDeviceIterator.java | 42 ++- ...edFastAlignedSeriesCompactionExecutor.java | 56 ++-- ...dChunkAlignedSeriesCompactionExecutor.java | 60 ++-- .../AlignedSeriesBatchCompactionUtils.java | 20 +- ...actionAlignedPageLazyLoadPointReader.java} | 49 ++- .../FastAlignedSeriesCompactionExecutor.java | 19 +- .../reader/CompactionAlignedChunkReader.java | 79 ++--- ...dChunkAlignedSeriesCompactionExecutor.java | 85 +++-- .../repair/RepairDataFileScanUtil.java | 2 +- .../repair/RepairTimePartitionScanTask.java | 6 +- .../schedule/CompactionScheduleContext.java | 40 ++- .../schedule/CompactionScheduler.java | 15 +- .../constant/CrossCompactionSelector.java | 6 +- .../InnerSequenceCompactionSelector.java | 2 +- .../InnerUnsequenceCompactionSelector.java | 2 +- .../impl/NewSizeTieredCompactionSelector.java | 4 +- .../RewriteCrossSpaceCompactionSelector.java | 15 +- .../selector/impl/SettleSelectorImpl.java | 16 +- .../impl/SizeTieredCompactionSelector.java | 21 +- .../utils/CrossSpaceCompactionCandidate.java | 4 +- .../utils/TsFileResourceCandidate.java | 1 + .../compaction/AbstractCompactionTest.java | 3 +- ...ssSpaceCompactionWithUnusualCasesTest.java | 37 +- ...sistentCompressionTypeAndEncodingTest.java | 3 +- .../FastInnerCompactionPerformerTest.java | 2 +- .../CrossSpaceCompactionSelectorTest.java | 63 ++-- ...sSpaceCompactionWithFastPerformerTest.java | 10 +- ...actionWithFastPerformerValidationTest.java | 136 ++++---- ...eCompactionWithReadPointPerformerTest.java | 10 +- ...nWithReadPointPerformerValidationTest.java | 130 ++++---- ...ertionCrossSpaceCompactionRecoverTest.java | 13 +- ...rtionCrossSpaceCompactionSelectorTest.java | 105 ++++-- .../InsertionCrossSpaceCompactionTest.java | 15 +- .../compaction/cross/MergeUpgradeTest.java | 4 +- .../RewriteCompactionFileSelectorTest.java | 42 +-- .../InnerSpaceCompactionSelectorTest.java | 25 +- .../SizeTieredCompactionSelectorTest.java | 7 +- .../RepairUnsortedFileCompactionTest.java | 28 +- .../settle/SettleCompactionSelectorTest.java | 63 ++-- .../settle/SettleCompactionTaskTest.java | 4 +- .../CompactionWithAllNullRowsTest.java | 315 ++++++++++++++++++ .../utils/BatchCompactionUtilsTest.java | 13 +- .../utils/CompactionCheckerUtils.java | 3 +- .../utils/MultiTsFileDeviceIteratorTest.java | 38 +-- pom.xml | 2 +- 56 files changed, 1175 insertions(+), 533 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/{BatchedCompactionAlignedPagePointReader.java => CompactionAlignedPageLazyLoadPointReader.java} (67%) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/CompactionWithAllNullRowsTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanOperator.java index 51fe0b0e952a..1fc8014b3ea8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanOperator.java @@ -64,8 +64,7 @@ public AlignedSeriesScanOperator( seriesScanOptions, context.getInstanceContext(), queryAllSensors, - dataTypes, - true); + dataTypes); this.valueColumnCount = seriesPath.getColumnNum(); this.maxReturnSize = Math.min( 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 6dff940b800e..f7ddaee472ea 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 @@ -59,7 +59,7 @@ public AlignedSeriesScanUtil( Ordering scanOrder, SeriesScanOptions scanOptions, FragmentInstanceContext context) { - this(seriesPath, scanOrder, scanOptions, context, false, null, true); + this(seriesPath, scanOrder, scanOptions, context, false, null); } public AlignedSeriesScanUtil( @@ -68,8 +68,7 @@ public AlignedSeriesScanUtil( SeriesScanOptions scanOptions, FragmentInstanceContext context, boolean queryAllSensors, - List givenDataTypes, - boolean ignoreAllNullRows) { + List givenDataTypes) { super(seriesPath, scanOrder, scanOptions, context); isAligned = true; this.dataTypes = @@ -79,7 +78,7 @@ public AlignedSeriesScanUtil( .map(IMeasurementSchema::getType) .collect(Collectors.toList()); this.queryAllSensors = queryAllSensors; - this.ignoreAllNullRows = ignoreAllNullRows; + this.ignoreAllNullRows = context.isIgnoreAllNullRows(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java index d3efac0a1e3a..909825497163 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java @@ -306,8 +306,7 @@ private AlignedSeriesScanUtil constructAlignedSeriesScanUtil(DeviceEntry deviceE seriesScanOptions, operatorContext.getInstanceContext(), true, - measurementColumnTSDataTypes, - false); + measurementColumnTSDataTypes); } public static AlignedFullPath constructAlignedPath( 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 5915be052079..062e8225ea21 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 @@ -313,6 +313,8 @@ public class DataRegion implements IDataRegionForQuery { private final AtomicBoolean isCompactionSelecting = new AtomicBoolean(false); + private boolean isTreeModel; + private static final QueryResourceMetricSet QUERY_RESOURCE_METRIC_SET = QueryResourceMetricSet.getInstance(); @@ -333,6 +335,7 @@ public DataRegion( this.dataRegionId = dataRegionId; this.databaseName = databaseName; this.fileFlushPolicy = fileFlushPolicy; + this.isTreeModel = this.databaseName.startsWith("root."); acquireDirectBufferMemory(); dataRegionSysDir = SystemFileFactory.INSTANCE.getFile(systemDir, dataRegionId); @@ -383,6 +386,7 @@ public DataRegion(String databaseName, String id) { this.dataRegionId = id; this.tsFileManager = new TsFileManager(databaseName, id, ""); this.partitionMaxFileVersions = new HashMap<>(); + this.isTreeModel = this.databaseName.startsWith("root."); partitionMaxFileVersions.put(0L, 0L); } @@ -2765,7 +2769,7 @@ public int executeCompaction() throws InterruptedException { // Sort the time partition from largest to smallest timePartitions.sort(Comparator.reverseOrder()); - CompactionScheduleContext context = new CompactionScheduleContext(); + CompactionScheduleContext context = new CompactionScheduleContext(this.isTreeModel); // schedule insert compaction trySubmitCount += executeInsertionCompaction(timePartitions, context); @@ -2808,7 +2812,7 @@ public int executeTTLCheck() throws InterruptedException { logger.info("[TTL] {}-{} Start ttl checking.", databaseName, dataRegionId); int trySubmitCount = 0; try { - CompactionScheduleContext context = new CompactionScheduleContext(); + CompactionScheduleContext context = new CompactionScheduleContext(this.isTreeModel); List timePartitions = new ArrayList<>(tsFileManager.getTimePartitions()); // Sort the time partition from smallest to largest Collections.sort(timePartitions); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/ICompactionPerformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/ICompactionPerformer.java index 1dc24d12f700..091a8ed1e5ad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/ICompactionPerformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/ICompactionPerformer.java @@ -40,6 +40,8 @@ public interface ICompactionPerformer { void setSummary(CompactionTaskSummary summary); + void setIgnoreAllNullRows(boolean ignoreAllNullRows); + default void setSourceFiles(List files) { throw new IllegalSourceFileTypeException( "Cannot set single type of source files to this kind of performer"); 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 59bc2d4dae1e..9708b4d80e88 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 @@ -91,6 +91,7 @@ public class FastCompactionPerformer modificationCache = new ConcurrentHashMap<>(); private final boolean isCrossCompaction; + private boolean ignoreAllNullRows = true; public FastCompactionPerformer( List seqFiles, @@ -115,7 +116,7 @@ public FastCompactionPerformer(boolean isCrossCompaction) { public void perform() throws Exception { this.subTaskSummary.setTemporalFileNum(targetFiles.size()); try (MultiTsFileDeviceIterator deviceIterator = - new MultiTsFileDeviceIterator(seqFiles, unseqFiles, readerCacheMap); + new MultiTsFileDeviceIterator(seqFiles, unseqFiles, readerCacheMap, ignoreAllNullRows); AbstractCompactionWriter compactionWriter = isCrossCompaction ? new FastCrossCompactionWriter(targetFiles, seqFiles, readerCacheMap) @@ -192,7 +193,7 @@ private void compactAlignedSeries( // end offset of each timeseries metadata, in order to facilitate the reading of chunkMetadata // directly by this offset later. Instead of deserializing chunk metadata later, we need to // deserialize chunk metadata here to get the schemas of all value measurements, because we - // should get schemas of all value measurement to startMeasruement() and compaction process is + // should get schemas of all value measurement to startMeasurement() and compaction process is // to read a batch of overlapped files each time, and we cannot make sure if the first batch of // overlapped tsfiles contain all the value measurements. for (Map.Entry>>> entry : @@ -210,7 +211,8 @@ private void compactAlignedSeries( sortedSourceFiles, measurementSchemas, deviceId, - taskSummary) + taskSummary, + ignoreAllNullRows) .call(); subTaskSummary.increase(taskSummary); } @@ -296,6 +298,11 @@ public void setSummary(CompactionTaskSummary summary) { this.subTaskSummary = (FastCompactionTaskSummary) summary; } + @Override + public void setIgnoreAllNullRows(boolean ignoreAllNullRows) { + this.ignoreAllNullRows = ignoreAllNullRows; + } + @Override public void setSourceFiles(List seqFiles, List unseqFiles) { this.seqFiles = seqFiles; 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 b19462c77933..d000d58b4c2c 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 @@ -55,6 +55,7 @@ public class ReadChunkCompactionPerformer implements ISeqCompactionPerformer { private CompactionTsFileWriter currentWriter; private long endedFileSize = 0; private int currentTargetFileIndex = 0; + private boolean ignoreAllNullRows = true; // memory budget for file writer is 5% of per compaction task memory budget private final long memoryBudgetForFileWriter = (long) @@ -86,7 +87,8 @@ public void perform() InterruptedException, StorageEngineException, PageException { - try (MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(seqFiles)) { + try (MultiTsFileDeviceIterator deviceIterator = + new MultiTsFileDeviceIterator(seqFiles, ignoreAllNullRows)) { schema = CompactionTableSchemaCollector.collectSchema(seqFiles, deviceIterator.getReaderMap()); while (deviceIterator.hasNextDevice()) { @@ -172,6 +174,11 @@ public void setSummary(CompactionTaskSummary summary) { this.summary = summary; } + @Override + public void setIgnoreAllNullRows(boolean ignoreAllNullRows) { + this.ignoreAllNullRows = ignoreAllNullRows; + } + private void compactAlignedSeries( IDeviceID device, TsFileResource targetResource, @@ -187,7 +194,7 @@ private void compactAlignedSeries( writer.startChunkGroup(device); BatchedReadChunkAlignedSeriesCompactionExecutor compactionExecutor = new BatchedReadChunkAlignedSeriesCompactionExecutor( - device, targetResource, readerAndChunkMetadataList, writer, summary); + device, targetResource, readerAndChunkMetadataList, writer, summary, ignoreAllNullRows); compactionExecutor.execute(); for (ChunkMetadata chunkMetadata : writer.getChunkMetadataListOfCurrentDeviceInMemory()) { if (chunkMetadata.getMeasurementUid().isEmpty()) { 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 28d1d0476add..f890cb788350 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 @@ -79,6 +79,7 @@ public class ReadPointCompactionPerformer private CompactionTaskSummary summary; protected List targetFiles = Collections.emptyList(); + protected boolean ignoreAllNullRows = true; public ReadPointCompactionPerformer( List seqFiles, @@ -112,7 +113,7 @@ public void perform() throws Exception { getCompactionWriter(seqFiles, unseqFiles, targetFiles)) { // Do not close device iterator, because tsfile reader is managed by FileReaderManager. MultiTsFileDeviceIterator deviceIterator = - new MultiTsFileDeviceIterator(seqFiles, unseqFiles); + new MultiTsFileDeviceIterator(seqFiles, unseqFiles, ignoreAllNullRows); List schemas = CompactionTableSchemaCollector.collectSchema( seqFiles, unseqFiles, deviceIterator.getReaderMap()); @@ -152,6 +153,11 @@ public void setSummary(CompactionTaskSummary summary) { this.summary = summary; } + @Override + public void setIgnoreAllNullRows(boolean ignoreAllNullRows) { + this.ignoreAllNullRows = ignoreAllNullRows; + } + private void compactAlignedSeries( IDeviceID device, MultiTsFileDeviceIterator deviceIterator, @@ -169,6 +175,8 @@ private void compactAlignedSeries( measurementSchemas.stream() .map(IMeasurementSchema::getMeasurementId) .collect(Collectors.toList()); + + fragmentInstanceContext.setIgnoreAllNullRows(ignoreAllNullRows); IDataBlockReader dataBlockReader = constructReader( device, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/RepairUnsortedFileCompactionPerformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/RepairUnsortedFileCompactionPerformer.java index 4269169a3c3f..a0f57c5efa44 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/RepairUnsortedFileCompactionPerformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/RepairUnsortedFileCompactionPerformer.java @@ -36,9 +36,10 @@ public class RepairUnsortedFileCompactionPerformer extends ReadPointCompactionPe private final boolean rewriteFile; - public RepairUnsortedFileCompactionPerformer(boolean rewriteFile) { + public RepairUnsortedFileCompactionPerformer(boolean rewriteFile, boolean ignoreAllNullRows) { super(); this.rewriteFile = rewriteFile; + this.ignoreAllNullRows = ignoreAllNullRows; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/RepairUnsortedFileCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/RepairUnsortedFileCompactionTask.java index ccc02e9f5edd..e2ac24b0dbae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/RepairUnsortedFileCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/RepairUnsortedFileCompactionTask.java @@ -63,13 +63,14 @@ public RepairUnsortedFileCompactionTask( TsFileManager tsFileManager, TsFileResource sourceFile, boolean sequence, - long serialId) { + long serialId, + boolean ignoreAllNullRows) { super( timePartition, tsFileManager, Collections.singletonList(sourceFile), sequence, - new RepairUnsortedFileCompactionPerformer(true), + new RepairUnsortedFileCompactionPerformer(true, ignoreAllNullRows), serialId); this.sourceFile = sourceFile; this.innerSpaceEstimator = new RepairUnsortedFileCompactionEstimator(); @@ -82,13 +83,14 @@ public RepairUnsortedFileCompactionTask( TsFileResource sourceFile, boolean sequence, boolean rewriteFile, - long serialId) { + long serialId, + boolean ignoreAllNullRows) { super( timePartition, tsFileManager, Collections.singletonList(sourceFile), sequence, - new RepairUnsortedFileCompactionPerformer(rewriteFile), + new RepairUnsortedFileCompactionPerformer(rewriteFile, ignoreAllNullRows), serialId); this.sourceFile = sourceFile; if (rewriteFile) { @@ -103,13 +105,14 @@ public RepairUnsortedFileCompactionTask( TsFileResource sourceFile, CountDownLatch latch, boolean sequence, - long serialId) { + long serialId, + boolean ignoreAllNullRows) { super( timePartition, tsFileManager, Collections.singletonList(sourceFile), sequence, - new RepairUnsortedFileCompactionPerformer(true), + new RepairUnsortedFileCompactionPerformer(true, ignoreAllNullRows), serialId); this.sourceFile = sourceFile; this.innerSpaceEstimator = new RepairUnsortedFileCompactionEstimator(); @@ -124,13 +127,14 @@ public RepairUnsortedFileCompactionTask( CountDownLatch latch, boolean sequence, boolean rewriteFile, - long serialId) { + long serialId, + boolean ignoreAllNullRows) { super( timePartition, tsFileManager, Collections.singletonList(sourceFile), sequence, - new RepairUnsortedFileCompactionPerformer(rewriteFile), + new RepairUnsortedFileCompactionPerformer(rewriteFile, ignoreAllNullRows), serialId); this.sourceFile = sourceFile; if (rewriteFile) { 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 a604648f6093..bdda2c6a1cbc 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 @@ -67,6 +67,8 @@ public class FastCompactionPerformerSubTask implements Callable { private final boolean isAligned; + private final boolean ignoreAllNullRows; + private IDeviceID deviceId; private List measurements; @@ -96,6 +98,7 @@ public FastCompactionPerformerSubTask( this.sortedSourceFiles = sortedSourceFiles; this.measurements = measurements; this.summary = summary; + this.ignoreAllNullRows = true; } /** Used for aligned timeseries. */ @@ -108,7 +111,8 @@ public FastCompactionPerformerSubTask( List sortedSourceFiles, List measurementSchemas, IDeviceID deviceId, - FastCompactionTaskSummary summary) { + FastCompactionTaskSummary summary, + boolean ignoreAllNullRows) { this.compactionWriter = compactionWriter; this.subTaskId = 0; this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap; @@ -119,6 +123,7 @@ public FastCompactionPerformerSubTask( this.sortedSourceFiles = sortedSourceFiles; this.measurementSchemas = measurementSchemas; this.summary = summary; + this.ignoreAllNullRows = ignoreAllNullRows; } @Override @@ -154,7 +159,8 @@ public Void call() deviceId, subTaskId, measurementSchemas, - summary); + summary, + ignoreAllNullRows); } else { seriesCompactionExecutor = new FastAlignedSeriesCompactionExecutor( @@ -166,7 +172,8 @@ public Void call() deviceId, subTaskId, measurementSchemas, - summary); + summary, + ignoreAllNullRows); } seriesCompactionExecutor.execute(); } 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 d2b2bf25b28e..6b84d5ca7a10 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 @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.AlignedPath; import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.utils.CommonDateTimeUtils; @@ -71,13 +72,15 @@ public class MultiTsFileDeviceIterator implements AutoCloseable { private final Map> modificationCache = new HashMap<>(); private Pair currentDevice = null; private long timeLowerBoundForCurrentDevice; + private boolean ignoreAllNullRows = true; /** * Used for compaction with read chunk performer. * * @throws IOException if io error occurred */ - public MultiTsFileDeviceIterator(List tsFileResources) throws IOException { + public MultiTsFileDeviceIterator(List tsFileResources, boolean ignoreAllNullRows) + throws IOException { this.tsFileResourcesSortedByDesc = new ArrayList<>(tsFileResources); this.tsFileResourcesSortedByAsc = new ArrayList<>(tsFileResources); // sort the files from the oldest to the newest @@ -101,6 +104,7 @@ public MultiTsFileDeviceIterator(List tsFileResources) throws IO } throw e; } + this.ignoreAllNullRows = ignoreAllNullRows; } /** @@ -109,7 +113,10 @@ public MultiTsFileDeviceIterator(List tsFileResources) throws IO * @throws IOException if io errors occurred */ public MultiTsFileDeviceIterator( - List seqResources, List unseqResources) throws IOException { + List seqResources, + List unseqResources, + boolean ignoreAllNullRows) + throws IOException { this.tsFileResourcesSortedByDesc = new ArrayList<>(seqResources); tsFileResourcesSortedByDesc.addAll(unseqResources); // sort the files from the newest to the oldest @@ -121,6 +128,7 @@ public MultiTsFileDeviceIterator( readerMap.put(tsFileResource, reader); deviceIteratorMap.put(tsFileResource, reader.getAllDevicesIteratorWithIsAligned()); } + this.ignoreAllNullRows = ignoreAllNullRows; } /** @@ -131,7 +139,8 @@ public MultiTsFileDeviceIterator( public MultiTsFileDeviceIterator( List seqResources, List unseqResources, - Map readerMap) + Map readerMap, + boolean ignoreAllNullRows) throws IOException { this.tsFileResourcesSortedByDesc = new ArrayList<>(seqResources); tsFileResourcesSortedByDesc.addAll(unseqResources); @@ -139,6 +148,7 @@ public MultiTsFileDeviceIterator( Collections.sort( this.tsFileResourcesSortedByDesc, TsFileResource::compareFileCreationOrderByDesc); this.readerMap = readerMap; + this.ignoreAllNullRows = ignoreAllNullRows; CompactionType type = null; if (!seqResources.isEmpty() && !unseqResources.isEmpty()) { @@ -382,7 +392,7 @@ public Map getAllSchemasOfCurrentDevice() throws IOEx TsFileSequenceReader reader = readerMap.get(tsFileResource); List alignedChunkMetadataList = reader.getAlignedChunkMetadataByMetadataIndexNode( - currentDevice.left, firstMeasurementNodeOfCurrentDevice); + currentDevice.left, firstMeasurementNodeOfCurrentDevice, ignoreAllNullRows); applyModificationForAlignedChunkMetadataList(tsFileResource, alignedChunkMetadataList); readerAndChunkMetadataList.add(new Pair<>(reader, alignedChunkMetadataList)); } @@ -422,10 +432,23 @@ private void applyModificationForAlignedChunkMetadataList( // construct the input params List> for QueryUtils.modifyAlignedChunkMetaData AlignedChunkMetadata alignedChunkMetadata = alignedChunkMetadataList.get(0); List valueChunkMetadataList = alignedChunkMetadata.getValueChunkMetadataList(); - List> modificationForCurDevice = new ArrayList<>(); + + // match time column modifications + List modificationForTimeColumn = new ArrayList<>(); + for (Modification modification : modifications) { + if (modification.getPath().matchFullPath(device, AlignedPath.VECTOR_PLACEHOLDER)) { + modificationForTimeColumn.add(modification); + } + } + if (ttlDeletion != null) { + modificationForTimeColumn.add(ttlDeletion); + } + + // match value column modifications + List> modificationForValueColumns = new ArrayList<>(); for (IChunkMetadata valueChunkMetadata : valueChunkMetadataList) { if (valueChunkMetadata == null) { - modificationForCurDevice.add(Collections.emptyList()); + modificationForValueColumns.add(Collections.emptyList()); continue; } List modificationList = new ArrayList<>(); @@ -440,12 +463,15 @@ private void applyModificationForAlignedChunkMetadataList( if (ttlDeletion != null) { modificationList.add(ttlDeletion); } - modificationForCurDevice.add( + modificationForValueColumns.add( modificationList.isEmpty() ? Collections.emptyList() : modificationList); } ModificationUtils.modifyAlignedChunkMetaData( - alignedChunkMetadataList, modificationForCurDevice); + alignedChunkMetadataList, + modificationForTimeColumn, + modificationForValueColumns, + ignoreAllNullRows); } public Map getReaderMap() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/BatchedFastAlignedSeriesCompactionExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/BatchedFastAlignedSeriesCompactionExecutor.java index 9a31990b11ad..b22ed5df6f54 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/BatchedFastAlignedSeriesCompactionExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/BatchedFastAlignedSeriesCompactionExecutor.java @@ -55,6 +55,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -84,7 +85,8 @@ public BatchedFastAlignedSeriesCompactionExecutor( IDeviceID deviceId, int subTaskId, List measurementSchemas, - FastCompactionTaskSummary summary) { + FastCompactionTaskSummary summary, + boolean ignoreAllNullRows) { super( compactionWriter, timeseriesMetadataOffsetMap, @@ -94,7 +96,8 @@ public BatchedFastAlignedSeriesCompactionExecutor( deviceId, subTaskId, measurementSchemas, - summary); + summary, + ignoreAllNullRows); timeSchema = measurementSchemas.remove(0); valueMeasurementSchemas = measurementSchemas; this.batchColumnSelection = @@ -107,7 +110,7 @@ public BatchedFastAlignedSeriesCompactionExecutor( private List getAlignedChunkMetadataListBySelectedValueColumn( TsFileResource tsFileResource, List selectedValueMeasurementSchemas) - throws IOException { + throws IOException, IllegalPathException { // 1. get Full AlignedChunkMetadata from cache List alignedChunkMetadataList = null; if (alignedChunkMetadataCache.containsKey(tsFileResource)) { @@ -140,28 +143,32 @@ public void execute() private void compactFirstBatch() throws PageException, IllegalPathException, IOException, WriteProcessException { + List selectedMeasurementSchemas; if (!batchColumnSelection.hasNext()) { - return; + if (ignoreAllNullRows) { + return; + } + selectedMeasurementSchemas = Collections.singletonList(timeSchema); + } else { + batchColumnSelection.next(); + selectedMeasurementSchemas = + new ArrayList<>(batchColumnSelection.getCurrentSelectedColumnSchemaList().size() + 1); + selectedMeasurementSchemas.add(timeSchema); + selectedMeasurementSchemas.addAll(batchColumnSelection.getCurrentSelectedColumnSchemaList()); } - batchColumnSelection.next(); - - List currentBatchMeasurementSchemas = - new ArrayList<>(batchColumnSelection.getCurrentSelectedColumnSchemaList().size() + 1); - currentBatchMeasurementSchemas.add(timeSchema); - currentBatchMeasurementSchemas.addAll( - batchColumnSelection.getCurrentSelectedColumnSchemaList()); FirstBatchFastAlignedSeriesCompactionExecutor executor = new FirstBatchFastAlignedSeriesCompactionExecutor( compactionWriter, - filterTimeseriesMetadataOffsetMap(currentBatchMeasurementSchemas), + filterTimeseriesMetadataOffsetMap(selectedMeasurementSchemas), readerCacheMap, modificationCacheMap, sortedSourceFiles, deviceId, subTaskId, - currentBatchMeasurementSchemas, - summary); + selectedMeasurementSchemas, + summary, + ignoreAllNullRows); executor.execute(); LOGGER.debug( "[Batch Compaction] current device is {}, first batch compacted time chunk is {}", @@ -188,7 +195,8 @@ private void compactLeftBatches() deviceId, subTaskId, currentBatchMeasurementSchemas, - summary); + summary, + ignoreAllNullRows); executor.execute(); } } @@ -218,7 +226,8 @@ public FirstBatchFastAlignedSeriesCompactionExecutor( IDeviceID deviceId, int subTaskId, List measurementSchemas, - FastCompactionTaskSummary summary) { + FastCompactionTaskSummary summary, + boolean ignoreAllNullRows) { super( compactionWriter, timeseriesMetadataOffsetMap, @@ -228,7 +237,8 @@ public FirstBatchFastAlignedSeriesCompactionExecutor( deviceId, subTaskId, measurementSchemas, - summary); + summary, + ignoreAllNullRows); isBatchedCompaction = true; } @@ -253,7 +263,7 @@ public void execute() @Override protected List getAlignedChunkMetadataList(TsFileResource resource) - throws IOException { + throws IOException, IllegalPathException { return getAlignedChunkMetadataListBySelectedValueColumn(resource, measurementSchemas); } @@ -291,7 +301,7 @@ protected ModifiedStatus isPageModified(PageElement pageElement) { ModifiedStatus modifiedStatus = AlignedSeriesBatchCompactionUtils.calculateAlignedPageModifiedStatus( - startTime, endTime, originAlignedChunkMetadata); + startTime, endTime, originAlignedChunkMetadata, ignoreAllNullRows); batchCompactionPlan.recordPageModifiedStatus( resource.getTsFile().getName(), new TimeRange(startTime, endTime), modifiedStatus); return modifiedStatus; @@ -313,7 +323,8 @@ public FollowingBatchFastAlignedSeriesCompactionExecutor( IDeviceID deviceId, int subTaskId, List measurementSchemas, - FastCompactionTaskSummary summary) { + FastCompactionTaskSummary summary, + boolean ignoreAllNullRows) { super( compactionWriter, timeseriesMetadataOffsetMap, @@ -323,7 +334,8 @@ public FollowingBatchFastAlignedSeriesCompactionExecutor( deviceId, subTaskId, measurementSchemas, - summary); + summary, + ignoreAllNullRows); isBatchedCompaction = true; } @@ -359,7 +371,7 @@ protected ModifiedStatus isPageModified(PageElement pageElement) { @Override protected List getAlignedChunkMetadataList(TsFileResource resource) - throws IOException { + throws IOException, IllegalPathException { return getAlignedChunkMetadataListBySelectedValueColumn(resource, measurementSchemas); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/BatchedReadChunkAlignedSeriesCompactionExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/BatchedReadChunkAlignedSeriesCompactionExecutor.java index a8245d62632b..60ba6635fe25 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/BatchedReadChunkAlignedSeriesCompactionExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/BatchedReadChunkAlignedSeriesCompactionExecutor.java @@ -26,8 +26,8 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.ModifiedStatus; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.AlignedSeriesBatchCompactionUtils; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.BatchCompactionPlan; -import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.BatchedCompactionAlignedPagePointReader; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.CompactChunkPlan; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.CompactionAlignedPageLazyLoadPointReader; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.FirstBatchCompactionAlignedChunkWriter; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.FollowingBatchCompactionAlignedChunkWriter; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.readchunk.ReadChunkAlignedSeriesCompactionExecutor; @@ -44,7 +44,8 @@ import org.apache.tsfile.read.TsFileSequenceReader; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.reader.IPointReader; -import org.apache.tsfile.read.reader.page.AlignedPageReader; +import org.apache.tsfile.read.reader.page.TimePageReader; +import org.apache.tsfile.read.reader.page.ValuePageReader; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.chunk.AlignedChunkWriterImpl; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -72,9 +73,10 @@ public BatchedReadChunkAlignedSeriesCompactionExecutor( TsFileResource targetResource, LinkedList>> readerAndChunkMetadataList, CompactionTsFileWriter writer, - CompactionTaskSummary summary) + CompactionTaskSummary summary, + boolean ignoreAllNullRows) throws IOException { - super(device, targetResource, readerAndChunkMetadataList, writer, summary); + super(device, targetResource, readerAndChunkMetadataList, writer, summary, ignoreAllNullRows); this.originReaderAndChunkMetadataList = readerAndChunkMetadataList; this.batchColumnSelection = new AlignedSeriesBatchCompactionUtils.BatchColumnSelection(schemaList, batchSize); @@ -95,15 +97,23 @@ public void execute() throws IOException, PageException { } private void compactFirstBatch() throws IOException, PageException { + List selectedColumnIndexList; + List selectedColumnSchemaList; if (!batchColumnSelection.hasNext()) { - return; + if (ignoreAllNullRows) { + return; + } + selectedColumnIndexList = Collections.emptyList(); + selectedColumnSchemaList = Collections.emptyList(); + } else { + batchColumnSelection.next(); + selectedColumnIndexList = batchColumnSelection.getSelectedColumnIndexList(); + selectedColumnSchemaList = batchColumnSelection.getCurrentSelectedColumnSchemaList(); } - batchColumnSelection.next(); LinkedList>> batchedReaderAndChunkMetadataList = - filterAlignedChunkMetadataList( - readerAndChunkMetadataList, batchColumnSelection.getSelectedColumnIndexList()); + filterAlignedChunkMetadataList(readerAndChunkMetadataList, selectedColumnIndexList); FirstBatchedReadChunkAlignedSeriesCompactionExecutor executor = new FirstBatchedReadChunkAlignedSeriesCompactionExecutor( @@ -113,7 +123,8 @@ private void compactFirstBatch() throws IOException, PageException { writer, summary, timeSchema, - batchColumnSelection.getCurrentSelectedColumnSchemaList()); + selectedColumnSchemaList, + ignoreAllNullRows); executor.execute(); LOGGER.debug( "[Batch Compaction] current device is {}, first batch compacted time chunk is {}", @@ -136,7 +147,8 @@ private void compactLeftBatches() throws PageException, IOException { writer, summary, timeSchema, - batchColumnSelection.getCurrentSelectedColumnSchemaList()); + batchColumnSelection.getCurrentSelectedColumnSchemaList(), + ignoreAllNullRows); executor.execute(); } } @@ -172,7 +184,8 @@ public FirstBatchedReadChunkAlignedSeriesCompactionExecutor( CompactionTsFileWriter writer, CompactionTaskSummary summary, IMeasurementSchema timeSchema, - List valueSchemaList) { + List valueSchemaList, + boolean ignoreAllNullRows) { super( device, targetResource, @@ -180,7 +193,8 @@ public FirstBatchedReadChunkAlignedSeriesCompactionExecutor( writer, summary, timeSchema, - valueSchemaList); + valueSchemaList, + ignoreAllNullRows); int compactionFileLevel = Integer.parseInt(this.targetResource.getTsFile().getName().split("-")[2]); this.flushController = @@ -229,7 +243,7 @@ protected boolean isAllValuePageEmpty(PageLoader timePage, List valu ModifiedStatus modifiedStatus = AlignedSeriesBatchCompactionUtils.calculateAlignedPageModifiedStatus( - startTime, endTime, originAlignedChunkMetadata); + startTime, endTime, originAlignedChunkMetadata, ignoreAllNullRows); batchCompactionPlan.recordPageModifiedStatus( file, new TimeRange(startTime, endTime), modifiedStatus); return modifiedStatus == ModifiedStatus.ALL_DELETED; @@ -247,9 +261,10 @@ protected void flushCurrentChunkWriter() throws IOException { } @Override - protected IPointReader getPointReader(AlignedPageReader alignedPageReader) throws IOException { - return new BatchedCompactionAlignedPagePointReader( - alignedPageReader.getTimePageReader(), alignedPageReader.getValuePageReaderList()); + protected IPointReader getPointReader( + TimePageReader timePageReader, List valuePageReaders) throws IOException { + // we have to set ignoreAllNullRows=false because we can not get the entire row here + return new CompactionAlignedPageLazyLoadPointReader(timePageReader, valuePageReaders, false); } private class FirstBatchReadChunkAlignedSeriesCompactionFlushController @@ -280,7 +295,8 @@ public FollowingBatchedReadChunkAlignedSeriesGroupCompactionExecutor( CompactionTsFileWriter writer, CompactionTaskSummary summary, IMeasurementSchema timeSchema, - List valueSchemaList) { + List valueSchemaList, + boolean ignoreAllNullRows) { super( device, targetResource, @@ -288,7 +304,8 @@ public FollowingBatchedReadChunkAlignedSeriesGroupCompactionExecutor( writer, summary, timeSchema, - valueSchemaList); + valueSchemaList, + ignoreAllNullRows); this.flushController = new FollowingBatchReadChunkAlignedSeriesCompactionFlushController(); this.chunkWriter = new FollowingBatchCompactionAlignedChunkWriter( @@ -355,9 +372,10 @@ protected void compactAlignedChunkByFlush(ChunkLoader timeChunk, List valuePageReaders) throws IOException { + // we have to set ignoreAllNullRows=false because we can not get the entire row here + return new CompactionAlignedPageLazyLoadPointReader(timePageReader, valuePageReaders, false); } private class FollowingBatchReadChunkAlignedSeriesCompactionFlushController diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/AlignedSeriesBatchCompactionUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/AlignedSeriesBatchCompactionUtils.java index 79e7b72a8ac5..3e5f69b88e5e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/AlignedSeriesBatchCompactionUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/AlignedSeriesBatchCompactionUtils.java @@ -105,7 +105,19 @@ public static AlignedChunkMetadata fillAlignedChunkMetadataBySchemaList( } public static ModifiedStatus calculateAlignedPageModifiedStatus( - long startTime, long endTime, AlignedChunkMetadata originAlignedChunkMetadata) { + long startTime, + long endTime, + AlignedChunkMetadata originAlignedChunkMetadata, + boolean ignoreAllNullRows) { + ModifiedStatus timePageModifiedStatus = + checkIsModified( + startTime, + endTime, + originAlignedChunkMetadata.getTimeChunkMetadata().getDeleteIntervalList()); + if (timePageModifiedStatus != ModifiedStatus.NONE_DELETED) { + return timePageModifiedStatus; + } + ModifiedStatus lastPageStatus = null; for (IChunkMetadata valueChunkMetadata : originAlignedChunkMetadata.getValueChunkMetadataList()) { @@ -128,7 +140,11 @@ public static ModifiedStatus calculateAlignedPageModifiedStatus( lastPageStatus = ModifiedStatus.NONE_DELETED; } } - return lastPageStatus; + + // keep the aligned table page with deletion only in value page + return (!ignoreAllNullRows && lastPageStatus == ModifiedStatus.ALL_DELETED) + ? ModifiedStatus.PARTIAL_DELETED + : lastPageStatus; } public static ModifiedStatus checkIsModified( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/BatchedCompactionAlignedPagePointReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/CompactionAlignedPageLazyLoadPointReader.java similarity index 67% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/BatchedCompactionAlignedPagePointReader.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/CompactionAlignedPageLazyLoadPointReader.java index 0a90fe49a15a..900e887b7daf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/BatchedCompactionAlignedPagePointReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/batch/utils/CompactionAlignedPageLazyLoadPointReader.java @@ -28,9 +28,10 @@ import java.io.IOException; import java.util.List; -public class BatchedCompactionAlignedPagePointReader implements IPointReader { +public class CompactionAlignedPageLazyLoadPointReader implements IPointReader { private final TimePageReader timeReader; private final List valueReaders; + private final boolean ignoreAllNullRows; private boolean hasNextRow = false; @@ -38,31 +39,43 @@ public class BatchedCompactionAlignedPagePointReader implements IPointReader { private long currentTime; private TsPrimitiveType currentRow; - public BatchedCompactionAlignedPagePointReader( - TimePageReader timeReader, List valueReaders) throws IOException { + public CompactionAlignedPageLazyLoadPointReader( + TimePageReader timeReader, List valueReaders, boolean ignoreAllNullRows) + throws IOException { this.timeIndex = -1; this.timeReader = timeReader; this.valueReaders = valueReaders; + this.ignoreAllNullRows = ignoreAllNullRows; prepareNextRow(); } private void prepareNextRow() throws IOException { - if (!timeReader.hasNextTime()) { - hasNextRow = false; - return; - } - currentTime = timeReader.nextTime(); - timeIndex++; - TsPrimitiveType[] valuesInThisRow = new TsPrimitiveType[valueReaders.size()]; - for (int i = 0; i < valueReaders.size(); i++) { - TsPrimitiveType value = - valueReaders.get(i) == null - ? null - : valueReaders.get(i).nextValue(currentTime, timeIndex); - valuesInThisRow[i] = value; + while (timeReader.hasNextTime()) { + currentTime = timeReader.nextTime(); + ++this.timeIndex; + boolean someValueNotNull = false; + TsPrimitiveType[] valuesInThisRow = new TsPrimitiveType[this.valueReaders.size()]; + + for (int i = 0; i < valueReaders.size(); ++i) { + TsPrimitiveType value = + valueReaders.get(i) == null + ? null + : this.valueReaders.get(i).nextValue(currentTime, timeIndex); + someValueNotNull = someValueNotNull || value != null; + valuesInThisRow[i] = value; + } + if (timeReader.isDeleted(currentTime)) { + continue; + } + + if (!ignoreAllNullRows || someValueNotNull) { + currentRow = new TsPrimitiveType.TsVector(valuesInThisRow); + hasNextRow = true; + return; + } } - currentRow = new TsPrimitiveType.TsVector(valuesInThisRow); - hasNextRow = true; + + hasNextRow = false; } @Override 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 3c9c195f6864..e997031f0a94 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 @@ -20,6 +20,7 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.fast; import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.AlignedPath; import org.apache.iotdb.commons.path.PatternTreeMap; import org.apache.iotdb.db.exception.WriteProcessException; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.subtask.FastCompactionTaskSummary; @@ -70,6 +71,7 @@ public class FastAlignedSeriesCompactionExecutor extends SeriesCompactionExecuto protected final List measurementSchemas; protected final IMeasurementSchema timeColumnMeasurementSchema; protected final Map measurementSchemaMap; + protected final boolean ignoreAllNullRows; @SuppressWarnings("squid:S107") public FastAlignedSeriesCompactionExecutor( @@ -82,7 +84,8 @@ public FastAlignedSeriesCompactionExecutor( IDeviceID deviceId, int subTaskId, List measurementSchemas, - FastCompactionTaskSummary summary) { + FastCompactionTaskSummary summary, + boolean ignoreAllNullRows) { super( compactionWriter, readerCacheMap, modificationCacheMap, deviceId, true, subTaskId, summary); this.timeseriesMetadataOffsetMap = timeseriesMetadataOffsetMap; @@ -91,6 +94,7 @@ public FastAlignedSeriesCompactionExecutor( this.measurementSchemaMap = new HashMap<>(); this.measurementSchemas.forEach( schema -> measurementSchemaMap.put(schema.getMeasurementId(), schema)); + this.ignoreAllNullRows = ignoreAllNullRows; // get source files which are sorted by the startTime of current device from old to new, // files that do not contain the current device have been filtered out as well. sortedSourceFiles.forEach(x -> fileList.add(new FileElement(x))); @@ -171,7 +175,7 @@ void deserializeFileIntoChunkMetadataQueue(List fileElements) } protected List getAlignedChunkMetadataList(TsFileResource resource) - throws IOException { + throws IOException, IllegalPathException { // read time chunk metadatas and value chunk metadatas in the current file List timeChunkMetadatas = null; List> valueChunkMetadatas = new ArrayList<>(); @@ -230,6 +234,10 @@ protected List getAlignedChunkMetadataList(TsFileResource alignedChunkMetadataList.add(alignedChunkMetadata); } + // get time modifications of this file + List timeModifications = + getModificationsFromCache( + resource, CompactionPathUtils.getPath(deviceId, AlignedPath.VECTOR_PLACEHOLDER)); // get value modifications of this file List> valueModifications = new ArrayList<>(); alignedChunkMetadataList @@ -252,7 +260,8 @@ protected List getAlignedChunkMetadataList(TsFileResource }); // modify aligned chunk metadatas - ModificationUtils.modifyAlignedChunkMetaData(alignedChunkMetadataList, valueModifications); + ModificationUtils.modifyAlignedChunkMetaData( + alignedChunkMetadataList, timeModifications, valueModifications, ignoreAllNullRows); } return alignedChunkMetadataList; } @@ -324,7 +333,7 @@ protected void deserializeChunkIntoPageQueue(ChunkMetadataElement chunkMetadataE alignedPageHeaders, timePages.get(i).right, alignedPageDatas, - new CompactionAlignedChunkReader(timeChunk, valueChunks), + new CompactionAlignedChunkReader(timeChunk, valueChunks, ignoreAllNullRows), chunkMetadataElement, i == timePages.size() - 1, isBatchedCompaction); @@ -411,6 +420,6 @@ protected ModifiedStatus isPageModified(PageElement pageElement) { AlignedChunkMetadata alignedChunkMetadata = (AlignedChunkMetadata) pageElement.getChunkMetadataElement().chunkMetadata; return AlignedSeriesBatchCompactionUtils.calculateAlignedPageModifiedStatus( - startTime, endTime, alignedChunkMetadata); + startTime, endTime, alignedChunkMetadata, ignoreAllNullRows); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/reader/CompactionAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/reader/CompactionAlignedChunkReader.java index 282035ee6619..ff41e5e840e7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/reader/CompactionAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/reader/CompactionAlignedChunkReader.java @@ -19,7 +19,7 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.fast.reader; -import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.BatchedCompactionAlignedPagePointReader; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.CompactionAlignedPageLazyLoadPointReader; import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.compress.IUnCompressor; @@ -31,7 +31,8 @@ import org.apache.tsfile.read.common.Chunk; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.reader.IPointReader; -import org.apache.tsfile.read.reader.page.AlignedPageReader; +import org.apache.tsfile.read.reader.page.TimePageReader; +import org.apache.tsfile.read.reader.page.ValuePageReader; import java.io.IOException; import java.nio.ByteBuffer; @@ -46,27 +47,33 @@ public class CompactionAlignedChunkReader { private final List valueChunkHeaderList = new ArrayList<>(); private final IUnCompressor timeUnCompressor; + private final boolean ignoreAllNullRows; private final Decoder timeDecoder = Decoder.getDecoderByType( TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()), TSDataType.INT64); + private final List timeDeleteIntervalList; // A list of deleted intervals - private final List> valueDeleteIntervalList = new ArrayList<>(); + private final List> valueDeleteIntervalList; /** * Constructor of ChunkReader without deserializing chunk into page. This is used for fast * compaction. */ - public CompactionAlignedChunkReader(Chunk timeChunk, List valueChunkList) { + public CompactionAlignedChunkReader( + Chunk timeChunk, List valueChunkList, boolean ignoreAllNullRows) { ChunkHeader timeChunkHeader = timeChunk.getHeader(); this.timeUnCompressor = IUnCompressor.getUnCompressor(timeChunkHeader.getCompressionType()); + this.timeDeleteIntervalList = timeChunk.getDeleteIntervalList(); + this.valueDeleteIntervalList = new ArrayList<>(valueChunkList.size()); valueChunkList.forEach( chunk -> { this.valueChunkHeaderList.add(chunk == null ? null : chunk.getHeader()); this.valueDeleteIntervalList.add(chunk == null ? null : chunk.getDeleteIntervalList()); }); + this.ignoreAllNullRows = ignoreAllNullRows; } /** @@ -80,10 +87,12 @@ public IPointReader getPagePointReader( ByteBuffer compressedTimePageData, List compressedValuePageDatas) throws IOException { - AlignedPageReader alignedPageReader = - getAlignedPageReader( - timePageHeader, valuePageHeaders, compressedTimePageData, compressedValuePageDatas); - return alignedPageReader.getLazyPointReader(); + return getPontReader( + timePageHeader, + valuePageHeaders, + compressedTimePageData, + compressedValuePageDatas, + ignoreAllNullRows); } public IPointReader getBatchedPagePointReader( @@ -92,58 +101,50 @@ public IPointReader getBatchedPagePointReader( ByteBuffer compressedTimePageData, List compressedValuePageDatas) throws IOException { - AlignedPageReader alignedPageReader = - getAlignedPageReader( - timePageHeader, valuePageHeaders, compressedTimePageData, compressedValuePageDatas); - return new BatchedCompactionAlignedPagePointReader( - alignedPageReader.getTimePageReader(), alignedPageReader.getValuePageReaderList()); + return getPontReader( + timePageHeader, valuePageHeaders, compressedTimePageData, compressedValuePageDatas, false); } - public AlignedPageReader getAlignedPageReader( + private IPointReader getPontReader( PageHeader timePageHeader, List valuePageHeaders, ByteBuffer compressedTimePageData, - List compressedValuePageDatas) + List compressedValuePageDatas, + boolean ignoreAllNullRows) throws IOException { // uncompress time page data ByteBuffer uncompressedTimePageData = uncompressPageData(timePageHeader, timeUnCompressor, compressedTimePageData); + TimePageReader timePageReader = + new TimePageReader(timePageHeader, uncompressedTimePageData, timeDecoder); + timePageReader.setDeleteIntervalList(timeDeleteIntervalList); + // uncompress value page datas - List uncompressedValuePageDatas = new ArrayList<>(); - List valueTypes = new ArrayList<>(); - List valueDecoders = new ArrayList<>(); + List valuePageReaders = new ArrayList<>(valuePageHeaders.size()); for (int i = 0; i < valuePageHeaders.size(); i++) { if (valuePageHeaders.get(i) == null) { - uncompressedValuePageDatas.add(null); - valueTypes.add(TSDataType.BOOLEAN); - valueDecoders.add(null); + valuePageReaders.add(null); } else { ChunkHeader valueChunkHeader = valueChunkHeaderList.get(i); - uncompressedValuePageDatas.add( + ByteBuffer uncompressedPageData = uncompressPageData( valuePageHeaders.get(i), IUnCompressor.getUnCompressor(valueChunkHeader.getCompressionType()), - compressedValuePageDatas.get(i))); + compressedValuePageDatas.get(i)); TSDataType valueType = valueChunkHeader.getDataType(); - valueDecoders.add(Decoder.getDecoderByType(valueChunkHeader.getEncodingType(), valueType)); - valueTypes.add(valueType); + ValuePageReader valuePageReader = + new ValuePageReader( + valuePageHeaders.get(i), + uncompressedPageData, + valueType, + Decoder.getDecoderByType(valueChunkHeader.getEncodingType(), valueType)); + valuePageReader.setDeleteIntervalList(valueDeleteIntervalList.get(i)); + valuePageReaders.add(valuePageReader); } } - // decode page data - AlignedPageReader alignedPageReader = - new AlignedPageReader( - timePageHeader, - uncompressedTimePageData, - timeDecoder, - valuePageHeaders, - uncompressedValuePageDatas, - valueTypes, - valueDecoders, - null); - alignedPageReader.initTsBlockBuilder(valueTypes); - alignedPageReader.setDeleteIntervalList(valueDeleteIntervalList); - return alignedPageReader; + return new CompactionAlignedPageLazyLoadPointReader( + timePageReader, valuePageReaders, ignoreAllNullRows); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/ReadChunkAlignedSeriesCompactionExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/ReadChunkAlignedSeriesCompactionExecutor.java index 65fc6c5c217e..373dbe536198 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/ReadChunkAlignedSeriesCompactionExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/ReadChunkAlignedSeriesCompactionExecutor.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CompactionTaskSummary; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.ModifiedStatus; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.AlignedSeriesBatchCompactionUtils; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.CompactionAlignedPageLazyLoadPointReader; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.readchunk.loader.ChunkLoader; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.readchunk.loader.InstantChunkLoader; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.readchunk.loader.InstantPageLoader; @@ -46,9 +47,9 @@ import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.TsFileSequenceReader; import org.apache.tsfile.read.common.Chunk; -import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.reader.IPointReader; -import org.apache.tsfile.read.reader.page.AlignedPageReader; +import org.apache.tsfile.read.reader.page.TimePageReader; +import org.apache.tsfile.read.reader.page.ValuePageReader; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.chunk.AlignedChunkWriterImpl; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -75,9 +76,9 @@ public class ReadChunkAlignedSeriesCompactionExecutor { protected AlignedChunkWriterImpl chunkWriter; protected IMeasurementSchema timeSchema; protected List schemaList; - protected Map measurementSchemaListIndexMap; protected ReadChunkAlignedSeriesCompactionFlushController flushController; protected final CompactionTaskSummary summary; + protected final boolean ignoreAllNullRows; private long lastWriteTimestamp = Long.MIN_VALUE; @@ -86,7 +87,8 @@ public ReadChunkAlignedSeriesCompactionExecutor( TsFileResource targetResource, LinkedList>> readerAndChunkMetadataList, CompactionTsFileWriter writer, - CompactionTaskSummary summary) + CompactionTaskSummary summary, + boolean ignoreAllNullRows) throws IOException { this.device = device; this.readerAndChunkMetadataList = readerAndChunkMetadataList; @@ -99,8 +101,10 @@ public ReadChunkAlignedSeriesCompactionExecutor( Integer.parseInt(this.targetResource.getTsFile().getName().split("-")[2]); flushController = new ReadChunkAlignedSeriesCompactionFlushController(compactionFileLevel); this.chunkWriter = constructAlignedChunkWriter(); + this.ignoreAllNullRows = ignoreAllNullRows; } + // used for batched column compaction public ReadChunkAlignedSeriesCompactionExecutor( IDeviceID device, TsFileResource targetResource, @@ -108,7 +112,8 @@ public ReadChunkAlignedSeriesCompactionExecutor( CompactionTsFileWriter writer, CompactionTaskSummary summary, IMeasurementSchema timeSchema, - List valueSchemaList) { + List valueSchemaList, + boolean ignoreAllNullRows) { this.device = device; this.readerAndChunkMetadataList = readerAndChunkMetadataList; this.writer = writer; @@ -120,6 +125,7 @@ public ReadChunkAlignedSeriesCompactionExecutor( this.timeSchema = timeSchema; this.schemaList = valueSchemaList; this.chunkWriter = constructAlignedChunkWriter(); + this.ignoreAllNullRows = ignoreAllNullRows; } private void collectValueColumnSchemaList() throws IOException { @@ -303,7 +309,12 @@ private void compactAlignedChunkByDeserialize( pageListOfValueColumn.isEmpty() ? getEmptyPage() : pageListOfValueColumn.get(i)); } - if (isAllValuePageEmpty(timePage, valuePages)) { + // used for tree model + if (ignoreAllNullRows && isAllValuePageEmpty(timePage, valuePages)) { + continue; + } + // used for table model deletion + if (!ignoreAllNullRows && timePage.isEmpty()) { continue; } @@ -356,49 +367,33 @@ private void compactAlignedPageByDeserialize(PageLoader timePage, List valuePageHeaders = new ArrayList<>(valuePages.size()); - List uncompressedValuePageDatas = new ArrayList<>(valuePages.size()); - List valueDataTypes = new ArrayList<>(valuePages.size()); - List valueDecoders = new ArrayList<>(valuePages.size()); - List> deleteIntervalLists = new ArrayList<>(valuePages.size()); + List valuePageReaders = new ArrayList<>(valuePages.size()); int nonEmptyPageNum = 1; for (int i = 0; i < valuePages.size(); i++) { PageLoader valuePage = valuePages.get(i); - if (valuePage.isEmpty()) { - valuePageHeaders.add(null); - uncompressedValuePageDatas.add(null); - valueDataTypes.add(schemaList.get(i).getType()); - valueDecoders.add( - Decoder.getDecoderByType( - schemaList.get(i).getEncodingType(), schemaList.get(i).getType())); - deleteIntervalLists.add(null); - } else { - valuePageHeaders.add(valuePage.getHeader()); - uncompressedValuePageDatas.add(valuePage.getUnCompressedData()); - valueDataTypes.add(valuePage.getDataType()); - valueDecoders.add( - Decoder.getDecoderByType(valuePage.getEncoding(), valuePage.getDataType())); - deleteIntervalLists.add(valuePage.getDeleteIntervalList()); - valuePage.clear(); + ValuePageReader valuePageReader = null; + if (!valuePage.isEmpty()) { + valuePageReader = + new ValuePageReader( + valuePage.getHeader(), + valuePage.getUnCompressedData(), + schemaList.get(i).getType(), + Decoder.getDecoderByType( + schemaList.get(i).getEncodingType(), schemaList.get(i).getType())); + valuePageReader.setDeleteIntervalList(valuePage.getDeleteIntervalList()); nonEmptyPageNum++; } + valuePage.clear(); + valuePageReaders.add(valuePageReader); } summary.increaseDeserializedPageNum(nonEmptyPageNum); - AlignedPageReader alignedPageReader = - new AlignedPageReader( - timePageHeader, - uncompressedTimePageData, - timeDecoder, - valuePageHeaders, - uncompressedValuePageDatas, - valueDataTypes, - valueDecoders, - null); - alignedPageReader.setDeleteIntervalList(deleteIntervalLists); long processedPointNum = 0; - IPointReader lazyPointReader = getPointReader(alignedPageReader); + IPointReader lazyPointReader = getPointReader(timePageReader, valuePageReaders); while (lazyPointReader.hasNextTimeValuePair()) { TimeValuePair timeValuePair = lazyPointReader.nextTimeValuePair(); long currentTime = timeValuePair.getTimestamp(); @@ -414,8 +409,10 @@ private void compactAlignedPageByDeserialize(PageLoader timePage, List valuePageReaders) throws IOException { + return new CompactionAlignedPageLazyLoadPointReader( + timePageReader, valuePageReaders, this.ignoreAllNullRows); } protected void checkAndUpdatePreviousTimestamp(long currentWritingTimestamp) { @@ -456,8 +453,8 @@ protected boolean canFlushCurrentChunkWriter() { private boolean canFlushChunk(ChunkLoader timeChunk, List valueChunks) throws IOException { boolean largeEnough = - timeChunk.getHeader().getDataSize() > targetChunkSize - || timeChunk.getChunkMetadata().getNumOfPoints() > targetChunkPointNum; + timeChunk.getHeader().getDataSize() >= targetChunkSize + || timeChunk.getChunkMetadata().getNumOfPoints() >= targetChunkPointNum; if (timeSchema.getEncodingType() != timeChunk.getHeader().getEncodingType() || timeSchema.getCompressor() != timeChunk.getHeader().getCompressionType()) { return false; @@ -475,7 +472,7 @@ private boolean canFlushChunk(ChunkLoader timeChunk, List valueChun if (valueChunk.getModifiedStatus() == ModifiedStatus.PARTIAL_DELETED) { return false; } - if (valueChunk.getHeader().getDataSize() > targetChunkSize) { + if (valueChunk.getHeader().getDataSize() >= targetChunkSize) { largeEnough = true; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairDataFileScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairDataFileScanUtil.java index 9bbd219b6c9a..64439887ae8a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairDataFileScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairDataFileScanUtil.java @@ -117,7 +117,7 @@ private void checkAlignedDeviceSeries( TsFileSequenceReader reader, IDeviceID device, MetadataIndexNode metadataIndexNode) throws IOException { List chunkMetadataList = - reader.getAlignedChunkMetadataByMetadataIndexNode(device, metadataIndexNode); + reader.getAlignedChunkMetadataByMetadataIndexNode(device, metadataIndexNode, false); for (AlignedChunkMetadata alignedChunkMetadata : chunkMetadataList) { IChunkMetadata timeChunkMetadata = alignedChunkMetadata.getTimeChunkMetadata(); Chunk timeChunk = reader.readMemChunk((ChunkMetadata) timeChunkMetadata); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairTimePartitionScanTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairTimePartitionScanTask.java index 828307e6397b..8bbae0af1036 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairTimePartitionScanTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairTimePartitionScanTask.java @@ -128,7 +128,8 @@ private void checkInternalUnsortedFileAndRepair(RepairTimePartition timePartitio sourceFile, latch, sourceFile.isSeq(), - tsFileManager.getNextCompactionTaskId()); + tsFileManager.getNextCompactionTaskId(), + timePartition.getDatabaseName().startsWith("root.")); if (!submitRepairFileTaskSafely(task)) { latch.countDown(); } @@ -160,7 +161,8 @@ private void checkOverlapInSequenceSpaceAndRepair(RepairTimePartition timePartit latch, true, false, - tsFileManager.getNextCompactionTaskId()); + tsFileManager.getNextCompactionTaskId(), + true); LOGGER.info( "[RepairScheduler] {} need to repair because it is overlapped with other files", overlapFile); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleContext.java index 205d6b620a4c..ec08df4fdf53 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleContext.java @@ -19,7 +19,11 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.schedule; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.storageengine.dataregion.compaction.constant.CompactionTaskType; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICrossCompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ISeqCompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.IUnseqCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.AbstractCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.SettleCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.DeviceInfo; @@ -37,11 +41,12 @@ public class CompactionScheduleContext { private int submitInsertionCrossSpaceCompactionTaskNum = 0; private int submitSettleCompactionTaskNum = 0; - // region TTL info + // region info private int fullyDirtyFileNum = 0; private int partiallyDirtyFileNum = 0; + private boolean isTreeModel = true; // end region private final Map> partitionFileDeviceInfoCache; @@ -50,6 +55,11 @@ public CompactionScheduleContext() { this.partitionFileDeviceInfoCache = new HashMap<>(); } + public CompactionScheduleContext(boolean isTreeModel) { + this(); + this.isTreeModel = isTreeModel; + } + public void addResourceDeviceTimeIndex( TsFileResource tsFileResource, Map deviceInfoMap) { partitionFileDeviceInfoCache.put(tsFileResource, deviceInfoMap); @@ -141,4 +151,32 @@ public int getFullyDirtyFileNum() { public int getPartiallyDirtyFileNum() { return partiallyDirtyFileNum; } + + public ISeqCompactionPerformer getSeqCompactionPerformer() { + ISeqCompactionPerformer seqCompactionPerformer = + IoTDBDescriptor.getInstance().getConfig().getInnerSeqCompactionPerformer().createInstance(); + seqCompactionPerformer.setIgnoreAllNullRows(isTreeModel); + return seqCompactionPerformer; + } + + public IUnseqCompactionPerformer getUnseqCompactionPerformer() { + IUnseqCompactionPerformer unseqCompactionPerformer = + IoTDBDescriptor.getInstance() + .getConfig() + .getInnerUnseqCompactionPerformer() + .createInstance(); + unseqCompactionPerformer.setIgnoreAllNullRows(isTreeModel); + return unseqCompactionPerformer; + } + + public ICrossCompactionPerformer getCrossCompactionPerformer() { + ICrossCompactionPerformer crossCompactionPerformer = + IoTDBDescriptor.getInstance().getConfig().getCrossCompactionPerformer().createInstance(); + crossCompactionPerformer.setIgnoreAllNullRows(isTreeModel); + return crossCompactionPerformer; + } + + public boolean isIgnoreAllNullRows() { + return isTreeModel; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduler.java index 589ea57c6c32..c465e6fa8631 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduler.java @@ -277,7 +277,8 @@ private static int tryToSubmitCrossSpaceCompactionTask( ICrossSpaceSelector crossSpaceCompactionSelector = config .getCrossCompactionSelector() - .createInstance(logicalStorageGroupName, dataRegionId, timePartition, tsFileManager); + .createInstance( + logicalStorageGroupName, dataRegionId, timePartition, tsFileManager, context); List taskList = crossSpaceCompactionSelector.selectCrossSpaceTask( @@ -297,10 +298,7 @@ private static int tryToSubmitCrossSpaceCompactionTask( tsFileManager, taskList.get(i).getSeqFiles(), taskList.get(i).getUnseqFiles(), - IoTDBDescriptor.getInstance() - .getConfig() - .getCrossCompactionPerformer() - .createInstance(), + context.getCrossCompactionPerformer(), memoryCost.get(i), tsFileManager.getNextCompactionTaskId()); task.setCompactionConfigVersion(compactionConfigVersionWhenSelectTask); @@ -323,7 +321,12 @@ public static int tryToSubmitSettleCompactionTask( String dataRegionId = tsFileManager.getDataRegionId(); SettleSelectorImpl settleSelector = new SettleSelectorImpl( - heavySelect, logicalStorageGroupName, dataRegionId, timePartition, tsFileManager); + heavySelect, + logicalStorageGroupName, + dataRegionId, + timePartition, + tsFileManager, + context); long startTime = System.currentTimeMillis(); List taskList = new ArrayList<>(); if (config.isEnableSeqSpaceCompaction()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/CrossCompactionSelector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/CrossCompactionSelector.java index 061b8e882d10..dc0b019dd47f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/CrossCompactionSelector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/CrossCompactionSelector.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.selector.constant; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.ICrossSpaceSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; @@ -39,12 +40,13 @@ public ICrossSpaceSelector createInstance( String logicalStorageGroupName, String virtualGroupId, long timePartition, - TsFileManager tsFileManager) { + TsFileManager tsFileManager, + CompactionScheduleContext context) { switch (this) { case REWRITE: default: return new RewriteCrossSpaceCompactionSelector( - logicalStorageGroupName, virtualGroupId, timePartition, tsFileManager); + logicalStorageGroupName, virtualGroupId, timePartition, tsFileManager, context); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/InnerSequenceCompactionSelector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/InnerSequenceCompactionSelector.java index 08bc8a1919ce..ea191f25853d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/InnerSequenceCompactionSelector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/InnerSequenceCompactionSelector.java @@ -54,7 +54,7 @@ public IInnerSeqSpaceSelector createInstance( case SIZE_TIERED_SINGLE_TARGET: default: return new SizeTieredCompactionSelector( - storageGroupName, dataRegionId, timePartition, true, tsFileManager); + storageGroupName, dataRegionId, timePartition, true, tsFileManager, context); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/InnerUnsequenceCompactionSelector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/InnerUnsequenceCompactionSelector.java index e6e4cc6a6fe9..1624254aa1d5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/InnerUnsequenceCompactionSelector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/constant/InnerUnsequenceCompactionSelector.java @@ -55,7 +55,7 @@ public IInnerUnseqSpaceSelector createInstance( case SIZE_TIERED_SINGLE_TARGET: default: return new SizeTieredCompactionSelector( - storageGroupName, dataRegionId, timePartition, false, tsFileManager); + storageGroupName, dataRegionId, timePartition, false, tsFileManager, context); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/NewSizeTieredCompactionSelector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/NewSizeTieredCompactionSelector.java index 802e0afd1407..3583ee90f7d4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/NewSizeTieredCompactionSelector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/NewSizeTieredCompactionSelector.java @@ -50,7 +50,6 @@ public class NewSizeTieredCompactionSelector extends SizeTieredCompactionSelecto private final int totalFileNumLowerBound; private final long singleFileSizeThreshold; private final int maxLevelGap; - private final CompactionScheduleContext context; private boolean isActiveTimePartition; public NewSizeTieredCompactionSelector( @@ -60,7 +59,7 @@ public NewSizeTieredCompactionSelector( boolean sequence, TsFileManager tsFileManager, CompactionScheduleContext context) { - super(storageGroupName, dataRegionId, timePartition, sequence, tsFileManager); + super(storageGroupName, dataRegionId, timePartition, sequence, tsFileManager, context); double availableDiskSpaceInByte = MetricService.getInstance() .getAutoGauge( @@ -80,7 +79,6 @@ public NewSizeTieredCompactionSelector( Math.min(config.getInnerCompactionTotalFileSizeThresholdInByte(), maxDiskSizeForTempFiles); this.singleFileSizeThreshold = Math.min(config.getTargetCompactionFileSize(), maxDiskSizeForTempFiles); - this.context = context; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/RewriteCrossSpaceCompactionSelector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/RewriteCrossSpaceCompactionSelector.java index 7a379a2b42a1..4c72c0dd0537 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/RewriteCrossSpaceCompactionSelector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/RewriteCrossSpaceCompactionSelector.java @@ -73,13 +73,14 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector private final long maxCrossCompactionFileSize; private final AbstractCrossSpaceEstimator compactionEstimator; - private CompactionScheduleContext context; + private final CompactionScheduleContext context; public RewriteCrossSpaceCompactionSelector( String logicalStorageGroupName, String dataRegionId, long timePartition, - TsFileManager tsFileManager) { + TsFileManager tsFileManager, + CompactionScheduleContext context) { this.logicalStorageGroupName = logicalStorageGroupName; this.dataRegionId = dataRegionId; this.timePartition = timePartition; @@ -98,16 +99,6 @@ public RewriteCrossSpaceCompactionSelector( (AbstractCrossSpaceEstimator) ICompactionSelector.getCompactionEstimator( IoTDBDescriptor.getInstance().getConfig().getCrossCompactionPerformer(), false); - this.context = null; - } - - public RewriteCrossSpaceCompactionSelector( - String logicalStorageGroupName, - String dataRegionId, - long timePartition, - TsFileManager tsFileManager, - CompactionScheduleContext context) { - this(logicalStorageGroupName, dataRegionId, timePartition, tsFileManager); this.context = context; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/SettleSelectorImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/SettleSelectorImpl.java index 5b72740933bb..cf60c55108e0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/SettleSelectorImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/SettleSelectorImpl.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.SettleCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionUtils; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.ISettleSelector; import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; @@ -68,18 +69,21 @@ public class SettleSelectorImpl implements ISettleSelector { private final long timePartition; private final TsFileManager tsFileManager; private boolean isSeq; + private final CompactionScheduleContext context; public SettleSelectorImpl( boolean heavySelect, String storageGroupName, String dataRegionId, long timePartition, - TsFileManager tsFileManager) { + TsFileManager tsFileManager, + CompactionScheduleContext context) { this.heavySelect = heavySelect; this.storageGroupName = storageGroupName; this.dataRegionId = dataRegionId; this.timePartition = timePartition; this.tsFileManager = tsFileManager; + this.context = context; } static class FileDirtyInfo { @@ -307,15 +311,7 @@ private List createTask( } private ICompactionPerformer createCompactionPerformer() { - return isSeq - ? IoTDBDescriptor.getInstance() - .getConfig() - .getInnerSeqCompactionPerformer() - .createInstance() - : IoTDBDescriptor.getInstance() - .getConfig() - .getInnerUnseqCompactionPerformer() - .createInstance(); + return isSeq ? context.getSeqCompactionPerformer() : context.getUnseqCompactionPerformer(); } enum DirtyStatus { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/SizeTieredCompactionSelector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/SizeTieredCompactionSelector.java index 65401dcef256..a23c4dbf4382 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/SizeTieredCompactionSelector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/impl/SizeTieredCompactionSelector.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.InnerSpaceCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.RepairUnsortedFileCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionTaskManager; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.comparator.ICompactionTaskComparator; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.IInnerSeqSpaceSelector; @@ -61,6 +62,7 @@ public class SizeTieredCompactionSelector protected static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); protected String storageGroupName; protected String dataRegionId; + protected final CompactionScheduleContext context; protected long timePartition; protected List tsFileResources; protected boolean sequence; @@ -72,13 +74,15 @@ public SizeTieredCompactionSelector( String dataRegionId, long timePartition, boolean sequence, - TsFileManager tsFileManager) { + TsFileManager tsFileManager, + CompactionScheduleContext context) { this.storageGroupName = storageGroupName; this.dataRegionId = dataRegionId; this.timePartition = timePartition; this.sequence = sequence; this.tsFileManager = tsFileManager; - hasNextTimePartition = tsFileManager.hasNextTimePartition(timePartition, sequence); + this.hasNextTimePartition = tsFileManager.hasNextTimePartition(timePartition, sequence); + this.context = context; } /** @@ -219,22 +223,15 @@ private List selectFileNeedToRepair() { tsFileManager, resource, sequence, - tsFileManager.getNextCompactionTaskId())); + tsFileManager.getNextCompactionTaskId(), + context.isIgnoreAllNullRows())); } } return taskList; } protected ICompactionPerformer createCompactionPerformer() { - return sequence - ? IoTDBDescriptor.getInstance() - .getConfig() - .getInnerSeqCompactionPerformer() - .createInstance() - : IoTDBDescriptor.getInstance() - .getConfig() - .getInnerUnseqCompactionPerformer() - .createInstance(); + return sequence ? context.getSeqCompactionPerformer() : context.getUnseqCompactionPerformer(); } protected int searchMaxFileLevel() throws IOException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/utils/CrossSpaceCompactionCandidate.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/utils/CrossSpaceCompactionCandidate.java index c12ec32741a7..a0a0c6f347dc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/utils/CrossSpaceCompactionCandidate.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/utils/CrossSpaceCompactionCandidate.java @@ -47,12 +47,12 @@ public class CrossSpaceCompactionCandidate { @TestOnly public CrossSpaceCompactionCandidate( List seqFiles, List unseqFiles) { - init(seqFiles, unseqFiles, null); + init(seqFiles, unseqFiles, new CompactionScheduleContext()); } public CrossSpaceCompactionCandidate( List seqFiles, List unseqFiles, long ttlLowerBound) { - this(seqFiles, unseqFiles, ttlLowerBound, null); + this(seqFiles, unseqFiles, ttlLowerBound, new CompactionScheduleContext()); } public CrossSpaceCompactionCandidate( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/utils/TsFileResourceCandidate.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/utils/TsFileResourceCandidate.java index 2832109714d0..7356f1aa6248 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/utils/TsFileResourceCandidate.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/selector/utils/TsFileResourceCandidate.java @@ -76,6 +76,7 @@ private void prepareDeviceInfos() throws IOException { if (deviceInfoMap == null && compactionScheduleContext != null) { // get device info from cache deviceInfoMap = compactionScheduleContext.getResourceDeviceInfo(this.resource); + hasDetailedDeviceInfo = true; } if (deviceInfoMap != null) { return; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/AbstractCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/AbstractCompactionTest.java index f35c766ea585..48f6e77a3358 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/AbstractCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/AbstractCompactionTest.java @@ -842,7 +842,8 @@ protected TsFileResource generateSingleAlignedSeriesFile( protected List getPaths(List resources) throws IOException, IllegalPathException { Set paths = new HashSet<>(); - try (MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(resources)) { + try (MultiTsFileDeviceIterator deviceIterator = + new MultiTsFileDeviceIterator(resources, false)) { while (deviceIterator.hasNextDevice()) { Pair iDeviceIDBooleanPair = deviceIterator.nextDevice(); IDeviceID deviceID = iDeviceIDBooleanPair.getLeft(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CrossSpaceCompactionWithUnusualCasesTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CrossSpaceCompactionWithUnusualCasesTest.java index b24f025dcdad..260068805fa9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CrossSpaceCompactionWithUnusualCasesTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CrossSpaceCompactionWithUnusualCasesTest.java @@ -25,6 +25,7 @@ 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.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.CrossCompactionTaskResource; import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.TsFileGeneratorUtils; @@ -113,7 +114,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -169,7 +171,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -237,7 +240,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -316,7 +320,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -399,7 +404,8 @@ public void testMultiUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSe tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -447,7 +453,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -533,7 +540,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -583,7 +591,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -639,7 +648,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -698,7 +708,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -765,7 +776,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -825,7 +837,8 @@ public void testUnSeqFileOverlapWithSeqFilesButOneDeviceNotExistInOverlapSeqFile tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List result = selector.selectCrossSpaceTask(seqResources, unseqResources); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithInconsistentCompressionTypeAndEncodingTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithInconsistentCompressionTypeAndEncodingTest.java index 5a9580fba025..55cb8625da37 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithInconsistentCompressionTypeAndEncodingTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithInconsistentCompressionTypeAndEncodingTest.java @@ -622,7 +622,8 @@ private void validateSingleTsFileWithAlignedSeries(TsFileSequenceReader reader) throws IOException { Map compressionTypeMap = new HashMap<>(); for (IDeviceID device : reader.getAllDevices()) { - List alignedChunkMetadataList = reader.getAlignedChunkMetadata(device); + List alignedChunkMetadataList = + reader.getAlignedChunkMetadata(device, true); for (AlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadataList) { IChunkMetadata timeChunkMetadata = alignedChunkMetadata.getTimeChunkMetadata(); List valueChunkMetadataList = 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 8d4abdeecf89..8c9088f5cebc 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 @@ -2049,7 +2049,7 @@ public void testMergeAlignedSeriesTimeValuePairFromDifferentFiles() try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource.getTsFilePath())) { List chunkMetadataList = reader.getAlignedChunkMetadata( - IDeviceID.Factory.DEFAULT_FACTORY.create("root.testsg.d1")); + IDeviceID.Factory.DEFAULT_FACTORY.create("root.testsg.d1"), true); for (AlignedChunkMetadata alignedChunkMetadata : chunkMetadataList) { ChunkMetadata timeChunkMetadata = (ChunkMetadata) alignedChunkMetadata.getTimeChunkMetadata(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionSelectorTest.java index 8b43043d9687..894de73da95f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionSelectorTest.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.exception.FileCannotTransitToCompactingException; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.AbstractCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CrossSpaceCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionTaskQueue; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.comparator.DefaultCompactionTaskComparatorImpl; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; @@ -74,7 +75,7 @@ public void testSelectWithEmptySeqFileList() throws IOException, MetadataException, WriteProcessException { createFiles(5, 2, 3, 50, 0, 10000, 50, 50, false, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(0, selected.size()); @@ -87,7 +88,7 @@ public void testSelectWithOneUnclosedSeqFile() createFiles(5, 2, 3, 50, 0, 10000, 50, 50, false, false); seqResources.get(0).setStatusForTest(TsFileResourceStatus.UNCLOSED); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(0, selected.size()); @@ -99,7 +100,7 @@ public void testSelectWithClosedSeqFileAndUnOverlapUnseqFile() createFiles(1, 2, 3, 50, 0, 10000, 50, 50, false, true); createFiles(5, 2, 3, 50, 0, 10000, 50, 50, false, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(1, selected.size()); @@ -120,7 +121,7 @@ public void testSelectWithClosedSeqFileAndUncloseSeqFile() createFiles(5, 2, 3, 50, 0, 10000, 50, 50, false, false); seqResources.get(1).setStatusForTest(TsFileResourceStatus.UNCLOSED); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(1, selected.size()); @@ -155,7 +156,7 @@ public void testSelectWithMultiUnseqFilesOverlapWithOneSeqFile() createFiles(1, 2, 3, 50, 0, 10000, 50, 50, false, false); createFiles(1, 2, 3, 50, 0, 10000, 50, 50, false, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(1, selected.size()); @@ -169,7 +170,7 @@ public void testSelectWithTooLargeSeqFile() createFiles(2, 2, 3, 50, 0, 10000, 50, 50, false, true); createFiles(5, 2, 3, 50, 0, 10000, 50, 50, false, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(1, selected.size()); @@ -196,7 +197,7 @@ public void testSelectWithTooManySourceFiles() createFiles(19, 2, 3, 50, 0, 10000, 50, 50, false, true); createFiles(1, 2, 3, 3000, 0, 10000, 50, 50, false, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(1, selected.size()); @@ -266,7 +267,8 @@ public void testSeqFileWithDeviceIndexBeenDeletedBeforeSelection() cd1.countDown(); cd2.await(); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( seqResources, unseqResources, System.currentTimeMillis() - Long.MAX_VALUE); @@ -335,7 +337,8 @@ public void testSeqFileWithDeviceIndexBeenDeletedDuringSelectionAndAfterCopyingL () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -446,7 +449,8 @@ public void testSeqFileWithDeviceIndexBeenDeletedDuringSelectionAndBeforeSetting () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -551,7 +555,8 @@ public void testSeqFileWithDeviceIndexBeenDeletedDuringSelectionAndBeforeSetting () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -671,7 +676,8 @@ public void testSeqFileWithFileIndexBeenDeletedDuringSelectionAndAfterCopyingLis () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -787,7 +793,8 @@ public void testSeqFileWithFileIndexBeenDeletedDuringSelectionAndBeforeSettingCa () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -896,7 +903,8 @@ public void testSeqFileWithFileIndexBeenDeletedDuringSelectionAndBeforeSettingCo () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -1018,7 +1026,8 @@ public void testSeqFileWithFileIndexBeenDeletedBeforeSelection() cd1.countDown(); cd2.await(); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( seqResources, unseqResources, System.currentTimeMillis() - Long.MAX_VALUE); @@ -1089,7 +1098,8 @@ public void testUnSeqFileWithDeviceIndexBeenDeletedBeforeSelection() cd1.countDown(); cd2.await(); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( seqResources, unseqResources, System.currentTimeMillis() - Long.MAX_VALUE); @@ -1158,7 +1168,8 @@ public void testUnSeqFileWithDeviceIndexBeenDeletedDuringSelectionAndAfterCopyin () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -1270,7 +1281,8 @@ public void testUnSeqFileWithDeviceIndexBeenDeletedDuringSelectionAndBeforeSetti () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -1376,7 +1388,8 @@ public void testUnSeqFileWithDeviceIndexBeenDeletedDuringSelectionAndBeforeSetti () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -1497,7 +1510,8 @@ public void testUnSeqFileWithFileIndexBeenDeletedDuringSelectionAndAfterCopyingL () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -1612,7 +1626,8 @@ public void testUnSeqFileWithFileIndexBeenDeletedDuringSelectionAndBeforeSetting () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -1721,7 +1736,8 @@ public void testUnSeqFileWithFileIndexBeenDeletedDuringSelectionAndBeforeSetting () -> { try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); // copy candidate source file list and add read lock CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( @@ -1843,7 +1859,8 @@ public void testUnSeqFileWithFileIndexBeenDeletedBeforeSelection() cd1.countDown(); cd2.await(); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); CrossSpaceCompactionCandidate candidate = new CrossSpaceCompactionCandidate( seqResources, unseqResources, System.currentTimeMillis() - Long.MAX_VALUE); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerTest.java index 6ce367f7a84d..e327114ca5c9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerTest.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.AbstractCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CrossSpaceCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionTaskManager; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.CrossCompactionTaskResource; @@ -411,7 +412,8 @@ public void testOneSeqFileAndSixUnseqFile() throws Exception { new CrossSpaceCompactionCandidate( seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList); index++; @@ -712,7 +714,8 @@ public void testFiveSeqFileAndOneUnseqFileWithSomeDeviceNotInSeqFiles() throws E new CrossSpaceCompactionCandidate( seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList); if (selected.size() > 0) { @@ -1011,7 +1014,8 @@ public void testFiveSeqFileAndOneUnseqFile() throws Exception { new CrossSpaceCompactionCandidate( seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList); if (selected.size() > 0) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java index c34144c0c1e3..b076755229ed 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java @@ -37,6 +37,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.InnerSpaceCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.subtask.FastCompactionTaskSummary; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionUtils; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.ICompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.ICrossSpaceSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; @@ -121,7 +122,7 @@ public void test1() throws MetadataException, IOException, WriteProcessException tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(1, selected.get(0).getSeqFiles().size()); @@ -163,7 +164,7 @@ public void test2() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -207,7 +208,7 @@ public void test3() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -257,7 +258,7 @@ public void test4() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -305,7 +306,7 @@ public void test5() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -354,7 +355,7 @@ public void test6() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -402,7 +403,7 @@ public void test7() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -451,7 +452,7 @@ public void test8() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -503,7 +504,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile10() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -553,7 +554,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile11() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -604,7 +605,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile12() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -653,7 +654,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile20() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -704,7 +705,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile21() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -756,7 +757,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile22() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -807,7 +808,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile30() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -860,7 +861,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile31() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -914,7 +915,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile32() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -967,7 +968,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile50() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1021,7 +1022,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile51() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1075,7 +1076,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile52() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1129,7 +1130,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile53() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1179,7 +1180,7 @@ public void testWithNewDeviceAndSensorInUnseqFile10() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1229,7 +1230,7 @@ public void testWithNewDeviceAndSensorInUnseqFile11() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1280,7 +1281,7 @@ public void testWithNewDeviceAndSensorInUnseqFile12() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1329,7 +1330,7 @@ public void testWithNewDeviceAndSensorInUnseqFile20() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1380,7 +1381,7 @@ public void testWithNewDeviceAndSensorInUnseqFile21() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1432,7 +1433,7 @@ public void testWithNewDeviceAndSensorInUnseqFile22() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1483,7 +1484,7 @@ public void testWithNewDeviceAndSensorInUnseqFile30() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1536,7 +1537,7 @@ public void testWithNewDeviceAndSensorInUnseqFile31() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1590,7 +1591,7 @@ public void testWithNewDeviceAndSensorInUnseqFile32() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1643,7 +1644,7 @@ public void testWithNewDeviceAndSensorInUnseqFile50() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1697,7 +1698,7 @@ public void testWithNewDeviceAndSensorInUnseqFile51() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1751,7 +1752,7 @@ public void testWithNewDeviceAndSensorInUnseqFile52() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1805,7 +1806,7 @@ public void testWithNewDeviceAndSensorInUnseqFile53() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1859,7 +1860,7 @@ public void testWithUnclosedSeqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1912,7 +1913,7 @@ public void testWithUnclosedSeqFileAndNewSensorInUnseqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); // Assert.assertEquals(0, result.length); @@ -1958,7 +1959,7 @@ public void testUnseqFileOverlapWithUnclosedSeqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -2010,7 +2011,7 @@ public void testUnseqFileOverlapWithUnclosedSeqFile2() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -2064,7 +2065,7 @@ public void testWithUnclosedUnSeqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -2124,7 +2125,8 @@ public void testSelectingFilesWhenSomeFilesBeingDeleted() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); // In the process of getting the file list and starting to select files, the file list is // updated (the file is deleted or the status is updated) List selected = @@ -2218,7 +2220,8 @@ public void testCompactionSchedule() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2260,7 +2263,8 @@ public void testCompactionSchedule() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getInnerSequenceCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager, null); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); Assert.assertEquals(0, innerSelector.selectInnerSpaceTask(targetResources).size()); // first compaction task finishes successfully @@ -2326,7 +2330,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles1() throws Exception { // inner seq space compact List taskResources = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : taskResources) { Assert.assertTrue(task.start()); @@ -2337,7 +2342,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles1() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2399,7 +2405,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles2() throws Exception { // inner seq space compact List taskResources = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : taskResources) { Assert.assertTrue(task.start()); @@ -2410,7 +2417,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles2() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2472,7 +2480,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles3() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { Assert.assertTrue(task.start()); @@ -2483,7 +2492,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles3() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2546,7 +2556,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles4() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { Assert.assertTrue(task.start()); @@ -2557,7 +2568,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles4() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2627,7 +2639,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles1() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { Assert.assertTrue(task.start()); @@ -2638,7 +2651,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles1() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2706,7 +2720,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles2() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { Assert.assertTrue(task.start()); @@ -2717,7 +2732,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles2() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2786,7 +2802,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles3() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { Assert.assertTrue(task.start()); @@ -2797,7 +2814,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles3() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2867,7 +2885,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles4() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { Assert.assertTrue(task.start()); @@ -2878,7 +2897,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles4() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithReadPointPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithReadPointPerformerTest.java index 9309736f07f5..3c194a4e3d14 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithReadPointPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithReadPointPerformerTest.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadPointCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.AbstractCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CrossSpaceCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionTaskManager; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.CrossCompactionTaskResource; @@ -410,7 +411,8 @@ public void testOneSeqFileAndSixUnseqFile() throws Exception { new CrossSpaceCompactionCandidate( seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList); index++; @@ -711,7 +713,8 @@ public void testFiveSeqFileAndOneUnseqFileWithSomeDeviceNotInSeqFiles() throws E new CrossSpaceCompactionCandidate( seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList); if (selected.size() > 0) { @@ -1010,7 +1013,8 @@ public void testFiveSeqFileAndOneUnseqFile() throws Exception { new CrossSpaceCompactionCandidate( seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqTsFileResourceList, unseqTsFileResourceList); if (selected.size() > 0) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithReadPointPerformerValidationTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithReadPointPerformerValidationTest.java index 690024f6bf28..77d5a66c301d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithReadPointPerformerValidationTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithReadPointPerformerValidationTest.java @@ -38,6 +38,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CrossSpaceCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.InnerSpaceCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionUtils; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.ICrossSpaceSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.SizeTieredCompactionSelector; @@ -120,7 +121,7 @@ public void test1() throws MetadataException, IOException, WriteProcessException tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); Assert.assertEquals(1, selected.get(0).getSeqFiles().size()); @@ -162,7 +163,7 @@ public void test2() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -206,7 +207,7 @@ public void test3() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -256,7 +257,7 @@ public void test4() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -304,7 +305,7 @@ public void test5() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -353,7 +354,7 @@ public void test6() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -401,7 +402,7 @@ public void test7() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -450,7 +451,7 @@ public void test8() throws MetadataException, IOException, WriteProcessException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -502,7 +503,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile10() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -552,7 +553,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile11() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -603,7 +604,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile12() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -652,7 +653,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile20() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -703,7 +704,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile21() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -755,7 +756,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile22() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -806,7 +807,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile30() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -859,7 +860,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile31() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -913,7 +914,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile32() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -966,7 +967,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile50() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1020,7 +1021,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile51() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1074,7 +1075,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile52() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1128,7 +1129,7 @@ public void testWithNewAlignedDeviceAndSensorInUnseqFile53() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1178,7 +1179,7 @@ public void testWithNewDeviceAndSensorInUnseqFile10() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1228,7 +1229,7 @@ public void testWithNewDeviceAndSensorInUnseqFile11() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1279,7 +1280,7 @@ public void testWithNewDeviceAndSensorInUnseqFile12() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1328,7 +1329,7 @@ public void testWithNewDeviceAndSensorInUnseqFile20() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1379,7 +1380,7 @@ public void testWithNewDeviceAndSensorInUnseqFile21() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1431,7 +1432,7 @@ public void testWithNewDeviceAndSensorInUnseqFile22() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1482,7 +1483,7 @@ public void testWithNewDeviceAndSensorInUnseqFile30() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1535,7 +1536,7 @@ public void testWithNewDeviceAndSensorInUnseqFile31() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1589,7 +1590,7 @@ public void testWithNewDeviceAndSensorInUnseqFile32() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1642,7 +1643,7 @@ public void testWithNewDeviceAndSensorInUnseqFile50() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1696,7 +1697,7 @@ public void testWithNewDeviceAndSensorInUnseqFile51() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1750,7 +1751,7 @@ public void testWithNewDeviceAndSensorInUnseqFile52() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1804,7 +1805,7 @@ public void testWithNewDeviceAndSensorInUnseqFile53() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1858,7 +1859,7 @@ public void testWithUnclosedSeqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -1911,7 +1912,7 @@ public void testWithUnclosedSeqFileAndNewSensorInUnseqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); // Assert.assertEquals(0, result.length); @@ -1957,7 +1958,7 @@ public void testUnseqFileOverlapWithUnclosedSeqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -2009,7 +2010,7 @@ public void testUnseqFileOverlapWithUnclosedSeqFile2() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -2063,7 +2064,7 @@ public void testWithUnclosedUnSeqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); @@ -2127,7 +2128,8 @@ public void testSelectingFilesWhenSomeFilesBeingDeleted() IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); // In the process of getting the file list and starting to select files, the file list is // updated (the file is deleted or the status is updated) List selected = @@ -2174,7 +2176,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles1() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { task.start(); @@ -2185,7 +2188,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles1() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2247,7 +2251,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles2() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { task.start(); @@ -2258,7 +2263,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles2() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2321,7 +2327,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles3() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { task.start(); @@ -2332,7 +2339,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles3() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2396,7 +2404,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles4() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { task.start(); @@ -2407,7 +2416,8 @@ public void testNonAlignedUnseqFilesNotOverlapWithSeqFiles4() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2477,7 +2487,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles1() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { task.start(); @@ -2488,7 +2499,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles1() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2557,7 +2569,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles2() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { task.start(); @@ -2568,7 +2581,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles2() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2638,7 +2652,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles3() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { task.start(); @@ -2649,7 +2664,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles3() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( @@ -2720,7 +2736,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles4() throws Exception { // inner seq space compact List innerSpaceCompactionTasks = - new SizeTieredCompactionSelector(COMPACTION_TEST_SG, "0", 0, true, tsFileManager) + new SizeTieredCompactionSelector( + COMPACTION_TEST_SG, "0", 0, true, tsFileManager, new CompactionScheduleContext()) .selectInnerSpaceTask(tsFileManager.getOrCreateSequenceListByTimePartition(0)); for (InnerSpaceCompactionTask task : innerSpaceCompactionTasks) { task.start(); @@ -2731,7 +2748,8 @@ public void testAlignedUnseqFilesNotOverlapWithSeqFiles4() throws Exception { IoTDBDescriptor.getInstance() .getConfig() .getCrossCompactionSelector() - .createInstance(COMPACTION_TEST_SG, "0", 0, tsFileManager); + .createInstance( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); CrossCompactionTaskResource sourceFiles = crossSpaceCompactionSelector .selectCrossSpaceTask( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionRecoverTest.java index 824e45af0a1c..c3071dc96b75 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionRecoverTest.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.InsertionCrossSpaceCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.log.CompactionLogger; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.log.SimpleCompactionLogger; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.CrossSpaceCompactionCandidate; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.InsertionCrossCompactionTaskResource; @@ -116,7 +117,8 @@ public void testRecoverWithTargetModFileNotExistedAndSourceModFileExisted() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource taskResource = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -203,7 +205,8 @@ public void testRecoverWithTargetModFileNotExistedAndSourceModNotExisted() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource taskResource = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -294,7 +297,8 @@ public void testRecoverWithAllTargetFileExisted() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource taskResource = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -387,7 +391,8 @@ public void testRecoverWithTargetFileNotExist() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource taskResource = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionSelectorTest.java index e29968cd7bff..6a86b3955f93 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionSelectorTest.java @@ -92,7 +92,8 @@ public void testSimpleInsertionCompaction() throws IOException, MergeException { tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -199,7 +200,8 @@ public void testSimpleInsertionCompactionWithMultiUnseqFiles() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -234,7 +236,8 @@ public void testSimpleInsertionCompactionWithFirstUnseqFileCannotSelect() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -268,7 +271,8 @@ public void testSimpleInsertionCompactionWithFirstUnseqFileInvalid() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -305,7 +309,8 @@ public void testSimpleInsertionCompactionWithFirstTwoUnseqFileCannotSelect() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -334,7 +339,8 @@ public void testSimpleInsertionCompactionWithUnseqDeviceNotExistInSeqSpace() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -364,7 +370,8 @@ public void testSimpleInsertionCompactionWithUnseqFileInsertFirstInSeqSpace() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -394,7 +401,8 @@ public void testSimpleInsertionCompactionWithUnseqFileInsertLastInSeqSpace() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -423,7 +431,8 @@ public void testSimpleInsertionCompactionWithCloseTimestamp() throws IOException tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -450,7 +459,8 @@ public void testSimpleInsertionCompactionWithOverlap() throws IOException, Merge tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -482,7 +492,8 @@ public void testSimpleInsertionCompactionWithPrevSeqFileInvalidCompactionCandida tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -514,7 +525,8 @@ public void testSimpleInsertionCompactionWithNextSeqFileInvalidCompactionCandida tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -549,7 +561,8 @@ public void testSimpleInsertionCompactionWithManySeqFiles() throws IOException, tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -593,7 +606,8 @@ public void testInsertionCompactionWithManySeqFilesManyDevices() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -637,7 +651,8 @@ public void testInsertionCompactionWithManySeqFilesManyDevices2() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -673,7 +688,8 @@ public void testInsertLast1() throws IOException, MergeException { tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -713,7 +729,8 @@ public void testInsertLast2() throws IOException, MergeException { tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -741,7 +758,8 @@ public void testInsertFirst() throws IOException, MergeException { tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource result = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -783,7 +801,8 @@ public void testInsertionCompactionWithManySeqFilesManyDevices3() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -829,7 +848,8 @@ public void testInsertionCompactionWithManySeqFilesManyDevices4() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -875,7 +895,8 @@ public void testInsertionCompactionWithManySeqFilesManyDevices5() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -917,7 +938,8 @@ public void testInsertionCompactionWithManySeqFilesManyDevices6() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -965,7 +987,8 @@ public void testInsertionCompactionWithManySeqFilesManyDevicesWithOverlap() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1061,7 +1084,8 @@ public void testInsertionSelectorWithNoSeqFiles() throws MergeException, IOExcep tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource taskResource; int i = 1; @@ -1123,7 +1147,8 @@ public void testInsertionSelectorWithNoUnseqFiles() throws MergeException { seqResources.add(seqResource3); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1169,7 +1194,8 @@ public void testInsertionSelectorWithOverlapUnseqFile() throws MergeException { unseqResources.add(unseqResource1); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1272,7 +1298,8 @@ public void testInsertionIntoCompactingSeqFiles() throws IOException, MergeExcep tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1360,7 +1387,8 @@ public void testInsertionSelectorWithUnclosedSeqFile() throws MergeException { unseqResources.add(unseqResource2); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1443,7 +1471,8 @@ public void testInsertionSelectorWithUnclosedUnSeqFile() throws MergeException { unseqResources.add(unseqResource2); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1544,7 +1573,8 @@ public void testInsertionSelectorWithNoSeqFilesAndFileTimeIndex() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource taskResource; int i = 1; @@ -1611,7 +1641,8 @@ public void testInsertionSelectorWithNoUnseqFilesAndFileTimeIndex() degradeTimeIndex(); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1664,7 +1695,8 @@ public void testInsertionSelectorWithOverlapUnseqFileAndFileTimeIndex() degradeTimeIndex(); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1774,7 +1806,8 @@ public void testInsertionIntoCompactingSeqFilesAndFileTimeIndex() tsFileManager.addAll(unseqResources, false); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1871,7 +1904,8 @@ public void testInsertionSelectorWithUnclosedSeqFileAndFileTimeIndex() degradeTimeIndex(); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); @@ -1965,7 +1999,8 @@ public void testInsertionSelectorWithUnclosedUnSeqFileAndFileTimeIndex() degradeTimeIndex(); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("root.testsg", "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "root.testsg", "0", 0, tsFileManager, new CompactionScheduleContext()); InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionTest.java index b286f7bdfa40..431b9bb93298 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionTest.java @@ -95,7 +95,8 @@ public void test1() throws IOException, InterruptedException { unseqResource1.setStatusForTest(TsFileResourceStatus.NORMAL); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); seqResources.add(seqResource1); seqResources.add(seqResource2); unseqResources.add(unseqResource1); @@ -137,7 +138,8 @@ public void test2() throws IOException, InterruptedException { unseqResource1.setStatusForTest(TsFileResourceStatus.NORMAL); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); seqResources.add(seqResource1); unseqResources.add(unseqResource1); tsFileManager.addAll(seqResources, true); @@ -177,7 +179,8 @@ public void test3() throws IOException, InterruptedException { unseqResource1.setStatusForTest(TsFileResourceStatus.NORMAL); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); seqResources.add(seqResource1); unseqResources.add(unseqResource1); tsFileManager.addAll(seqResources, true); @@ -221,7 +224,8 @@ public void test4() throws IOException, InterruptedException { unseqResource1.setStatusForTest(TsFileResourceStatus.NORMAL); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); seqResources.add(seqResource1); seqResources.add(seqResource2); unseqResources.add(unseqResource1); @@ -259,7 +263,8 @@ public void test5() throws IOException, InterruptedException { unseqResource1.setStatusForTest(TsFileResourceStatus.NORMAL); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector(COMPACTION_TEST_SG, "0", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); unseqResources.add(unseqResource1); tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/MergeUpgradeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/MergeUpgradeTest.java index b326a7dd0dd3..0a1d5b02f7e1 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/MergeUpgradeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/MergeUpgradeTest.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.MergeException; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.CrossCompactionTaskResource; import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionConfigRestorer; @@ -87,7 +88,8 @@ public void testMergeUpgradeSelect() throws MergeException { tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, true); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, tsFileManager); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, tsFileManager, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); assertEquals(0, selected.size()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCompactionFileSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCompactionFileSelectorTest.java index 2c80a0c85e85..16ca83ccc16e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCompactionFileSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCompactionFileSelectorTest.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.MergeException; import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.RewriteCrossSpaceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.CrossCompactionTaskResource; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.CrossSpaceCompactionCandidate; @@ -85,7 +86,7 @@ public void tearDown() throws StorageEngineException, IOException { @Test public void testFullSelection() throws MergeException, IOException { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); List seqSelected = selected.get(0).getSeqFiles(); @@ -93,14 +94,16 @@ public void testFullSelection() throws MergeException, IOException { assertEquals(seqResources, seqSelected); assertEquals(unseqResources, unseqSelected); - selector = new RewriteCrossSpaceCompactionSelector("", "", 0, null); + selector = + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); selected = selector.selectCrossSpaceTask(seqResources.subList(0, 1), unseqResources); seqSelected = selected.get(0).getSeqFiles(); unseqSelected = selected.get(0).getUnseqFiles(); assertEquals(seqResources.subList(0, 1), seqSelected); assertEquals(unseqResources, unseqSelected); - selector = new RewriteCrossSpaceCompactionSelector("", "", 0, null); + selector = + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); selected = selector.selectCrossSpaceTask(seqResources, unseqResources.subList(0, 1)); seqSelected = selected.get(0).getSeqFiles(); unseqSelected = selected.get(0).getUnseqFiles(); @@ -113,7 +116,7 @@ public void testWithFewMemoryBudgeSelection() throws MergeException, IOException CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); assertEquals(1, selected.size()); @@ -124,7 +127,7 @@ public void testRestrictedSelection() throws MergeException, IOException { CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); List seqSelected = selected.get(0).getSeqFiles(); @@ -182,7 +185,7 @@ public void testFileOpenSelection() List newUnseqResources = new ArrayList<>(); newUnseqResources.add(largeUnseqTsFileResource); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, newUnseqResources); assertEquals(0, selected.size()); @@ -271,7 +274,7 @@ public void testFileSelectionAboutLastSeqFile() CrossSpaceCompactionCandidate resource = new CrossSpaceCompactionCandidate(seqResources, unseqResources); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); assertEquals(2, selected.get(0).getSeqFiles().size()); @@ -334,7 +337,8 @@ public void testSelectContinuousUnseqFile() 29000L * IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()); try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector( + "", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqList, unseqList); assertEquals(1, selected.get(0).getSeqFiles().size()); @@ -417,7 +421,7 @@ public void testUnseqFilesOverlappedWithOneSeqFile() * IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()); try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqList, unseqList); Assert.assertEquals(1, selected.size()); @@ -490,7 +494,7 @@ public void testOneUnseqFileOverlappedWithOneSeqFile() * IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()); try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqList, unseqList); Assert.assertEquals(1, selected.size()); @@ -563,7 +567,7 @@ public void testUnseqFilesOverlapped() throws IOException, WriteProcessException * IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()); try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqList, unseqList); Assert.assertEquals(1, selected.size()); @@ -639,7 +643,7 @@ public void testAllUnseqFilesOverlapped() * IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()); try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqList, unseqList); Assert.assertEquals(1, selected.size()); @@ -718,7 +722,7 @@ public void testAllUnseqFilesOverlappedWithSeqFileOpen() * IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()); try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqList, unseqList); Assert.assertEquals(1, selected.size()); @@ -950,7 +954,7 @@ public void testMultiFileOverlapWithOneFile() * IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()); try { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqList, unseqList); Assert.assertEquals(1, selected.size()); @@ -967,7 +971,7 @@ public void testMaxFileSelection() throws MergeException, IOException { IoTDBDescriptor.getInstance().getConfig().getFileLimitPerCrossTask(); IoTDBDescriptor.getInstance().getConfig().setFileLimitPerCrossTask(5); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); assertEquals(1, selected.size()); @@ -987,7 +991,7 @@ public void testAtLeastOneUnseqFileBeenSelected() throws IOException, MergeExcep IoTDBDescriptor.getInstance().getConfig().setFileLimitPerCrossTask(1); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources); assertEquals(1, selected.size()); @@ -1002,7 +1006,7 @@ public void testAtLeastOneUnseqFileBeenSelected() throws IOException, MergeExcep @Test public void testDeleteInSelection() throws Exception { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); AtomicBoolean fail = new AtomicBoolean(false); Thread thread1 = new Thread( @@ -1038,7 +1042,7 @@ public void testDeleteInSelection() throws Exception { @Test public void testDeleteAndDegradeInSelection() throws Exception { RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); AtomicBoolean fail = new AtomicBoolean(false); Thread thread1 = new Thread( @@ -1078,7 +1082,7 @@ public void testFirstZeroLevelUnseqFileIsLarge() { .getConfig() .setTargetCompactionFileSize(unseqResources.get(5).getTsFileSize()); RewriteCrossSpaceCompactionSelector selector = - new RewriteCrossSpaceCompactionSelector("", "", 0, null); + new RewriteCrossSpaceCompactionSelector("", "", 0, null, new CompactionScheduleContext()); List selected = selector.selectCrossSpaceTask(seqResources, unseqResources.subList(5, 6)); Assert.assertEquals(1, selected.size()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSpaceCompactionSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSpaceCompactionSelectorTest.java index 082357a9b0e8..ddc223faa40b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSpaceCompactionSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSpaceCompactionSelectorTest.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.exception.FileCannotTransitToCompactingException; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.InnerSpaceCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.SizeTieredCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; @@ -81,7 +82,8 @@ public void testFileWithDeviceIndexBeenDeletedBeforeSelection() // the file is deleted before selection cd.await(); SizeTieredCompactionSelector selector = - new SizeTieredCompactionSelector("", "", 0, true, tsFileManager); + new SizeTieredCompactionSelector( + "", "", 0, true, tsFileManager, new CompactionScheduleContext()); List resources = tsFileManager.getOrCreateSequenceListByTimePartition(0); List innerSpaceCompactionTasks = @@ -145,7 +147,8 @@ public void testFileWithDeviceIndexBeenDeletedDuringSelectionAndBeforeSettingCan () -> { try { SizeTieredCompactionSelector selector = - new SizeTieredCompactionSelector("", "", 0, true, tsFileManager); + new SizeTieredCompactionSelector( + "", "", 0, true, tsFileManager, new CompactionScheduleContext()); // copy candidate source file list List resources = tsFileManager.getOrCreateSequenceListByTimePartition(0); @@ -251,7 +254,8 @@ public void testFileWithDeviceIndexBeenDeletedDuringSelectionAndBeforeSettingCom () -> { try { SizeTieredCompactionSelector selector = - new SizeTieredCompactionSelector("", "", 0, true, tsFileManager); + new SizeTieredCompactionSelector( + "", "", 0, true, tsFileManager, new CompactionScheduleContext()); // copy candidate source file list List resources = tsFileManager.getOrCreateSequenceListByTimePartition(0); @@ -373,7 +377,8 @@ public void testFileWithFileIndexBeenDeletedBeforeSelection() cd1.countDown(); cd2.await(); SizeTieredCompactionSelector selector = - new SizeTieredCompactionSelector("", "", 0, true, tsFileManager); + new SizeTieredCompactionSelector( + "", "", 0, true, tsFileManager, new CompactionScheduleContext()); List resources = tsFileManager.getOrCreateSequenceListByTimePartition(0); List innerSpaceCompactionTasks = @@ -441,7 +446,8 @@ public void testFileWithFileIndexBeenDeletedDuringSelectionAndBeforeSettingCandi () -> { try { SizeTieredCompactionSelector selector = - new SizeTieredCompactionSelector("", "", 0, true, tsFileManager); + new SizeTieredCompactionSelector( + "", "", 0, true, tsFileManager, new CompactionScheduleContext()); // copy candidate source file list List resources = tsFileManager.getOrCreateSequenceListByTimePartition(0); @@ -550,7 +556,8 @@ public void testFileWithFileIndexBeenDeletedDuringSelectionAndBeforeSettingCompa () -> { try { SizeTieredCompactionSelector selector = - new SizeTieredCompactionSelector("", "", 0, true, tsFileManager); + new SizeTieredCompactionSelector( + "", "", 0, true, tsFileManager, new CompactionScheduleContext()); // copy candidate source file list List resources = tsFileManager.getOrCreateSequenceListByTimePartition(0); @@ -665,7 +672,8 @@ public void testSelectWhenModsFileGreaterThan50M() mockModiFicationFile.write(new Deletion(new MeasurementPath("root.a.b"), 1, 1)); seqResources.get(0).setModFile(mockModiFicationFile); SizeTieredCompactionSelector selector = - new SizeTieredCompactionSelector("", "", 0, true, tsFileManager); + new SizeTieredCompactionSelector( + "", "", 0, true, tsFileManager, new CompactionScheduleContext()); // copy candidate source file list List resources = tsFileManager.getOrCreateSequenceListByTimePartition(0); List innerSpaceCompactionTasks = @@ -689,7 +697,8 @@ public void testSelectWhenModsFileGreaterThan50MWithTsFileResourceStatusIsNotNor seqResources.get(0).setModFile(mockModiFicationFile); seqResources.get(0).setStatusForTest(TsFileResourceStatus.COMPACTING); SizeTieredCompactionSelector selector = - new SizeTieredCompactionSelector("", "", 0, true, tsFileManager); + new SizeTieredCompactionSelector( + "", "", 0, true, tsFileManager, new CompactionScheduleContext()); // copy candidate source file list List resources = tsFileManager.getOrCreateSequenceListByTimePartition(0); List innerSpaceCompactionTasks = diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/SizeTieredCompactionSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/SizeTieredCompactionSelectorTest.java index 1e7cc0450da1..ec87a22ac24c 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/SizeTieredCompactionSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/SizeTieredCompactionSelectorTest.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.inner.sizetiered; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.SizeTieredCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.tsfile.FakedTsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; @@ -57,14 +58,16 @@ public void testSubmitWhenNextTimePartitionExists() { for (long i = 0; i < 9; ++i) { Assert.assertEquals( 1, - new SizeTieredCompactionSelector("root.test", "0", i, true, manager) + new SizeTieredCompactionSelector( + "root.test", "0", i, true, manager, new CompactionScheduleContext()) .selectInnerSpaceTask(manager.getOrCreateSequenceListByTimePartition(i)) .size()); } Assert.assertEquals( 0, - new SizeTieredCompactionSelector("root.test", "0", 9, true, manager) + new SizeTieredCompactionSelector( + "root.test", "0", 9, true, manager, new CompactionScheduleContext()) .selectInnerSpaceTask(manager.getOrCreateSequenceListByTimePartition(9)) .size()); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileCompactionTest.java index 374969a0a4d1..9e58259d722b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileCompactionTest.java @@ -113,7 +113,7 @@ public void testRepairUnsortedDataBetweenPageWithNonAlignedSeries() throws IOExc } Assert.assertFalse(TsFileResourceUtils.validateTsFileDataCorrectness(resource)); RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0, true); task.start(); Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); @@ -137,7 +137,7 @@ public void testRepairUnsortedDataBetweenPageWithAlignedSeries() throws IOExcept } Assert.assertFalse(TsFileResourceUtils.validateTsFileDataCorrectness(resource)); RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0, true); task.start(); Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); @@ -165,7 +165,7 @@ public void testRepairUnsortedDataInOnePageWithNonAlignedSeries() throws IOExcep } Assert.assertFalse(TsFileResourceUtils.validateTsFileDataCorrectness(resource)); RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0, true); task.start(); Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); @@ -197,7 +197,7 @@ public void testRepairUnsortedDataInOnePageWithMultiNonAlignedSeries() throws IO } Assert.assertFalse(TsFileResourceUtils.validateTsFileDataCorrectness(resource)); RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0, true); task.start(); Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); @@ -225,7 +225,7 @@ public void testRepairUnsortedDataInOnePageWithUnseqFile() throws IOException { } Assert.assertFalse(TsFileResourceUtils.validateTsFileDataCorrectness(resource)); RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0, true); task.start(); Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); @@ -253,7 +253,7 @@ public void testRepairUnsortedDataInOnePageWithAlignedSeries() throws IOExceptio } Assert.assertFalse(TsFileResourceUtils.validateTsFileDataCorrectness(resource)); RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0, true); task.start(); Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); @@ -522,7 +522,7 @@ public void testRepairOverlapBetweenFile() throws IOException { Assert.assertFalse(TsFileResourceUtils.validateTsFileResourcesHasNoOverlap(seqResources)); RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, seqResource2, true, false, 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, seqResource2, true, false, 0, true); Assert.assertTrue(task.start()); Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); @@ -572,7 +572,7 @@ public void testRepairOverlapBetweenFileWithModFile() throws IOException, Illega Assert.assertFalse(TsFileResourceUtils.validateTsFileResourcesHasNoOverlap(seqResources)); RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, seqResource2, true, false, 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, seqResource2, true, false, 0, true); Assert.assertTrue(task.start()); Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); @@ -631,9 +631,9 @@ public void testEstimateRepairCompactionMemory() throws IOException { writer.endFile(); } RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, true, 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, true, 0, true); Assert.assertTrue(task.getEstimatedMemoryCost() > 0); - task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, false, 0); + task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, false, 0, true); Assert.assertEquals(0, task.getEstimatedMemoryCost()); } @@ -658,13 +658,13 @@ public void testMergeAlignedSeriesPointWithSameTimestamp() throws IOException { writer.endFile(); } RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, true, 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, true, 0, true); Assert.assertTrue(task.start()); TsFileResource target = tsFileManager.getTsFileList(false).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(target.getTsFilePath())) { List chunkMetadataList = reader.getAlignedChunkMetadata( - IDeviceID.Factory.DEFAULT_FACTORY.create("root.testsg.d1")); + IDeviceID.Factory.DEFAULT_FACTORY.create("root.testsg.d1"), true); for (AlignedChunkMetadata alignedChunkMetadata : chunkMetadataList) { ChunkMetadata timeChunkMetadata = (ChunkMetadata) alignedChunkMetadata.getTimeChunkMetadata(); @@ -706,13 +706,13 @@ public void testSplitChunk() throws IOException { writer.endFile(); } RepairUnsortedFileCompactionTask task = - new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, true, 0); + new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, true, 0, true); Assert.assertTrue(task.start()); TsFileResource target = tsFileManager.getTsFileList(false).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(target.getTsFilePath())) { List chunkMetadataList = reader.getAlignedChunkMetadata( - IDeviceID.Factory.DEFAULT_FACTORY.create("root.testsg.d1")); + IDeviceID.Factory.DEFAULT_FACTORY.create("root.testsg.d1"), true); Assert.assertEquals(3, chunkMetadataList.size()); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionSelectorTest.java index 0ae1dbf2cd04..58d8d5ed7f1d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionSelectorTest.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.queryengine.plan.analyze.cache.schema.DataNodeTTLCache; import org.apache.iotdb.db.storageengine.dataregion.compaction.AbstractCompactionTest; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.SettleCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.SettleSelectorImpl; import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; @@ -80,7 +81,8 @@ public void testSelectContinuousFileWithLightSelect() // inner task, continuous // select first time SettleSelectorImpl settleSelector = - new SettleSelectorImpl(false, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + false, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(1, seqTasks.size()); @@ -143,7 +145,8 @@ public void testSelectUnContinuousFileWithLightSelect() // settle task, not continuous // select first time SettleSelectorImpl settleSelector = - new SettleSelectorImpl(false, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + false, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); int num = 10; for (int i = 0; i < 5; i++) { List seqTasks = settleSelector.selectSettleTask(seqResources); @@ -188,7 +191,8 @@ public void testSelectContinuousFilesBaseOnModsSizeWithHeavySelect() // select first time SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(0, seqTasks.size()); @@ -221,7 +225,8 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelect() // select first time, file 0 is all deleted, file 1 2 3 4 is not satisfied by heavy select SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(1, seqTasks.size()); Assert.assertEquals(1, seqTasks.get(0).getFullyDirtyFiles().size()); @@ -231,7 +236,9 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelect() Assert.assertEquals(4, tsFileManager.getTsFileList(true).size()); // select second time - settleSelector = new SettleSelectorImpl(false, COMPACTION_TEST_SG, "0", 0, tsFileManager); + settleSelector = + new SettleSelectorImpl( + false, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(1, seqTasks.size()); Assert.assertEquals(0, seqTasks.get(0).getFullyDirtyFiles().size()); @@ -272,7 +279,8 @@ public void testSelectContinuousFileBaseOnDirtyDataRateWithHeavySelect() // select first time, none partial_deleted and all_deleted files SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(0, seqTasks.size()); @@ -341,7 +349,8 @@ public void testSelectContinuousFileBaseOnDirtyDataOutdatedTooLongWithHeavySelec // inner task, continuous // select first time, none partial_deleted and all_deleted files SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(0, seqTasks.size()); @@ -408,7 +417,8 @@ public void testSelectUncontinuousFileBaseOnDirtyDataRateWithHeavySelect() } } SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(0, seqTasks.size()); @@ -502,7 +512,8 @@ public void testSelectFileBaseOnDirtyDataRateWithHeavySelect() } // compact all_deleted file and partial_deleted file 0 2 SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(4, seqTasks.size()); Assert.assertEquals(1, seqTasks.get(0).getFullyDirtyFiles().size()); @@ -574,7 +585,8 @@ public void testSelectFileBaseOnDirtyDataRateWithHeavySelect2() // compact all_deleted file and partial_deleted file 0 SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(3, seqTasks.size()); Assert.assertEquals(3, seqTasks.get(0).getFullyDirtyFiles().size()); @@ -616,7 +628,8 @@ public void testSelectContinuousFileWithLightSelectAligned() // inner task, continuous // select first time SettleSelectorImpl settleSelector = - new SettleSelectorImpl(false, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + false, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(1, seqTasks.size()); @@ -680,7 +693,8 @@ public void testSelectUnContinuousFileWithLightSelectAligned() // settle task, not continuous // select first time SettleSelectorImpl settleSelector = - new SettleSelectorImpl(false, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + false, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); int num = 10; for (int i = 0; i < 5; i++) { List seqTasks = settleSelector.selectSettleTask(seqResources); @@ -725,7 +739,8 @@ public void testSelectContinuousFilesBaseOnModsSizeWithHeavySelectAligned() // select first time SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(0, seqTasks.size()); @@ -758,7 +773,8 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelectAligned // select first time, file 0 is all deleted, file 1 2 3 4 is not satisfied by heavy select SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(1, seqTasks.size()); Assert.assertEquals(1, seqTasks.get(0).getFullyDirtyFiles().size()); @@ -768,7 +784,9 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelectAligned Assert.assertEquals(4, tsFileManager.getTsFileList(true).size()); // select second time - settleSelector = new SettleSelectorImpl(false, COMPACTION_TEST_SG, "0", 0, tsFileManager); + settleSelector = + new SettleSelectorImpl( + false, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(1, seqTasks.size()); Assert.assertEquals(0, seqTasks.get(0).getFullyDirtyFiles().size()); @@ -809,7 +827,8 @@ public void testSelectContinuousFileBaseOnDirtyDataRateWithHeavySelectAligned() // select first time, none partial_deleted and all_deleted files SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(0, seqTasks.size()); @@ -878,7 +897,8 @@ public void testSelectContinuousFileBaseOnDirtyDataOutdatedTooLongWithHeavySelec // inner task, continuous // select first time, none partial_deleted and all_deleted files SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(0, seqTasks.size()); @@ -945,7 +965,8 @@ public void testSelectUncontinuousFileBaseOnDirtyDataRateWithHeavySelectAligned( } } SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(0, seqTasks.size()); @@ -1040,7 +1061,8 @@ public void testSelectFileBaseOnDirtyDataRateWithHeavySelectAligned() } // compact all_deleted file and partial_deleted file 0 2 SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(4, seqTasks.size()); Assert.assertEquals(1, seqTasks.get(0).getFullyDirtyFiles().size()); @@ -1113,7 +1135,8 @@ public void testSelectFileBaseOnDirtyDataRateWithHeavySelect2Aligned() // compact all_deleted file and partial_deleted file 0 SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); Assert.assertEquals(3, seqTasks.size()); Assert.assertEquals(3, seqTasks.get(0).getFullyDirtyFiles().size()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionTaskTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionTaskTest.java index cbedc6d69e2a..9d040b00bed6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionTaskTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionTaskTest.java @@ -38,6 +38,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadPointCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.InnerSpaceCompactionTask; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.SettleCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleContext; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.impl.SettleSelectorImpl; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; @@ -158,7 +159,8 @@ public void settleWithOnlyAllDirtyFilesByMods() tsFileManager.addAll(unseqResources, false); SettleSelectorImpl settleSelector = - new SettleSelectorImpl(true, COMPACTION_TEST_SG, "0", 0, tsFileManager); + new SettleSelectorImpl( + true, COMPACTION_TEST_SG, "0", 0, tsFileManager, new CompactionScheduleContext()); List seqTasks = settleSelector.selectSettleTask(seqResources); List unseqTasks = settleSelector.selectSettleTask(unseqResources); Assert.assertEquals(1, seqTasks.size()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/CompactionWithAllNullRowsTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/CompactionWithAllNullRowsTest.java new file mode 100644 index 000000000000..641933f70a5a --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/CompactionWithAllNullRowsTest.java @@ -0,0 +1,315 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.compaction.tablemodel; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.db.storageengine.buffer.BloomFilterCache; +import org.apache.iotdb.db.storageengine.buffer.ChunkCache; +import org.apache.iotdb.db.storageengine.buffer.TimeSeriesMetadataCache; +import org.apache.iotdb.db.storageengine.dataregion.compaction.AbstractCompactionTest; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.constant.InnerSeqCompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.constant.InnerUnseqCompactionPerformer; +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.InnerSpaceCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.read.common.TimeRange; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; + +@RunWith(Parameterized.class) +public class CompactionWithAllNullRowsTest extends AbstractCompactionTest { + + private final String performerType; + private String threadName; + private final int maxAlignedSeriesNumInOneBatch; + private int defaultMaxAlignedSeriesNumInOneBatch; + + @Before + public void setUp() + throws IOException, WriteProcessException, MetadataException, InterruptedException { + this.threadName = Thread.currentThread().getName(); + Thread.currentThread().setName("pool-1-IoTDB-Compaction-Worker-1"); + this.defaultMaxAlignedSeriesNumInOneBatch = + IoTDBDescriptor.getInstance().getConfig().getCompactionMaxAlignedSeriesNumInOneBatch(); + IoTDBDescriptor.getInstance() + .getConfig() + .setCompactionMaxAlignedSeriesNumInOneBatch(maxAlignedSeriesNumInOneBatch); + } + + @After + public void tearDown() throws IOException, StorageEngineException { + ChunkCache.getInstance().clear(); + TimeSeriesMetadataCache.getInstance().clear(); + BloomFilterCache.getInstance().clear(); + Thread.currentThread().setName(threadName); + IoTDBDescriptor.getInstance() + .getConfig() + .setCompactionMaxAlignedSeriesNumInOneBatch(defaultMaxAlignedSeriesNumInOneBatch); + } + + @Parameterized.Parameters(name = "type={0} batch_size={1}") + public static Collection data() { + return Arrays.asList( + new Object[][] { + {"read_chunk", 10}, {"read_chunk", 2}, {"fast", 10}, {"fast", 2}, {"read_point", 10}, + }); + } + + public CompactionWithAllNullRowsTest(String performerType, int maxAlignedSeriesNumInOneBatch) { + this.performerType = performerType; + this.maxAlignedSeriesNumInOneBatch = maxAlignedSeriesNumInOneBatch; + } + + public ICompactionPerformer getPerformer() { + ICompactionPerformer performer; + if (performerType.equalsIgnoreCase(InnerSeqCompactionPerformer.READ_CHUNK.toString())) { + performer = new ReadChunkCompactionPerformer(); + } else if (performerType.equalsIgnoreCase(InnerUnseqCompactionPerformer.FAST.toString())) { + performer = new FastCompactionPerformer(false); + } else { + performer = new ReadPointCompactionPerformer(); + } + performer.setIgnoreAllNullRows(false); + return performer; + } + + @Test + public void testCompactionWithAllNullRows1() throws IOException { + TsFileResource resource1 = createEmptyFileAndResource(true); + try (CompactionTableModelTestFileWriter writer = + new CompactionTableModelTestFileWriter(resource1)) { + writer.registerTableSchema("t1", Arrays.asList("id1", "id2")); + writer.startChunkGroup("t1", Arrays.asList("id_field1", "id_field2")); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8", "s9"), + new TimeRange[][][] {new TimeRange[][] {new TimeRange[] {new TimeRange(10, 12)}}}, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(true, true, true, true, true, true, true, true, true, true)); + writer.endChunkGroup(); + writer.endFile(); + } + seqResources.add(resource1); + InnerSpaceCompactionTask task = + new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); + Assert.assertTrue(task.start()); + TsFileResource target = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(10, target.getFileStartTime()); + Assert.assertEquals(12, target.getFileEndTime()); + InnerSpaceCompactionTask task2 = + new InnerSpaceCompactionTask( + 0, tsFileManager, tsFileManager.getTsFileList(true), true, getPerformer(), 0); + Assert.assertTrue(task2.start()); + TsFileResource target2 = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(10, target2.getFileStartTime()); + Assert.assertEquals(12, target2.getFileEndTime()); + } + + @Test + public void testCompactionWithAllNullRows2() throws IOException { + TsFileResource resource1 = createEmptyFileAndResource(true); + try (CompactionTableModelTestFileWriter writer = + new CompactionTableModelTestFileWriter(resource1)) { + writer.registerTableSchema("t1", Arrays.asList("id1", "id2")); + writer.startChunkGroup("t1", Arrays.asList("id_field1", "id_field2")); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8", "s9"), + new TimeRange[][][] { + new TimeRange[][] {new TimeRange[] {new TimeRange(1, 9), new TimeRange(10, 12)}} + }, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(true, true, true, true, true, true, true, true, true, true)); + + writer.endChunkGroup(); + writer.endFile(); + } + seqResources.add(resource1); + InnerSpaceCompactionTask task = + new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); + Assert.assertTrue(task.start()); + TsFileResource target = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(1, target.getFileStartTime()); + Assert.assertEquals(12, target.getFileEndTime()); + InnerSpaceCompactionTask task2 = + new InnerSpaceCompactionTask( + 0, tsFileManager, tsFileManager.getTsFileList(true), true, getPerformer(), 0); + Assert.assertTrue(task2.start()); + TsFileResource target2 = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(1, target2.getFileStartTime()); + Assert.assertEquals(12, target2.getFileEndTime()); + } + + @Test + public void testCompactionWithAllNullRows3() throws IOException { + TsFileResource resource1 = createEmptyFileAndResource(true); + try (CompactionTableModelTestFileWriter writer = + new CompactionTableModelTestFileWriter(resource1)) { + writer.registerTableSchema("t1", Arrays.asList("id1", "id2")); + writer.startChunkGroup("t1", Arrays.asList("id_field1", "id_field2")); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8", "s9"), + new TimeRange[][] {new TimeRange[] {new TimeRange(10000, 19999)}}, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(true, true, true, true, true, true, true, true, true, true)); + + writer.endChunkGroup(); + writer.endFile(); + } + seqResources.add(resource1); + InnerSpaceCompactionTask task = + new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); + Assert.assertTrue(task.start()); + TsFileResource target = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(10000, target.getFileStartTime()); + Assert.assertEquals(19999, target.getFileEndTime()); + InnerSpaceCompactionTask task2 = + new InnerSpaceCompactionTask( + 0, tsFileManager, tsFileManager.getTsFileList(true), true, getPerformer(), 0); + Assert.assertTrue(task2.start()); + TsFileResource target2 = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(10000, target2.getFileStartTime()); + Assert.assertEquals(19999, target2.getFileEndTime()); + } + + @Test + public void testCompactionWithAllNullRows4() throws IOException { + TsFileResource resource1 = createEmptyFileAndResource(true); + try (CompactionTableModelTestFileWriter writer = + new CompactionTableModelTestFileWriter(resource1)) { + writer.registerTableSchema("t1", Arrays.asList("id1", "id2")); + writer.startChunkGroup("t1", Arrays.asList("id_field1", "id_field2")); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8", "s9"), + new TimeRange[] {new TimeRange(100000, 199999)}, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(true, true, true, true, true, true, true, true, true, true)); + + writer.endChunkGroup(); + writer.endFile(); + } + seqResources.add(resource1); + InnerSpaceCompactionTask task = + new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); + Assert.assertTrue(task.start()); + TsFileResource target = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(100000, target.getFileStartTime()); + Assert.assertEquals(199999, target.getFileEndTime()); + InnerSpaceCompactionTask task2 = + new InnerSpaceCompactionTask( + 0, tsFileManager, tsFileManager.getTsFileList(true), true, getPerformer(), 0); + Assert.assertTrue(task2.start()); + TsFileResource target2 = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(100000, target2.getFileStartTime()); + Assert.assertEquals(199999, target2.getFileEndTime()); + } + + @Test + @Ignore + public void testCompactionWithAllDeletion() throws IOException, IllegalPathException { + TsFileResource resource1 = createEmptyFileAndResource(true); + IDeviceID deviceID = null; + try (CompactionTableModelTestFileWriter writer = + new CompactionTableModelTestFileWriter(resource1)) { + writer.registerTableSchema("t1", Arrays.asList("id1", "id2")); + deviceID = writer.startChunkGroup("t1", Arrays.asList("id_field1", "id_field2")); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8", "s9"), + new TimeRange[][][] {new TimeRange[][] {new TimeRange[] {new TimeRange(10, 12)}}}, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(true, true, true, true, true, true, true, true, true, true)); + writer.endChunkGroup(); + writer.endFile(); + } + resource1 + .getModFile() + .write(new Deletion(new MeasurementPath(deviceID, ""), Long.MAX_VALUE, Long.MAX_VALUE)); + resource1.getModFile().close(); + seqResources.add(resource1); + InnerSpaceCompactionTask task = + new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); + Assert.assertTrue(task.start()); + Assert.assertTrue(tsFileManager.getTsFileList(true).isEmpty()); + } + + @Test + public void testCompactionWithAllValueColumnDeletion() throws IOException, IllegalPathException { + TsFileResource resource1 = createEmptyFileAndResource(true); + IDeviceID deviceID = null; + try (CompactionTableModelTestFileWriter writer = + new CompactionTableModelTestFileWriter(resource1)) { + writer.registerTableSchema("t1", Arrays.asList("id1", "id2")); + deviceID = writer.startChunkGroup("t1", Arrays.asList("id_field1", "id_field2")); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1", "s2", "s3"), + new TimeRange[][][] {new TimeRange[][] {new TimeRange[] {new TimeRange(10, 12)}}}, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(true, true, true, true)); + writer.endChunkGroup(); + writer.endFile(); + } + resource1 + .getModFile() + .write(new Deletion(new MeasurementPath(deviceID, "s0"), Long.MAX_VALUE, 11)); + resource1 + .getModFile() + .write(new Deletion(new MeasurementPath(deviceID, "s1"), Long.MAX_VALUE, 11)); + resource1 + .getModFile() + .write(new Deletion(new MeasurementPath(deviceID, "s2"), Long.MAX_VALUE, 11)); + resource1 + .getModFile() + .write(new Deletion(new MeasurementPath(deviceID, "s3"), Long.MAX_VALUE, 11)); + resource1.getModFile().close(); + seqResources.add(resource1); + InnerSpaceCompactionTask task = + new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); + Assert.assertTrue(task.start()); + TsFileResource target = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(10, target.getFileStartTime()); + Assert.assertEquals(12, target.getFileEndTime()); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/BatchCompactionUtilsTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/BatchCompactionUtilsTest.java index 45e19a9a24d3..5755964a4366 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/BatchCompactionUtilsTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/BatchCompactionUtilsTest.java @@ -23,9 +23,9 @@ import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.storageengine.dataregion.compaction.AbstractCompactionTest; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.AlignedSeriesBatchCompactionUtils; -import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.BatchedCompactionAlignedPagePointReader; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.CompactChunkPlan; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.CompactPagePlan; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.CompactionAlignedPageLazyLoadPointReader; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.FirstBatchCompactionAlignedChunkWriter; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.FollowingBatchCompactionAlignedChunkWriter; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -108,7 +108,8 @@ public void testBatchCompactionPointReader() throws IOException { new TsFileSequenceReader(seqResource1.getTsFile().getAbsolutePath())) { AlignedChunkMetadata alignedChunkMetadata = reader - .getAlignedChunkMetadata(IDeviceID.Factory.DEFAULT_FACTORY.create("root.testsg.d0")) + .getAlignedChunkMetadata( + IDeviceID.Factory.DEFAULT_FACTORY.create("root.testsg.d0"), true) .get(0); ChunkMetadata timeChunkMetadata = (ChunkMetadata) alignedChunkMetadata.getTimeChunkMetadata(); List valueChunkMetadataList = @@ -126,9 +127,11 @@ public void testBatchCompactionPointReader() throws IOException { AlignedChunkReader alignedChunkReader = new AlignedChunkReader(timeChunk, valueChunks); AlignedPageReader iPageReader = (AlignedPageReader) alignedChunkReader.loadPageReaderList().get(0); - BatchedCompactionAlignedPagePointReader batchCompactionPointReader = - new BatchedCompactionAlignedPagePointReader( - iPageReader.getTimePageReader(), iPageReader.getValuePageReaderList().subList(1, 2)); + CompactionAlignedPageLazyLoadPointReader batchCompactionPointReader = + new CompactionAlignedPageLazyLoadPointReader( + iPageReader.getTimePageReader(), + iPageReader.getValuePageReaderList().subList(1, 2), + false); int readPointNum = 0; while (batchCompactionPointReader.hasNextTimeValuePair()) { TimeValuePair timeValuePair = batchCompactionPointReader.nextTimeValuePair(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionCheckerUtils.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionCheckerUtils.java index 7c9b4ccd6e11..13ee5d972462 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionCheckerUtils.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionCheckerUtils.java @@ -526,7 +526,8 @@ public static Map> getAllDataByQuery( public static List getAllPathsOfResources(List resources) throws IOException, IllegalPathException { Set paths = new HashSet<>(); - try (MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(resources)) { + try (MultiTsFileDeviceIterator deviceIterator = + new MultiTsFileDeviceIterator(resources, false)) { while (deviceIterator.hasNextDevice()) { Pair iDeviceIDBooleanPair = deviceIterator.nextDevice(); IDeviceID deviceID = iDeviceIDBooleanPair.getLeft(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/MultiTsFileDeviceIteratorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/MultiTsFileDeviceIteratorTest.java index d26b03964b74..b9d653efc841 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/MultiTsFileDeviceIteratorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/MultiTsFileDeviceIteratorTest.java @@ -126,7 +126,7 @@ public void testMeasurementIterator() throws IOException, MetadataException { List measurementSet = new ArrayList<>(4000); try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(seqResources)) { + new MultiTsFileDeviceIterator(seqResources, false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceIsAlignedPair = multiTsFileDeviceIterator.nextDevice(); IDeviceID device = deviceIsAlignedPair.getLeft(); @@ -172,7 +172,7 @@ public void getNonAlignedDevicesFromDifferentFilesWithFourLayersInNodeTreeTest() int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(seqResources, unseqResources)) { + new MultiTsFileDeviceIterator(seqResources, unseqResources, false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -203,7 +203,7 @@ public void getAlignedDevicesFromDifferentFilesWithOneLayerInNodeTreeTest() int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(seqResources, unseqResources)) { + new MultiTsFileDeviceIterator(seqResources, unseqResources, false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -241,7 +241,7 @@ public void getNonAlignedDevicesFromDifferentFilesWithFourLayersInNodeTreeTestUs int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(seqResources, unseqResources)) { + new MultiTsFileDeviceIterator(seqResources, unseqResources, false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -280,7 +280,7 @@ public void getAlignedDevicesFromDifferentFilesWithOneLayerInNodeTreeTestUsingFi int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(seqResources, unseqResources)) { + new MultiTsFileDeviceIterator(seqResources, unseqResources, false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -332,7 +332,7 @@ public void getDeletedDevicesWithSameNameFromDifferentFilesWithFourLayersInNodeT int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(seqResources, unseqResources)) { + new MultiTsFileDeviceIterator(seqResources, unseqResources, false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -384,7 +384,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesWithFourLayersInNodeTreeTes int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(seqResources)) { + new MultiTsFileDeviceIterator(seqResources, false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -447,7 +447,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer() int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -526,7 +526,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer() deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -601,7 +601,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer2() int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -681,7 +681,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer2() deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -751,7 +751,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer() thro int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -830,7 +830,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer() thro deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -901,7 +901,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer2() thr int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -990,7 +990,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer2() thr deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1061,7 +1061,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer() throws Ex int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1140,7 +1140,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer() throws Ex deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1211,7 +1211,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer2() throws E int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1300,7 +1300,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer2() throws E deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true), false)) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); diff --git a/pom.xml b/pom.xml index 0e46b02a0769..88a6e1007abc 100644 --- a/pom.xml +++ b/pom.xml @@ -166,7 +166,7 @@ 0.14.1 1.9 1.5.6-3 - 1.2.0-4467fa07-SNAPSHOT + 1.2.0-240920-SNAPSHOT