You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/04/12 09:01:12 UTC
[iotdb] branch master updated: [IOTDB-2817] Clear Compaction Framework (#5437)
This is an automated email from the ASF dual-hosted git repository.
qiaojialin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/master by this push:
new c889624e3e [IOTDB-2817] Clear Compaction Framework (#5437)
c889624e3e is described below
commit c889624e3e3163878e32d14be168077e37ac22de
Author: Liu Xuxin <37...@users.noreply.github.com>
AuthorDate: Tue Apr 12 02:01:06 2022 -0700
[IOTDB-2817] Clear Compaction Framework (#5437)
---
.../resources/conf/iotdb-engine.properties | 28 +-
.../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 82 ++++-
.../org/apache/iotdb/db/conf/IoTDBDescriptor.java | 40 ++-
.../{task => }/CompactionExceptionHandler.java | 3 +-
.../compaction/CompactionMetricsManager.java | 12 +-
.../{task => }/CompactionRecoverManager.java | 9 +-
.../db/engine/compaction/CompactionScheduler.java | 165 +++++----
.../engine/compaction/CompactionTaskManager.java | 5 +-
.../db/engine/compaction/CompactionUtils.java | 371 +++++----------------
.../DefaultCompactionTaskComparatorImpl.java} | 34 +-
.../ICompactionTaskComparator.java} | 37 +-
.../CrossCompactionPerformer.java} | 30 +-
.../CrossCompactionSelector.java} | 40 +--
.../InnerSeqCompactionPerformer.java} | 30 +-
.../constant/InnerSequenceCompactionSelector.java} | 41 +--
.../InnerUnseqCompactionPerformer.java} | 30 +-
.../InnerUnsequenceCompactionSelector.java} | 43 ++-
.../AbstractCrossSpaceCompactionSelector.java | 56 ----
.../cross/AbstractCrossSpaceCompactionTask.java | 113 -------
.../compaction/cross/CrossSpaceCompactionTask.java | 281 ++++++++++++++++
.../ICrossSpaceSelector.java} | 19 +-
...va => RewriteCrossSpaceCompactionResource.java} | 8 +-
.../RewriteCrossSpaceCompactionSelector.java | 60 ++--
.../selector/ICrossSpaceMergeFileSelector.java | 2 -
.../cross/rewrite/selector/NaivePathSelector.java | 55 ---
.../selector/RewriteCompactionFileSelector.java | 12 +-
...ionTask.java => ReadPointPerformerSubTask.java} | 10 +-
.../task/RewriteCrossSpaceCompactionTask.java | 254 --------------
.../AbstractInnerSpaceCompactionSelector.java | 59 ----
.../inner/AbstractInnerSpaceCompactionTask.java | 160 ---------
.../IInnerSeqSpaceSelector.java} | 17 +-
.../IInnerUnseqSpaceSelector.java} | 15 +-
.../compaction/inner/InnerCompactionStrategy.java | 80 -----
...tionTask.java => InnerSpaceCompactionTask.java} | 225 +++++++++----
.../sizetiered/SizeTieredCompactionSelector.java | 70 ++--
.../inner/utils/InnerSpaceCompactionUtils.java | 302 -----------------
.../{utils => }/log/CompactionLogAnalyzer.java | 21 +-
.../{utils => }/log/CompactionLogger.java | 3 +-
.../compaction/{ => log}/TsFileIdentifier.java | 2 +-
.../compaction/performer/ICompactionPerformer.java | 49 +++
.../ICrossCompactionPerformer.java} | 12 +-
.../ISeqCompactionPerformer.java} | 12 +-
.../IUnseqCompactionPerformer.java} | 12 +-
.../impl/ReadChunkCompactionPerformer.java | 166 +++++++++
.../impl/ReadPointCompactionPerformer.java} | 270 ++++++---------
.../compaction/task/AbstractCompactionTask.java | 4 +-
.../compaction/task/CompactionRecoverTask.java | 6 +-
.../compaction/task/ICompactionSelector.java | 54 +++
.../iotdb/db/engine/storagegroup/DataRegion.java | 2 +-
.../compaction/CompactionTaskComparatorTest.java | 31 +-
.../compaction/CompactionTaskManagerTest.java | 101 ++++--
....java => ReadPointCompactionPerformerTest.java} | 57 ++--
.../db/engine/compaction/TsFileIdentifierUT.java | 3 +-
.../cross/CrossSpaceCompactionExceptionTest.java | 23 +-
.../compaction/cross/CrossSpaceCompactionTest.java | 81 ++---
.../engine/compaction/cross/MergeUpgradeTest.java | 6 +-
.../cross/RewriteCompactionFileSelectorTest.java | 58 ++--
.../RewriteCrossSpaceCompactionRecoverTest.java | 25 +-
.../cross/RewriteCrossSpaceCompactionTest.java | 46 ++-
.../compaction/inner/InnerCompactionLogTest.java | 2 +-
.../inner/InnerCompactionSchedulerTest.java | 18 +-
.../compaction/inner/InnerSeqCompactionTest.java | 40 ++-
.../inner/InnerSpaceCompactionExceptionTest.java | 44 ++-
.../compaction/inner/InnerUnseqCompactionTest.java | 14 +-
...> ReadChunkCompactionPerformerAlignedTest.java} | 18 +-
...ReadChunkCompactionPerformerNoAlignedTest.java} | 22 +-
...va => ReadChunkCompactionPerformerOldTest.java} | 17 +-
.../SizeTieredCompactionRecoverTest.java | 77 +++--
...eCrossSpaceCompactionRecoverCompatibleTest.java | 2 +-
.../SizeTieredCompactionRecoverCompatibleTest.java | 20 +-
.../recover/SizeTieredCompactionRecoverTest.java | 49 +--
.../task/FakedCrossSpaceCompactionTask.java | 62 ----
.../task/FakedInnerSpaceCompactionTask.java | 97 ------
.../task/FakedInnerSpaceCompactionTaskFactory.java | 50 ---
.../compaction/utils/CompactionClearUtils.java | 2 +-
.../compaction/utils/CompactionConfigRestorer.java | 13 +-
.../storagegroup/StorageGroupProcessorTest.java | 12 +-
77 files changed, 1841 insertions(+), 2560 deletions(-)
diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index de12695ec6..85a6418304 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -453,13 +453,29 @@ timestamp_precision=ms
# Datatype: boolean
# enable_cross_space_compaction=true
-# the strategy of cross space compaction task
-# Options: rewrite_compaction
-# cross_compaction_strategy=rewrite_compaction
+# the selector of cross space compaction task
+# Options: rewrite
+# cross_selector=rewrite
-# the strategy of inner space compaction task
-# Options: size_tiered_compaction
-# inner_compaction_strategy=size_tiered_compaction
+# the compaction performer of cross space compaction task
+# Options: read_point
+# cross_performer=read_point
+
+# the selector of inner sequence space compaction task
+# Options: size_tiered
+# inner_seq_selector=size_tiered
+
+# the performer of inner sequence space compaction task
+# Options: read_chunk
+# inner_seq_performer=read_chunk
+
+# the selector of inner unsequence space compaction task
+# Options: size_tiered
+# inner_unseq_selector=size_tiered
+
+# the performer of inner unsequence space compaction task
+# Options: read_point
+# inner_seq_performer=read_point
# The priority of compaction execution
# INNER_CROSS: prioritize inner space compaction, reduce the number of files first
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index a6932e3a93..edee341863 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -21,8 +21,12 @@ package org.apache.iotdb.db.conf;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.directories.DirectoryManager;
import org.apache.iotdb.db.engine.compaction.constant.CompactionPriority;
-import org.apache.iotdb.db.engine.compaction.cross.CrossCompactionStrategy;
-import org.apache.iotdb.db.engine.compaction.inner.InnerCompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.constant.InnerSeqCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.constant.InnerSequenceCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.constant.InnerUnseqCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.constant.InnerUnsequenceCompactionSelector;
import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel;
import org.apache.iotdb.db.exception.LoadConfigurationException;
import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
@@ -362,10 +366,10 @@ public class IoTDBConfig {
/** When average series point number reaches this, flush the memtable to disk */
private int avgSeriesPointNumberThreshold = 10000;
- /** Enable inner space copaction for sequence files */
+ /** Enable inner space compaction for sequence files */
private boolean enableSeqSpaceCompaction = true;
- /** Enable inner space copaction for unsequence files */
+ /** Enable inner space compaction for unsequence files */
private boolean enableUnseqSpaceCompaction = true;
/** Compact the unsequence files into the overlapped sequence files */
@@ -375,15 +379,25 @@ public class IoTDBConfig {
* The strategy of inner space compaction task. There are just one inner space compaction strategy
* SIZE_TIRED_COMPACTION:
*/
- private InnerCompactionStrategy innerCompactionStrategy =
- InnerCompactionStrategy.SIZE_TIERED_COMPACTION;
+ private InnerSequenceCompactionSelector innerSequenceCompactionSelector =
+ InnerSequenceCompactionSelector.SIZE_TIERED;
+
+ private InnerSeqCompactionPerformer innerSeqCompactionPerformer =
+ InnerSeqCompactionPerformer.READ_CHUNK;
+
+ private InnerUnsequenceCompactionSelector innerUnsequenceCompactionSelector =
+ InnerUnsequenceCompactionSelector.SIZE_TIERED;
+
+ private InnerUnseqCompactionPerformer innerUnseqCompactionPerformer =
+ InnerUnseqCompactionPerformer.READ_POINT;
/**
* The strategy of cross space compaction task. There are just one cross space compaction strategy
* SIZE_TIRED_COMPACTION:
*/
- private CrossCompactionStrategy crossCompactionStrategy =
- CrossCompactionStrategy.REWRITE_COMPACTION;
+ private CrossCompactionSelector crossCompactionSelector = CrossCompactionSelector.REWRITE;
+
+ private CrossCompactionPerformer crossCompactionPerformer = CrossCompactionPerformer.READ_POINT;
/**
* The priority of compaction task execution. There are three priority strategy INNER_CROSS:
@@ -2455,20 +2469,56 @@ public class IoTDBConfig {
this.enableCrossSpaceCompaction = enableCrossSpaceCompaction;
}
- public InnerCompactionStrategy getInnerCompactionStrategy() {
- return innerCompactionStrategy;
+ public InnerSequenceCompactionSelector getInnerSequenceCompactionSelector() {
+ return innerSequenceCompactionSelector;
+ }
+
+ public void setInnerSequenceCompactionSelector(
+ InnerSequenceCompactionSelector innerSequenceCompactionSelector) {
+ this.innerSequenceCompactionSelector = innerSequenceCompactionSelector;
+ }
+
+ public InnerUnsequenceCompactionSelector getInnerUnsequenceCompactionSelector() {
+ return innerUnsequenceCompactionSelector;
+ }
+
+ public void setInnerUnsequenceCompactionSelector(
+ InnerUnsequenceCompactionSelector innerUnsequenceCompactionSelector) {
+ this.innerUnsequenceCompactionSelector = innerUnsequenceCompactionSelector;
+ }
+
+ public InnerSeqCompactionPerformer getInnerSeqCompactionPerformer() {
+ return innerSeqCompactionPerformer;
+ }
+
+ public void setInnerSeqCompactionPerformer(
+ InnerSeqCompactionPerformer innerSeqCompactionPerformer) {
+ this.innerSeqCompactionPerformer = innerSeqCompactionPerformer;
+ }
+
+ public InnerUnseqCompactionPerformer getInnerUnseqCompactionPerformer() {
+ return innerUnseqCompactionPerformer;
+ }
+
+ public void setInnerUnseqCompactionPerformer(
+ InnerUnseqCompactionPerformer innerUnseqCompactionPerformer) {
+ this.innerUnseqCompactionPerformer = innerUnseqCompactionPerformer;
+ }
+
+ public CrossCompactionSelector getCrossCompactionSelector() {
+ return crossCompactionSelector;
}
- public void setInnerCompactionStrategy(InnerCompactionStrategy innerCompactionStrategy) {
- this.innerCompactionStrategy = innerCompactionStrategy;
+ public void setCrossCompactionSelector(CrossCompactionSelector crossCompactionSelector) {
+ this.crossCompactionSelector = crossCompactionSelector;
}
- public CrossCompactionStrategy getCrossCompactionStrategy() {
- return crossCompactionStrategy;
+ public CrossCompactionPerformer getCrossCompactionPerformer() {
+ return crossCompactionPerformer;
}
- public void setCrossCompactionStrategy(CrossCompactionStrategy crossCompactionStrategy) {
- this.crossCompactionStrategy = crossCompactionStrategy;
+ public void setCrossCompactionPerformer(CrossCompactionPerformer crossCompactionPerformer) {
+ this.crossCompactionPerformer = crossCompactionPerformer;
}
public CompactionPriority getCompactionPriority() {
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 2cb530bebb..e48f56989c 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -22,8 +22,12 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.directories.DirectoryManager;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.engine.compaction.constant.CompactionPriority;
-import org.apache.iotdb.db.engine.compaction.cross.CrossCompactionStrategy;
-import org.apache.iotdb.db.engine.compaction.inner.InnerCompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.constant.InnerSeqCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.constant.InnerSequenceCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.constant.InnerUnseqCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.constant.InnerUnsequenceCompactionSelector;
import org.apache.iotdb.db.exception.BadNodeUrlFormatException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.qp.utils.DatetimeUtils;
@@ -359,15 +363,35 @@ public class IoTDBDescriptor {
"enable_unseq_space_compaction",
Boolean.toString(conf.isEnableUnseqSpaceCompaction()))));
- conf.setCrossCompactionStrategy(
- CrossCompactionStrategy.getCrossCompactionStrategy(
+ conf.setCrossCompactionSelector(
+ CrossCompactionSelector.getCrossCompactionStrategy(
properties.getProperty(
- "cross_compaction_strategy", conf.getCrossCompactionStrategy().toString())));
+ "cross_selector", conf.getCrossCompactionSelector().toString())));
- conf.setInnerCompactionStrategy(
- InnerCompactionStrategy.getInnerCompactionStrategy(
+ conf.setInnerSequenceCompactionSelector(
+ InnerSequenceCompactionSelector.getInnerSequenceCompactionStrategy(
properties.getProperty(
- "inner_compaction_strategy", conf.getInnerCompactionStrategy().toString())));
+ "inner_seq_selector", conf.getInnerSequenceCompactionSelector().toString())));
+
+ conf.setInnerUnsequenceCompactionSelector(
+ InnerUnsequenceCompactionSelector.getInnerUnsequenceCompactionStrategy(
+ properties.getProperty(
+ "inner_unseq_selector", conf.getInnerUnsequenceCompactionSelector().toString())));
+
+ conf.setInnerSeqCompactionPerformer(
+ InnerSeqCompactionPerformer.getInnerSeqCompactionPerformer(
+ properties.getProperty(
+ "inner_seq_performer", conf.getInnerUnseqCompactionPerformer().toString())));
+
+ conf.setInnerUnseqCompactionPerformer(
+ InnerUnseqCompactionPerformer.getInnerUnseqCompactionPerformer(
+ properties.getProperty(
+ "inner_unseq_performer", conf.getInnerUnseqCompactionPerformer().toString())));
+
+ conf.setCrossCompactionPerformer(
+ CrossCompactionPerformer.getCrossCompactionPerformer(
+ properties.getProperty(
+ "cross_performer", conf.getCrossCompactionPerformer().toString())));
conf.setCompactionPriority(
CompactionPriority.valueOf(
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionExceptionHandler.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionExceptionHandler.java
similarity index 98%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionExceptionHandler.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionExceptionHandler.java
index 91cca7b621..3523394333 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionExceptionHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionExceptionHandler.java
@@ -17,11 +17,10 @@
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.task;
+package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionUtils;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionMetricsManager.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionMetricsManager.java
index 4c6b195ca5..f852dd9d61 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionMetricsManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionMetricsManager.java
@@ -21,8 +21,8 @@ package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.db.engine.compaction.constant.CompactionTaskStatus;
import org.apache.iotdb.db.engine.compaction.constant.CompactionType;
import org.apache.iotdb.db.engine.compaction.constant.ProcessChunkType;
-import org.apache.iotdb.db.engine.compaction.cross.AbstractCrossSpaceCompactionTask;
-import org.apache.iotdb.db.engine.compaction.inner.AbstractInnerSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
import org.apache.iotdb.db.service.metrics.Metric;
import org.apache.iotdb.db.service.metrics.MetricsService;
@@ -89,10 +89,10 @@ public class CompactionMetricsManager {
}
String taskType = "unknown";
boolean isInnerTask = false;
- if (task instanceof AbstractInnerSpaceCompactionTask) {
+ if (task instanceof InnerSpaceCompactionTask) {
isInnerTask = true;
taskType = "inner";
- } else if (task instanceof AbstractCrossSpaceCompactionTask) {
+ } else if (task instanceof CrossSpaceCompactionTask) {
taskType = "cross";
}
@@ -154,9 +154,7 @@ public class CompactionMetricsManager {
Tag.NAME.toString(),
"inner_compaction_count",
Tag.TYPE.toString(),
- ((AbstractInnerSpaceCompactionTask) task).isSequence()
- ? "sequence"
- : "unsequence");
+ ((InnerSpaceCompactionTask) task).isSequence() ? "sequence" : "unsequence");
} else {
MetricsService.getInstance()
.getMetricManager()
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionRecoverManager.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionRecoverManager.java
similarity index 91%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionRecoverManager.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionRecoverManager.java
index fe46eb0c71..4e23df5ac4 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionRecoverManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionRecoverManager.java
@@ -16,11 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.task;
+package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.directories.DirectoryManager;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.task.CompactionRecoverTask;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
@@ -31,8 +32,8 @@ import java.io.File;
import java.util.List;
import java.util.regex.Pattern;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.CROSS_COMPACTION_LOG_NAME_FROM_OLD;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.CROSS_COMPACTION_LOG_NAME_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX_FROM_OLD;
/**
* CompactionRecoverManager searches compaction log and call {@link CompactionRecoverTask} to
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java
index d9b641f472..02eca44554 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionScheduler.java
@@ -19,101 +19,152 @@
package org.apache.iotdb.db.engine.compaction;
+import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.cross.AbstractCrossSpaceCompactionSelector;
-import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTaskFactory;
-import org.apache.iotdb.db.engine.compaction.inner.AbstractInnerSpaceCompactionSelector;
-import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTaskFactory;
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.cross.ICrossSpaceSelector;
+import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.performer.ICompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.ICompactionSelector;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
/**
* CompactionScheduler schedules and submits the compaction task periodically, and it counts the
* total number of running compaction task. There are three compaction strategy: BALANCE,
* INNER_CROSS, CROSS_INNER. Difference strategies will lead to different compaction preferences.
* For different types of compaction task(e.g. InnerSpaceCompaction), CompactionScheduler will call
- * the corresponding {@link org.apache.iotdb.db.engine.compaction.task.AbstractCompactionSelector
- * selector} according to the compaction machanism of the task(e.g. LevelCompaction,
- * SizeTiredCompaction), and the selection and submission process is carried out in the {@link
- * AbstractCompactionSelector#selectAndSubmit() selectAndSubmit()} in selector.
+ * the corresponding {@link ICompactionSelector selector} according to the compaction machanism of
+ * the task(e.g. LevelCompaction, SizeTiredCompaction), and the selection and submission process is
+ * carried out in the {@link ICompactionSelector#selectInnerSpaceTask(List)} () and {@link
+ * ICompactionSelector#selectCrossSpaceTask(List, List)}} in selector.
*/
public class CompactionScheduler {
-
+ private static final Logger LOGGER =
+ LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
private static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
public static void scheduleCompaction(TsFileManager tsFileManager, long timePartition) {
if (!tsFileManager.isAllowCompaction()) {
return;
}
- tryToSubmitCrossSpaceCompactionTask(
- tsFileManager.getStorageGroupName(),
- tsFileManager.getDataRegion(),
- tsFileManager.getStorageGroupDir(),
- timePartition,
- tsFileManager,
- new CrossSpaceCompactionTaskFactory());
- tryToSubmitInnerSpaceCompactionTask(
- tsFileManager.getStorageGroupName(),
- tsFileManager.getDataRegion(),
- timePartition,
- tsFileManager,
- true,
- new InnerSpaceCompactionTaskFactory());
- tryToSubmitInnerSpaceCompactionTask(
- tsFileManager.getStorageGroupName(),
- tsFileManager.getDataRegion(),
- timePartition,
- tsFileManager,
- false,
- new InnerSpaceCompactionTaskFactory());
+ try {
+ tryToSubmitCrossSpaceCompactionTask(
+ tsFileManager.getStorageGroupName(),
+ tsFileManager.getDataRegion(),
+ tsFileManager.getStorageGroupDir(),
+ timePartition,
+ tsFileManager);
+ tryToSubmitInnerSpaceCompactionTask(
+ tsFileManager.getStorageGroupName(),
+ tsFileManager.getDataRegion(),
+ timePartition,
+ tsFileManager,
+ true);
+ tryToSubmitInnerSpaceCompactionTask(
+ tsFileManager.getStorageGroupName(),
+ tsFileManager.getDataRegion(),
+ timePartition,
+ tsFileManager,
+ false);
+ } catch (InterruptedException e) {
+ LOGGER.error("Exception occurs when selecting compaction tasks", e);
+ }
}
public static void tryToSubmitInnerSpaceCompactionTask(
String logicalStorageGroupName,
- String virtualStorageGroupName,
+ String dataRegionId,
long timePartition,
TsFileManager tsFileManager,
- boolean sequence,
- InnerSpaceCompactionTaskFactory taskFactory) {
+ boolean sequence)
+ throws InterruptedException {
if ((!config.isEnableSeqSpaceCompaction() && sequence)
|| (!config.isEnableUnseqSpaceCompaction() && !sequence)) {
return;
}
- AbstractInnerSpaceCompactionSelector innerSpaceCompactionSelector =
- config
- .getInnerCompactionStrategy()
- .getCompactionSelector(
- logicalStorageGroupName,
- virtualStorageGroupName,
- timePartition,
- tsFileManager,
- sequence,
- taskFactory);
- innerSpaceCompactionSelector.selectAndSubmit();
+ ICompactionSelector innerSpaceCompactionSelector = null;
+ if (sequence) {
+ innerSpaceCompactionSelector =
+ config
+ .getInnerSequenceCompactionSelector()
+ .getCompactionSelector(
+ logicalStorageGroupName, dataRegionId, timePartition, tsFileManager);
+ } else {
+ innerSpaceCompactionSelector =
+ config
+ .getInnerUnsequenceCompactionSelector()
+ .getCompactionSelector(
+ logicalStorageGroupName, dataRegionId, timePartition, tsFileManager);
+ }
+ List<List<TsFileResource>> taskList =
+ innerSpaceCompactionSelector.selectInnerSpaceTask(
+ sequence
+ ? tsFileManager.getSequenceListByTimePartition(timePartition)
+ : tsFileManager.getUnsequenceListByTimePartition(timePartition));
+ for (List<TsFileResource> task : taskList) {
+ ICompactionPerformer performer =
+ sequence
+ ? IoTDBDescriptor.getInstance()
+ .getConfig()
+ .getInnerSeqCompactionPerformer()
+ .getCompactionPerformer()
+ : IoTDBDescriptor.getInstance()
+ .getConfig()
+ .getInnerUnseqCompactionPerformer()
+ .getCompactionPerformer();
+ CompactionTaskManager.getInstance()
+ .addTaskToWaitingQueue(
+ new InnerSpaceCompactionTask(
+ timePartition,
+ tsFileManager,
+ task,
+ sequence,
+ performer,
+ CompactionTaskManager.currentTaskNum));
+ }
}
private static void tryToSubmitCrossSpaceCompactionTask(
String logicalStorageGroupName,
- String virtualStorageGroupName,
+ String dataRegionId,
String storageGroupDir,
long timePartition,
- TsFileManager tsFileManager,
- CrossSpaceCompactionTaskFactory taskFactory) {
+ TsFileManager tsFileManager)
+ throws InterruptedException {
if (!config.isEnableCrossSpaceCompaction()) {
return;
}
- AbstractCrossSpaceCompactionSelector crossSpaceCompactionSelector =
+ ICrossSpaceSelector crossSpaceCompactionSelector =
config
- .getCrossCompactionStrategy()
+ .getCrossCompactionSelector()
.getCompactionSelector(
- logicalStorageGroupName,
- virtualStorageGroupName,
- storageGroupDir,
- timePartition,
- tsFileManager,
- taskFactory);
- crossSpaceCompactionSelector.selectAndSubmit();
+ logicalStorageGroupName, dataRegionId, timePartition, tsFileManager);
+ List<Pair<List<TsFileResource>, List<TsFileResource>>> taskList =
+ crossSpaceCompactionSelector.selectCrossSpaceTask(
+ tsFileManager.getSequenceListByTimePartition(timePartition),
+ tsFileManager.getUnsequenceListByTimePartition(timePartition));
+ for (Pair<List<TsFileResource>, List<TsFileResource>> selectedFilesPair : taskList) {
+ CompactionTaskManager.getInstance()
+ .addTaskToWaitingQueue(
+ new CrossSpaceCompactionTask(
+ timePartition,
+ tsFileManager,
+ selectedFilesPair.left,
+ selectedFilesPair.right,
+ IoTDBDescriptor.getInstance()
+ .getConfig()
+ .getCrossCompactionPerformer()
+ .getCompactionPerformer(),
+ CompactionTaskManager.currentTaskNum));
+ }
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java
index bf54c18b91..7e9b7a9636 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.commons.service.IService;
import org.apache.iotdb.commons.service.ServiceType;
import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.comparator.DefaultCompactionTaskComparatorImpl;
import org.apache.iotdb.db.engine.compaction.constant.CompactionTaskStatus;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
import org.apache.iotdb.db.utils.datastructure.FixedPriorityBlockingQueue;
@@ -62,7 +63,7 @@ public class CompactionTaskManager implements IService {
public static volatile AtomicInteger currentTaskNum = new AtomicInteger(0);
private FixedPriorityBlockingQueue<AbstractCompactionTask> candidateCompactionTaskQueue =
- new FixedPriorityBlockingQueue<>(1024, new CompactionTaskComparator());
+ new FixedPriorityBlockingQueue<>(1024, new DefaultCompactionTaskComparatorImpl());
// <fullStorageGroupName,futureSet>, it is used to store all compaction tasks under each
// virtualStorageGroup
private Map<String, Map<AbstractCompactionTask, Future<Void>>> storageGroupTasks =
@@ -289,7 +290,7 @@ public class CompactionTaskManager implements IService {
}
}
- public synchronized void removeRunningTaskFromList(AbstractCompactionTask task) {
+ public synchronized void removeRunningTaskFuture(AbstractCompactionTask task) {
String storageGroupName = task.getFullStorageGroupName();
if (storageGroupTasks.containsKey(storageGroupName)) {
storageGroupTasks.get(storageGroupName).remove(task);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java
index a4328449ca..ac3d9bcfae 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java
@@ -20,39 +20,19 @@ package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.SubCompactionTask;
-import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
-import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
-import org.apache.iotdb.db.engine.compaction.writer.CrossSpaceCompactionWriter;
-import org.apache.iotdb.db.engine.compaction.writer.InnerSpaceCompactionWriter;
+import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.ICrossSpaceMergeFileSelector;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.RewriteCompactionFileSelector;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.modification.ModificationFile;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-import org.apache.iotdb.db.metadata.idtable.IDTableManager;
-import org.apache.iotdb.db.metadata.path.AlignedPath;
-import org.apache.iotdb.db.metadata.path.MeasurementPath;
-import org.apache.iotdb.db.metadata.path.PartialPath;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.QueryResourceManager;
-import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
+import org.apache.iotdb.db.query.control.FileReaderManager;
import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
-import org.apache.iotdb.tsfile.read.common.BatchData;
-import org.apache.iotdb.tsfile.read.reader.IBatchReader;
-import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -60,20 +40,14 @@ import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.stream.Collectors;
/**
* This tool can be used to perform inner space or cross space compaction of aligned and non aligned
- * timeseries . Currently, we use {@link
- * org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils} to speed up if it is
- * an seq inner space compaction.
+ * timeseries.
*/
public class CompactionUtils {
private static final Logger logger =
@@ -81,249 +55,6 @@ public class CompactionUtils {
private static final int subTaskNum =
IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
- public static void compact(
- List<TsFileResource> seqFileResources,
- List<TsFileResource> unseqFileResources,
- List<TsFileResource> targetFileResources)
- throws IOException, MetadataException, StorageEngineException, InterruptedException {
- long queryId = QueryResourceManager.getInstance().assignCompactionQueryId();
- QueryContext queryContext = new QueryContext(queryId);
- QueryDataSource queryDataSource = new QueryDataSource(seqFileResources, unseqFileResources);
- QueryResourceManager.getInstance()
- .getQueryFileManager()
- .addUsedFilesForQuery(queryId, queryDataSource);
-
- try (AbstractCompactionWriter compactionWriter =
- getCompactionWriter(seqFileResources, unseqFileResources, targetFileResources)) {
- // Do not close device iterator, because tsfile reader is managed by FileReaderManager.
- MultiTsFileDeviceIterator deviceIterator =
- new MultiTsFileDeviceIterator(seqFileResources, unseqFileResources);
- while (deviceIterator.hasNextDevice()) {
- checkThreadInterrupted(targetFileResources);
- Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
- String device = deviceInfo.left;
- boolean isAligned = deviceInfo.right;
- QueryUtils.fillOrderIndexes(queryDataSource, device, true);
-
- if (isAligned) {
- compactAlignedSeries(
- device, deviceIterator, compactionWriter, queryContext, queryDataSource);
- } else {
- compactNonAlignedSeries(
- device, deviceIterator, compactionWriter, queryContext, queryDataSource);
- }
- }
-
- compactionWriter.endFile();
- updateDeviceStartTimeAndEndTime(targetFileResources, compactionWriter);
- updatePlanIndexes(targetFileResources, seqFileResources, unseqFileResources);
- } finally {
- QueryResourceManager.getInstance().endQuery(queryId);
- }
- }
-
- private static void compactAlignedSeries(
- String device,
- MultiTsFileDeviceIterator deviceIterator,
- AbstractCompactionWriter compactionWriter,
- QueryContext queryContext,
- QueryDataSource queryDataSource)
- throws IOException, MetadataException {
- MultiTsFileDeviceIterator.AlignedMeasurementIterator alignedMeasurementIterator =
- deviceIterator.iterateAlignedSeries(device);
- Set<String> allMeasurements = alignedMeasurementIterator.getAllMeasurements();
- List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
- for (String measurement : allMeasurements) {
- try {
- if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
- measurementSchemas.add(IDTableManager.getInstance().getSeriesSchema(device, measurement));
- } else {
- measurementSchemas.add(
- IoTDB.schemaProcessor.getSeriesSchema(new PartialPath(device, measurement)));
- }
- } catch (PathNotExistException e) {
- logger.info("A deleted path is skipped: {}", e.getMessage());
- }
- }
- if (measurementSchemas.isEmpty()) {
- return;
- }
- List<String> existedMeasurements =
- measurementSchemas.stream()
- .map(IMeasurementSchema::getMeasurementId)
- .collect(Collectors.toList());
- IBatchReader dataBatchReader =
- constructReader(
- device,
- existedMeasurements,
- measurementSchemas,
- allMeasurements,
- queryContext,
- queryDataSource,
- true);
-
- if (dataBatchReader.hasNextBatch()) {
- // chunkgroup is serialized only when at least one timeseries under this device has data
- compactionWriter.startChunkGroup(device, true);
- compactionWriter.startMeasurement(measurementSchemas, 0);
- writeWithReader(compactionWriter, dataBatchReader, 0);
- compactionWriter.endMeasurement(0);
- compactionWriter.endChunkGroup();
- }
- }
-
- private static void compactNonAlignedSeries(
- String device,
- MultiTsFileDeviceIterator deviceIterator,
- AbstractCompactionWriter compactionWriter,
- QueryContext queryContext,
- QueryDataSource queryDataSource)
- throws IOException, InterruptedException {
- MultiTsFileDeviceIterator.MeasurementIterator measurementIterator =
- deviceIterator.iterateNotAlignedSeries(device, false);
- Set<String> allMeasurements = measurementIterator.getAllMeasurements();
- int subTaskNums = Math.min(allMeasurements.size(), subTaskNum);
-
- // assign all measurements to different sub tasks
- Set<String>[] measurementsForEachSubTask = new HashSet[subTaskNums];
- int idx = 0;
- for (String measurement : allMeasurements) {
- if (measurementsForEachSubTask[idx % subTaskNums] == null) {
- measurementsForEachSubTask[idx % subTaskNums] = new HashSet<String>();
- }
- measurementsForEachSubTask[idx++ % subTaskNums].add(measurement);
- }
-
- // construct sub tasks and start compacting measurements in parallel
- List<Future<Void>> futures = new ArrayList<>();
- compactionWriter.startChunkGroup(device, false);
- for (int i = 0; i < subTaskNums; i++) {
- futures.add(
- CompactionTaskManager.getInstance()
- .submitSubTask(
- new SubCompactionTask(
- device,
- measurementsForEachSubTask[i],
- queryContext,
- queryDataSource,
- compactionWriter,
- i)));
- }
-
- // wait for all sub tasks finish
- for (int i = 0; i < subTaskNums; i++) {
- try {
- futures.get(i).get();
- } catch (InterruptedException | ExecutionException e) {
- logger.error("SubCompactionTask meet errors ", e);
- Thread.interrupted();
- throw new InterruptedException();
- }
- }
-
- compactionWriter.endChunkGroup();
- }
-
- public static void writeWithReader(
- AbstractCompactionWriter writer, IBatchReader reader, int subTaskId) throws IOException {
- while (reader.hasNextBatch()) {
- BatchData batchData = reader.nextBatch();
- while (batchData.hasCurrent()) {
- writer.write(batchData.currentTime(), batchData.currentValue(), subTaskId);
- batchData.next();
- }
- }
- }
-
- /**
- * @param measurementIds if device is aligned, then measurementIds contain all measurements. If
- * device is not aligned, then measurementIds only contain one measurement.
- */
- public static IBatchReader constructReader(
- String deviceId,
- List<String> measurementIds,
- List<IMeasurementSchema> measurementSchemas,
- Set<String> allSensors,
- QueryContext queryContext,
- QueryDataSource queryDataSource,
- boolean isAlign)
- throws IllegalPathException {
- PartialPath seriesPath;
- TSDataType tsDataType;
- if (isAlign) {
- seriesPath = new AlignedPath(deviceId, measurementIds, measurementSchemas);
- tsDataType = TSDataType.VECTOR;
- } else {
- seriesPath = new MeasurementPath(deviceId, measurementIds.get(0), measurementSchemas.get(0));
- tsDataType = measurementSchemas.get(0).getType();
- }
- return new SeriesRawDataBatchReader(
- seriesPath, allSensors, tsDataType, queryContext, queryDataSource, null, null, null, true);
- }
-
- private static AbstractCompactionWriter getCompactionWriter(
- List<TsFileResource> seqFileResources,
- List<TsFileResource> unseqFileResources,
- List<TsFileResource> targetFileResources)
- throws IOException {
- if (!seqFileResources.isEmpty() && !unseqFileResources.isEmpty()) {
- // cross space
- return new CrossSpaceCompactionWriter(targetFileResources, seqFileResources);
- } else {
- // inner space
- return new InnerSpaceCompactionWriter(targetFileResources.get(0));
- }
- }
-
- private static void updateDeviceStartTimeAndEndTime(
- List<TsFileResource> targetResources, AbstractCompactionWriter compactionWriter) {
- List<TsFileIOWriter> targetFileWriters = compactionWriter.getFileIOWriter();
- for (int i = 0; i < targetFileWriters.size(); i++) {
- TsFileIOWriter fileIOWriter = targetFileWriters.get(i);
- TsFileResource fileResource = targetResources.get(i);
- // The tmp target file may does not have any data points written due to the existence of the
- // mods file, and it will be deleted after compaction. So skip the target file that has been
- // deleted.
- if (!fileResource.getTsFile().exists()) {
- continue;
- }
- for (Map.Entry<String, List<TimeseriesMetadata>> entry :
- fileIOWriter.getDeviceTimeseriesMetadataMap().entrySet()) {
- String device = entry.getKey();
- for (TimeseriesMetadata timeseriesMetadata : entry.getValue()) {
- fileResource.updateStartTime(device, timeseriesMetadata.getStatistics().getStartTime());
- fileResource.updateEndTime(device, timeseriesMetadata.getStatistics().getEndTime());
- }
- }
- }
- }
-
- private static void updatePlanIndexes(
- List<TsFileResource> targetResources,
- List<TsFileResource> seqResources,
- List<TsFileResource> unseqResources) {
- // as the new file contains data of other files, track their plan indexes in the new file
- // so that we will be able to compare data across different IoTDBs that share the same index
- // generation policy
- // however, since the data of unseq files are mixed together, we won't be able to know
- // which files are exactly contained in the new file, so we have to record all unseq files
- // in the new file
- for (int i = 0; i < targetResources.size(); i++) {
- TsFileResource targetResource = targetResources.get(i);
- // remove the target file that has been deleted from list
- if (!targetResource.getTsFile().exists()) {
- targetResources.remove(i--);
- continue;
- }
- for (TsFileResource unseqResource : unseqResources) {
- targetResource.updatePlanIndexes(unseqResource);
- }
- for (TsFileResource seqResource : seqResources) {
- targetResource.updatePlanIndexes(seqResource);
- }
- }
- }
-
/**
* Update the targetResource. Move tmp target file to target file and serialize
* xxx.tsfile.resource.
@@ -370,7 +101,7 @@ public class CompactionUtils {
* Collect all the compaction modification files of source files, and combines them as the
* modification file of target file.
*/
- public static void combineModsInCompaction(
+ public static void combineModsInCrossCompaction(
List<TsFileResource> seqResources,
List<TsFileResource> unseqResources,
List<TsFileResource> targetResources)
@@ -390,6 +121,31 @@ public class CompactionUtils {
}
}
+ /**
+ * Collect all the compaction modification files of source files, and combines them as the
+ * modification file of target file.
+ */
+ public static void combineModsInInnerCompaction(
+ Collection<TsFileResource> sourceFiles, TsFileResource targetTsFile) throws IOException {
+ List<Modification> modifications = new ArrayList<>();
+ for (TsFileResource mergeTsFile : sourceFiles) {
+ try (ModificationFile sourceCompactionModificationFile =
+ ModificationFile.getCompactionMods(mergeTsFile)) {
+ modifications.addAll(sourceCompactionModificationFile.getModifications());
+ }
+ }
+ if (!modifications.isEmpty()) {
+ try (ModificationFile modificationFile = ModificationFile.getNormalMods(targetTsFile)) {
+ for (Modification modification : modifications) {
+ // we have to set modification offset to MAX_VALUE, as the offset of source chunk may
+ // change after compaction
+ modification.setFileOffset(Long.MAX_VALUE);
+ modificationFile.write(modification);
+ }
+ }
+ }
+ }
+
private static void updateOneTargetMods(
TsFileResource targetFile, TsFileResource seqFile, List<TsFileResource> unseqFiles)
throws IOException {
@@ -429,12 +185,59 @@ public class CompactionUtils {
}
}
- private static void checkThreadInterrupted(List<TsFileResource> tsFileResource)
- throws InterruptedException {
- if (Thread.currentThread().isInterrupted()) {
- throw new InterruptedException(
- String.format(
- "[Compaction] compaction for target file %s abort", tsFileResource.toString()));
+ public static boolean deleteTsFilesInDisk(
+ Collection<TsFileResource> mergeTsFiles, String storageGroupName) {
+ logger.info("{} [Compaction] Compaction starts to delete real file ", storageGroupName);
+ boolean result = true;
+ for (TsFileResource mergeTsFile : mergeTsFiles) {
+ if (!deleteTsFile(mergeTsFile)) {
+ result = false;
+ }
+ logger.info(
+ "{} [Compaction] delete TsFile {}", storageGroupName, mergeTsFile.getTsFilePath());
+ }
+ return result;
+ }
+
+ public static boolean deleteTsFile(TsFileResource seqFile) {
+ try {
+ FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile.getTsFilePath());
+ seqFile.setStatus(TsFileResourceStatus.DELETED);
+ seqFile.delete();
+ } catch (IOException e) {
+ logger.error(e.getMessage(), e);
+ return false;
+ }
+ return true;
+ }
+
+ /** Delete all modification files for source files */
+ public static void deleteModificationForSourceFile(
+ Collection<TsFileResource> sourceFiles, String storageGroupName) throws IOException {
+ logger.info("{} [Compaction] Start to delete modifications of source files", storageGroupName);
+ for (TsFileResource tsFileResource : sourceFiles) {
+ ModificationFile compactionModificationFile =
+ ModificationFile.getCompactionMods(tsFileResource);
+ if (compactionModificationFile.exists()) {
+ compactionModificationFile.remove();
+ }
+
+ ModificationFile normalModification = ModificationFile.getNormalMods(tsFileResource);
+ if (normalModification.exists()) {
+ normalModification.remove();
+ }
+ }
+ }
+
+ public static ICrossSpaceMergeFileSelector getCrossSpaceFileSelector(
+ long budget, RewriteCrossSpaceCompactionResource resource) {
+ CrossCompactionSelector strategy =
+ IoTDBDescriptor.getInstance().getConfig().getCrossCompactionSelector();
+ switch (strategy) {
+ case REWRITE:
+ return new RewriteCompactionFileSelector(resource, budget);
+ default:
+ throw new UnsupportedOperationException("Unknown CrossSpaceFileStrategy " + strategy);
}
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskComparator.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/comparator/DefaultCompactionTaskComparatorImpl.java
similarity index 75%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskComparator.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/comparator/DefaultCompactionTaskComparatorImpl.java
index f05f5c8105..04a213b7e6 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskComparator.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/comparator/DefaultCompactionTaskComparatorImpl.java
@@ -17,47 +17,45 @@
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction;
+package org.apache.iotdb.db.engine.compaction.comparator;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.constant.CompactionPriority;
-import org.apache.iotdb.db.engine.compaction.cross.AbstractCrossSpaceCompactionTask;
-import org.apache.iotdb.db.engine.compaction.inner.AbstractInnerSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import java.util.Comparator;
import java.util.List;
-public class CompactionTaskComparator implements Comparator<AbstractCompactionTask> {
+public class DefaultCompactionTaskComparatorImpl implements ICompactionTaskComparator {
private IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
@Override
public int compare(AbstractCompactionTask o1, AbstractCompactionTask o2) {
- if ((((o1 instanceof AbstractInnerSpaceCompactionTask)
- && (o2 instanceof AbstractCrossSpaceCompactionTask))
- || ((o2 instanceof AbstractInnerSpaceCompactionTask)
- && (o1 instanceof AbstractCrossSpaceCompactionTask)))) {
+ if ((((o1 instanceof InnerSpaceCompactionTask) && (o2 instanceof CrossSpaceCompactionTask))
+ || ((o2 instanceof InnerSpaceCompactionTask)
+ && (o1 instanceof CrossSpaceCompactionTask)))) {
if (config.getCompactionPriority() == CompactionPriority.BALANCE) {
return 0;
} else if (config.getCompactionPriority() == CompactionPriority.INNER_CROSS) {
- return o1 instanceof AbstractInnerSpaceCompactionTask ? -1 : 1;
+ return o1 instanceof InnerSpaceCompactionTask ? -1 : 1;
} else {
- return o1 instanceof AbstractCrossSpaceCompactionTask ? -1 : 1;
+ return o1 instanceof CrossSpaceCompactionTask ? -1 : 1;
}
}
- if (o1 instanceof AbstractInnerSpaceCompactionTask) {
+ if (o1 instanceof InnerSpaceCompactionTask) {
return compareInnerSpaceCompactionTask(
- (AbstractInnerSpaceCompactionTask) o1, (AbstractInnerSpaceCompactionTask) o2);
+ (InnerSpaceCompactionTask) o1, (InnerSpaceCompactionTask) o2);
} else {
return compareCrossSpaceCompactionTask(
- (AbstractCrossSpaceCompactionTask) o1, (AbstractCrossSpaceCompactionTask) o2);
+ (CrossSpaceCompactionTask) o1, (CrossSpaceCompactionTask) o2);
}
}
- private int compareInnerSpaceCompactionTask(
- AbstractInnerSpaceCompactionTask o1, AbstractInnerSpaceCompactionTask o2) {
+ public int compareInnerSpaceCompactionTask(
+ InnerSpaceCompactionTask o1, InnerSpaceCompactionTask o2) {
if (o1.isSequence() ^ o2.isSequence()) {
// prioritize sequence file compaction
return o1.isSequence() ? -1 : 1;
@@ -98,8 +96,8 @@ public class CompactionTaskComparator implements Comparator<AbstractCompactionTa
return 0;
}
- private int compareCrossSpaceCompactionTask(
- AbstractCrossSpaceCompactionTask o1, AbstractCrossSpaceCompactionTask o2) {
+ public int compareCrossSpaceCompactionTask(
+ CrossSpaceCompactionTask o1, CrossSpaceCompactionTask o2) {
if (o1.getSelectedSequenceFiles().size() != o2.getSelectedSequenceFiles().size()) {
// we prefer the task with fewer sequence files
// because this type of tasks consume fewer memory during execution
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTaskFactory.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/comparator/ICompactionTaskComparator.java
similarity index 50%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTaskFactory.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/comparator/ICompactionTaskComparator.java
index 49ba832071..e8a2dae539 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTaskFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/comparator/ICompactionTaskComparator.java
@@ -16,33 +16,20 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.comparator;
-package org.apache.iotdb.db.engine.compaction.inner;
-
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import java.util.List;
+import java.util.Comparator;
+
+public interface ICompactionTaskComparator extends Comparator<AbstractCompactionTask> {
+ int compare(AbstractCompactionTask task1, AbstractCompactionTask task2);
+
+ int compareInnerSpaceCompactionTask(
+ InnerSpaceCompactionTask task1, InnerSpaceCompactionTask task2);
-public class InnerSpaceCompactionTaskFactory {
- public AbstractCompactionTask createTask(
- String logicalStorageGroupName,
- String virtualStorageGroup,
- long timePartition,
- TsFileManager tsFileManager,
- List<TsFileResource> selectedTsFileResourceList,
- boolean sequence) {
- return IoTDBDescriptor.getInstance()
- .getConfig()
- .getInnerCompactionStrategy()
- .getCompactionTask(
- logicalStorageGroupName,
- virtualStorageGroup,
- timePartition,
- tsFileManager,
- selectedTsFileResourceList,
- sequence);
- }
+ int compareCrossSpaceCompactionTask(
+ CrossSpaceCompactionTask task1, CrossSpaceCompactionTask task2);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/CrossCompactionPerformer.java
similarity index 52%
copy from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
copy to server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/CrossCompactionPerformer.java
index b0d36564ad..9dd13cc995 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/CrossCompactionPerformer.java
@@ -16,20 +16,26 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.constant;
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
-import org.apache.iotdb.db.exception.MergeException;
+public enum CrossCompactionPerformer {
+ READ_POINT;
-import java.util.List;
+ public static CrossCompactionPerformer getCrossCompactionPerformer(String name) {
+ if (READ_POINT.toString().equalsIgnoreCase(name)) {
+ return READ_POINT;
+ }
+ throw new RuntimeException("Illegal compaction performer for cross compaction " + name);
+ }
-/**
- * IMergeFileSelector selects a set of files from given seqFiles and unseqFiles which can be merged
- * without exceeding given memory budget.
- */
-public interface ICrossSpaceMergeFileSelector {
-
- List[] select() throws MergeException;
-
- int getConcurrentMergeNum();
+ public ICrossCompactionPerformer getCompactionPerformer() {
+ switch (this) {
+ case READ_POINT:
+ default:
+ return new ReadPointCompactionPerformer();
+ }
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossCompactionStrategy.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/CrossCompactionSelector.java
similarity index 65%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossCompactionStrategy.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/CrossCompactionSelector.java
index 05ae8acf5a..caf0810530 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossCompactionStrategy.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/CrossCompactionSelector.java
@@ -16,64 +16,56 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.cross;
+package org.apache.iotdb.db.engine.compaction.constant;
import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.cross.ICrossSpaceSelector;
import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionSelector;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.RewriteCrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import java.util.List;
-public enum CrossCompactionStrategy {
- REWRITE_COMPACTION;
+public enum CrossCompactionSelector {
+ REWRITE;
- public static CrossCompactionStrategy getCrossCompactionStrategy(String name) {
+ public static CrossCompactionSelector getCrossCompactionStrategy(String name) {
if ("REWRITE_COMPACTION".equalsIgnoreCase(name)) {
- return REWRITE_COMPACTION;
+ return REWRITE;
}
throw new RuntimeException("Illegal Cross Compaction Strategy " + name);
}
- public AbstractCrossSpaceCompactionTask getCompactionTask(
- String logicalStorageGroupName,
- String virtualStorageGroupName,
+ public CrossSpaceCompactionTask getCompactionTask(
long timePartitionId,
TsFileManager tsFileManager,
List<TsFileResource> selectedSeqTsFileResourceList,
List<TsFileResource> selectedUnSeqTsFileResourceList) {
switch (this) {
- case REWRITE_COMPACTION:
+ case REWRITE:
default:
- return new RewriteCrossSpaceCompactionTask(
- logicalStorageGroupName,
- virtualStorageGroupName,
+ return new CrossSpaceCompactionTask(
timePartitionId,
tsFileManager,
selectedSeqTsFileResourceList,
selectedUnSeqTsFileResourceList,
+ new ReadPointCompactionPerformer(),
CompactionTaskManager.currentTaskNum);
}
}
- public AbstractCrossSpaceCompactionSelector getCompactionSelector(
+ public ICrossSpaceSelector getCompactionSelector(
String logicalStorageGroupName,
String virtualGroupId,
- String storageGroupDir,
long timePartition,
- TsFileManager tsFileManager,
- CrossSpaceCompactionTaskFactory taskFactory) {
+ TsFileManager tsFileManager) {
switch (this) {
- case REWRITE_COMPACTION:
+ case REWRITE:
default:
return new RewriteCrossSpaceCompactionSelector(
- logicalStorageGroupName,
- virtualGroupId,
- storageGroupDir,
- timePartition,
- tsFileManager,
- taskFactory);
+ logicalStorageGroupName, virtualGroupId, timePartition, tsFileManager);
}
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerSeqCompactionPerformer.java
similarity index 52%
copy from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
copy to server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerSeqCompactionPerformer.java
index b0d36564ad..57a7319782 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerSeqCompactionPerformer.java
@@ -16,20 +16,26 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.constant;
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
+import org.apache.iotdb.db.engine.compaction.performer.ISeqCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
-import org.apache.iotdb.db.exception.MergeException;
+public enum InnerSeqCompactionPerformer {
+ READ_CHUNK;
-import java.util.List;
+ public static InnerSeqCompactionPerformer getInnerSeqCompactionPerformer(String name) {
+ if (READ_CHUNK.toString().equalsIgnoreCase(name)) {
+ return READ_CHUNK;
+ }
+ throw new RuntimeException("Illegal compaction performer for seq inner compaction " + name);
+ }
-/**
- * IMergeFileSelector selects a set of files from given seqFiles and unseqFiles which can be merged
- * without exceeding given memory budget.
- */
-public interface ICrossSpaceMergeFileSelector {
-
- List[] select() throws MergeException;
-
- int getConcurrentMergeNum();
+ public ISeqCompactionPerformer getCompactionPerformer() {
+ switch (this) {
+ case READ_CHUNK:
+ default:
+ return new ReadChunkCompactionPerformer();
+ }
+ }
}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedCrossSpaceCompactionTaskFactory.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerSequenceCompactionSelector.java
similarity index 52%
rename from server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedCrossSpaceCompactionTaskFactory.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerSequenceCompactionSelector.java
index 1fb4b2e3fc..f9fe30aaf6 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedCrossSpaceCompactionTaskFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerSequenceCompactionSelector.java
@@ -17,31 +17,32 @@
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.task;
+package org.apache.iotdb.db.engine.compaction.constant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.inner.IInnerSeqSpaceSelector;
+import org.apache.iotdb.db.engine.compaction.inner.sizetiered.SizeTieredCompactionSelector;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import java.util.List;
+public enum InnerSequenceCompactionSelector {
+ SIZE_TIERED;
-public class FakedCrossSpaceCompactionTaskFactory {
- public AbstractCompactionTask createTask(
+ public static InnerSequenceCompactionSelector getInnerSequenceCompactionStrategy(String name) {
+ if ("SIZE_TIERED_COMPACTION".equalsIgnoreCase(name)) {
+ return SIZE_TIERED;
+ }
+ throw new RuntimeException("Illegal Compaction Strategy " + name);
+ }
+
+ public IInnerSeqSpaceSelector getCompactionSelector(
String logicalStorageGroupName,
String virtualStorageGroupName,
- long timePartitionId,
- TsFileManager tsFileManager,
- List<TsFileResource> selectedSeqTsFileResourceList,
- List<TsFileResource> selectedUnSeqTsFileResourceList) {
- return IoTDBDescriptor.getInstance()
- .getConfig()
- .getCrossCompactionStrategy()
- .getCompactionTask(
- logicalStorageGroupName,
- virtualStorageGroupName,
- timePartitionId,
- tsFileManager,
- selectedSeqTsFileResourceList,
- selectedUnSeqTsFileResourceList);
+ long timePartition,
+ TsFileManager tsFileManager) {
+ switch (this) {
+ case SIZE_TIERED:
+ default:
+ return new SizeTieredCompactionSelector(
+ logicalStorageGroupName, virtualStorageGroupName, timePartition, tsFileManager, true);
+ }
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerUnseqCompactionPerformer.java
similarity index 52%
copy from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
copy to server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerUnseqCompactionPerformer.java
index b0d36564ad..75ef19ff9f 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerUnseqCompactionPerformer.java
@@ -16,20 +16,26 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.constant;
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
+import org.apache.iotdb.db.engine.compaction.performer.IUnseqCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
-import org.apache.iotdb.db.exception.MergeException;
+public enum InnerUnseqCompactionPerformer {
+ READ_POINT;
-import java.util.List;
+ public static InnerUnseqCompactionPerformer getInnerUnseqCompactionPerformer(String name) {
+ if (READ_POINT.toString().equalsIgnoreCase(name)) {
+ return READ_POINT;
+ }
+ throw new RuntimeException("Illegal compaction performer for unseq inner compaction " + name);
+ }
-/**
- * IMergeFileSelector selects a set of files from given seqFiles and unseqFiles which can be merged
- * without exceeding given memory budget.
- */
-public interface ICrossSpaceMergeFileSelector {
-
- List[] select() throws MergeException;
-
- int getConcurrentMergeNum();
+ public IUnseqCompactionPerformer getCompactionPerformer() {
+ switch (this) {
+ case READ_POINT:
+ default:
+ return new ReadPointCompactionPerformer();
+ }
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTaskFactory.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerUnsequenceCompactionSelector.java
similarity index 51%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTaskFactory.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerUnsequenceCompactionSelector.java
index 3c98410ef3..f325a9b19a 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTaskFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/constant/InnerUnsequenceCompactionSelector.java
@@ -16,34 +16,33 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.constant;
-package org.apache.iotdb.db.engine.compaction.cross;
-
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
+import org.apache.iotdb.db.engine.compaction.inner.IInnerUnseqSpaceSelector;
+import org.apache.iotdb.db.engine.compaction.inner.sizetiered.SizeTieredCompactionSelector;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import java.util.List;
+public enum InnerUnsequenceCompactionSelector {
+ SIZE_TIERED;
-public class CrossSpaceCompactionTaskFactory {
+ public static InnerUnsequenceCompactionSelector getInnerUnsequenceCompactionStrategy(
+ String name) {
+ if (SIZE_TIERED.toString().equalsIgnoreCase(name)) {
+ return SIZE_TIERED;
+ }
+ throw new RuntimeException("Illegal Compaction Strategy " + name);
+ }
- public AbstractCompactionTask createTask(
+ public IInnerUnseqSpaceSelector getCompactionSelector(
String logicalStorageGroupName,
String virtualStorageGroupName,
- long timePartitionId,
- TsFileManager tsFileManager,
- List<TsFileResource> selectedSeqTsFileResourceList,
- List<TsFileResource> selectedUnSeqTsFileResourceList) {
- return IoTDBDescriptor.getInstance()
- .getConfig()
- .getCrossCompactionStrategy()
- .getCompactionTask(
- logicalStorageGroupName,
- virtualStorageGroupName,
- timePartitionId,
- tsFileManager,
- selectedSeqTsFileResourceList,
- selectedUnSeqTsFileResourceList);
+ long timePartition,
+ TsFileManager tsFileManager) {
+ switch (this) {
+ case SIZE_TIERED:
+ default:
+ return new SizeTieredCompactionSelector(
+ logicalStorageGroupName, virtualStorageGroupName, timePartition, tsFileManager, false);
+ }
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/AbstractCrossSpaceCompactionSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/AbstractCrossSpaceCompactionSelector.java
deleted file mode 100644
index 26c557af5b..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/AbstractCrossSpaceCompactionSelector.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.compaction.cross;
-
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionSelector;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-
-import java.util.List;
-
-public abstract class AbstractCrossSpaceCompactionSelector extends AbstractCompactionSelector {
- protected String logicalStorageGroupName;
- protected String dataRegionId;
- protected String storageGroupDir;
- protected long timePartition;
- protected TsFileManager tsFileManager;
- protected List<TsFileResource> sequenceFileList;
- protected List<TsFileResource> unsequenceFileList;
- protected CrossSpaceCompactionTaskFactory taskFactory;
-
- public AbstractCrossSpaceCompactionSelector(
- String logicalStorageGroupName,
- String dataRegionId,
- String storageGroupDir,
- long timePartition,
- TsFileManager tsFileManager,
- CrossSpaceCompactionTaskFactory taskFactory) {
- this.logicalStorageGroupName = logicalStorageGroupName;
- this.dataRegionId = dataRegionId;
- this.storageGroupDir = storageGroupDir;
- this.timePartition = timePartition;
- this.sequenceFileList = tsFileManager.getSequenceListByTimePartition(timePartition);
- this.unsequenceFileList = tsFileManager.getUnsequenceListByTimePartition(timePartition);
- this.taskFactory = taskFactory;
- this.tsFileManager = tsFileManager;
- }
-
- @Override
- public abstract void selectAndSubmit();
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/AbstractCrossSpaceCompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/AbstractCrossSpaceCompactionTask.java
deleted file mode 100644
index 633fb8350e..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/AbstractCrossSpaceCompactionTask.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.cross;
-
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
-
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public abstract class AbstractCrossSpaceCompactionTask extends AbstractCompactionTask {
- List<TsFileResource> selectedSequenceFiles;
- List<TsFileResource> selectedUnsequenceFiles;
-
- public AbstractCrossSpaceCompactionTask(
- String fullStorageGroupName,
- long timePartition,
- AtomicInteger currentTaskNum,
- List<TsFileResource> selectedSequenceFiles,
- List<TsFileResource> selectedUnsequenceFiles,
- TsFileManager tsFileManager) {
- super(fullStorageGroupName, timePartition, tsFileManager, currentTaskNum);
- this.selectedSequenceFiles = selectedSequenceFiles;
- this.selectedUnsequenceFiles = selectedUnsequenceFiles;
- }
-
- @Override
- public void setSourceFilesToCompactionCandidate() {
- this.selectedSequenceFiles.forEach(x -> x.setStatus(TsFileResourceStatus.COMPACTION_CANDIDATE));
- this.selectedUnsequenceFiles.forEach(
- x -> x.setStatus(TsFileResourceStatus.COMPACTION_CANDIDATE));
- }
-
- public List<TsFileResource> getSelectedSequenceFiles() {
- return selectedSequenceFiles;
- }
-
- public List<TsFileResource> getSelectedUnsequenceFiles() {
- return selectedUnsequenceFiles;
- }
-
- @Override
- public boolean checkValidAndSetMerging() {
- if (!tsFileManager.isAllowCompaction()) {
- return false;
- }
-
- for (TsFileResource resource : selectedSequenceFiles) {
- if (resource.isCompacting() || !resource.isClosed() || !resource.getTsFile().exists()) {
- return false;
- }
- }
-
- for (TsFileResource resource : selectedUnsequenceFiles) {
- if (resource.isCompacting() || !resource.isClosed() || !resource.getTsFile().exists()) {
- return false;
- }
- }
-
- for (TsFileResource resource : selectedSequenceFiles) {
- resource.setStatus(TsFileResourceStatus.COMPACTING);
- }
-
- for (TsFileResource resource : selectedUnsequenceFiles) {
- resource.setStatus(TsFileResourceStatus.COMPACTING);
- }
-
- return true;
- }
-
- @Override
- public String toString() {
- return new StringBuilder()
- .append(fullStorageGroupName)
- .append("-")
- .append(timePartition)
- .append(" task seq files are ")
- .append(selectedSequenceFiles.toString())
- .append(" , unseq files are ")
- .append(selectedUnsequenceFiles.toString())
- .toString();
- }
-
- @Override
- public int hashCode() {
- return toString().hashCode();
- }
-
- @Override
- public void resetCompactionCandidateStatusForAllSourceFiles() {
- selectedSequenceFiles.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
- selectedUnsequenceFiles.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTask.java
new file mode 100644
index 0000000000..c4e1866f3e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTask.java
@@ -0,0 +1,281 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.engine.compaction.cross;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.db.engine.compaction.CompactionExceptionHandler;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
+import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
+import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_SOURCE_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_TARGET_FILES;
+
+public class CrossSpaceCompactionTask extends AbstractCompactionTask {
+ private static final Logger LOGGER =
+ LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+ protected List<TsFileResource> selectedSequenceFiles;
+ protected List<TsFileResource> selectedUnsequenceFiles;
+ protected TsFileResourceList seqTsFileResourceList;
+ protected TsFileResourceList unseqTsFileResourceList;
+ private File logFile;
+ protected List<TsFileResource> targetTsfileResourceList;
+ protected List<TsFileResource> holdReadLockList = new ArrayList<>();
+ protected List<TsFileResource> holdWriteLockList = new ArrayList<>();
+
+ public CrossSpaceCompactionTask(
+ long timePartition,
+ TsFileManager tsFileManager,
+ List<TsFileResource> selectedSequenceFiles,
+ List<TsFileResource> selectedUnsequenceFiles,
+ ICrossCompactionPerformer performer,
+ AtomicInteger currentTaskNum) {
+ super(
+ tsFileManager.getStorageGroupName() + "-" + tsFileManager.getDataRegion(),
+ timePartition,
+ tsFileManager,
+ currentTaskNum);
+ this.selectedSequenceFiles = selectedSequenceFiles;
+ this.selectedUnsequenceFiles = selectedUnsequenceFiles;
+ this.seqTsFileResourceList = tsFileManager.getSequenceListByTimePartition(timePartition);
+ this.unseqTsFileResourceList = tsFileManager.getUnsequenceListByTimePartition(timePartition);
+ this.performer = performer;
+ }
+
+ @Override
+ protected void doCompaction() throws Exception {
+ try {
+ if (!tsFileManager.isAllowCompaction()) {
+ return;
+ }
+ long startTime = System.currentTimeMillis();
+ targetTsfileResourceList =
+ TsFileNameGenerator.getCrossCompactionTargetFileResources(selectedSequenceFiles);
+
+ if (targetTsfileResourceList.isEmpty()
+ || selectedSequenceFiles.isEmpty()
+ || selectedUnsequenceFiles.isEmpty()) {
+ LOGGER.info(
+ "{} [Compaction] Cross space compaction file list is empty, end it",
+ fullStorageGroupName);
+ return;
+ }
+
+ LOGGER.info(
+ "{} [Compaction] CrossSpaceCompactionTask start. Sequence files : {}, unsequence files : {}",
+ fullStorageGroupName,
+ selectedSequenceFiles,
+ selectedUnsequenceFiles);
+ logFile =
+ new File(
+ selectedSequenceFiles.get(0).getTsFile().getParent()
+ + File.separator
+ + targetTsfileResourceList.get(0).getTsFile().getName()
+ + CompactionLogger.CROSS_COMPACTION_LOG_NAME_SUFFIX);
+
+ try (CompactionLogger compactionLogger = new CompactionLogger(logFile)) {
+ // print the path of the temporary file first for priority check during recovery
+ compactionLogger.logFiles(selectedSequenceFiles, STR_SOURCE_FILES);
+ compactionLogger.logFiles(selectedUnsequenceFiles, STR_SOURCE_FILES);
+ compactionLogger.logFiles(targetTsfileResourceList, STR_TARGET_FILES);
+ // indicates that the cross compaction is complete and the result can be reused during a
+ // restart recovery
+ compactionLogger.close();
+
+ performer.setSourceFiles(selectedSequenceFiles, selectedUnsequenceFiles);
+ performer.setTargetFiles(targetTsfileResourceList);
+ performer.perform();
+
+ CompactionUtils.moveTargetFile(targetTsfileResourceList, false, fullStorageGroupName);
+ CompactionUtils.combineModsInCrossCompaction(
+ selectedSequenceFiles, selectedUnsequenceFiles, targetTsfileResourceList);
+
+ // update tsfile resource in memory
+ tsFileManager.replace(
+ selectedSequenceFiles,
+ selectedUnsequenceFiles,
+ targetTsfileResourceList,
+ timePartition,
+ true);
+
+ releaseReadAndLockWrite(selectedSequenceFiles);
+ releaseReadAndLockWrite(selectedUnsequenceFiles);
+
+ deleteOldFiles(selectedSequenceFiles);
+ deleteOldFiles(selectedUnsequenceFiles);
+ CompactionUtils.deleteCompactionModsFile(selectedSequenceFiles, selectedUnsequenceFiles);
+
+ if (logFile.exists()) {
+ FileUtils.delete(logFile);
+ }
+ LOGGER.info(
+ "{} [Compaction] CrossSpaceCompactionTask Costs {} s",
+ fullStorageGroupName,
+ (System.currentTimeMillis() - startTime) / 1000);
+ }
+ } catch (Throwable throwable) {
+ // catch throwable instead of exception to handle OOM errors
+ LOGGER.error("Meet errors in cross space compaction, {}", throwable.getMessage());
+ CompactionExceptionHandler.handleException(
+ fullStorageGroupName,
+ logFile,
+ targetTsfileResourceList,
+ selectedSequenceFiles,
+ selectedUnsequenceFiles,
+ tsFileManager,
+ timePartition,
+ false,
+ true);
+ throw throwable;
+ } finally {
+ releaseAllLock();
+ }
+ }
+
+ @Override
+ public boolean equalsOtherTask(AbstractCompactionTask otherTask) {
+ if (!(otherTask instanceof CrossSpaceCompactionTask)) {
+ return false;
+ }
+ CrossSpaceCompactionTask otherCrossCompactionTask = (CrossSpaceCompactionTask) otherTask;
+ return this.selectedSequenceFiles.equals(otherCrossCompactionTask.selectedSequenceFiles)
+ && this.selectedUnsequenceFiles.equals(otherCrossCompactionTask.selectedUnsequenceFiles)
+ && this.performer.getClass().isInstance(otherCrossCompactionTask.performer);
+ }
+
+ private void releaseAllLock() {
+ selectedSequenceFiles.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
+ selectedUnsequenceFiles.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
+ for (TsFileResource tsFileResource : holdReadLockList) {
+ tsFileResource.readUnlock();
+ tsFileResource.setStatus(TsFileResourceStatus.CLOSED);
+ }
+ for (TsFileResource tsFileResource : holdWriteLockList) {
+ tsFileResource.writeUnlock();
+ tsFileResource.setStatus(TsFileResourceStatus.CLOSED);
+ }
+ holdReadLockList.clear();
+ holdWriteLockList.clear();
+ }
+
+ public List<TsFileResource> getSelectedSequenceFiles() {
+ return selectedSequenceFiles;
+ }
+
+ @Override
+ public void setSourceFilesToCompactionCandidate() {
+ this.selectedSequenceFiles.forEach(x -> x.setStatus(TsFileResourceStatus.COMPACTION_CANDIDATE));
+ this.selectedUnsequenceFiles.forEach(
+ x -> x.setStatus(TsFileResourceStatus.COMPACTION_CANDIDATE));
+ }
+
+ public List<TsFileResource> getSelectedUnsequenceFiles() {
+ return selectedUnsequenceFiles;
+ }
+
+ @Override
+ public String toString() {
+ return new StringBuilder()
+ .append(fullStorageGroupName)
+ .append("-")
+ .append(timePartition)
+ .append(" task seq files are ")
+ .append(selectedSequenceFiles.toString())
+ .append(" , unseq files are ")
+ .append(selectedUnsequenceFiles.toString())
+ .toString();
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public void resetCompactionCandidateStatusForAllSourceFiles() {
+ selectedSequenceFiles.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
+ selectedUnsequenceFiles.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
+ }
+
+ private void deleteOldFiles(List<TsFileResource> tsFileResourceList) throws IOException {
+ for (TsFileResource tsFileResource : tsFileResourceList) {
+ FileReaderManager.getInstance().closeFileAndRemoveReader(tsFileResource.getTsFilePath());
+ tsFileResource.setStatus(TsFileResourceStatus.DELETED);
+ tsFileResource.remove();
+ LOGGER.info(
+ "[CrossSpaceCompaction] Delete TsFile :{}.",
+ tsFileResource.getTsFile().getAbsolutePath());
+ }
+ }
+
+ private void releaseReadAndLockWrite(List<TsFileResource> tsFileResourceList) {
+ for (TsFileResource tsFileResource : tsFileResourceList) {
+ tsFileResource.readUnlock();
+ holdReadLockList.remove(tsFileResource);
+ tsFileResource.writeLock();
+ holdWriteLockList.add(tsFileResource);
+ }
+ }
+
+ @Override
+ public boolean checkValidAndSetMerging() {
+ return addReadLock(selectedSequenceFiles) && addReadLock(selectedUnsequenceFiles);
+ }
+
+ private boolean addReadLock(List<TsFileResource> tsFileResourceList) {
+ if (!tsFileManager.isAllowCompaction()) {
+ return false;
+ }
+ for (TsFileResource tsFileResource : tsFileResourceList) {
+ tsFileResource.readLock();
+ holdReadLockList.add(tsFileResource);
+ if (tsFileResource.isCompacting()
+ || !tsFileResource.isClosed()
+ || !tsFileResource.getTsFile().exists()
+ || tsFileResource.isDeleted()) {
+ releaseAllLock();
+ return false;
+ }
+ tsFileResource.setStatus(TsFileResourceStatus.COMPACTING);
+ }
+ return true;
+ }
+
+ public String getStorageGroupName() {
+ return fullStorageGroupName;
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/AbstractCompactionSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/ICrossSpaceSelector.java
similarity index 63%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/task/AbstractCompactionSelector.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/ICrossSpaceSelector.java
index 96cd85a5bc..061e8d2f26 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/AbstractCompactionSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/ICrossSpaceSelector.java
@@ -16,14 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.task;
+package org.apache.iotdb.db.engine.compaction.cross;
-/**
- * AbstractCompactionSelector is the base class of all CompactionSelector. It runs the file
- * selection process, if there still threads availabe for compaction task, it will submit a
- * compaction task to {@link org.apache.iotdb.db.engine.compaction.CompactionTaskManager} and
- * increase the global compaction task count.
- */
-public abstract class AbstractCompactionSelector {
- public abstract void selectAndSubmit();
+import org.apache.iotdb.db.engine.compaction.task.ICompactionSelector;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.List;
+
+public interface ICrossSpaceSelector extends ICompactionSelector {
+ List<Pair<List<TsFileResource>, List<TsFileResource>>> selectCrossSpaceTask(
+ List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/manage/CrossSpaceCompactionResource.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionResource.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/manage/CrossSpaceCompactionResource.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionResource.java
index fecdd78caf..8af36afe6b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/manage/CrossSpaceCompactionResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionResource.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.manage;
+package org.apache.iotdb.db.engine.compaction.cross.rewrite;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -46,7 +46,7 @@ import java.util.stream.Collectors;
* CrossSpaceMergeResource manages files and caches of readers, writers, MeasurementSchemas and
* modifications to avoid unnecessary object creations and file openings.
*/
-public class CrossSpaceCompactionResource {
+public class RewriteCrossSpaceCompactionResource {
private List<TsFileResource> seqFiles;
private List<TsFileResource> unseqFiles = new ArrayList<>();
@@ -62,7 +62,7 @@ public class CrossSpaceCompactionResource {
private boolean cacheDeviceMeta = false;
- public CrossSpaceCompactionResource(
+ public RewriteCrossSpaceCompactionResource(
List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
this.seqFiles = seqFiles.stream().filter(this::filterSeqResource).collect(Collectors.toList());
filterUnseqResource(unseqFiles);
@@ -89,7 +89,7 @@ public class CrossSpaceCompactionResource {
}
}
- public CrossSpaceCompactionResource(
+ public RewriteCrossSpaceCompactionResource(
Collection<TsFileResource> seqFiles, List<TsFileResource> unseqFiles, long ttlLowerBound) {
this.ttlLowerBound = ttlLowerBound;
this.seqFiles = seqFiles.stream().filter(this::filterSeqResource).collect(Collectors.toList());
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionSelector.java
index 86bc05f738..3950afc75c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/RewriteCrossSpaceCompactionSelector.java
@@ -22,43 +22,41 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.cross.AbstractCrossSpaceCompactionSelector;
-import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTaskFactory;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.manage.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.cross.ICrossSpaceSelector;
import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.ICrossSpaceMergeFileSelector;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.exception.MergeException;
+import org.apache.iotdb.tsfile.utils.Pair;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.Iterator;
import java.util.List;
-public class RewriteCrossSpaceCompactionSelector extends AbstractCrossSpaceCompactionSelector {
+public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector {
private static final Logger LOGGER =
LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+ protected String logicalStorageGroupName;
+ protected String dataRegionId;
+ protected long timePartition;
+ protected TsFileManager tsFileManager;
public RewriteCrossSpaceCompactionSelector(
String logicalStorageGroupName,
- String virtualStorageGroupId,
- String storageGroupDir,
+ String dataRegionId,
long timePartition,
- TsFileManager tsFileManager,
- CrossSpaceCompactionTaskFactory taskFactory) {
- super(
- logicalStorageGroupName,
- virtualStorageGroupId,
- storageGroupDir,
- timePartition,
- tsFileManager,
- taskFactory);
+ TsFileManager tsFileManager) {
+ this.logicalStorageGroupName = logicalStorageGroupName;
+ this.dataRegionId = dataRegionId;
+ this.timePartition = timePartition;
+ this.tsFileManager = tsFileManager;
}
/**
@@ -70,10 +68,11 @@ public class RewriteCrossSpaceCompactionSelector extends AbstractCrossSpaceCompa
* @return Returns whether the file was found and submits the merge task
*/
@Override
- public void selectAndSubmit() {
+ public List selectCrossSpaceTask(
+ List<TsFileResource> sequenceFileList, List<TsFileResource> unsequenceFileList) {
if ((CompactionTaskManager.currentTaskNum.get() >= config.getConcurrentCompactionThread())
|| (!config.isEnableCrossSpaceCompaction())) {
- return;
+ return Collections.emptyList();
}
Iterator<TsFileResource> seqIterator = sequenceFileList.iterator();
Iterator<TsFileResource> unSeqIterator = unsequenceFileList.iterator();
@@ -86,15 +85,15 @@ public class RewriteCrossSpaceCompactionSelector extends AbstractCrossSpaceCompa
unSeqFileList.add(unSeqIterator.next());
}
if (seqFileList.isEmpty() || unSeqFileList.isEmpty()) {
- return;
+ return Collections.emptyList();
}
long budget = config.getCrossCompactionMemoryBudget();
long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
- CrossSpaceCompactionResource mergeResource =
- new CrossSpaceCompactionResource(seqFileList, unSeqFileList, timeLowerBound);
+ RewriteCrossSpaceCompactionResource mergeResource =
+ new RewriteCrossSpaceCompactionResource(seqFileList, unSeqFileList, timeLowerBound);
ICrossSpaceMergeFileSelector fileSelector =
- InnerSpaceCompactionUtils.getCrossSpaceFileSelector(budget, mergeResource);
+ CompactionUtils.getCrossSpaceFileSelector(budget, mergeResource);
try {
List[] mergeFiles = fileSelector.select();
// avoid pending tasks holds the metadata and streams
@@ -107,7 +106,7 @@ public class RewriteCrossSpaceCompactionSelector extends AbstractCrossSpaceCompa
logicalStorageGroupName,
budget);
}
- return;
+ return Collections.emptyList();
}
LOGGER.info(
"select files for cross compaction, sequence files: {}, unsequence files {}",
@@ -115,24 +114,17 @@ public class RewriteCrossSpaceCompactionSelector extends AbstractCrossSpaceCompa
mergeFiles[1]);
if (mergeFiles[0].size() > 0 && mergeFiles[1].size() > 0) {
- AbstractCompactionTask compactionTask =
- taskFactory.createTask(
- logicalStorageGroupName,
- dataRegionId,
- timePartition,
- tsFileManager,
- mergeFiles[0],
- mergeFiles[1]);
- CompactionTaskManager.getInstance().addTaskToWaitingQueue(compactionTask);
LOGGER.info(
"{} [Compaction] submit a task with {} sequence file and {} unseq files",
logicalStorageGroupName + "-" + dataRegionId,
mergeResource.getSeqFiles().size(),
mergeResource.getUnseqFiles().size());
+ return Collections.singletonList(new Pair<>(mergeFiles[0], mergeFiles[1]));
}
- } catch (MergeException | IOException | InterruptedException e) {
+ } catch (MergeException | IOException e) {
LOGGER.error("{} cannot select file for cross space compaction", logicalStorageGroupName, e);
}
+ return Collections.emptyList();
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
index b0d36564ad..70243dc5c8 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/ICrossSpaceMergeFileSelector.java
@@ -30,6 +30,4 @@ import java.util.List;
public interface ICrossSpaceMergeFileSelector {
List[] select() throws MergeException;
-
- int getConcurrentMergeNum();
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/NaivePathSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/NaivePathSelector.java
deleted file mode 100644
index d3e73c3b72..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/NaivePathSelector.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
-
-import org.apache.iotdb.db.metadata.path.PartialPath;
-
-import java.util.List;
-import java.util.NoSuchElementException;
-
-public class NaivePathSelector implements IMergePathSelector {
-
- private List<PartialPath> paths;
- private int idx;
- private int maxSeriesNum;
-
- public NaivePathSelector(List<PartialPath> paths, int maxSeriesNum) {
- this.paths = paths;
- this.maxSeriesNum = maxSeriesNum;
- }
-
- @Override
- public boolean hasNext() {
- return idx < paths.size();
- }
-
- @Override
- public List<PartialPath> next() {
- if (!hasNext()) {
- throw new NoSuchElementException();
- }
- List<PartialPath> ret =
- idx + maxSeriesNum <= paths.size()
- ? paths.subList(idx, idx + maxSeriesNum)
- : paths.subList(idx, paths.size());
- idx += maxSeriesNum;
- return ret;
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/RewriteCompactionFileSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/RewriteCompactionFileSelector.java
index bdf8a683f3..f7d1ffbf1d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/RewriteCompactionFileSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/RewriteCompactionFileSelector.java
@@ -20,7 +20,7 @@
package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.manage.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionResource;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.exception.MergeException;
import org.apache.iotdb.db.utils.MergeUtils;
@@ -46,7 +46,7 @@ public class RewriteCompactionFileSelector implements ICrossSpaceMergeFileSelect
private static final Logger logger = LoggerFactory.getLogger(RewriteCompactionFileSelector.class);
private static final String LOG_FILE_COST = "Memory cost of file {} is {}";
- CrossSpaceCompactionResource resource;
+ RewriteCrossSpaceCompactionResource resource;
long totalCost;
private long memoryBudget;
@@ -70,7 +70,8 @@ public class RewriteCompactionFileSelector implements ICrossSpaceMergeFileSelect
private boolean[] seqSelected;
private int seqSelectedNum;
- public RewriteCompactionFileSelector(CrossSpaceCompactionResource resource, long memoryBudget) {
+ public RewriteCompactionFileSelector(
+ RewriteCrossSpaceCompactionResource resource, long memoryBudget) {
this.resource = resource;
this.memoryBudget = memoryBudget;
this.maxCrossCompactionFileNum =
@@ -391,9 +392,4 @@ public class RewriteCompactionFileSelector implements ICrossSpaceMergeFileSelect
long maxCost = unseqFile.getTsFileSize();
return Math.min(multiSeriesCost, maxCost);
}
-
- @Override
- public int getConcurrentMergeNum() {
- return concurrentMergeNum;
- }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/SubCompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/ReadPointPerformerSubTask.java
similarity index 91%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/SubCompactionTask.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/ReadPointPerformerSubTask.java
index 315122e3cf..b7a19b641d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/SubCompactionTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/ReadPointPerformerSubTask.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.db.engine.compaction.cross.rewrite.task;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
@@ -45,7 +45,7 @@ import java.util.concurrent.Callable;
* parallel in the compaction. Currently, it only works for nonAligned data in cross space
* compaction and unseq inner space compaction.
*/
-public class SubCompactionTask implements Callable<Void> {
+public class ReadPointPerformerSubTask implements Callable<Void> {
private static final Logger logger =
LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
private final String device;
@@ -55,7 +55,7 @@ public class SubCompactionTask implements Callable<Void> {
private final AbstractCompactionWriter compactionWriter;
private final int taskId;
- public SubCompactionTask(
+ public ReadPointPerformerSubTask(
String device,
Set<String> measurementList,
QueryContext queryContext,
@@ -87,7 +87,7 @@ public class SubCompactionTask implements Callable<Void> {
}
IBatchReader dataBatchReader =
- CompactionUtils.constructReader(
+ ReadPointCompactionPerformer.constructReader(
device,
Collections.singletonList(measurement),
measurementSchemas,
@@ -98,7 +98,7 @@ public class SubCompactionTask implements Callable<Void> {
if (dataBatchReader.hasNextBatch()) {
compactionWriter.startMeasurement(measurementSchemas, taskId);
- CompactionUtils.writeWithReader(compactionWriter, dataBatchReader, taskId);
+ ReadPointCompactionPerformer.writeWithReader(compactionWriter, dataBatchReader, taskId);
compactionWriter.endMeasurement(taskId);
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/RewriteCrossSpaceCompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/RewriteCrossSpaceCompactionTask.java
deleted file mode 100644
index 5426bfae3b..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/RewriteCrossSpaceCompactionTask.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.task;
-
-import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.db.engine.compaction.CompactionUtils;
-import org.apache.iotdb.db.engine.compaction.cross.AbstractCrossSpaceCompactionTask;
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
-import org.apache.iotdb.db.engine.compaction.task.CompactionExceptionHandler;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
-
-import org.apache.commons.io.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_SOURCE_FILES;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_TARGET_FILES;
-
-public class RewriteCrossSpaceCompactionTask extends AbstractCrossSpaceCompactionTask {
- private static final Logger logger =
- LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
- protected List<TsFileResource> selectedSeqTsFileResourceList;
- protected List<TsFileResource> selectedUnSeqTsFileResourceList;
- protected TsFileResourceList seqTsFileResourceList;
- protected TsFileResourceList unseqTsFileResourceList;
- private File logFile;
-
- private List<TsFileResource> targetTsfileResourceList;
- private List<TsFileResource> holdReadLockList = new ArrayList<>();
- private List<TsFileResource> holdWriteLockList = new ArrayList<>();
-
- public RewriteCrossSpaceCompactionTask(
- String logicalStorageGroupName,
- String virtualStorageGroupName,
- long timePartitionId,
- TsFileManager tsFileManager,
- List<TsFileResource> selectedSeqTsFileResourceList,
- List<TsFileResource> selectedUnSeqTsFileResourceList,
- AtomicInteger currentTaskNum) {
- super(
- logicalStorageGroupName + "-" + virtualStorageGroupName,
- timePartitionId,
- currentTaskNum,
- selectedSeqTsFileResourceList,
- selectedUnSeqTsFileResourceList,
- tsFileManager);
- this.selectedSeqTsFileResourceList = selectedSeqTsFileResourceList;
- this.selectedUnSeqTsFileResourceList = selectedUnSeqTsFileResourceList;
- this.seqTsFileResourceList = tsFileManager.getSequenceListByTimePartition(timePartition);
- this.unseqTsFileResourceList = tsFileManager.getUnsequenceListByTimePartition(timePartition);
- }
-
- @Override
- protected void doCompaction() throws Exception {
- try {
- executeCompaction();
- } catch (Throwable throwable) {
- // catch throwable instead of exception to handle OOM errors
- logger.error("Meet errors in cross space compaction, {}", throwable.getMessage());
- CompactionExceptionHandler.handleException(
- fullStorageGroupName,
- logFile,
- targetTsfileResourceList,
- selectedSeqTsFileResourceList,
- selectedUnSeqTsFileResourceList,
- tsFileManager,
- timePartition,
- false,
- true);
- throw throwable;
- } finally {
- releaseAllLock();
- }
- }
-
- private void executeCompaction()
- throws IOException, StorageEngineException, MetadataException, InterruptedException,
- WriteProcessException {
- if (!tsFileManager.isAllowCompaction()) {
- return;
- }
- long startTime = System.currentTimeMillis();
- targetTsfileResourceList =
- TsFileNameGenerator.getCrossCompactionTargetFileResources(selectedSeqTsFileResourceList);
-
- if (targetTsfileResourceList.isEmpty()
- || selectedSeqTsFileResourceList.isEmpty()
- || selectedUnSeqTsFileResourceList.isEmpty()) {
- logger.info(
- "{} [Compaction] Cross space compaction file list is empty, end it",
- fullStorageGroupName);
- return;
- }
-
- logger.info(
- "{} [Compaction] CrossSpaceCompactionTask start. Sequence files : {}, unsequence files : {}",
- fullStorageGroupName,
- selectedSeqTsFileResourceList,
- selectedUnSeqTsFileResourceList);
- logFile =
- new File(
- selectedSeqTsFileResourceList.get(0).getTsFile().getParent()
- + File.separator
- + targetTsfileResourceList.get(0).getTsFile().getName()
- + CompactionLogger.CROSS_COMPACTION_LOG_NAME_SUFFIX);
-
- try (CompactionLogger compactionLogger = new CompactionLogger(logFile)) {
- // print the path of the temporary file first for priority check during recovery
- compactionLogger.logFiles(selectedSeqTsFileResourceList, STR_SOURCE_FILES);
- compactionLogger.logFiles(selectedUnSeqTsFileResourceList, STR_SOURCE_FILES);
- compactionLogger.logFiles(targetTsfileResourceList, STR_TARGET_FILES);
- // indicates that the cross compaction is complete and the result can be reused during a
- // restart recovery
- compactionLogger.close();
-
- CompactionUtils.compact(
- selectedSeqTsFileResourceList, selectedUnSeqTsFileResourceList, targetTsfileResourceList);
-
- checkInterrupted();
- CompactionUtils.moveTargetFile(targetTsfileResourceList, false, fullStorageGroupName);
- CompactionUtils.combineModsInCompaction(
- selectedSeqTsFileResourceList, selectedUnSeqTsFileResourceList, targetTsfileResourceList);
-
- checkInterrupted();
- // update tsfile resource in memory
- tsFileManager.replace(
- selectedSeqTsFileResourceList,
- selectedUnSeqTsFileResourceList,
- targetTsfileResourceList,
- timePartition,
- true);
-
- releaseReadAndLockWrite(selectedSeqTsFileResourceList);
- checkInterrupted();
- releaseReadAndLockWrite(selectedUnSeqTsFileResourceList);
- checkInterrupted();
-
- deleteOldFiles(selectedSeqTsFileResourceList);
- deleteOldFiles(selectedUnSeqTsFileResourceList);
- CompactionUtils.deleteCompactionModsFile(
- selectedSeqTsFileResourceList, selectedUnSeqTsFileResourceList);
-
- if (logFile.exists()) {
- FileUtils.delete(logFile);
- }
- logger.info(
- "{} [Compaction] CrossSpaceCompactionTask Costs {} s",
- fullStorageGroupName,
- (System.currentTimeMillis() - startTime) / 1000);
- }
- }
-
- private boolean addReadLock(List<TsFileResource> tsFileResourceList) {
- for (TsFileResource tsFileResource : tsFileResourceList) {
- tsFileResource.readLock();
- holdReadLockList.add(tsFileResource);
- if (tsFileResource.isCompacting()
- || !tsFileResource.isClosed()
- || !tsFileResource.getTsFile().exists()
- || tsFileResource.isDeleted()) {
- releaseAllLock();
- return false;
- }
- tsFileResource.setStatus(TsFileResourceStatus.COMPACTING);
- }
- return true;
- }
-
- @Override
- public boolean checkValidAndSetMerging() {
- return addReadLock(selectedSeqTsFileResourceList)
- && addReadLock(selectedUnSeqTsFileResourceList);
- }
-
- private void releaseReadAndLockWrite(List<TsFileResource> tsFileResourceList) {
- for (TsFileResource tsFileResource : tsFileResourceList) {
- tsFileResource.readUnlock();
- holdReadLockList.remove(tsFileResource);
- tsFileResource.writeLock();
- holdWriteLockList.add(tsFileResource);
- }
- }
-
- private void releaseAllLock() {
- selectedSeqTsFileResourceList.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
- selectedUnSeqTsFileResourceList.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
- for (TsFileResource tsFileResource : holdReadLockList) {
- tsFileResource.readUnlock();
- tsFileResource.setStatus(TsFileResourceStatus.CLOSED);
- }
- for (TsFileResource tsFileResource : holdWriteLockList) {
- tsFileResource.writeUnlock();
- tsFileResource.setStatus(TsFileResourceStatus.CLOSED);
- }
- holdReadLockList.clear();
- holdWriteLockList.clear();
- }
-
- private void deleteOldFiles(List<TsFileResource> tsFileResourceList) throws IOException {
- for (TsFileResource tsFileResource : tsFileResourceList) {
- FileReaderManager.getInstance().closeFileAndRemoveReader(tsFileResource.getTsFilePath());
- tsFileResource.setStatus(TsFileResourceStatus.DELETED);
- tsFileResource.remove();
- logger.info(
- "[CrossSpaceCompaction] Delete TsFile :{}.",
- tsFileResource.getTsFile().getAbsolutePath());
- }
- }
-
- public String getStorageGroupName() {
- return fullStorageGroupName;
- }
-
- @Override
- public boolean equalsOtherTask(AbstractCompactionTask other) {
- if (other instanceof RewriteCrossSpaceCompactionTask) {
- RewriteCrossSpaceCompactionTask otherTask = (RewriteCrossSpaceCompactionTask) other;
- return otherTask.selectedSeqTsFileResourceList.equals(selectedSeqTsFileResourceList)
- && otherTask.selectedUnSeqTsFileResourceList.equals(selectedUnSeqTsFileResourceList);
- }
- return false;
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/AbstractInnerSpaceCompactionSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/AbstractInnerSpaceCompactionSelector.java
deleted file mode 100644
index cb7ec1029a..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/AbstractInnerSpaceCompactionSelector.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.compaction.inner;
-
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionSelector;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-
-import java.util.List;
-
-public abstract class AbstractInnerSpaceCompactionSelector extends AbstractCompactionSelector {
- protected String logicalStorageGroupName;
- protected String dataRegionName;
- protected long timePartition;
- protected List<TsFileResource> tsFileResources;
- protected boolean sequence;
- protected InnerSpaceCompactionTaskFactory taskFactory;
- protected TsFileManager tsFileManager;
-
- public AbstractInnerSpaceCompactionSelector(
- String logicalStorageGroupName,
- String dataRegionName,
- long timePartition,
- TsFileManager tsFileManager,
- boolean sequence,
- InnerSpaceCompactionTaskFactory taskFactory) {
- this.logicalStorageGroupName = logicalStorageGroupName;
- this.dataRegionName = dataRegionName;
- this.timePartition = timePartition;
- this.tsFileManager = tsFileManager;
- this.sequence = sequence;
- this.taskFactory = taskFactory;
- if (sequence) {
- tsFileResources = tsFileManager.getSequenceListByTimePartition(timePartition).getArrayList();
- } else {
- tsFileResources =
- tsFileManager.getUnsequenceListByTimePartition(timePartition).getArrayList();
- }
- }
-
- @Override
- public abstract void selectAndSubmit();
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/AbstractInnerSpaceCompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/AbstractInnerSpaceCompactionTask.java
deleted file mode 100644
index 8b60330fbd..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/AbstractInnerSpaceCompactionTask.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.inner;
-
-import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public abstract class AbstractInnerSpaceCompactionTask extends AbstractCompactionTask {
- private static final Logger LOGGER =
- LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
-
- protected List<TsFileResource> selectedTsFileResourceList;
- protected boolean sequence;
- protected long selectedFileSize;
- protected int sumOfCompactionCount;
- protected long maxFileVersion;
- protected int maxCompactionCount;
-
- public AbstractInnerSpaceCompactionTask(
- String storageGroupName,
- long timePartition,
- AtomicInteger currentTaskNum,
- boolean sequence,
- List<TsFileResource> selectedTsFileResourceList,
- TsFileManager tsFileManager) {
- super(storageGroupName, timePartition, tsFileManager, currentTaskNum);
- this.selectedTsFileResourceList = selectedTsFileResourceList;
- this.sequence = sequence;
- collectSelectedFilesInfo();
- }
-
- @Override
- public void setSourceFilesToCompactionCandidate() {
- this.selectedTsFileResourceList.forEach(
- tsFileResource -> {
- try {
- tsFileResource.setStatus(TsFileResourceStatus.COMPACTION_CANDIDATE);
- } catch (Exception e) {
- LOGGER.error("Exception occurs when setting compaction candidate", e);
- }
- });
- }
-
- private void collectSelectedFilesInfo() {
- selectedFileSize = 0L;
- sumOfCompactionCount = 0;
- maxFileVersion = -1L;
- maxCompactionCount = -1;
- if (selectedTsFileResourceList == null) {
- return;
- }
- for (TsFileResource resource : selectedTsFileResourceList) {
- try {
- selectedFileSize += resource.getTsFileSize();
- TsFileNameGenerator.TsFileName fileName =
- TsFileNameGenerator.getTsFileName(resource.getTsFile().getName());
- sumOfCompactionCount += fileName.getInnerCompactionCnt();
- if (fileName.getInnerCompactionCnt() > maxCompactionCount) {
- maxCompactionCount = fileName.getInnerCompactionCnt();
- }
- if (fileName.getVersion() > maxFileVersion) {
- maxFileVersion = fileName.getVersion();
- }
- } catch (IOException e) {
- LOGGER.warn("Fail to get the tsfile name of {}", resource.getTsFile(), e);
- }
- }
- }
-
- public List<TsFileResource> getSelectedTsFileResourceList() {
- return selectedTsFileResourceList;
- }
-
- public boolean isSequence() {
- return sequence;
- }
-
- public long getSelectedFileSize() {
- return selectedFileSize;
- }
-
- public int getSumOfCompactionCount() {
- return sumOfCompactionCount;
- }
-
- public long getMaxFileVersion() {
- return maxFileVersion;
- }
-
- @Override
- public boolean checkValidAndSetMerging() {
- if (!tsFileManager.isAllowCompaction()) {
- return false;
- }
-
- for (TsFileResource resource : selectedTsFileResourceList) {
- if (resource.isCompacting() | !resource.isClosed() || !resource.getTsFile().exists()) {
- return false;
- }
- }
-
- for (TsFileResource resource : selectedTsFileResourceList) {
- resource.setStatus(TsFileResourceStatus.COMPACTING);
- }
- return true;
- }
-
- @Override
- public String toString() {
- return new StringBuilder()
- .append(fullStorageGroupName)
- .append("-")
- .append(timePartition)
- .append(" task file num is ")
- .append(selectedTsFileResourceList.size())
- .append(", files is ")
- .append(selectedTsFileResourceList)
- .append(", total compaction count is ")
- .append(sumOfCompactionCount)
- .toString();
- }
-
- @Override
- public int hashCode() {
- return toString().hashCode();
- }
-
- @Override
- public void resetCompactionCandidateStatusForAllSourceFiles() {
- selectedTsFileResourceList.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/IMergePathSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/IInnerSeqSpaceSelector.java
similarity index 65%
copy from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/IMergePathSelector.java
copy to server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/IInnerSeqSpaceSelector.java
index bf60b193e2..7a6b326a8e 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/IMergePathSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/IInnerSeqSpaceSelector.java
@@ -7,7 +7,7 @@
* "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
+ * 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
@@ -16,16 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.inner;
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
+import org.apache.iotdb.db.engine.compaction.task.ICompactionSelector;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.metadata.path.PartialPath;
-
-import java.util.Iterator;
import java.util.List;
-/**
- * IMergePathSelector select paths to be merged at a time if all paths of a device cannot be merged
- * at the same time.
- */
-public interface IMergePathSelector extends Iterator<List<PartialPath>> {}
+public interface IInnerSeqSpaceSelector extends ICompactionSelector {
+ List<List<TsFileResource>> selectInnerSpaceTask(List<TsFileResource> resources);
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/IMergePathSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/IInnerUnseqSpaceSelector.java
similarity index 70%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/IMergePathSelector.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/IInnerUnseqSpaceSelector.java
index bf60b193e2..c7f02f5f65 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/IMergePathSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/IInnerUnseqSpaceSelector.java
@@ -16,16 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.inner;
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
+import org.apache.iotdb.db.engine.compaction.task.ICompactionSelector;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.metadata.path.PartialPath;
-
-import java.util.Iterator;
import java.util.List;
-/**
- * IMergePathSelector select paths to be merged at a time if all paths of a device cannot be merged
- * at the same time.
- */
-public interface IMergePathSelector extends Iterator<List<PartialPath>> {}
+public interface IInnerUnseqSpaceSelector extends ICompactionSelector {
+ List<List<TsFileResource>> selectInnerSpaceTask(List<TsFileResource> resources);
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionStrategy.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionStrategy.java
deleted file mode 100644
index 35d8c7426c..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionStrategy.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.inner;
-
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.inner.sizetiered.SizeTieredCompactionSelector;
-import org.apache.iotdb.db.engine.compaction.inner.sizetiered.SizeTieredCompactionTask;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-
-import java.util.List;
-
-public enum InnerCompactionStrategy {
- SIZE_TIERED_COMPACTION;
-
- public static InnerCompactionStrategy getInnerCompactionStrategy(String name) {
- if ("SIZE_TIERED_COMPACTION".equalsIgnoreCase(name)) {
- return SIZE_TIERED_COMPACTION;
- }
- throw new RuntimeException("Illegal Compaction Strategy " + name);
- }
-
- public AbstractInnerSpaceCompactionTask getCompactionTask(
- String logicalStorageGroupName,
- String virtualStorageGroup,
- long timePartition,
- TsFileManager tsFileManager,
- List<TsFileResource> selectedTsFileResourceList,
- boolean sequence) {
- switch (this) {
- case SIZE_TIERED_COMPACTION:
- default:
- return new SizeTieredCompactionTask(
- logicalStorageGroupName,
- virtualStorageGroup,
- timePartition,
- tsFileManager,
- selectedTsFileResourceList,
- sequence,
- CompactionTaskManager.currentTaskNum);
- }
- }
-
- public AbstractInnerSpaceCompactionSelector getCompactionSelector(
- String logicalStorageGroupName,
- String virtualStorageGroupName,
- long timePartition,
- TsFileManager tsFileManager,
- boolean sequence,
- InnerSpaceCompactionTaskFactory taskFactory) {
- switch (this) {
- case SIZE_TIERED_COMPACTION:
- default:
- return new SizeTieredCompactionSelector(
- logicalStorageGroupName,
- virtualStorageGroupName,
- timePartition,
- tsFileManager,
- sequence,
- taskFactory);
- }
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTask.java
similarity index 60%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionTask.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTask.java
index b19e43a2d8..a42ea6d2a1 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionTask.java
@@ -7,7 +7,7 @@
* "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
+ * 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
@@ -16,15 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.inner.sizetiered;
+
+package org.apache.iotdb.db.engine.compaction.inner;
import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.db.engine.compaction.CompactionExceptionHandler;
import org.apache.iotdb.db.engine.compaction.CompactionUtils;
-import org.apache.iotdb.db.engine.compaction.inner.AbstractInnerSpaceCompactionTask;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.ICompactionPerformer;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
-import org.apache.iotdb.db.engine.compaction.task.CompactionExceptionHandler;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -37,36 +37,42 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
+import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
-/**
- * SizeTiredCompactionTask compact several inner space files selected by {@link
- * SizeTieredCompactionSelector} into one file.
- */
-public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
+public class InnerSpaceCompactionTask extends AbstractCompactionTask {
private static final Logger LOGGER =
LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+
+ protected List<TsFileResource> selectedTsFileResourceList;
+ protected TsFileResource targetTsFileResource;
+ protected boolean sequence;
+ protected long selectedFileSize;
+ protected int sumOfCompactionCount;
+ protected long maxFileVersion;
+ protected int maxCompactionCount;
+
protected TsFileResourceList tsFileResourceList;
protected boolean[] isHoldingReadLock;
protected boolean[] isHoldingWriteLock;
- public SizeTieredCompactionTask(
- String logicalStorageGroupName,
- String virtualStorageGroupName,
+ public InnerSpaceCompactionTask(
long timePartition,
TsFileManager tsFileManager,
List<TsFileResource> selectedTsFileResourceList,
boolean sequence,
+ ICompactionPerformer performer,
AtomicInteger currentTaskNum) {
super(
- logicalStorageGroupName + "-" + virtualStorageGroupName,
+ tsFileManager.getStorageGroupName() + "-" + tsFileManager.getDataRegion(),
timePartition,
- currentTaskNum,
- sequence,
- selectedTsFileResourceList,
- tsFileManager);
+ tsFileManager,
+ currentTaskNum);
+ this.selectedTsFileResourceList = selectedTsFileResourceList;
+ this.sequence = sequence;
+ this.performer = performer;
isHoldingReadLock = new boolean[selectedTsFileResourceList.size()];
isHoldingWriteLock = new boolean[selectedTsFileResourceList.size()];
for (int i = 0; i < selectedTsFileResourceList.size(); ++i) {
@@ -78,6 +84,7 @@ public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
} else {
tsFileResourceList = tsFileManager.getUnsequenceListByTimePartition(timePartition);
}
+ collectSelectedFilesInfo();
}
@Override
@@ -89,7 +96,7 @@ public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
// get resource of target file
String dataDirectory = selectedTsFileResourceList.get(0).getTsFile().getParent();
// Here is tmpTargetFile, which is xxx.target
- TsFileResource targetTsFileResource =
+ targetTsFileResource =
TsFileNameGenerator.getInnerCompactionTargetFileResource(
selectedTsFileResourceList, sequence);
LOGGER.info(
@@ -102,35 +109,33 @@ public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
+ File.separator
+ targetTsFileResource.getTsFile().getName()
+ CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX);
- CompactionLogger sizeTieredCompactionLogger = null;
+ CompactionLogger compactionLogger = null;
try {
- sizeTieredCompactionLogger = new CompactionLogger(logFile);
- sizeTieredCompactionLogger.logFiles(
- selectedTsFileResourceList, CompactionLogger.STR_SOURCE_FILES);
- sizeTieredCompactionLogger.logFiles(
+ compactionLogger = new CompactionLogger(logFile);
+ compactionLogger.logFiles(selectedTsFileResourceList, CompactionLogger.STR_SOURCE_FILES);
+ compactionLogger.logFiles(
Collections.singletonList(targetTsFileResource), CompactionLogger.STR_TARGET_FILES);
- LOGGER.info("{} [SizeTiredCompactionTask] Close the logger", fullStorageGroupName);
- sizeTieredCompactionLogger.close();
+ LOGGER.info("{} [InnerSpaceCompactionTask] Close the logger", fullStorageGroupName);
+ compactionLogger.close();
LOGGER.info(
"{} [Compaction] compaction with {}", fullStorageGroupName, selectedTsFileResourceList);
// carry out the compaction
- if (sequence) {
- InnerSpaceCompactionUtils.compact(targetTsFileResource, selectedTsFileResourceList);
- } else {
- CompactionUtils.compact(
- Collections.emptyList(),
- selectedTsFileResourceList,
- Collections.singletonList(targetTsFileResource));
- }
+ performer.setSourceFiles(selectedTsFileResourceList);
+ performer.setTargetFiles(Collections.singletonList(targetTsFileResource));
+ performer.perform();
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, fullStorageGroupName);
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, fullStorageGroupName);
- LOGGER.info("{} [SizeTiredCompactionTask] start to rename mods file", fullStorageGroupName);
- InnerSpaceCompactionUtils.combineModsInCompaction(
+ LOGGER.info("{} [InnerSpaceCompactionTask] start to rename mods file", fullStorageGroupName);
+ CompactionUtils.combineModsInInnerCompaction(
selectedTsFileResourceList, targetTsFileResource);
- checkInterrupted();
+ if (Thread.currentThread().isInterrupted()) {
+ throw new InterruptedException(
+ String.format("%s [Compaction] abort", fullStorageGroupName));
+ }
// replace the old files with new file, the new is in same position as the old
if (sequence) {
@@ -159,7 +164,6 @@ public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
isHoldingReadLock[i] = false;
selectedTsFileResourceList.get(i).writeLock();
isHoldingWriteLock[i] = true;
- checkInterrupted();
}
if (targetTsFileResource.getTsFile().length()
@@ -174,14 +178,13 @@ public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
LOGGER.info(
"{} [Compaction] compaction finish, start to delete old files", fullStorageGroupName);
// delete the old files
- InnerSpaceCompactionUtils.deleteTsFilesInDisk(
- selectedTsFileResourceList, fullStorageGroupName);
- InnerSpaceCompactionUtils.deleteModificationForSourceFile(
+ CompactionUtils.deleteTsFilesInDisk(selectedTsFileResourceList, fullStorageGroupName);
+ CompactionUtils.deleteModificationForSourceFile(
selectedTsFileResourceList, fullStorageGroupName);
long costTime = System.currentTimeMillis() - startTime;
LOGGER.info(
- "{} [SizeTiredCompactionTask] all compaction task finish, target file is {},"
+ "{} [InnerSpaceCompactionTask] all compaction task finish, target file is {},"
+ "time cost is {} s",
fullStorageGroupName,
targetTsFileResource.getTsFile().getName(),
@@ -196,8 +199,8 @@ public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
fullStorageGroupName,
throwable);
LOGGER.warn("{} [Compaction] Start to handle exception", fullStorageGroupName);
- if (sizeTieredCompactionLogger != null) {
- sizeTieredCompactionLogger.close();
+ if (compactionLogger != null) {
+ compactionLogger.close();
}
if (isSequence()) {
CompactionExceptionHandler.handleException(
@@ -228,19 +231,119 @@ public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
}
@Override
- public boolean equalsOtherTask(AbstractCompactionTask other) {
- if (other instanceof SizeTieredCompactionTask) {
- SizeTieredCompactionTask otherSizeTieredTask = (SizeTieredCompactionTask) other;
- if (!selectedTsFileResourceList.equals(otherSizeTieredTask.selectedTsFileResourceList)) {
- return false;
+ public boolean equalsOtherTask(AbstractCompactionTask otherTask) {
+ if (!(otherTask instanceof InnerSpaceCompactionTask)) {
+ return false;
+ }
+ InnerSpaceCompactionTask task = (InnerSpaceCompactionTask) otherTask;
+ return this.selectedTsFileResourceList.equals(task.selectedTsFileResourceList)
+ && this.performer.getClass().isInstance(task.performer);
+ }
+
+ @Override
+ public void setSourceFilesToCompactionCandidate() {
+ this.selectedTsFileResourceList.forEach(
+ tsFileResource -> {
+ try {
+ tsFileResource.setStatus(TsFileResourceStatus.COMPACTION_CANDIDATE);
+ } catch (Exception e) {
+ LOGGER.error("Exception occurs when setting compaction candidate", e);
+ }
+ });
+ }
+
+ private void collectSelectedFilesInfo() {
+ selectedFileSize = 0L;
+ sumOfCompactionCount = 0;
+ maxFileVersion = -1L;
+ maxCompactionCount = -1;
+ if (selectedTsFileResourceList == null) {
+ return;
+ }
+ for (TsFileResource resource : selectedTsFileResourceList) {
+ try {
+ selectedFileSize += resource.getTsFileSize();
+ TsFileNameGenerator.TsFileName fileName =
+ TsFileNameGenerator.getTsFileName(resource.getTsFile().getName());
+ sumOfCompactionCount += fileName.getInnerCompactionCnt();
+ if (fileName.getInnerCompactionCnt() > maxCompactionCount) {
+ maxCompactionCount = fileName.getInnerCompactionCnt();
+ }
+ if (fileName.getVersion() > maxFileVersion) {
+ maxFileVersion = fileName.getVersion();
+ }
+ } catch (IOException e) {
+ LOGGER.warn("Fail to get the tsfile name of {}", resource.getTsFile(), e);
+ }
+ }
+ }
+
+ public List<TsFileResource> getSelectedTsFileResourceList() {
+ return selectedTsFileResourceList;
+ }
+
+ public boolean isSequence() {
+ return sequence;
+ }
+
+ public long getSelectedFileSize() {
+ return selectedFileSize;
+ }
+
+ public int getSumOfCompactionCount() {
+ return sumOfCompactionCount;
+ }
+
+ public long getMaxFileVersion() {
+ return maxFileVersion;
+ }
+
+ @Override
+ public String toString() {
+ return new StringBuilder()
+ .append(fullStorageGroupName)
+ .append("-")
+ .append(timePartition)
+ .append(" task file num is ")
+ .append(selectedTsFileResourceList.size())
+ .append(", files is ")
+ .append(selectedTsFileResourceList)
+ .append(", total compaction count is ")
+ .append(sumOfCompactionCount)
+ .toString();
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public void resetCompactionCandidateStatusForAllSourceFiles() {
+ selectedTsFileResourceList.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED));
+ }
+
+ /**
+ * release the read lock and write lock of files if it is held, and set the merging status of
+ * selected files to false
+ */
+ protected void releaseFileLocksAndResetMergingStatus() {
+ for (int i = 0; i < selectedTsFileResourceList.size(); ++i) {
+ if (isHoldingReadLock[i]) {
+ selectedTsFileResourceList.get(i).readUnlock();
+ }
+ if (isHoldingWriteLock[i]) {
+ selectedTsFileResourceList.get(i).writeUnlock();
}
- return true;
+ selectedTsFileResourceList.get(i).setStatus(TsFileResourceStatus.CLOSED);
}
- return false;
}
@Override
public boolean checkValidAndSetMerging() {
+ if (!tsFileManager.isAllowCompaction()) {
+ return false;
+ }
for (int i = 0; i < selectedTsFileResourceList.size(); ++i) {
TsFileResource resource = selectedTsFileResourceList.get(i);
resource.readLock();
@@ -261,20 +364,4 @@ public class SizeTieredCompactionTask extends AbstractInnerSpaceCompactionTask {
}
return true;
}
-
- /**
- * release the read lock and write lock of files if it is held, and set the merging status of
- * selected files to false
- */
- private void releaseFileLocksAndResetMergingStatus() {
- for (int i = 0; i < selectedTsFileResourceList.size(); ++i) {
- if (isHoldingReadLock[i]) {
- selectedTsFileResourceList.get(i).readUnlock();
- }
- if (isHoldingWriteLock[i]) {
- selectedTsFileResourceList.get(i).writeUnlock();
- }
- selectedTsFileResourceList.get(i).setStatus(TsFileResourceStatus.CLOSED);
- }
- }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionSelector.java
index 9ef486a973..dc413a1370 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionSelector.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionSelector.java
@@ -22,9 +22,8 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.inner.AbstractInnerSpaceCompactionSelector;
-import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTaskFactory;
-import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
+import org.apache.iotdb.db.engine.compaction.inner.IInnerSeqSpaceSelector;
+import org.apache.iotdb.db.engine.compaction.inner.IInnerUnseqSpaceSelector;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -36,8 +35,9 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.Comparator;
-import java.util.Iterator;
+import java.util.LinkedList;
import java.util.List;
import java.util.PriorityQueue;
@@ -46,30 +46,34 @@ import java.util.PriorityQueue;
* selector traverses the file list from old to new. If the size of selected files or the number of
* select files exceed given threshold, a compaction task will be submitted to task queue in
* CompactionTaskManager. In CompactionTaskManager, tasks are ordered by {@link
- * org.apache.iotdb.db.engine.compaction.CompactionTaskComparator}. To maximize compaction
- * efficiency, selector searches compaction task from 0 compaction files(that is, file that never
- * been compacted, named level 0 file) to higher level files. If a compaction task is found in some
- * level, selector will not search higher level anymore.
+ * org.apache.iotdb.db.engine.compaction.comparator.ICompactionTaskComparator}. To maximize
+ * compaction efficiency, selector searches compaction task from 0 compaction files(that is, file
+ * that never been compacted, named level 0 file) to higher level files. If a compaction task is
+ * found in some level, selector will not search higher level anymore.
*/
-public class SizeTieredCompactionSelector extends AbstractInnerSpaceCompactionSelector {
+public class SizeTieredCompactionSelector
+ implements IInnerSeqSpaceSelector, IInnerUnseqSpaceSelector {
private static final Logger LOGGER =
LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
- private static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+ private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+ protected String logicalStorageGroupName;
+ protected String dataRegionId;
+ protected long timePartition;
+ protected List<TsFileResource> tsFileResources;
+ protected TsFileManager tsFileManager;
+ protected boolean sequence;
public SizeTieredCompactionSelector(
String logicalStorageGroupName,
- String virtualStorageGroupName,
+ String dataRegionId,
long timePartition,
TsFileManager tsFileManager,
- boolean sequence,
- InnerSpaceCompactionTaskFactory taskFactory) {
- super(
- logicalStorageGroupName,
- virtualStorageGroupName,
- timePartition,
- tsFileManager,
- sequence,
- taskFactory);
+ boolean sequence) {
+ this.logicalStorageGroupName = logicalStorageGroupName;
+ this.dataRegionId = dataRegionId;
+ this.timePartition = timePartition;
+ this.tsFileManager = tsFileManager;
+ this.sequence = sequence;
}
/**
@@ -81,7 +85,8 @@ public class SizeTieredCompactionSelector extends AbstractInnerSpaceCompactionSe
* @return Returns whether the file was found and submits the merge task
*/
@Override
- public void selectAndSubmit() {
+ public List<List<TsFileResource>> selectInnerSpaceTask(List<TsFileResource> tsFileResources) {
+ this.tsFileResources = tsFileResources;
PriorityQueue<Pair<List<TsFileResource>, Long>> taskPriorityQueue =
new PriorityQueue<>(new SizeTieredCompactionTaskComparator());
try {
@@ -91,12 +96,16 @@ public class SizeTieredCompactionSelector extends AbstractInnerSpaceCompactionSe
break;
}
}
+ List<List<TsFileResource>> taskList = new LinkedList<>();
while (taskPriorityQueue.size() > 0) {
- createAndSubmitTask(taskPriorityQueue.poll().left);
+ List<TsFileResource> resources = taskPriorityQueue.poll().left;
+ taskList.add(resources);
}
+ return taskList;
} catch (Exception e) {
LOGGER.error("Exception occurs while selecting files", e);
}
+ return Collections.emptyList();
}
/**
@@ -155,9 +164,7 @@ public class SizeTieredCompactionSelector extends AbstractInnerSpaceCompactionSe
private int searchMaxFileLevel() throws IOException {
int maxLevel = -1;
- Iterator<TsFileResource> iterator = tsFileResources.iterator();
- while (iterator.hasNext()) {
- TsFileResource currentFile = iterator.next();
+ for (TsFileResource currentFile : tsFileResources) {
TsFileNameGenerator.TsFileName currentName =
TsFileNameGenerator.getTsFileName(currentFile.getTsFile().getName());
if (currentName.getInnerCompactionCnt() > maxLevel) {
@@ -167,19 +174,6 @@ public class SizeTieredCompactionSelector extends AbstractInnerSpaceCompactionSe
return maxLevel;
}
- private boolean createAndSubmitTask(List<TsFileResource> selectedFileList)
- throws InterruptedException {
- AbstractCompactionTask compactionTask =
- taskFactory.createTask(
- logicalStorageGroupName,
- dataRegionName,
- timePartition,
- tsFileManager,
- selectedFileList,
- sequence);
- return CompactionTaskManager.getInstance().addTaskToWaitingQueue(compactionTask);
- }
-
private class SizeTieredCompactionTaskComparator
implements Comparator<Pair<List<TsFileResource>, Long>> {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/InnerSpaceCompactionUtils.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/InnerSpaceCompactionUtils.java
deleted file mode 100644
index d1753cb3ae..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/InnerSpaceCompactionUtils.java
+++ /dev/null
@@ -1,302 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.inner.utils;
-
-import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.cross.CrossCompactionStrategy;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.manage.CrossSpaceCompactionResource;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.ICrossSpaceMergeFileSelector;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.RewriteCompactionFileSelector;
-import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.modification.ModificationFile;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-import org.apache.iotdb.db.metadata.idtable.IDTableManager;
-import org.apache.iotdb.db.metadata.path.PartialPath;
-import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
-import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
-import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
-import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
-
-import org.apache.commons.io.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-
-public class InnerSpaceCompactionUtils {
-
- private static final Logger logger =
- LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
-
- private InnerSpaceCompactionUtils() {
- throw new IllegalStateException("Utility class");
- }
-
- public static void compact(TsFileResource targetResource, List<TsFileResource> tsFileResources)
- throws IOException, MetadataException, InterruptedException {
-
- try (MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(tsFileResources);
- TsFileIOWriter writer = new TsFileIOWriter(targetResource.getTsFile())) {
- while (deviceIterator.hasNextDevice()) {
- Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
- String device = deviceInfo.left;
- boolean aligned = deviceInfo.right;
-
- writer.startChunkGroup(device);
- if (aligned) {
- compactAlignedSeries(device, targetResource, writer, deviceIterator);
- } else {
- compactNotAlignedSeries(device, targetResource, writer, deviceIterator);
- }
- writer.endChunkGroup();
- }
-
- for (TsFileResource tsFileResource : tsFileResources) {
- targetResource.updatePlanIndexes(tsFileResource);
- }
- writer.endFile();
- targetResource.close();
- }
- }
-
- private static void checkThreadInterrupted(TsFileResource tsFileResource)
- throws InterruptedException {
- if (Thread.currentThread().isInterrupted()) {
- throw new InterruptedException(
- String.format(
- "[Compaction] compaction for target file %s abort", tsFileResource.toString()));
- }
- }
-
- private static void compactNotAlignedSeries(
- String device,
- TsFileResource targetResource,
- TsFileIOWriter writer,
- MultiTsFileDeviceIterator deviceIterator)
- throws IOException, MetadataException, InterruptedException {
- MultiTsFileDeviceIterator.MeasurementIterator seriesIterator =
- deviceIterator.iterateNotAlignedSeries(device, true);
- while (seriesIterator.hasNextSeries()) {
- checkThreadInterrupted(targetResource);
- // TODO: we can provide a configuration item to enable concurrent between each series
- PartialPath p = new PartialPath(device, seriesIterator.nextSeries());
- IMeasurementSchema measurementSchema;
- // TODO: seriesIterator needs to be refactor.
- // This statement must be called before next hasNextSeries() called, or it may be trapped in a
- // dead-loop.
- LinkedList<Pair<TsFileSequenceReader, List<ChunkMetadata>>> readerAndChunkMetadataList =
- seriesIterator.getMetadataListForCurrentSeries();
- try {
- if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
- measurementSchema =
- IDTableManager.getInstance().getSeriesSchema(device, p.getMeasurement());
- } else {
- measurementSchema = IoTDB.schemaProcessor.getSeriesSchema(p);
- }
- } catch (PathNotExistException e) {
- logger.info("A deleted path is skipped: {}", e.getMessage());
- continue;
- }
- SingleSeriesCompactionExecutor compactionExecutorOfCurrentTimeSeries =
- new SingleSeriesCompactionExecutor(
- p, measurementSchema, readerAndChunkMetadataList, writer, targetResource);
- compactionExecutorOfCurrentTimeSeries.execute();
- }
- }
-
- private static void compactAlignedSeries(
- String device,
- TsFileResource targetResource,
- TsFileIOWriter writer,
- MultiTsFileDeviceIterator deviceIterator)
- throws IOException, InterruptedException {
- checkThreadInterrupted(targetResource);
- LinkedList<Pair<TsFileSequenceReader, List<AlignedChunkMetadata>>> readerAndChunkMetadataList =
- deviceIterator.getReaderAndChunkMetadataForCurrentAlignedSeries();
- AlignedSeriesCompactionExecutor compactionExecutor =
- new AlignedSeriesCompactionExecutor(
- device, targetResource, readerAndChunkMetadataList, writer);
- compactionExecutor.execute();
- }
-
- public static boolean deleteTsFilesInDisk(
- Collection<TsFileResource> mergeTsFiles, String storageGroupName) {
- logger.info("{} [Compaction] Compaction starts to delete real file ", storageGroupName);
- boolean result = true;
- for (TsFileResource mergeTsFile : mergeTsFiles) {
- if (!deleteTsFile(mergeTsFile)) {
- result = false;
- }
- logger.info(
- "{} [Compaction] delete TsFile {}", storageGroupName, mergeTsFile.getTsFilePath());
- }
- return result;
- }
-
- /** Delete all modification files for source files */
- public static void deleteModificationForSourceFile(
- Collection<TsFileResource> sourceFiles, String storageGroupName) throws IOException {
- logger.info("{} [Compaction] Start to delete modifications of source files", storageGroupName);
- for (TsFileResource tsFileResource : sourceFiles) {
- ModificationFile compactionModificationFile =
- ModificationFile.getCompactionMods(tsFileResource);
- if (compactionModificationFile.exists()) {
- compactionModificationFile.remove();
- }
-
- ModificationFile normalModification = ModificationFile.getNormalMods(tsFileResource);
- if (normalModification.exists()) {
- normalModification.remove();
- }
- }
- }
-
- /**
- * This method is called to recover modifications while an exception occurs during compaction. It
- * append new modifications of each selected tsfile to its corresponding old mods file and delete
- * the compaction mods file.
- *
- * @param selectedTsFileResources
- * @throws IOException
- */
- public static void appendNewModificationsToOldModsFile(
- List<TsFileResource> selectedTsFileResources) throws IOException {
- for (TsFileResource sourceFile : selectedTsFileResources) {
- // if there are modifications to this seqFile during compaction
- if (sourceFile.getCompactionModFile().exists()) {
- ModificationFile compactionModificationFile =
- ModificationFile.getCompactionMods(sourceFile);
- Collection<Modification> newModification = compactionModificationFile.getModifications();
- compactionModificationFile.close();
- // write the new modifications to its old modification file
- try (ModificationFile oldModificationFile = sourceFile.getModFile()) {
- for (Modification modification : newModification) {
- oldModificationFile.write(modification);
- }
- }
- FileUtils.delete(new File(ModificationFile.getCompactionMods(sourceFile).getFilePath()));
- }
- }
- }
-
- /**
- * Collect all the compaction modification files of source files, and combines them as the
- * modification file of target file.
- */
- public static void combineModsInCompaction(
- Collection<TsFileResource> mergeTsFiles, TsFileResource targetTsFile) throws IOException {
- List<Modification> modifications = new ArrayList<>();
- for (TsFileResource mergeTsFile : mergeTsFiles) {
- try (ModificationFile sourceCompactionModificationFile =
- ModificationFile.getCompactionMods(mergeTsFile)) {
- modifications.addAll(sourceCompactionModificationFile.getModifications());
- }
- }
- if (!modifications.isEmpty()) {
- try (ModificationFile modificationFile = ModificationFile.getNormalMods(targetTsFile)) {
- for (Modification modification : modifications) {
- // we have to set modification offset to MAX_VALUE, as the offset of source chunk may
- // change after compaction
- modification.setFileOffset(Long.MAX_VALUE);
- modificationFile.write(modification);
- }
- }
- }
- }
-
- public static boolean deleteTsFile(TsFileResource seqFile) {
- try {
- FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile.getTsFilePath());
- seqFile.setStatus(TsFileResourceStatus.DELETED);
- seqFile.delete();
- } catch (IOException e) {
- logger.error(e.getMessage(), e);
- return false;
- }
- return true;
- }
-
- public static ICrossSpaceMergeFileSelector getCrossSpaceFileSelector(
- long budget, CrossSpaceCompactionResource resource) {
- CrossCompactionStrategy strategy =
- IoTDBDescriptor.getInstance().getConfig().getCrossCompactionStrategy();
- switch (strategy) {
- case REWRITE_COMPACTION:
- return new RewriteCompactionFileSelector(resource, budget);
- default:
- throw new UnsupportedOperationException("Unknown CrossSpaceFileStrategy " + strategy);
- }
- }
-
- public static class TsFileNameComparator implements Comparator<TsFileSequenceReader> {
-
- @Override
- public int compare(TsFileSequenceReader o1, TsFileSequenceReader o2) {
- return TsFileManager.compareFileName(new File(o1.getFileName()), new File(o2.getFileName()));
- }
- }
- /**
- * Update the targetResource. Move xxx.target to xxx.tsfile and serialize xxx.tsfile.resource .
- *
- * @param targetResource the old tsfile to be moved, which is xxx.target
- */
- public static void moveTargetFile(TsFileResource targetResource, String fullStorageGroupName)
- throws IOException {
- if (!targetResource.getTsFilePath().endsWith(IoTDBConstant.INNER_COMPACTION_TMP_FILE_SUFFIX)) {
- logger.warn(
- "{} [Compaction] Tmp target tsfile {} should be end with {}",
- fullStorageGroupName,
- targetResource.getTsFilePath(),
- IoTDBConstant.INNER_COMPACTION_TMP_FILE_SUFFIX);
- return;
- }
- File oldFile = targetResource.getTsFile();
-
- // move TsFile and delete old tsfile
- String newFilePath =
- targetResource
- .getTsFilePath()
- .replace(IoTDBConstant.INNER_COMPACTION_TMP_FILE_SUFFIX, TsFileConstant.TSFILE_SUFFIX);
- File newFile = new File(newFilePath);
- FSFactoryProducer.getFSFactory().moveFile(oldFile, newFile);
-
- // serialize xxx.tsfile.resource
- targetResource.setFile(newFile);
- targetResource.serialize();
- targetResource.close();
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/log/CompactionLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/CompactionLogAnalyzer.java
similarity index 84%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/log/CompactionLogAnalyzer.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/log/CompactionLogAnalyzer.java
index c827cf8a45..62c293d090 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/log/CompactionLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/CompactionLogAnalyzer.java
@@ -16,10 +16,9 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.utils.log;
+package org.apache.iotdb.db.engine.compaction.log;
import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.db.engine.compaction.TsFileIdentifier;
import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
import java.io.BufferedReader;
@@ -29,15 +28,15 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.SEQUENCE_NAME_FROM_OLD;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_MERGE_START_FROM_OLD;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_SEQ_FILES_FROM_OLD;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_SOURCE_FILES;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_SOURCE_FILES_FROM_OLD;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_TARGET_FILES;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_TARGET_FILES_FROM_OLD;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_UNSEQ_FILES_FROM_OLD;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.UNSEQUENCE_NAME_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.SEQUENCE_NAME_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_MERGE_START_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_SEQ_FILES_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_SOURCE_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_SOURCE_FILES_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_TARGET_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_TARGET_FILES_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_UNSEQ_FILES_FROM_OLD;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.UNSEQUENCE_NAME_FROM_OLD;
public class CompactionLogAnalyzer {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/log/CompactionLogger.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/CompactionLogger.java
similarity index 96%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/log/CompactionLogger.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/log/CompactionLogger.java
index c22f82ddf3..8f8204ade3 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/utils/log/CompactionLogger.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/CompactionLogger.java
@@ -17,9 +17,8 @@
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction.utils.log;
+package org.apache.iotdb.db.engine.compaction.log;
-import org.apache.iotdb.db.engine.compaction.TsFileIdentifier;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import java.io.BufferedWriter;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileIdentifier.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/TsFileIdentifier.java
similarity index 99%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileIdentifier.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/log/TsFileIdentifier.java
index 0087ef5ee4..2e3ccaf039 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileIdentifier.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/TsFileIdentifier.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction;
+package org.apache.iotdb.db.engine.compaction.log;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ICompactionPerformer.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ICompactionPerformer.java
new file mode 100644
index 0000000000..93baeb0d55
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ICompactionPerformer.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.engine.compaction.performer;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * CompactionPerformer is used to compact multiple files into one or multiple files. Different
+ * performers may use different implementation to achieve this goal. Some may read chunk directly
+ * from tsfile, and some may using query tools to read data point by point from tsfile. Notice, not
+ * all kinds of Performer can be used for all kinds of compaction tasks!
+ */
+public interface ICompactionPerformer {
+
+ void perform()
+ throws IOException, MetadataException, StorageEngineException, InterruptedException;
+
+ void setTargetFiles(List<TsFileResource> targetFiles);
+
+ default void setSourceFiles(List<TsFileResource> files) {
+ throw new RuntimeException("Cannot set single type of source files to this kind of performer");
+ }
+
+ default void setSourceFiles(List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
+ throw new RuntimeException(
+ "Cannot set both seq files and unseq files to this kind of performer");
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ICrossCompactionPerformer.java
similarity index 72%
copy from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java
copy to server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ICrossCompactionPerformer.java
index 17328a3d12..d96e576c26 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ICrossCompactionPerformer.java
@@ -16,12 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.performer;
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-public enum MergeFileStrategy {
- MAX_SERIES_NUM,
- MAX_FILE_NUM,
- // TODO: HOW?
- TRADE_OFF,
+import java.util.List;
+
+public interface ICrossCompactionPerformer extends ICompactionPerformer {
+ void setSourceFiles(List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ISeqCompactionPerformer.java
similarity index 75%
copy from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java
copy to server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ISeqCompactionPerformer.java
index 17328a3d12..8835bb544f 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/ISeqCompactionPerformer.java
@@ -16,12 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.performer;
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-public enum MergeFileStrategy {
- MAX_SERIES_NUM,
- MAX_FILE_NUM,
- // TODO: HOW?
- TRADE_OFF,
+import java.util.List;
+
+public interface ISeqCompactionPerformer extends ICompactionPerformer {
+ void setSourceFiles(List<TsFileResource> seqFiles);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/IUnseqCompactionPerformer.java
similarity index 74%
rename from server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java
rename to server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/IUnseqCompactionPerformer.java
index 17328a3d12..3c48d41df0 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/selector/MergeFileStrategy.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/IUnseqCompactionPerformer.java
@@ -16,12 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.iotdb.db.engine.compaction.performer;
-package org.apache.iotdb.db.engine.compaction.cross.rewrite.selector;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-public enum MergeFileStrategy {
- MAX_SERIES_NUM,
- MAX_FILE_NUM,
- // TODO: HOW?
- TRADE_OFF,
+import java.util.List;
+
+public interface IUnseqCompactionPerformer extends ICompactionPerformer {
+ void setSourceFiles(List<TsFileResource> unseqFiles);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadChunkCompactionPerformer.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadChunkCompactionPerformer.java
new file mode 100644
index 0000000000..28cfa86963
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadChunkCompactionPerformer.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.engine.compaction.performer.impl;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.inner.utils.AlignedSeriesCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.inner.utils.SingleSeriesCompactionExecutor;
+import org.apache.iotdb.db.engine.compaction.performer.ISeqCompactionPerformer;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+public class ReadChunkCompactionPerformer implements ISeqCompactionPerformer {
+ private static final Logger LOGGER =
+ LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+ private TsFileResource targetResource;
+ private List<TsFileResource> seqFiles;
+
+ public ReadChunkCompactionPerformer(List<TsFileResource> sourceFiles, TsFileResource targetFile) {
+ this.seqFiles = sourceFiles;
+ this.targetResource = targetFile;
+ }
+
+ public ReadChunkCompactionPerformer(List<TsFileResource> sourceFiles) {
+ this.seqFiles = sourceFiles;
+ }
+
+ public ReadChunkCompactionPerformer() {}
+
+ @Override
+ public void perform()
+ throws IOException, MetadataException, InterruptedException, StorageEngineException {
+ try (MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(seqFiles);
+ TsFileIOWriter writer = new TsFileIOWriter(targetResource.getTsFile())) {
+ while (deviceIterator.hasNextDevice()) {
+ Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
+ String device = deviceInfo.left;
+ boolean aligned = deviceInfo.right;
+
+ writer.startChunkGroup(device);
+ if (aligned) {
+ compactAlignedSeries(device, targetResource, writer, deviceIterator);
+ } else {
+ compactNotAlignedSeries(device, targetResource, writer, deviceIterator);
+ }
+ writer.endChunkGroup();
+ }
+
+ for (TsFileResource tsFileResource : seqFiles) {
+ targetResource.updatePlanIndexes(tsFileResource);
+ }
+ writer.endFile();
+ targetResource.close();
+ }
+ }
+
+ @Override
+ public void setTargetFiles(List<TsFileResource> targetFiles) {
+ if (targetFiles.size() != 1) {
+ throw new RuntimeException(
+ String.format(
+ "Current performer only supports for one target file while getting %d target files",
+ targetFiles.size()));
+ }
+ this.targetResource = targetFiles.get(0);
+ }
+
+ private void compactAlignedSeries(
+ String device,
+ TsFileResource targetResource,
+ TsFileIOWriter writer,
+ MultiTsFileDeviceIterator deviceIterator)
+ throws IOException, InterruptedException {
+ checkThreadInterrupted();
+ LinkedList<Pair<TsFileSequenceReader, List<AlignedChunkMetadata>>> readerAndChunkMetadataList =
+ deviceIterator.getReaderAndChunkMetadataForCurrentAlignedSeries();
+ AlignedSeriesCompactionExecutor compactionExecutor =
+ new AlignedSeriesCompactionExecutor(
+ device, targetResource, readerAndChunkMetadataList, writer);
+ compactionExecutor.execute();
+ }
+
+ private void checkThreadInterrupted() throws InterruptedException {
+ if (Thread.currentThread().isInterrupted()) {
+ throw new InterruptedException(
+ String.format(
+ "[Compaction] compaction for target file %s abort", targetResource.toString()));
+ }
+ }
+
+ private void compactNotAlignedSeries(
+ String device,
+ TsFileResource targetResource,
+ TsFileIOWriter writer,
+ MultiTsFileDeviceIterator deviceIterator)
+ throws IOException, MetadataException, InterruptedException {
+ MultiTsFileDeviceIterator.MeasurementIterator seriesIterator =
+ deviceIterator.iterateNotAlignedSeries(device, true);
+ while (seriesIterator.hasNextSeries()) {
+ checkThreadInterrupted();
+ // TODO: we can provide a configuration item to enable concurrent between each series
+ PartialPath p = new PartialPath(device, seriesIterator.nextSeries());
+ IMeasurementSchema measurementSchema;
+ // TODO: seriesIterator needs to be refactor.
+ // This statement must be called before next hasNextSeries() called, or it may be trapped in a
+ // dead-loop.
+ LinkedList<Pair<TsFileSequenceReader, List<ChunkMetadata>>> readerAndChunkMetadataList =
+ seriesIterator.getMetadataListForCurrentSeries();
+ try {
+ if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
+ measurementSchema =
+ IDTableManager.getInstance().getSeriesSchema(device, p.getMeasurement());
+ } else {
+ measurementSchema = IoTDB.schemaProcessor.getSeriesSchema(p);
+ }
+ } catch (PathNotExistException e) {
+ LOGGER.info("A deleted path is skipped: {}", e.getMessage());
+ continue;
+ }
+ SingleSeriesCompactionExecutor compactionExecutorOfCurrentTimeSeries =
+ new SingleSeriesCompactionExecutor(
+ p, measurementSchema, readerAndChunkMetadataList, writer, targetResource);
+ compactionExecutorOfCurrentTimeSeries.execute();
+ }
+ }
+
+ @Override
+ public void setSourceFiles(List<TsFileResource> seqFiles) {
+ this.seqFiles = seqFiles;
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java
similarity index 66%
copy from server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java
copy to server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java
index a4328449ca..6ddf8b0688 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/performer/impl/ReadPointCompactionPerformer.java
@@ -7,7 +7,7 @@
* "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
+ * 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
@@ -16,19 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.engine.compaction;
+package org.apache.iotdb.db.engine.compaction.performer.impl;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.SubCompactionTask;
+import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.ReadPointPerformerSubTask;
import org.apache.iotdb.db.engine.compaction.inner.utils.MultiTsFileDeviceIterator;
+import org.apache.iotdb.db.engine.compaction.performer.ICrossCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.performer.IUnseqCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.writer.AbstractCompactionWriter;
import org.apache.iotdb.db.engine.compaction.writer.CrossSpaceCompactionWriter;
import org.apache.iotdb.db.engine.compaction.writer.InnerSpaceCompactionWriter;
-import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.exception.StorageEngineException;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
@@ -43,11 +43,8 @@ import org.apache.iotdb.db.query.control.QueryResourceManager;
import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader;
import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.db.utils.QueryUtils;
-import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
-import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.reader.IBatchReader;
import org.apache.iotdb.tsfile.utils.Pair;
@@ -57,10 +54,9 @@ import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -69,37 +65,50 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.stream.Collectors;
-/**
- * This tool can be used to perform inner space or cross space compaction of aligned and non aligned
- * timeseries . Currently, we use {@link
- * org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils} to speed up if it is
- * an seq inner space compaction.
- */
-public class CompactionUtils {
- private static final Logger logger =
- LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+public class ReadPointCompactionPerformer
+ implements ICrossCompactionPerformer, IUnseqCompactionPerformer {
+ private Logger LOGGER = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
+ private List<TsFileResource> seqFiles = Collections.emptyList();
+ private List<TsFileResource> unseqFiles = Collections.emptyList();
private static final int subTaskNum =
IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
- public static void compact(
- List<TsFileResource> seqFileResources,
- List<TsFileResource> unseqFileResources,
- List<TsFileResource> targetFileResources)
+ private List<TsFileResource> targetFiles = Collections.emptyList();
+
+ public ReadPointCompactionPerformer(
+ List<TsFileResource> seqFiles,
+ List<TsFileResource> unseqFiles,
+ List<TsFileResource> targetFiles) {
+ this.seqFiles = seqFiles;
+ this.unseqFiles = unseqFiles;
+ this.targetFiles = targetFiles;
+ }
+
+ public ReadPointCompactionPerformer(
+ List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
+ this.seqFiles = seqFiles;
+ this.unseqFiles = unseqFiles;
+ }
+
+ public ReadPointCompactionPerformer() {}
+
+ @Override
+ public void perform()
throws IOException, MetadataException, StorageEngineException, InterruptedException {
long queryId = QueryResourceManager.getInstance().assignCompactionQueryId();
QueryContext queryContext = new QueryContext(queryId);
- QueryDataSource queryDataSource = new QueryDataSource(seqFileResources, unseqFileResources);
+ QueryDataSource queryDataSource = new QueryDataSource(seqFiles, unseqFiles);
QueryResourceManager.getInstance()
.getQueryFileManager()
.addUsedFilesForQuery(queryId, queryDataSource);
try (AbstractCompactionWriter compactionWriter =
- getCompactionWriter(seqFileResources, unseqFileResources, targetFileResources)) {
+ getCompactionWriter(seqFiles, unseqFiles, targetFiles)) {
// Do not close device iterator, because tsfile reader is managed by FileReaderManager.
MultiTsFileDeviceIterator deviceIterator =
- new MultiTsFileDeviceIterator(seqFileResources, unseqFileResources);
+ new MultiTsFileDeviceIterator(seqFiles, unseqFiles);
while (deviceIterator.hasNextDevice()) {
- checkThreadInterrupted(targetFileResources);
+ checkThreadInterrupted();
Pair<String, Boolean> deviceInfo = deviceIterator.nextDevice();
String device = deviceInfo.left;
boolean isAligned = deviceInfo.right;
@@ -115,14 +124,19 @@ public class CompactionUtils {
}
compactionWriter.endFile();
- updateDeviceStartTimeAndEndTime(targetFileResources, compactionWriter);
- updatePlanIndexes(targetFileResources, seqFileResources, unseqFileResources);
+ updateDeviceStartTimeAndEndTime(targetFiles, compactionWriter);
+ updatePlanIndexes(targetFiles, seqFiles, unseqFiles);
} finally {
QueryResourceManager.getInstance().endQuery(queryId);
}
}
- private static void compactAlignedSeries(
+ @Override
+ public void setTargetFiles(List<TsFileResource> targetFiles) {
+ this.targetFiles = targetFiles;
+ }
+
+ private void compactAlignedSeries(
String device,
MultiTsFileDeviceIterator deviceIterator,
AbstractCompactionWriter compactionWriter,
@@ -142,7 +156,7 @@ public class CompactionUtils {
IoTDB.schemaProcessor.getSeriesSchema(new PartialPath(device, measurement)));
}
} catch (PathNotExistException e) {
- logger.info("A deleted path is skipped: {}", e.getMessage());
+ LOGGER.info("A deleted path is skipped: {}", e.getMessage());
}
}
if (measurementSchemas.isEmpty()) {
@@ -172,13 +186,14 @@ public class CompactionUtils {
}
}
- private static void compactNonAlignedSeries(
+ private void compactNonAlignedSeries(
String device,
MultiTsFileDeviceIterator deviceIterator,
AbstractCompactionWriter compactionWriter,
QueryContext queryContext,
QueryDataSource queryDataSource)
- throws IOException, InterruptedException {
+ throws MetadataException, IOException, InterruptedException {
+ boolean hasStartChunkGroup = false;
MultiTsFileDeviceIterator.MeasurementIterator measurementIterator =
deviceIterator.iterateNotAlignedSeries(device, false);
Set<String> allMeasurements = measurementIterator.getAllMeasurements();
@@ -201,7 +216,7 @@ public class CompactionUtils {
futures.add(
CompactionTaskManager.getInstance()
.submitSubTask(
- new SubCompactionTask(
+ new ReadPointPerformerSubTask(
device,
measurementsForEachSubTask[i],
queryContext,
@@ -215,7 +230,7 @@ public class CompactionUtils {
try {
futures.get(i).get();
} catch (InterruptedException | ExecutionException e) {
- logger.error("SubCompactionTask meet errors ", e);
+ LOGGER.error("SubCompactionTask meet errors ", e);
Thread.interrupted();
throw new InterruptedException();
}
@@ -224,13 +239,25 @@ public class CompactionUtils {
compactionWriter.endChunkGroup();
}
- public static void writeWithReader(
- AbstractCompactionWriter writer, IBatchReader reader, int subTaskId) throws IOException {
- while (reader.hasNextBatch()) {
- BatchData batchData = reader.nextBatch();
- while (batchData.hasCurrent()) {
- writer.write(batchData.currentTime(), batchData.currentValue(), subTaskId);
- batchData.next();
+ private static void updateDeviceStartTimeAndEndTime(
+ List<TsFileResource> targetResources, AbstractCompactionWriter compactionWriter) {
+ List<TsFileIOWriter> targetFileWriters = compactionWriter.getFileIOWriter();
+ for (int i = 0; i < targetFileWriters.size(); i++) {
+ TsFileIOWriter fileIOWriter = targetFileWriters.get(i);
+ TsFileResource fileResource = targetResources.get(i);
+ // The tmp target file may does not have any data points written due to the existence of the
+ // mods file, and it will be deleted after compaction. So skip the target file that has been
+ // deleted.
+ if (!fileResource.getTsFile().exists()) {
+ continue;
+ }
+ for (Map.Entry<String, List<TimeseriesMetadata>> entry :
+ fileIOWriter.getDeviceTimeseriesMetadataMap().entrySet()) {
+ String device = entry.getKey();
+ for (TimeseriesMetadata timeseriesMetadata : entry.getValue()) {
+ fileResource.updateStartTime(device, timeseriesMetadata.getStatistics().getStartTime());
+ fileResource.updateEndTime(device, timeseriesMetadata.getStatistics().getEndTime());
+ }
}
}
}
@@ -261,7 +288,18 @@ public class CompactionUtils {
seriesPath, allSensors, tsDataType, queryContext, queryDataSource, null, null, null, true);
}
- private static AbstractCompactionWriter getCompactionWriter(
+ public static void writeWithReader(
+ AbstractCompactionWriter writer, IBatchReader reader, int subTaskId) throws IOException {
+ while (reader.hasNextBatch()) {
+ BatchData batchData = reader.nextBatch();
+ while (batchData.hasCurrent()) {
+ writer.write(batchData.currentTime(), batchData.currentValue(), subTaskId);
+ batchData.next();
+ }
+ }
+ }
+
+ private AbstractCompactionWriter getCompactionWriter(
List<TsFileResource> seqFileResources,
List<TsFileResource> unseqFileResources,
List<TsFileResource> targetFileResources)
@@ -275,29 +313,6 @@ public class CompactionUtils {
}
}
- private static void updateDeviceStartTimeAndEndTime(
- List<TsFileResource> targetResources, AbstractCompactionWriter compactionWriter) {
- List<TsFileIOWriter> targetFileWriters = compactionWriter.getFileIOWriter();
- for (int i = 0; i < targetFileWriters.size(); i++) {
- TsFileIOWriter fileIOWriter = targetFileWriters.get(i);
- TsFileResource fileResource = targetResources.get(i);
- // The tmp target file may does not have any data points written due to the existence of the
- // mods file, and it will be deleted after compaction. So skip the target file that has been
- // deleted.
- if (!fileResource.getTsFile().exists()) {
- continue;
- }
- for (Map.Entry<String, List<TimeseriesMetadata>> entry :
- fileIOWriter.getDeviceTimeseriesMetadataMap().entrySet()) {
- String device = entry.getKey();
- for (TimeseriesMetadata timeseriesMetadata : entry.getValue()) {
- fileResource.updateStartTime(device, timeseriesMetadata.getStatistics().getStartTime());
- fileResource.updateEndTime(device, timeseriesMetadata.getStatistics().getEndTime());
- }
- }
- }
- }
-
private static void updatePlanIndexes(
List<TsFileResource> targetResources,
List<TsFileResource> seqResources,
@@ -310,7 +325,7 @@ public class CompactionUtils {
// in the new file
for (int i = 0; i < targetResources.size(); i++) {
TsFileResource targetResource = targetResources.get(i);
- // remove the target file that has been deleted from list
+ // remove the target file been deleted from list
if (!targetResource.getTsFile().exists()) {
targetResources.remove(i--);
continue;
@@ -324,117 +339,22 @@ public class CompactionUtils {
}
}
- /**
- * Update the targetResource. Move tmp target file to target file and serialize
- * xxx.tsfile.resource.
- */
- public static void moveTargetFile(
- List<TsFileResource> targetResources, boolean isInnerSpace, String fullStorageGroupName)
- throws IOException, WriteProcessException {
- String fileSuffix;
- if (isInnerSpace) {
- fileSuffix = IoTDBConstant.INNER_COMPACTION_TMP_FILE_SUFFIX;
- } else {
- fileSuffix = IoTDBConstant.CROSS_COMPACTION_TMP_FILE_SUFFIX;
- }
- for (TsFileResource targetResource : targetResources) {
- moveOneTargetFile(targetResource, fileSuffix, fullStorageGroupName);
- }
- }
-
- private static void moveOneTargetFile(
- TsFileResource targetResource, String tmpFileSuffix, String fullStorageGroupName)
- throws IOException {
- // move to target file and delete old tmp target file
- if (!targetResource.getTsFile().exists()) {
- logger.info(
- "{} [Compaction] Tmp target tsfile {} may be deleted after compaction.",
- fullStorageGroupName,
- targetResource.getTsFilePath());
- return;
- }
- File newFile =
- new File(
- targetResource.getTsFilePath().replace(tmpFileSuffix, TsFileConstant.TSFILE_SUFFIX));
- if (!newFile.exists()) {
- FSFactoryProducer.getFSFactory().moveFile(targetResource.getTsFile(), newFile);
- }
-
- // serialize xxx.tsfile.resource
- targetResource.setFile(newFile);
- targetResource.serialize();
- targetResource.close();
- }
-
- /**
- * Collect all the compaction modification files of source files, and combines them as the
- * modification file of target file.
- */
- public static void combineModsInCompaction(
- List<TsFileResource> seqResources,
- List<TsFileResource> unseqResources,
- List<TsFileResource> targetResources)
- throws IOException {
- // target file may less than source seq files, so we should find each target file with its
- // corresponding source seq file.
- Map<String, TsFileResource> seqFileInfoMap = new HashMap<>();
- for (TsFileResource tsFileResource : seqResources) {
- seqFileInfoMap.put(
- TsFileNameGenerator.increaseCrossCompactionCnt(tsFileResource.getTsFile()).getName(),
- tsFileResource);
- }
- // update each target mods file.
- for (TsFileResource tsFileResource : targetResources) {
- updateOneTargetMods(
- tsFileResource, seqFileInfoMap.get(tsFileResource.getTsFile().getName()), unseqResources);
- }
- }
-
- private static void updateOneTargetMods(
- TsFileResource targetFile, TsFileResource seqFile, List<TsFileResource> unseqFiles)
- throws IOException {
- // write mods in the seq file
- if (seqFile != null) {
- ModificationFile seqCompactionModificationFile = ModificationFile.getCompactionMods(seqFile);
- for (Modification modification : seqCompactionModificationFile.getModifications()) {
- targetFile.getModFile().write(modification);
- }
- }
- // write mods in all unseq files
- for (TsFileResource unseqFile : unseqFiles) {
- ModificationFile compactionUnseqModificationFile =
- ModificationFile.getCompactionMods(unseqFile);
- for (Modification modification : compactionUnseqModificationFile.getModifications()) {
- targetFile.getModFile().write(modification);
- }
+ private void checkThreadInterrupted() throws InterruptedException {
+ if (Thread.currentThread().isInterrupted()) {
+ throw new InterruptedException(
+ String.format(
+ "[Compaction] compaction for target file %s abort", targetFiles.toString()));
}
- targetFile.getModFile().close();
}
- public static void deleteCompactionModsFile(
- List<TsFileResource> selectedSeqTsFileResourceList,
- List<TsFileResource> selectedUnSeqTsFileResourceList)
- throws IOException {
- for (TsFileResource seqFile : selectedSeqTsFileResourceList) {
- ModificationFile modificationFile = seqFile.getCompactionModFile();
- if (modificationFile.exists()) {
- modificationFile.remove();
- }
- }
- for (TsFileResource unseqFile : selectedUnSeqTsFileResourceList) {
- ModificationFile modificationFile = unseqFile.getCompactionModFile();
- if (modificationFile.exists()) {
- modificationFile.remove();
- }
- }
+ @Override
+ public void setSourceFiles(List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
+ this.seqFiles = seqFiles;
+ this.unseqFiles = unseqFiles;
}
- private static void checkThreadInterrupted(List<TsFileResource> tsFileResource)
- throws InterruptedException {
- if (Thread.currentThread().isInterrupted()) {
- throw new InterruptedException(
- String.format(
- "[Compaction] compaction for target file %s abort", tsFileResource.toString()));
- }
+ @Override
+ public void setSourceFiles(List<TsFileResource> unseqFiles) {
+ this.unseqFiles = unseqFiles;
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/AbstractCompactionTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/AbstractCompactionTask.java
index 79d45dfcfa..2cc19c6e54 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/AbstractCompactionTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/AbstractCompactionTask.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.engine.compaction.task;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
+import org.apache.iotdb.db.engine.compaction.performer.ICompactionPerformer;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.slf4j.Logger;
@@ -45,6 +46,7 @@ public abstract class AbstractCompactionTask implements Callable<Void> {
protected long timeCost = 0L;
protected volatile boolean ran = false;
protected volatile boolean finished = false;
+ protected ICompactionPerformer performer;
public AbstractCompactionTask(
String fullStorageGroupName,
@@ -72,7 +74,7 @@ public abstract class AbstractCompactionTask implements Callable<Void> {
LOGGER.error(e.getMessage(), e);
} finally {
this.currentTaskNum.decrementAndGet();
- CompactionTaskManager.getInstance().removeRunningTaskFromList(this);
+ CompactionTaskManager.getInstance().removeRunningTaskFuture(this);
timeCost = System.currentTimeMillis() - startTime;
finished = true;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionRecoverTask.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionRecoverTask.java
index a4f82eb118..cdb107ee5c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionRecoverTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/CompactionRecoverTask.java
@@ -21,9 +21,9 @@ package org.apache.iotdb.db.engine.compaction.task;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.CompactionUtils;
-import org.apache.iotdb.db.engine.compaction.TsFileIdentifier;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogAnalyzer;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogAnalyzer;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.log.TsFileIdentifier;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/ICompactionSelector.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/ICompactionSelector.java
new file mode 100644
index 0000000000..34acc5fa73
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/task/ICompactionSelector.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.engine.compaction.task;
+
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.List;
+
+/**
+ * AbstractCompactionSelector is the base class of all CompactionSelector. It runs the file
+ * selection process, if there still threads availabe for compaction task, it will submit a
+ * compaction task to {@link org.apache.iotdb.db.engine.compaction.CompactionTaskManager} and
+ * increase the global compaction task count.
+ */
+public interface ICompactionSelector {
+ /*
+ * This method should be implemented by all SequenceSpaceInnerSelector and UnsequenceSpaceInnerSelector.
+ * It takes the list of tsfile in a time partition as input, and returns a list of list. Each list in
+ * the returned list is the source files of one compaction tasks.
+ */
+ default List<List<TsFileResource>> selectInnerSpaceTask(List<TsFileResource> resources) {
+ throw new RuntimeException("This kind of selector cannot be used to select inner space task");
+ }
+
+ /*
+ * This method should be implemented by all CrossSpaceSelector. It takes the list of sequence files and
+ * list of unsequence files as input, and returns a list of pair of list and list. Each pair in the returned
+ * list contains two list: the left one is the selected sequence files, the right one is the selected
+ * unsequence files. Each pair is corresponding to a cross space compaction task.
+ */
+ default List<Pair<List<TsFileResource>, List<TsFileResource>>> selectCrossSpaceTask(
+ List<TsFileResource> seqFiles, List<TsFileResource> unseqFiles) {
+ {
+ throw new RuntimeException("This kind of selector cannot be used to select cross space task");
+ }
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
index 05b330f5c4..7434cd136e 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
@@ -29,10 +29,10 @@ import org.apache.iotdb.db.conf.directories.DirectoryManager;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.engine.cache.ChunkCache;
import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
+import org.apache.iotdb.db.engine.compaction.CompactionRecoverManager;
import org.apache.iotdb.db.engine.compaction.CompactionScheduler;
import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
-import org.apache.iotdb.db.engine.compaction.task.CompactionRecoverManager;
import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
import org.apache.iotdb.db.engine.flush.CloseFileListener;
import org.apache.iotdb.db.engine.flush.FlushListener;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskComparatorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskComparatorTest.java
index 53b039f4ba..0475026486 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskComparatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskComparatorTest.java
@@ -19,9 +19,12 @@
package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.comparator.DefaultCompactionTaskComparatorImpl;
import org.apache.iotdb.db.engine.compaction.constant.CompactionPriority;
-import org.apache.iotdb.db.engine.compaction.cross.AbstractCrossSpaceCompactionTask;
-import org.apache.iotdb.db.engine.compaction.inner.AbstractInnerSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
@@ -46,7 +49,7 @@ public class CompactionTaskComparatorTest {
private final Logger LOGGER = LoggerFactory.getLogger(CompactionTaskComparatorTest.class);
private final AtomicInteger taskNum = new AtomicInteger(0);
private FixedPriorityBlockingQueue<AbstractCompactionTask> compactionTaskQueue =
- new FixedPriorityBlockingQueue<>(1024, new CompactionTaskComparator());
+ new FixedPriorityBlockingQueue<>(1024, new DefaultCompactionTaskComparatorImpl());
private TsFileManager tsFileManager = new TsFileManager("fakeSg", "0", "/");
@Before
@@ -127,7 +130,9 @@ public class CompactionTaskComparatorTest {
@Test
public void testPriorityQueueSizeLimit() {
MinMaxPriorityQueue<AbstractCompactionTask> limitQueue =
- MinMaxPriorityQueue.orderedBy(new CompactionTaskComparator()).maximumSize(50).create();
+ MinMaxPriorityQueue.orderedBy(new DefaultCompactionTaskComparatorImpl())
+ .maximumSize(50)
+ .create();
AbstractCompactionTask[] compactionTasks = new AbstractCompactionTask[100];
for (int i = 0; i < 100; ++i) {
List<TsFileResource> resources = new ArrayList<>();
@@ -261,7 +266,7 @@ public class CompactionTaskComparatorTest {
}
}
- private static class FakedInnerSpaceCompactionTask extends AbstractInnerSpaceCompactionTask {
+ private static class FakedInnerSpaceCompactionTask extends InnerSpaceCompactionTask {
public FakedInnerSpaceCompactionTask(
String storageGroupName,
@@ -271,12 +276,12 @@ public class CompactionTaskComparatorTest {
boolean sequence,
List<TsFileResource> selectedTsFileResourceList) {
super(
- storageGroupName,
timePartition,
- currentTaskNum,
- sequence,
+ tsFileManager,
selectedTsFileResourceList,
- tsFileManager);
+ sequence,
+ new ReadChunkCompactionPerformer(),
+ currentTaskNum);
}
@Override
@@ -293,7 +298,7 @@ public class CompactionTaskComparatorTest {
}
}
- private static class FakeCrossSpaceCompactionTask extends AbstractCrossSpaceCompactionTask {
+ private static class FakeCrossSpaceCompactionTask extends CrossSpaceCompactionTask {
public FakeCrossSpaceCompactionTask(
String fullStorageGroupName,
@@ -303,12 +308,12 @@ public class CompactionTaskComparatorTest {
List<TsFileResource> selectedSequenceFiles,
List<TsFileResource> selectedUnsequenceFiles) {
super(
- fullStorageGroupName,
timePartition,
- currentTaskNum,
+ tsFileManager,
selectedSequenceFiles,
selectedUnsequenceFiles,
- tsFileManager);
+ new ReadPointCompactionPerformer(),
+ currentTaskNum);
}
@Override
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManagerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManagerTest.java
index 511cdb144f..977ebd1a43 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManagerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManagerTest.java
@@ -19,9 +19,11 @@
package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.RewriteCrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.cross.CrossSpaceCompactionTask;
import org.apache.iotdb.db.engine.compaction.inner.InnerCompactionTest;
-import org.apache.iotdb.db.engine.compaction.inner.sizetiered.SizeTieredCompactionTask;
+import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -68,12 +70,22 @@ public class CompactionTaskManagerTest extends InnerCompactionTest {
TsFileManager tsFileManager =
new TsFileManager("root.compactionTest", "0", tempSGDir.getAbsolutePath());
tsFileManager.addAll(seqResources, true);
- SizeTieredCompactionTask task1 =
- new SizeTieredCompactionTask(
- "root.compactionTest", "0", 0, tsFileManager, seqResources, true, new AtomicInteger(0));
- SizeTieredCompactionTask task2 =
- new SizeTieredCompactionTask(
- "root.compactionTest", "0", 0, tsFileManager, seqResources, true, new AtomicInteger(0));
+ InnerSpaceCompactionTask task1 =
+ new InnerSpaceCompactionTask(
+ 0,
+ tsFileManager,
+ seqResources,
+ true,
+ new ReadChunkCompactionPerformer(seqResources),
+ new AtomicInteger(0));
+ InnerSpaceCompactionTask task2 =
+ new InnerSpaceCompactionTask(
+ 0,
+ tsFileManager,
+ seqResources,
+ true,
+ new ReadChunkCompactionPerformer(seqResources),
+ new AtomicInteger(0));
seqResources.get(0).readLock();
CompactionTaskManager manager = CompactionTaskManager.getInstance();
try {
@@ -119,12 +131,22 @@ public class CompactionTaskManagerTest extends InnerCompactionTest {
TsFileManager tsFileManager =
new TsFileManager("root.compactionTest", "0", tempSGDir.getAbsolutePath());
tsFileManager.addAll(seqResources, true);
- SizeTieredCompactionTask task1 =
- new SizeTieredCompactionTask(
- "root.compactionTest", "0", 0, tsFileManager, seqResources, true, new AtomicInteger(0));
- SizeTieredCompactionTask task2 =
- new SizeTieredCompactionTask(
- "root.compactionTest", "0", 0, tsFileManager, seqResources, true, new AtomicInteger(0));
+ InnerSpaceCompactionTask task1 =
+ new InnerSpaceCompactionTask(
+ 0,
+ tsFileManager,
+ seqResources,
+ true,
+ new ReadChunkCompactionPerformer(seqResources),
+ new AtomicInteger(0));
+ InnerSpaceCompactionTask task2 =
+ new InnerSpaceCompactionTask(
+ 0,
+ tsFileManager,
+ seqResources,
+ true,
+ new ReadChunkCompactionPerformer(seqResources),
+ new AtomicInteger(0));
seqResources.get(0).readLock();
try {
CompactionTaskManager manager = CompactionTaskManager.getInstance();
@@ -171,12 +193,22 @@ public class CompactionTaskManagerTest extends InnerCompactionTest {
TsFileManager tsFileManager =
new TsFileManager("root.compactionTest", "0", tempSGDir.getAbsolutePath());
tsFileManager.addAll(seqResources, true);
- SizeTieredCompactionTask task1 =
- new SizeTieredCompactionTask(
- "root.compactionTest", "0", 0, tsFileManager, seqResources, true, new AtomicInteger(0));
- SizeTieredCompactionTask task2 =
- new SizeTieredCompactionTask(
- "root.compactionTest", "0", 0, tsFileManager, seqResources, true, new AtomicInteger(0));
+ InnerSpaceCompactionTask task1 =
+ new InnerSpaceCompactionTask(
+ 0,
+ tsFileManager,
+ seqResources,
+ true,
+ new ReadChunkCompactionPerformer(seqResources),
+ new AtomicInteger(0));
+ InnerSpaceCompactionTask task2 =
+ new InnerSpaceCompactionTask(
+ 0,
+ tsFileManager,
+ seqResources,
+ true,
+ new ReadChunkCompactionPerformer(seqResources),
+ new AtomicInteger(0));
CompactionTaskManager manager = CompactionTaskManager.getInstance();
manager.addTaskToWaitingQueue(task1);
manager.submitTaskFromTaskQueue();
@@ -209,9 +241,14 @@ public class CompactionTaskManagerTest extends InnerCompactionTest {
TsFileManager tsFileManager =
new TsFileManager("root.compactionTest", "0", tempSGDir.getAbsolutePath());
tsFileManager.addAll(seqResources, true);
- SizeTieredCompactionTask task1 =
- new SizeTieredCompactionTask(
- "root.compactionTest", "0", 0, tsFileManager, seqResources, true, new AtomicInteger(0));
+ InnerSpaceCompactionTask task1 =
+ new InnerSpaceCompactionTask(
+ 0,
+ tsFileManager,
+ seqResources,
+ true,
+ new ReadChunkCompactionPerformer(seqResources),
+ new AtomicInteger(0));
CompactionTaskManager manager = CompactionTaskManager.getInstance();
manager.restart();
seqResources.get(0).readLock();
@@ -248,9 +285,14 @@ public class CompactionTaskManagerTest extends InnerCompactionTest {
TsFileManager tsFileManager =
new TsFileManager("root.compactionTest", "0", tempSGDir.getAbsolutePath());
tsFileManager.addAll(seqResources, true);
- SizeTieredCompactionTask task =
- new SizeTieredCompactionTask(
- "root.compactionTest", "0", 0, tsFileManager, seqResources, true, new AtomicInteger(0));
+ InnerSpaceCompactionTask task =
+ new InnerSpaceCompactionTask(
+ 0,
+ tsFileManager,
+ seqResources,
+ true,
+ new ReadChunkCompactionPerformer(seqResources),
+ new AtomicInteger(0));
CompactionTaskManager.getInstance().addTaskToWaitingQueue(task);
for (TsFileResource resource : seqResources) {
@@ -270,14 +312,13 @@ public class CompactionTaskManagerTest extends InnerCompactionTest {
new TsFileManager("root.compactionTest", "0", tempSGDir.getAbsolutePath());
tsFileManager.addAll(seqResources, true);
tsFileManager.addAll(unseqResources, false);
- RewriteCrossSpaceCompactionTask task =
- new RewriteCrossSpaceCompactionTask(
- COMPACTION_TEST_SG,
- "0",
+ CrossSpaceCompactionTask task =
+ new CrossSpaceCompactionTask(
0,
tsFileManager,
seqResources,
unseqResources,
+ new ReadPointCompactionPerformer(),
new AtomicInteger(0));
for (TsFileResource resource : seqResources) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionUtilsTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/ReadPointCompactionPerformerTest.java
similarity index 98%
rename from server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionUtilsTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/ReadPointCompactionPerformerTest.java
index 69ff88dee0..efb5947981 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/CompactionUtilsTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/ReadPointCompactionPerformerTest.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.exception.StorageEngineException;
@@ -61,7 +62,7 @@ import java.util.Map;
import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_SEPARATOR;
import static org.junit.Assert.assertEquals;
-public class CompactionUtilsTest extends AbstractCompactionTest {
+public class ReadPointCompactionPerformerTest extends AbstractCompactionTest {
private final String oldThreadName = Thread.currentThread().getName();
@Before
@@ -120,7 +121,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources, true);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
tsFilesReader =
@@ -205,7 +206,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources, true);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
assertEquals(
0, targetResources.get(0).getStartTime(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0"));
@@ -307,7 +308,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = 0; i < 2; i++) {
@@ -412,7 +413,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = 0; i < 9; i++) {
@@ -547,7 +548,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = 0; i < 5; i++) {
@@ -672,7 +673,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = 0; i < 5; i++) {
@@ -783,7 +784,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = 0; i < 5; i++) {
@@ -863,7 +864,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources, true);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -971,7 +972,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources, true);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -1089,7 +1090,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources, true);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -1213,7 +1214,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -1386,7 +1387,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -1540,7 +1541,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -1646,7 +1647,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
}
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(unseqResources, false);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -1739,7 +1740,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
tsFilesReader =
@@ -1852,7 +1853,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
List<String> deviceIdList = new ArrayList<>();
@@ -2046,7 +2047,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
List<String> deviceIdList = new ArrayList<>();
@@ -2237,7 +2238,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
List<String> deviceIdList = new ArrayList<>();
@@ -2418,7 +2419,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
Assert.assertEquals(2, targetResources.size());
@@ -2590,7 +2591,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
Assert.assertEquals(4, targetResources.size());
@@ -2761,7 +2762,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
tsFilesReader =
@@ -2888,7 +2889,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -3088,7 +3089,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
Assert.assertEquals(4, targetResources.size());
@@ -3323,7 +3324,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -3533,7 +3534,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
for (int i = TsFileGeneratorUtils.getAlignDeviceOffset();
@@ -3618,7 +3619,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
Assert.assertEquals(4, targetResources.size());
@@ -3688,7 +3689,7 @@ public class CompactionUtilsTest extends AbstractCompactionTest {
List<TsFileResource> targetResources =
CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
Assert.assertEquals(2, targetResources.size());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/TsFileIdentifierUT.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/TsFileIdentifierUT.java
index 0a18c1d21b..5ca09317fa 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/TsFileIdentifierUT.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/TsFileIdentifierUT.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.log.TsFileIdentifier;
import org.apache.iotdb.db.utils.FileUtils;
import org.junit.Assert;
@@ -31,7 +32,7 @@ import java.io.IOException;
import java.lang.reflect.Field;
import java.nio.file.Files;
-import static org.apache.iotdb.db.engine.compaction.TsFileIdentifier.INFO_SEPARATOR;
+import static org.apache.iotdb.db.engine.compaction.log.TsFileIdentifier.INFO_SEPARATOR;
public class TsFileIdentifierUT {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionExceptionTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionExceptionTest.java
index e9ab9d8735..e77babf24b 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionExceptionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionExceptionTest.java
@@ -21,11 +21,12 @@ package org.apache.iotdb.db.engine.compaction.cross;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
+import org.apache.iotdb.db.engine.compaction.CompactionExceptionHandler;
import org.apache.iotdb.db.engine.compaction.CompactionUtils;
-import org.apache.iotdb.db.engine.compaction.task.CompactionExceptionHandler;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -46,8 +47,8 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_SOURCE_FILES;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_TARGET_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_SOURCE_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_TARGET_FILES;
import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
public class CrossSpaceCompactionExceptionTest extends AbstractCompactionTest {
@@ -90,7 +91,7 @@ public class CrossSpaceCompactionExceptionTest extends AbstractCompactionTest {
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
compactionLogger.close();
CompactionExceptionHandler.handleException(
COMPACTION_TEST_SG,
@@ -161,7 +162,7 @@ public class CrossSpaceCompactionExceptionTest extends AbstractCompactionTest {
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
compactionLogger.close();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
CompactionExceptionHandler.handleException(
@@ -233,7 +234,7 @@ public class CrossSpaceCompactionExceptionTest extends AbstractCompactionTest {
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
for (TsFileResource resource : seqResources) {
tsFileManager.getSequenceListByTimePartition(0).remove(resource);
@@ -315,7 +316,7 @@ public class CrossSpaceCompactionExceptionTest extends AbstractCompactionTest {
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
compactionLogger.close();
for (int i = 0; i < seqResources.size(); i++) {
@@ -334,7 +335,7 @@ public class CrossSpaceCompactionExceptionTest extends AbstractCompactionTest {
CompactionFileGeneratorUtils.generateMods(deleteMap, unseqResources.get(i), true);
CompactionFileGeneratorUtils.generateMods(deleteMap, unseqResources.get(i), false);
}
- CompactionUtils.combineModsInCompaction(seqResources, unseqResources, targetResources);
+ CompactionUtils.combineModsInCrossCompaction(seqResources, unseqResources, targetResources);
for (TsFileResource resource : seqResources) {
tsFileManager.getSequenceListByTimePartition(0).remove(resource);
}
@@ -429,7 +430,7 @@ public class CrossSpaceCompactionExceptionTest extends AbstractCompactionTest {
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
compactionLogger.close();
for (int i = 0; i < seqResources.size(); i++) {
@@ -448,7 +449,7 @@ public class CrossSpaceCompactionExceptionTest extends AbstractCompactionTest {
CompactionFileGeneratorUtils.generateMods(deleteMap, unseqResources.get(i), true);
CompactionFileGeneratorUtils.generateMods(deleteMap, unseqResources.get(i), false);
}
- CompactionUtils.combineModsInCompaction(seqResources, unseqResources, targetResources);
+ CompactionUtils.combineModsInCrossCompaction(seqResources, unseqResources, targetResources);
CompactionExceptionHandler.handleException(
COMPACTION_TEST_SG,
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTest.java
index 5e0824911e..7dae07dfcd 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/CrossSpaceCompactionTest.java
@@ -19,10 +19,11 @@
package org.apache.iotdb.db.engine.compaction.cross;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.cache.ChunkCache;
import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.manage.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionResource;
import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.ICrossSpaceMergeFileSelector;
import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.RewriteCompactionFileSelector;
import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask;
@@ -418,8 +419,8 @@ public class CrossSpaceCompactionTest {
TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
unseqTsFileResourceList.addAll(unseqResources);
long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
- CrossSpaceCompactionResource mergeResource =
- new CrossSpaceCompactionResource(
+ RewriteCrossSpaceCompactionResource mergeResource =
+ new RewriteCrossSpaceCompactionResource(
seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
ICrossSpaceMergeFileSelector fileSelector =
new RewriteCompactionFileSelector(mergeResource, Long.MAX_VALUE);
@@ -427,18 +428,18 @@ public class CrossSpaceCompactionTest {
mergeResource.clear();
index++;
if (mergeFiles.length > 0) {
- CrossSpaceCompactionTaskFactory taskFactory = new CrossSpaceCompactionTaskFactory();
AbstractCompactionTask compactionTask =
- taskFactory.createTask(
- COMPACTION_TEST_SG,
- "0",
- 0,
- new TsFileManager(
- "root.compactionTest",
- "0",
- "target\\data\\sequence\\test\\root.compactionTest\\0\\0\\"),
- mergeResource.getSeqFiles(),
- mergeResource.getUnseqFiles());
+ IoTDBDescriptor.getInstance()
+ .getConfig()
+ .getCrossCompactionSelector()
+ .getCompactionTask(
+ 0,
+ new TsFileManager(
+ "root.compactionTest",
+ "0",
+ "target\\data\\sequence\\test\\root.compactionTest\\0\\0\\"),
+ mergeResource.getSeqFiles(),
+ mergeResource.getUnseqFiles());
compactionTask.call();
List<TsFileResource> targetTsfileResourceList = new ArrayList<>();
for (TsFileResource seqResource : seqResources) {
@@ -722,26 +723,26 @@ public class CrossSpaceCompactionTest {
TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
unseqTsFileResourceList.addAll(unseqResources);
long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
- CrossSpaceCompactionResource mergeResource =
- new CrossSpaceCompactionResource(
+ RewriteCrossSpaceCompactionResource mergeResource =
+ new RewriteCrossSpaceCompactionResource(
seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
ICrossSpaceMergeFileSelector fileSelector =
new RewriteCompactionFileSelector(mergeResource, Long.MAX_VALUE);
List[] mergeFiles = fileSelector.select();
mergeResource.clear();
if (mergeFiles.length > 0) {
- CrossSpaceCompactionTaskFactory taskFactory = new CrossSpaceCompactionTaskFactory();
AbstractCompactionTask compactionTask =
- taskFactory.createTask(
- COMPACTION_TEST_SG,
- "0",
- 0,
- new TsFileManager(
- "root.compactionTest",
- "0",
- "target\\data\\sequence\\test\\root.compactionTest\\0\\0\\"),
- mergeResource.getSeqFiles(),
- mergeResource.getUnseqFiles());
+ IoTDBDescriptor.getInstance()
+ .getConfig()
+ .getCrossCompactionSelector()
+ .getCompactionTask(
+ 0,
+ new TsFileManager(
+ "root.compactionTest",
+ "0",
+ "target\\data\\sequence\\test\\root.compactionTest\\0\\0\\"),
+ mergeResource.getSeqFiles(),
+ mergeResource.getUnseqFiles());
compactionTask.call();
List<TsFileResource> targetTsfileResourceList = new ArrayList<>();
for (TsFileResource seqResource : seqResources.subList(1, 4)) {
@@ -1024,26 +1025,26 @@ public class CrossSpaceCompactionTest {
TsFileResourceList unseqTsFileResourceList = new TsFileResourceList();
unseqTsFileResourceList.addAll(unseqResources);
long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
- CrossSpaceCompactionResource mergeResource =
- new CrossSpaceCompactionResource(
+ RewriteCrossSpaceCompactionResource mergeResource =
+ new RewriteCrossSpaceCompactionResource(
seqTsFileResourceList, unseqTsFileResourceList, timeLowerBound);
ICrossSpaceMergeFileSelector fileSelector =
new RewriteCompactionFileSelector(mergeResource, Long.MAX_VALUE);
List[] mergeFiles = fileSelector.select();
mergeResource.clear();
if (mergeFiles.length > 0) {
- CrossSpaceCompactionTaskFactory taskFactory = new CrossSpaceCompactionTaskFactory();
AbstractCompactionTask compactionTask =
- taskFactory.createTask(
- COMPACTION_TEST_SG,
- "0",
- 0,
- new TsFileManager(
- "root.compactionTest",
- "0",
- "target\\data\\sequence\\test\\root.compactionTest\\0\\0\\"),
- mergeResource.getSeqFiles(),
- mergeResource.getUnseqFiles());
+ IoTDBDescriptor.getInstance()
+ .getConfig()
+ .getCrossCompactionSelector()
+ .getCompactionTask(
+ 0,
+ new TsFileManager(
+ "root.compactionTest",
+ "0",
+ "target\\data\\sequence\\test\\root.compactionTest\\0\\0\\"),
+ mergeResource.getSeqFiles(),
+ mergeResource.getUnseqFiles());
compactionTask.call();
List<TsFileResource> targetTsfileResourceList = new ArrayList<>();
for (TsFileResource seqResource : seqResources.subList(1, 4)) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/MergeUpgradeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/MergeUpgradeTest.java
index 2f3bcffc4c..410822ebc8 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/MergeUpgradeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/MergeUpgradeTest.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.engine.compaction.cross;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.manage.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionResource;
import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.RewriteCompactionFileSelector;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -79,8 +79,8 @@ public class MergeUpgradeTest {
@Test
public void testMergeUpgradeSelect() throws MergeException {
- CrossSpaceCompactionResource resource =
- new CrossSpaceCompactionResource(seqResources, unseqResources);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqResources, unseqResources);
RewriteCompactionFileSelector mergeFileSelector =
new RewriteCompactionFileSelector(resource, Long.MAX_VALUE);
List[] result = mergeFileSelector.select();
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCompactionFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCompactionFileSelectorTest.java
index 5345c6449b..0b5ca8d891 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCompactionFileSelectorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCompactionFileSelectorTest.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.engine.compaction.cross;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.manage.CrossSpaceCompactionResource;
+import org.apache.iotdb.db.engine.compaction.cross.rewrite.RewriteCrossSpaceCompactionResource;
import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.ICrossSpaceMergeFileSelector;
import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.RewriteCompactionFileSelector;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -52,8 +52,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
@Test
public void testFullSelection() throws MergeException, IOException {
- CrossSpaceCompactionResource resource =
- new CrossSpaceCompactionResource(seqResources, unseqResources);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqResources, unseqResources);
ICrossSpaceMergeFileSelector mergeFileSelector =
new RewriteCompactionFileSelector(resource, Long.MAX_VALUE);
List[] result = mergeFileSelector.select();
@@ -63,7 +63,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
assertEquals(unseqResources, unseqSelected);
resource.clear();
- resource = new CrossSpaceCompactionResource(seqResources.subList(0, 1), unseqResources);
+ resource = new RewriteCrossSpaceCompactionResource(seqResources.subList(0, 1), unseqResources);
mergeFileSelector = new RewriteCompactionFileSelector(resource, Long.MAX_VALUE);
result = mergeFileSelector.select();
seqSelected = result[0];
@@ -72,7 +72,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
assertEquals(unseqResources, unseqSelected);
resource.clear();
- resource = new CrossSpaceCompactionResource(seqResources, unseqResources.subList(0, 1));
+ resource = new RewriteCrossSpaceCompactionResource(seqResources, unseqResources.subList(0, 1));
mergeFileSelector = new RewriteCompactionFileSelector(resource, Long.MAX_VALUE);
result = mergeFileSelector.select();
seqSelected = result[0];
@@ -84,8 +84,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
@Test
public void testNonSelection() throws MergeException, IOException {
- CrossSpaceCompactionResource resource =
- new CrossSpaceCompactionResource(seqResources, unseqResources);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqResources, unseqResources);
ICrossSpaceMergeFileSelector mergeFileSelector = new RewriteCompactionFileSelector(resource, 1);
List[] result = mergeFileSelector.select();
assertEquals(0, result.length);
@@ -94,8 +94,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
@Test
public void testRestrictedSelection() throws MergeException, IOException {
- CrossSpaceCompactionResource resource =
- new CrossSpaceCompactionResource(seqResources, unseqResources);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqResources, unseqResources);
ICrossSpaceMergeFileSelector mergeFileSelector =
new RewriteCompactionFileSelector(resource, 400000);
List[] result = mergeFileSelector.select();
@@ -152,8 +152,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
List<TsFileResource> newUnseqResources = new ArrayList<>();
newUnseqResources.add(largeUnseqTsFileResource);
- CrossSpaceCompactionResource resource =
- new CrossSpaceCompactionResource(seqResources, newUnseqResources);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqResources, newUnseqResources);
ICrossSpaceMergeFileSelector mergeFileSelector =
new RewriteCompactionFileSelector(resource, Long.MAX_VALUE);
List[] result = mergeFileSelector.select();
@@ -207,8 +207,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
newUnseqResources.add(largeUnseqTsFileResource);
long ttlLowerBound = System.currentTimeMillis() - Long.MAX_VALUE;
- CrossSpaceCompactionResource mergeResource =
- new CrossSpaceCompactionResource(seqResources, newUnseqResources, ttlLowerBound);
+ RewriteCrossSpaceCompactionResource mergeResource =
+ new RewriteCrossSpaceCompactionResource(seqResources, newUnseqResources, ttlLowerBound);
assertEquals(5, mergeResource.getSeqFiles().size());
assertEquals(1, mergeResource.getUnseqFiles().size());
mergeResource.clear();
@@ -244,8 +244,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
unseqResources.clear();
unseqResources.add(largeUnseqTsFileResource);
- CrossSpaceCompactionResource resource =
- new CrossSpaceCompactionResource(seqResources, unseqResources);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqResources, unseqResources);
ICrossSpaceMergeFileSelector mergeFileSelector =
new RewriteCompactionFileSelector(resource, Long.MAX_VALUE);
List[] result = mergeFileSelector.select();
@@ -302,7 +302,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
prepareFile(unseqList.get(1), 0, 100, 20);
prepareFile(unseqList.get(2), 99, 1, 30);
- CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqList, unseqList);
// the budget is enough to select unseq0 and unseq2, but not unseq1
// the first selection should only contain seq0 and unseq0
ICrossSpaceMergeFileSelector mergeFileSelector =
@@ -315,7 +316,7 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
resource.clear();
resource =
- new CrossSpaceCompactionResource(
+ new RewriteCrossSpaceCompactionResource(
seqList.subList(1, seqList.size()), unseqList.subList(1, unseqList.size()));
// the second selection should be empty
mergeFileSelector = new RewriteCompactionFileSelector(resource, 29000);
@@ -379,7 +380,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
unseqList.add(fileResource);
}
- CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqList, unseqList);
Assert.assertEquals(5, resource.getSeqFiles().size());
Assert.assertEquals(10, resource.getUnseqFiles().size());
ICrossSpaceMergeFileSelector mergeFileSelector =
@@ -442,7 +444,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
unseqList.add(fileResource);
}
- CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqList, unseqList);
Assert.assertEquals(5, resource.getSeqFiles().size());
Assert.assertEquals(1, resource.getUnseqFiles().size());
ICrossSpaceMergeFileSelector mergeFileSelector =
@@ -505,7 +508,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
prepareFile(unseqList.get(0), 7, 3, 7);
prepareFile(unseqList.get(1), 10, 4, 10);
- CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqList, unseqList);
Assert.assertEquals(5, resource.getSeqFiles().size());
Assert.assertEquals(2, resource.getUnseqFiles().size());
ICrossSpaceMergeFileSelector mergeFileSelector =
@@ -571,7 +575,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
prepareFile(unseqList.get(2), 14, 3, 14);
prepareFile(unseqList.get(3), 17, 2, 17);
- CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqList, unseqList);
Assert.assertEquals(5, resource.getSeqFiles().size());
Assert.assertEquals(4, resource.getUnseqFiles().size());
ICrossSpaceMergeFileSelector mergeFileSelector =
@@ -640,7 +645,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
prepareFile(unseqList.get(2), 14, 3, 14);
prepareFile(unseqList.get(3), 17, 2, 17);
- CrossSpaceCompactionResource resource = new CrossSpaceCompactionResource(seqList, unseqList);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqList, unseqList);
Assert.assertEquals(5, resource.getSeqFiles().size());
Assert.assertEquals(4, resource.getUnseqFiles().size());
ICrossSpaceMergeFileSelector mergeFileSelector =
@@ -863,8 +869,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
fileWriter.flushAllChunkGroups();
fileWriter.close();
- CrossSpaceCompactionResource compactionResource =
- new CrossSpaceCompactionResource(seqList, unseqList);
+ RewriteCrossSpaceCompactionResource compactionResource =
+ new RewriteCrossSpaceCompactionResource(seqList, unseqList);
RewriteCompactionFileSelector selector =
new RewriteCompactionFileSelector(compactionResource, 500 * 1024 * 1024);
List[] result = selector.select();
@@ -877,8 +883,8 @@ public class RewriteCompactionFileSelectorTest extends MergeTest {
int oldMaxCrossCompactionCandidateFileNum =
IoTDBDescriptor.getInstance().getConfig().getMaxCrossCompactionCandidateFileNum();
IoTDBDescriptor.getInstance().getConfig().setMaxCrossCompactionCandidateFileNum(5);
- CrossSpaceCompactionResource resource =
- new CrossSpaceCompactionResource(seqResources, unseqResources);
+ RewriteCrossSpaceCompactionResource resource =
+ new RewriteCrossSpaceCompactionResource(seqResources, unseqResources);
ICrossSpaceMergeFileSelector mergeFileSelector =
new RewriteCompactionFileSelector(resource, Long.MAX_VALUE);
List[] result = mergeFileSelector.select();
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCrossSpaceCompactionRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCrossSpaceCompactionRecoverTest.java
index 736e6aa762..240e497af3 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCrossSpaceCompactionRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCrossSpaceCompactionRecoverTest.java
@@ -22,9 +22,10 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.task.CompactionRecoverTask;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -45,8 +46,8 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_SOURCE_FILES;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_TARGET_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_SOURCE_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_TARGET_FILES;
import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTest {
@@ -87,7 +88,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
compactionLogger.close();
new CompactionRecoverTask(COMPACTION_TEST_SG, "0", tsFileManager, compactionLogFile, false)
.doCompaction();
@@ -147,7 +148,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
// Target files may not exist
for (int i = 0; i < targetResources.size(); i++) {
if (i < 2) {
@@ -215,7 +216,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
compactionLogger.close();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
new CompactionRecoverTask(COMPACTION_TEST_SG, "0", tsFileManager, compactionLogFile, false)
@@ -276,7 +277,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
seqResources.get(0).getTsFile().delete();
compactionLogger.close();
@@ -338,7 +339,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
compactionLogger.close();
for (int i = 0; i < seqResources.size(); i++) {
@@ -357,7 +358,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
CompactionFileGeneratorUtils.generateMods(deleteMap, unseqResources.get(i), true);
CompactionFileGeneratorUtils.generateMods(deleteMap, unseqResources.get(i), false);
}
- CompactionUtils.combineModsInCompaction(seqResources, unseqResources, targetResources);
+ CompactionUtils.combineModsInCrossCompaction(seqResources, unseqResources, targetResources);
seqResources.get(0).remove();
new CompactionRecoverTask(COMPACTION_TEST_SG, "0", tsFileManager, compactionLogFile, false)
@@ -433,7 +434,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
compactionLogger.close();
CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG);
for (int i = 0; i < seqResources.size(); i++) {
@@ -452,7 +453,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
CompactionFileGeneratorUtils.generateMods(deleteMap, unseqResources.get(i), true);
CompactionFileGeneratorUtils.generateMods(deleteMap, unseqResources.get(i), false);
}
- CompactionUtils.combineModsInCompaction(seqResources, unseqResources, targetResources);
+ CompactionUtils.combineModsInCrossCompaction(seqResources, unseqResources, targetResources);
new CompactionRecoverTask(COMPACTION_TEST_SG, "0", tsFileManager, compactionLogFile, false)
.doCompaction();
@@ -540,7 +541,7 @@ public class RewriteCrossSpaceCompactionRecoverTest extends AbstractCompactionTe
compactionLogger.logFiles(targetResources, STR_TARGET_FILES);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(unseqResources, STR_SOURCE_FILES);
- CompactionUtils.compact(seqResources, unseqResources, targetResources);
+ new ReadPointCompactionPerformer(seqResources, unseqResources, targetResources).perform();
// Target files may not exist
for (int i = 0; i < targetResources.size(); i++) {
if (i < 2) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCrossSpaceCompactionTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCrossSpaceCompactionTest.java
index cc66af6cd1..55cb5bf429 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCrossSpaceCompactionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/cross/RewriteCrossSpaceCompactionTest.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.db.engine.compaction.cross;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.RewriteCrossSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
import org.apache.iotdb.db.engine.storagegroup.DataRegion;
@@ -219,16 +219,15 @@ public class RewriteCrossSpaceCompactionTest extends AbstractCompactionTest {
new TsFileManager(COMPACTION_TEST_SG, "0", STORAGE_GROUP_DIR.getPath());
tsFileManager.addAll(seqResources, true);
tsFileManager.addAll(unseqResources, false);
- RewriteCrossSpaceCompactionTask rewriteCrossSpaceCompactionTask =
- new RewriteCrossSpaceCompactionTask(
- COMPACTION_TEST_SG,
- "0",
+ CrossSpaceCompactionTask task =
+ new CrossSpaceCompactionTask(
0,
tsFileManager,
seqResources,
unseqResources,
+ new ReadPointCompactionPerformer(),
new AtomicInteger(0));
- rewriteCrossSpaceCompactionTask.call();
+ task.call();
for (TsFileResource resource : seqResources) {
resource.resetModFile();
@@ -456,16 +455,15 @@ public class RewriteCrossSpaceCompactionTest extends AbstractCompactionTest {
new TsFileManager(COMPACTION_TEST_SG, "0", STORAGE_GROUP_DIR.getPath());
tsFileManager.addAll(seqResources, true);
tsFileManager.addAll(unseqResources, false);
- RewriteCrossSpaceCompactionTask rewriteCrossSpaceCompactionTask =
- new RewriteCrossSpaceCompactionTask(
- COMPACTION_TEST_SG,
- "0",
+ CrossSpaceCompactionTask task =
+ new CrossSpaceCompactionTask(
0,
tsFileManager,
seqResources,
unseqResources,
+ new ReadPointCompactionPerformer(),
new AtomicInteger(0));
- rewriteCrossSpaceCompactionTask.call();
+ task.call();
for (TsFileResource resource : seqResources) {
Assert.assertFalse(resource.getModFile().exists());
@@ -603,17 +601,16 @@ public class RewriteCrossSpaceCompactionTest extends AbstractCompactionTest {
0,
null);
- RewriteCrossSpaceCompactionTask rewriteCrossSpaceCompactionTask =
- new RewriteCrossSpaceCompactionTask(
- COMPACTION_TEST_SG,
- "0",
+ CrossSpaceCompactionTask task =
+ new CrossSpaceCompactionTask(
0,
vsgp.getTsFileResourceManager(),
seqResources,
unseqResources,
+ new ReadPointCompactionPerformer(),
new AtomicInteger(0));
- rewriteCrossSpaceCompactionTask.setSourceFilesToCompactionCandidate();
- rewriteCrossSpaceCompactionTask.checkValidAndSetMerging();
+ task.setSourceFilesToCompactionCandidate();
+ task.checkValidAndSetMerging();
// delete data in source file during compaction
vsgp.delete(
new PartialPath(
@@ -646,7 +643,7 @@ public class RewriteCrossSpaceCompactionTest extends AbstractCompactionTest {
Assert.assertTrue(resource.getModFile().exists());
Assert.assertEquals(2, resource.getModFile().getModifications().size());
}
- rewriteCrossSpaceCompactionTask.call();
+ task.call();
for (TsFileResource resource : seqResources) {
Assert.assertFalse(resource.getTsFile().exists());
Assert.assertFalse(resource.getModFile().exists());
@@ -713,17 +710,16 @@ public class RewriteCrossSpaceCompactionTest extends AbstractCompactionTest {
0,
null);
- RewriteCrossSpaceCompactionTask rewriteCrossSpaceCompactionTask =
- new RewriteCrossSpaceCompactionTask(
- COMPACTION_TEST_SG,
- "0",
+ CrossSpaceCompactionTask task =
+ new CrossSpaceCompactionTask(
0,
vsgp.getTsFileResourceManager(),
seqResources,
unseqResources,
+ new ReadPointCompactionPerformer(),
new AtomicInteger(0));
- rewriteCrossSpaceCompactionTask.setSourceFilesToCompactionCandidate();
- rewriteCrossSpaceCompactionTask.checkValidAndSetMerging();
+ task.setSourceFilesToCompactionCandidate();
+ task.checkValidAndSetMerging();
// delete data in source file during compaction
vsgp.delete(
new PartialPath(
@@ -768,7 +764,7 @@ public class RewriteCrossSpaceCompactionTest extends AbstractCompactionTest {
Assert.assertTrue(resource.getModFile().exists());
Assert.assertEquals(3, resource.getModFile().getModifications().size());
}
- rewriteCrossSpaceCompactionTask.call();
+ task.call();
for (TsFileResource resource : seqResources) {
Assert.assertFalse(resource.getTsFile().exists());
Assert.assertFalse(resource.getModFile().exists());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionLogTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionLogTest.java
index bac5394f4d..bb6a7b7b0a 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionLogTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionLogTest.java
@@ -35,7 +35,7 @@ import org.junit.Test;
import java.io.File;
import java.io.IOException;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX;
import static org.junit.Assert.assertFalse;
public class InnerCompactionLogTest extends InnerCompactionTest {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionSchedulerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionSchedulerTest.java
index 0208d32789..2a6d2e26ea 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionSchedulerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionSchedulerTest.java
@@ -22,7 +22,6 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
import org.apache.iotdb.db.engine.compaction.CompactionScheduler;
import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.task.FakedInnerSpaceCompactionTaskFactory;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList;
import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
@@ -78,7 +77,8 @@ public class InnerCompactionSchedulerTest extends AbstractCompactionTest {
}
@Test
- public void testFileSelector1() throws IOException, MetadataException, WriteProcessException {
+ public void testFileSelector1()
+ throws IOException, MetadataException, WriteProcessException, InterruptedException {
IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(true);
IoTDBDescriptor.getInstance().getConfig().setEnableUnseqSpaceCompaction(true);
IoTDBDescriptor.getInstance().getConfig().setConcurrentCompactionThread(50);
@@ -93,8 +93,7 @@ public class InnerCompactionSchedulerTest extends AbstractCompactionTest {
TsFileManager tsFileManager = new TsFileManager("testSG", "0", "tmp");
tsFileManager.addAll(seqResources, true);
- CompactionScheduler.tryToSubmitInnerSpaceCompactionTask(
- "testSG", "0", 0L, tsFileManager, true, new FakedInnerSpaceCompactionTaskFactory());
+ CompactionScheduler.tryToSubmitInnerSpaceCompactionTask("testSG", "0", 0L, tsFileManager, true);
CompactionTaskManager.getInstance().submitTaskFromTaskQueue();
try {
Thread.sleep(5000);
@@ -105,7 +104,8 @@ public class InnerCompactionSchedulerTest extends AbstractCompactionTest {
}
@Test
- public void testFileSelector2() throws IOException, MetadataException, WriteProcessException {
+ public void testFileSelector2()
+ throws IOException, MetadataException, WriteProcessException, InterruptedException {
IoTDBDescriptor.getInstance().getConfig().setConcurrentCompactionThread(50);
IoTDBDescriptor.getInstance().getConfig().setMaxInnerCompactionCandidateFileNum(50);
TsFileResourceList tsFileResources = new TsFileResourceList();
@@ -115,8 +115,7 @@ public class InnerCompactionSchedulerTest extends AbstractCompactionTest {
seqResources.get(0).setStatus(TsFileResourceStatus.COMPACTING);
TsFileManager tsFileManager = new TsFileManager("testSG", "0", "tmp");
tsFileManager.addAll(seqResources, true);
- CompactionScheduler.tryToSubmitInnerSpaceCompactionTask(
- "testSG", "0", 0L, tsFileManager, true, new FakedInnerSpaceCompactionTaskFactory());
+ CompactionScheduler.tryToSubmitInnerSpaceCompactionTask("testSG", "0", 0L, tsFileManager, true);
CompactionTaskManager.getInstance().submitTaskFromTaskQueue();
long waitingTime = 0;
@@ -137,7 +136,7 @@ public class InnerCompactionSchedulerTest extends AbstractCompactionTest {
@Test
public void testFileSelectorWithUnclosedFile()
- throws IOException, MetadataException, WriteProcessException {
+ throws IOException, MetadataException, WriteProcessException, InterruptedException {
IoTDBDescriptor.getInstance().getConfig().setConcurrentCompactionThread(50);
IoTDBDescriptor.getInstance().getConfig().setMaxInnerCompactionCandidateFileNum(50);
TsFileResourceList tsFileResources = new TsFileResourceList();
@@ -146,8 +145,7 @@ public class InnerCompactionSchedulerTest extends AbstractCompactionTest {
seqResources.get(3).setStatus(TsFileResourceStatus.UNCLOSED);
TsFileManager tsFileManager = new TsFileManager("testSG", "0", "tmp");
tsFileManager.addAll(seqResources, true);
- CompactionScheduler.tryToSubmitInnerSpaceCompactionTask(
- "testSG", "0", 0L, tsFileManager, true, new FakedInnerSpaceCompactionTaskFactory());
+ CompactionScheduler.tryToSubmitInnerSpaceCompactionTask("testSG", "0", 0L, tsFileManager, true);
CompactionTaskManager.getInstance().submitTaskFromTaskQueue();
long waitingTime = 0;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSeqCompactionTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSeqCompactionTest.java
index 95f2ae0a7f..a327b5e33e 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSeqCompactionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSeqCompactionTest.java
@@ -22,7 +22,8 @@ package org.apache.iotdb.db.engine.compaction.inner;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.cache.ChunkCache;
import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionCheckerUtils;
import org.apache.iotdb.db.engine.compaction.utils.CompactionClearUtils;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
@@ -36,6 +37,7 @@ import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.db.utils.EnvironmentUtils;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -112,7 +114,7 @@ public class InnerSeqCompactionTest {
}
@Test
- public void testDeserializePage() throws MetadataException, IOException {
+ public void testDeserializePage() throws MetadataException, IOException, WriteProcessException {
long chunkSizeLowerBoundInCompaction =
IoTDBDescriptor.getInstance().getConfig().getChunkSizeLowerBoundInCompaction();
@@ -227,10 +229,10 @@ public class InnerSeqCompactionTest {
timeValuePair.getTimestamp() >= 250L
&& timeValuePair.getTimestamp() <= 300L);
}
- InnerSpaceCompactionUtils.compact(targetTsFileResource, sourceResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
- InnerSpaceCompactionUtils.combineModsInCompaction(
- sourceResources, targetTsFileResource);
+ new ReadChunkCompactionPerformer(sourceResources, targetTsFileResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
+ CompactionUtils.combineModsInInnerCompaction(sourceResources, targetTsFileResource);
List<TsFileResource> targetTsFileResources = new ArrayList<>();
targetTsFileResources.add(targetTsFileResource);
// check data
@@ -334,7 +336,7 @@ public class InnerSeqCompactionTest {
}
}
}
- } catch (InterruptedException e) {
+ } catch (InterruptedException | StorageEngineException e) {
e.printStackTrace();
} finally {
IoTDBDescriptor.getInstance()
@@ -347,7 +349,9 @@ public class InnerSeqCompactionTest {
}
@Test
- public void testAppendPage() throws IOException, MetadataException, InterruptedException {
+ public void testAppendPage()
+ throws IOException, MetadataException, InterruptedException, StorageEngineException,
+ WriteProcessException {
for (int toMergeFileNum : toMergeFileNums) {
for (CompactionTimeseriesType compactionTimeseriesType : compactionTimeseriesTypes) {
@@ -452,10 +456,10 @@ public class InnerSeqCompactionTest {
timeValuePair ->
timeValuePair.getTimestamp() >= 250L && timeValuePair.getTimestamp() <= 300L);
}
- InnerSpaceCompactionUtils.compact(targetTsFileResource, toMergeResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
- InnerSpaceCompactionUtils.combineModsInCompaction(
- toMergeResources, targetTsFileResource);
+ new ReadChunkCompactionPerformer(toMergeResources, targetTsFileResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
+ CompactionUtils.combineModsInInnerCompaction(toMergeResources, targetTsFileResource);
List<TsFileResource> targetTsFileResources = new ArrayList<>();
targetTsFileResources.add(targetTsFileResource);
CompactionCheckerUtils.checkDataAndResource(sourceData, targetTsFileResources);
@@ -609,7 +613,9 @@ public class InnerSeqCompactionTest {
}
@Test
- public void testAppendChunk() throws IOException, IllegalPathException, MetadataException {
+ public void testAppendChunk()
+ throws IOException, IllegalPathException, MetadataException, StorageEngineException,
+ WriteProcessException {
long prevChunkPointNumLowerBoundInCompaction =
IoTDBDescriptor.getInstance().getConfig().getChunkPointNumLowerBoundInCompaction();
IoTDBDescriptor.getInstance().getConfig().setChunkPointNumLowerBoundInCompaction(1);
@@ -727,10 +733,10 @@ public class InnerSeqCompactionTest {
timeValuePair.getTimestamp() >= 250L
&& timeValuePair.getTimestamp() <= 300L);
}
- InnerSpaceCompactionUtils.compact(targetTsFileResource, toMergeResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
- InnerSpaceCompactionUtils.combineModsInCompaction(
- toMergeResources, targetTsFileResource);
+ new ReadChunkCompactionPerformer(toMergeResources, targetTsFileResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
+ CompactionUtils.combineModsInInnerCompaction(toMergeResources, targetTsFileResource);
List<TsFileResource> targetTsFileResources = new ArrayList<>();
targetTsFileResources.add(targetTsFileResource);
CompactionCheckerUtils.checkDataAndResource(sourceData, targetTsFileResources);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionExceptionTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionExceptionTest.java
index 541e18309c..1b4ad64e6c 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionExceptionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionExceptionTest.java
@@ -19,10 +19,11 @@
package org.apache.iotdb.db.engine.compaction.inner;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
-import org.apache.iotdb.db.engine.compaction.task.CompactionExceptionHandler;
+import org.apache.iotdb.db.engine.compaction.CompactionExceptionHandler;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
@@ -63,8 +64,9 @@ public class InnerSpaceCompactionExceptionTest extends AbstractInnerSpaceCompact
compactionLogger.logFiles(seqResources, CompactionLogger.STR_SOURCE_FILES);
compactionLogger.logFiles(
Collections.singletonList(targetResource), CompactionLogger.STR_TARGET_FILES);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
try (FileOutputStream os = new FileOutputStream(targetResource.getTsFile(), true);
FileChannel channel = os.getChannel()) {
channel.truncate(targetResource.getTsFileSize() - 10);
@@ -111,8 +113,9 @@ public class InnerSpaceCompactionExceptionTest extends AbstractInnerSpaceCompact
compactionLogger.logFiles(seqResources, CompactionLogger.STR_SOURCE_FILES);
compactionLogger.logFiles(
Collections.singletonList(targetResource), CompactionLogger.STR_TARGET_FILES);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
compactionLogger.close();
CompactionExceptionHandler.handleException(
COMPACTION_TEST_SG,
@@ -155,8 +158,9 @@ public class InnerSpaceCompactionExceptionTest extends AbstractInnerSpaceCompact
compactionLogger.logFiles(seqResources, CompactionLogger.STR_SOURCE_FILES);
compactionLogger.logFiles(
Collections.singletonList(targetResource), CompactionLogger.STR_TARGET_FILES);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
for (TsFileResource resource : seqResources) {
tsFileManager.getSequenceListByTimePartition(0).remove(resource);
}
@@ -210,8 +214,9 @@ public class InnerSpaceCompactionExceptionTest extends AbstractInnerSpaceCompact
compactionLogger.logFiles(seqResources, CompactionLogger.STR_SOURCE_FILES);
compactionLogger.logFiles(
Collections.singletonList(targetResource), CompactionLogger.STR_TARGET_FILES);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
seqResources.get(0).remove();
try (FileOutputStream os = new FileOutputStream(targetResource.getTsFile(), true);
FileChannel channel = os.getChannel()) {
@@ -263,8 +268,9 @@ public class InnerSpaceCompactionExceptionTest extends AbstractInnerSpaceCompact
compactionLogger.logFiles(seqResources, CompactionLogger.STR_SOURCE_FILES);
compactionLogger.logFiles(
Collections.singletonList(targetResource), CompactionLogger.STR_TARGET_FILES);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
for (int i = 0; i < seqResources.size(); i++) {
Map<String, Pair<Long, Long>> deleteMap = new HashMap<>();
deleteMap.put(
@@ -272,7 +278,7 @@ public class InnerSpaceCompactionExceptionTest extends AbstractInnerSpaceCompact
new Pair<>(i * ptNum, i * ptNum + 10));
CompactionFileGeneratorUtils.generateMods(deleteMap, seqResources.get(i), true);
}
- InnerSpaceCompactionUtils.combineModsInCompaction(seqResources, targetResource);
+ CompactionUtils.combineModsInInnerCompaction(seqResources, targetResource);
seqResources.get(0).remove();
compactionLogger.close();
@@ -337,8 +343,9 @@ public class InnerSpaceCompactionExceptionTest extends AbstractInnerSpaceCompact
new Pair<>(i * ptNum, i * ptNum + 10));
CompactionFileGeneratorUtils.generateMods(deleteMap, seqResources.get(i), false);
}
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
seqResources.get(0).remove();
compactionLogger.close();
@@ -395,8 +402,9 @@ public class InnerSpaceCompactionExceptionTest extends AbstractInnerSpaceCompact
new Pair<>(i * ptNum, i * ptNum + 5));
CompactionFileGeneratorUtils.generateMods(deleteMap, seqResources.get(i), false);
}
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
for (int i = 0; i < seqResources.size(); i++) {
Map<String, Pair<Long, Long>> deleteMap = new HashMap<>();
deleteMap.put(
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerUnseqCompactionTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerUnseqCompactionTest.java
index 42419d9411..5b36608ece 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerUnseqCompactionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerUnseqCompactionTest.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.db.engine.compaction.inner;
import org.apache.iotdb.db.engine.cache.ChunkCache;
import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
import org.apache.iotdb.db.engine.compaction.CompactionUtils;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadPointCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionCheckerUtils;
import org.apache.iotdb.db.engine.compaction.utils.CompactionClearUtils;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
@@ -366,14 +366,14 @@ public class InnerUnseqCompactionTest {
timeValuePair.getTimestamp() >= 250L
&& timeValuePair.getTimestamp() <= 300L);
}
- CompactionUtils.compact(
- Collections.emptyList(),
- toMergeResources,
- Collections.singletonList(targetTsFileResource));
+ new ReadPointCompactionPerformer(
+ Collections.emptyList(),
+ toMergeResources,
+ Collections.singletonList(targetTsFileResource))
+ .perform();
CompactionUtils.moveTargetFile(
Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
- InnerSpaceCompactionUtils.combineModsInCompaction(
- toMergeResources, targetTsFileResource);
+ CompactionUtils.combineModsInInnerCompaction(toMergeResources, targetTsFileResource);
List<TsFileResource> targetTsFileResources = new ArrayList<>();
targetTsFileResources.add(targetTsFileResource);
CompactionCheckerUtils.checkDataAndResource(sourceData, targetTsFileResources);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsAlignedTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerAlignedTest.java
similarity index 97%
rename from server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsAlignedTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerAlignedTest.java
index 8541f39066..819a816942 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsAlignedTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerAlignedTest.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.db.constant.TestConstant;
import org.apache.iotdb.db.engine.cache.ChunkCache;
import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
import org.apache.iotdb.db.engine.compaction.TestUtilsForAlignedSeries;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionCheckerUtils;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
@@ -52,7 +52,7 @@ import java.util.List;
import java.util.Map;
import java.util.Random;
-public class InnerSpaceCompactionUtilsAlignedTest {
+public class ReadChunkCompactionPerformerAlignedTest {
private static final String storageGroup = "root.testAlignedCompaction";
private static File dataDirectory =
new File(
@@ -133,7 +133,7 @@ public class InnerSpaceCompactionUtilsAlignedTest {
Map<PartialPath, List<TimeValuePair>> originData =
CompactionCheckerUtils.getDataByQuery(
fullPaths, iMeasurementSchemas, resources, new ArrayList<>());
- InnerSpaceCompactionUtils.compact(targetResource, resources);
+ new ReadChunkCompactionPerformer(resources, targetResource).perform();
Map<PartialPath, List<TimeValuePair>> compactedData =
CompactionCheckerUtils.getDataByQuery(
fullPaths,
@@ -201,7 +201,7 @@ public class InnerSpaceCompactionUtilsAlignedTest {
Map<PartialPath, List<TimeValuePair>> originData =
CompactionCheckerUtils.getDataByQuery(
fullPaths, iMeasurementSchemas, resources, new ArrayList<>());
- InnerSpaceCompactionUtils.compact(targetResource, resources);
+ new ReadChunkCompactionPerformer(resources, targetResource).perform();
Map<PartialPath, List<TimeValuePair>> compactedData =
CompactionCheckerUtils.getDataByQuery(
fullPaths,
@@ -263,7 +263,7 @@ public class InnerSpaceCompactionUtilsAlignedTest {
Map<PartialPath, List<TimeValuePair>> originData =
CompactionCheckerUtils.getDataByQuery(
fullPaths, iMeasurementSchemas, resources, new ArrayList<>());
- InnerSpaceCompactionUtils.compact(targetResource, resources);
+ new ReadChunkCompactionPerformer(resources, targetResource).perform();
Map<PartialPath, List<TimeValuePair>> compactedData =
CompactionCheckerUtils.getDataByQuery(
fullPaths,
@@ -328,7 +328,7 @@ public class InnerSpaceCompactionUtilsAlignedTest {
Map<PartialPath, List<TimeValuePair>> originData =
CompactionCheckerUtils.getDataByQuery(
fullPaths, iMeasurementSchemas, resources, new ArrayList<>());
- InnerSpaceCompactionUtils.compact(targetResource, resources);
+ new ReadChunkCompactionPerformer(resources, targetResource).perform();
Map<PartialPath, List<TimeValuePair>> compactedData =
CompactionCheckerUtils.getDataByQuery(
fullPaths,
@@ -391,7 +391,7 @@ public class InnerSpaceCompactionUtilsAlignedTest {
Map<PartialPath, List<TimeValuePair>> originData =
CompactionCheckerUtils.getDataByQuery(
fullPaths, iMeasurementSchemas, resources, new ArrayList<>());
- InnerSpaceCompactionUtils.compact(targetResource, resources);
+ new ReadChunkCompactionPerformer(resources, targetResource).perform();
Map<PartialPath, List<TimeValuePair>> compactedData =
CompactionCheckerUtils.getDataByQuery(
fullPaths,
@@ -456,7 +456,7 @@ public class InnerSpaceCompactionUtilsAlignedTest {
Map<PartialPath, List<TimeValuePair>> originData =
CompactionCheckerUtils.getDataByQuery(
fullPaths, iMeasurementSchemas, resources, new ArrayList<>());
- InnerSpaceCompactionUtils.compact(targetResource, resources);
+ new ReadChunkCompactionPerformer(resources, targetResource).perform();
Map<PartialPath, List<TimeValuePair>> compactedData =
CompactionCheckerUtils.getDataByQuery(
fullPaths,
@@ -522,7 +522,7 @@ public class InnerSpaceCompactionUtilsAlignedTest {
Map<PartialPath, List<TimeValuePair>> originData =
CompactionCheckerUtils.getDataByQuery(
fullPaths, iMeasurementSchemas, resources, new ArrayList<>());
- InnerSpaceCompactionUtils.compact(targetResource, resources);
+ new ReadChunkCompactionPerformer(resources, targetResource).perform();
Map<PartialPath, List<TimeValuePair>> compactedData =
CompactionCheckerUtils.getDataByQuery(
fullPaths,
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsNoAlignedTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerNoAlignedTest.java
similarity index 98%
rename from server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsNoAlignedTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerNoAlignedTest.java
index e63f53094f..ebc9522ed0 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsNoAlignedTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerNoAlignedTest.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.constant.TestConstant;
import org.apache.iotdb.db.engine.cache.ChunkCache;
import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionCheckerUtils;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
@@ -60,7 +60,7 @@ import java.util.Set;
* This class is used to test InnerSpaceCompactionUtils.compact. Notice, it just tests not aligned
* timeseries.
*/
-public class InnerSpaceCompactionUtilsNoAlignedTest {
+public class ReadChunkCompactionPerformerNoAlignedTest {
private final String storageGroup = "root.compactionTest";
private final String[] devices = new String[] {"device0", "device1", "device2", "device3"};
private PartialPath[] devicePath = new PartialPath[devices.length];
@@ -201,7 +201,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
long[] points = new long[fileNum];
for (int i = 1; i <= fileNum; i++) {
@@ -291,7 +291,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
// outer list is a chunk, inner list is point num in each page
for (String path : fullPathSet) {
@@ -383,7 +383,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
// outer list is a chunk, inner list is point num in each page
for (String path : fullPathSet) {
@@ -462,7 +462,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
// outer list is a chunk, inner list is point num in each page
List<List<Long>> chunkPointsArray = new ArrayList<>();
@@ -571,7 +571,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
// outer list is a chunk, inner list is point num in each page
for (String path : fullPathSet) {
@@ -650,7 +650,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
// outer list is a chunk, inner list is point num in each page
List<List<Long>> chunkPointsArray = new ArrayList<>();
@@ -732,7 +732,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
// outer list is a chunk, inner list is point num in each page
List<List<Long>> chunkPointsArray = new ArrayList<>();
@@ -808,7 +808,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
// outer list is a chunk, inner list is point num in each page
for (String path : fullPathSet) {
@@ -886,7 +886,7 @@ public class InnerSpaceCompactionUtilsNoAlignedTest {
tsFileName.getVersion(),
tsFileName.getInnerCompactionCnt() + 1,
tsFileName.getCrossCompactionCnt())));
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
Map<String, List<List<Long>>> chunkPagePointsNumMerged = new HashMap<>();
// outer list is a chunk, inner list is point num in each page
for (String path : fullPathSet) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsOldTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerOldTest.java
similarity index 86%
rename from server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsOldTest.java
rename to server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerOldTest.java
index 234cb889a9..74699ec670 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSpaceCompactionUtilsOldTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/ReadChunkCompactionPerformerOldTest.java
@@ -21,11 +21,13 @@ package org.apache.iotdb.db.engine.compaction.inner;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.exception.StorageEngineException;
import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
import org.apache.iotdb.tsfile.read.TsFileReader;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -50,7 +52,7 @@ import static org.junit.Assert.assertTrue;
* current compaction. However, due to this test's strong coupling with an older version of
* compaction, we may remove it in the future.
*/
-public class InnerSpaceCompactionUtilsOldTest extends InnerCompactionTest {
+public class ReadChunkCompactionPerformerOldTest extends InnerCompactionTest {
File tempSGDir;
@@ -72,7 +74,9 @@ public class InnerSpaceCompactionUtilsOldTest extends InnerCompactionTest {
}
@Test
- public void testCompact() throws IOException, MetadataException, InterruptedException {
+ public void testCompact()
+ throws IOException, MetadataException, InterruptedException, StorageEngineException,
+ WriteProcessException {
TsFileResource targetTsFileResource =
new TsFileResource(
new File(
@@ -105,8 +109,9 @@ public class InnerSpaceCompactionUtilsOldTest extends InnerCompactionTest {
new CompactionLogger(
new File(targetTsFileResource.getTsFilePath().concat(".compaction.log")));
sizeTieredCompactionLogger.logFiles(seqResources, CompactionLogger.STR_SOURCE_FILES);
- InnerSpaceCompactionUtils.compact(targetTsFileResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetTsFileResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
sizeTieredCompactionLogger.close();
Path path = new Path(deviceIds[0], measurementSchemas[0].getMeasurementId());
try (TsFileSequenceReader reader =
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionRecoverTest.java
index 277b916a34..ae6104830e 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionRecoverTest.java
@@ -22,12 +22,13 @@ package org.apache.iotdb.db.engine.compaction.inner.sizetiered;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.engine.cache.ChunkCache;
import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
import org.apache.iotdb.db.engine.compaction.inner.AbstractInnerSpaceCompactionTest;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.task.CompactionRecoverTask;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
@@ -64,9 +65,9 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_SOURCE_FILES;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_TARGET_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_SOURCE_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_TARGET_FILES;
import static org.junit.Assert.assertEquals;
public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactionTest {
@@ -143,10 +144,12 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
compactionLogger.logFiles(tmpSeqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetTsFileResource), STR_TARGET_FILES);
deleteFileIfExists(targetTsFileResource.getTsFile());
- InnerSpaceCompactionUtils.compact(
- targetTsFileResource, new ArrayList<>(seqResources.subList(0, 3)));
+ new ReadChunkCompactionPerformer(
+ new ArrayList<>(seqResources.subList(0, 3)), targetTsFileResource)
+ .perform();
compactionLogger.close();
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
BufferedReader logReader = new BufferedReader(new FileReader(compactionLogFile));
List<String> logs = new ArrayList<>();
String line;
@@ -263,8 +266,9 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
compactionLogger.logFiles(tmpSeqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetTsFileResource), STR_TARGET_FILES);
deleteFileIfExists(targetTsFileResource.getTsFile());
- InnerSpaceCompactionUtils.compact(
- targetTsFileResource, new ArrayList<>(seqResources.subList(0, 3)));
+ new ReadChunkCompactionPerformer(
+ new ArrayList<>(seqResources.subList(0, 3)), targetTsFileResource)
+ .perform();
compactionLogger.close();
new CompactionRecoverTask(COMPACTION_TEST_SG, "0", tsFileManager, compactionLogFile, true)
.doCompaction();
@@ -373,8 +377,9 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
compactionLogger.logFiles(tmpSeqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetTsFileResource), STR_TARGET_FILES);
deleteFileIfExists(targetTsFileResource.getTsFile());
- InnerSpaceCompactionUtils.compact(
- targetTsFileResource, new ArrayList<>(seqResources.subList(0, 3)));
+ new ReadChunkCompactionPerformer(
+ new ArrayList<>(seqResources.subList(0, 3)), targetTsFileResource)
+ .perform();
// target file may not exist
targetTsFileResource.remove();
compactionLogger.close();
@@ -485,9 +490,11 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
compactionLogger.logFiles(tmpSeqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetTsFileResource), STR_TARGET_FILES);
deleteFileIfExists(targetTsFileResource.getTsFile());
- InnerSpaceCompactionUtils.compact(
- targetTsFileResource, new ArrayList<>(seqResources.subList(0, 3)));
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(
+ new ArrayList<>(seqResources.subList(0, 3)), targetTsFileResource)
+ .perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
// delete one source file
seqResources.get(0).remove();
compactionLogger.close();
@@ -558,8 +565,9 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
CompactionLogger compactionLogger = new CompactionLogger(logFile);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
for (int i = 0; i < seqResources.size(); i++) {
Map<String, Pair<Long, Long>> deleteMap = new HashMap<>();
deleteMap.put(
@@ -568,7 +576,7 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
CompactionFileGeneratorUtils.generateMods(deleteMap, seqResources.get(i), true);
CompactionFileGeneratorUtils.generateMods(deleteMap, seqResources.get(i), false);
}
- InnerSpaceCompactionUtils.combineModsInCompaction(seqResources, targetResource);
+ CompactionUtils.combineModsInInnerCompaction(seqResources, targetResource);
compactionLogger.close();
new CompactionRecoverTask(COMPACTION_TEST_SG, "0", tsFileManager, logFile, true).doCompaction();
@@ -626,7 +634,7 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
CompactionLogger compactionLogger = new CompactionLogger(logFile);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
// target file may not exist
targetResource.remove();
for (int i = 0; i < seqResources.size(); i++) {
@@ -697,8 +705,9 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
CompactionLogger compactionLogger = new CompactionLogger(logFile);
compactionLogger.logFiles(seqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
for (int i = 0; i < seqResources.size(); i++) {
Map<String, Pair<Long, Long>> deleteMap = new HashMap<>();
deleteMap.put(
@@ -707,7 +716,7 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
CompactionFileGeneratorUtils.generateMods(deleteMap, seqResources.get(i), true);
CompactionFileGeneratorUtils.generateMods(deleteMap, seqResources.get(i), false);
}
- InnerSpaceCompactionUtils.combineModsInCompaction(seqResources, targetResource);
+ CompactionUtils.combineModsInInnerCompaction(seqResources, targetResource);
seqResources.get(0).remove();
compactionLogger.close();
@@ -808,10 +817,12 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
compactionLogger.logFiles(tmpSeqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetTsFileResource), STR_TARGET_FILES);
deleteFileIfExists(targetTsFileResource.getTsFile());
- InnerSpaceCompactionUtils.compact(
- targetTsFileResource, new ArrayList<>(seqResources.subList(0, 3)));
+ new ReadChunkCompactionPerformer(
+ new ArrayList<>(seqResources.subList(0, 3)), targetTsFileResource)
+ .perform();
compactionLogger.close();
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
tsFileManager.add(targetTsFileResource, true);
new CompactionRecoverTask(COMPACTION_TEST_SG, "0", tsFileManager, compactionLogFile, true)
.doCompaction();
@@ -905,9 +916,11 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
compactionLogger.logFiles(tmpSeqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetTsFileResource), STR_TARGET_FILES);
deleteFileIfExists(targetTsFileResource.getTsFile());
- InnerSpaceCompactionUtils.compact(
- targetTsFileResource, new ArrayList<>(seqResources.subList(0, 3)));
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(
+ new ArrayList<>(seqResources.subList(0, 3)), targetTsFileResource)
+ .perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
compactionLogger.close();
for (TsFileResource resource : new ArrayList<>(seqResources.subList(0, 3))) {
deleteFileIfExists(resource.getTsFile());
@@ -1005,10 +1018,12 @@ public class SizeTieredCompactionRecoverTest extends AbstractInnerSpaceCompactio
compactionLogger.logFiles(tmpSeqResources, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetTsFileResource), STR_TARGET_FILES);
deleteFileIfExists(targetTsFileResource.getTsFile());
- InnerSpaceCompactionUtils.compact(
- targetTsFileResource, new ArrayList<>(seqResources.subList(0, 3)));
+ new ReadChunkCompactionPerformer(
+ new ArrayList<>(seqResources.subList(0, 3)), targetTsFileResource)
+ .perform();
compactionLogger.close();
- InnerSpaceCompactionUtils.moveTargetFile(targetTsFileResource, COMPACTION_TEST_SG);
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetTsFileResource), true, COMPACTION_TEST_SG);
deleteFileIfExists(compactionLogFile);
for (TsFileResource resource : new ArrayList<>(seqResources.subList(0, 3))) {
tsFileManager.remove(resource, true);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/RewriteCrossSpaceCompactionRecoverCompatibleTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/RewriteCrossSpaceCompactionRecoverCompatibleTest.java
index 99357e64a0..257122e9b4 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/RewriteCrossSpaceCompactionRecoverCompatibleTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/RewriteCrossSpaceCompactionRecoverCompatibleTest.java
@@ -20,9 +20,9 @@ package org.apache.iotdb.db.engine.compaction.recover;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
import org.apache.iotdb.db.engine.compaction.task.CompactionRecoverTask;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.modification.Deletion;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/SizeTieredCompactionRecoverCompatibleTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/SizeTieredCompactionRecoverCompatibleTest.java
index ad5877da52..165958f1d3 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/SizeTieredCompactionRecoverCompatibleTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/SizeTieredCompactionRecoverCompatibleTest.java
@@ -20,10 +20,11 @@
package org.apache.iotdb.db.engine.compaction.recover;
import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.task.CompactionRecoverTask;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -42,6 +43,7 @@ import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.io.RandomAccessFile;
+import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@@ -67,7 +69,7 @@ public class SizeTieredCompactionRecoverCompatibleTest extends AbstractCompactio
registerTimeseriesInMManger(2, 3, false);
TsFileResource targetResource =
TsFileNameGenerator.getInnerCompactionTargetFileResource(seqResources, true);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
RandomAccessFile targetFile = new RandomAccessFile(targetResource.getTsFile(), "rw");
long fileLength = targetFile.length();
targetFile.getChannel().truncate(fileLength - 20);
@@ -124,8 +126,9 @@ public class SizeTieredCompactionRecoverCompatibleTest extends AbstractCompactio
registerTimeseriesInMManger(2, 3, false);
TsFileResource targetResource =
TsFileNameGenerator.getInnerCompactionTargetFileResource(seqResources, true);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, "root.compactionTest");
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, "root.compactionTest");
// first source file does not exist
seqResources.get(0).delete();
@@ -180,7 +183,7 @@ public class SizeTieredCompactionRecoverCompatibleTest extends AbstractCompactio
registerTimeseriesInMManger(2, 3, false);
TsFileResource targetResource =
TsFileNameGenerator.getInnerCompactionTargetFileResource(seqResources, true);
- InnerSpaceCompactionUtils.compact(targetResource, seqResources);
+ new ReadChunkCompactionPerformer(seqResources, targetResource).perform();
RandomAccessFile targetFile = new RandomAccessFile(targetResource.getTsFile(), "rw");
long fileLength = targetFile.length();
targetFile.getChannel().truncate(fileLength - 20);
@@ -217,8 +220,9 @@ public class SizeTieredCompactionRecoverCompatibleTest extends AbstractCompactio
registerTimeseriesInMManger(2, 3, false);
TsFileResource targetResource =
TsFileNameGenerator.getInnerCompactionTargetFileResource(unseqResources, true);
- InnerSpaceCompactionUtils.compact(targetResource, unseqResources);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, "root.compactionTest");
+ new ReadChunkCompactionPerformer(unseqResources, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, "root.compactionTest");
// first source file does not exist
unseqResources.get(0).delete();
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/SizeTieredCompactionRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/SizeTieredCompactionRecoverTest.java
index e25d8b1b82..5dd5abe88b 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/SizeTieredCompactionRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/recover/SizeTieredCompactionRecoverTest.java
@@ -22,11 +22,12 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.compaction.inner.utils.InnerSpaceCompactionUtils;
+import org.apache.iotdb.db.engine.compaction.CompactionUtils;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.task.CompactionRecoverTask;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
import org.apache.iotdb.db.engine.storagegroup.DataRegion;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
@@ -61,8 +62,8 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_SOURCE_FILES;
-import static org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger.STR_TARGET_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_SOURCE_FILES;
+import static org.apache.iotdb.db.engine.compaction.log.CompactionLogger.STR_TARGET_FILES;
public class SizeTieredCompactionRecoverTest {
@@ -234,8 +235,9 @@ public class SizeTieredCompactionRecoverTest {
compactionLogger.logFiles(sourceFiles, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
compactionLogger.close();
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
CompactionRecoverTask recoverTask =
new CompactionRecoverTask(
COMPACTION_TEST_SG, "0", tsFileManager, new File(logFilePath), true);
@@ -284,8 +286,9 @@ public class SizeTieredCompactionRecoverTest {
compactionLogger.logFiles(sourceFiles, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
compactionLogger.close();
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
FileOutputStream targetStream = new FileOutputStream(targetResource.getTsFile(), true);
FileChannel channel = targetStream.getChannel();
channel.truncate(targetResource.getTsFile().length() - 100);
@@ -339,8 +342,9 @@ public class SizeTieredCompactionRecoverTest {
logger.logFiles(sourceFiles, CompactionLogger.STR_SOURCE_FILES);
logger.logFiles(Collections.singletonList(targetResource), CompactionLogger.STR_TARGET_FILES);
logger.close();
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
CompactionRecoverTask recoverTask =
new CompactionRecoverTask(
COMPACTION_TEST_SG, "0", tsFileManager, new File(logFilePath), true);
@@ -389,8 +393,9 @@ public class SizeTieredCompactionRecoverTest {
compactionLogger.logFiles(sourceFiles, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
compactionLogger.close();
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
FileOutputStream targetStream = new FileOutputStream(targetResource.getTsFile(), true);
FileChannel channel = targetStream.getChannel();
channel.truncate(targetResource.getTsFile().length() - 100);
@@ -447,8 +452,9 @@ public class SizeTieredCompactionRecoverTest {
compactionLogger.logFiles(sourceFiles, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
compactionLogger.close();
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
long sizeOfTargetFile = targetResource.getTsFileSize();
FileUtils.moveDirectory(
new File(TestConstant.BASE_OUTPUT_PATH + File.separator + "data"),
@@ -543,8 +549,9 @@ public class SizeTieredCompactionRecoverTest {
compactionLogger.logFiles(sourceFiles, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
compactionLogger.close();
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
FileOutputStream targetStream = new FileOutputStream(targetResource.getTsFile(), true);
FileChannel channel = targetStream.getChannel();
channel.truncate(targetResource.getTsFile().length() - 100);
@@ -636,8 +643,9 @@ public class SizeTieredCompactionRecoverTest {
compactionLogger.logFiles(sourceFiles, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
compactionLogger.close();
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
long sizeOfTargetFile = targetResource.getTsFileSize();
FileUtils.moveDirectory(
new File(TestConstant.BASE_OUTPUT_PATH + File.separator + "data"),
@@ -732,8 +740,9 @@ public class SizeTieredCompactionRecoverTest {
compactionLogger.logFiles(sourceFiles, STR_SOURCE_FILES);
compactionLogger.logFiles(Collections.singletonList(targetResource), STR_TARGET_FILES);
compactionLogger.close();
- InnerSpaceCompactionUtils.compact(targetResource, sourceFiles);
- InnerSpaceCompactionUtils.moveTargetFile(targetResource, COMPACTION_TEST_SG);
+ new ReadChunkCompactionPerformer(sourceFiles, targetResource).perform();
+ CompactionUtils.moveTargetFile(
+ Collections.singletonList(targetResource), true, COMPACTION_TEST_SG);
FileOutputStream targetStream = new FileOutputStream(targetResource.getTsFile(), true);
FileChannel channel = targetStream.getChannel();
channel.truncate(targetResource.getTsFile().length() - 100);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedCrossSpaceCompactionTask.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedCrossSpaceCompactionTask.java
deleted file mode 100644
index 315ffc51e3..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedCrossSpaceCompactionTask.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.db.engine.compaction.task;
-
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.cross.rewrite.task.RewriteCrossSpaceCompactionTask;
-import org.apache.iotdb.db.engine.storagegroup.FakedTsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-
-import java.util.List;
-
-public class FakedCrossSpaceCompactionTask extends RewriteCrossSpaceCompactionTask {
- public FakedCrossSpaceCompactionTask(
- String logicalStorageGroupName,
- String virtualStorageGroupName,
- long timePartitionId,
- TsFileManager tsFileManager,
- List<TsFileResource> selectedSeqTsFileResourceList,
- List<TsFileResource> selectedUnSeqTsFileResourceList) {
- super(
- logicalStorageGroupName,
- virtualStorageGroupName,
- timePartitionId,
- tsFileManager,
- selectedSeqTsFileResourceList,
- selectedUnSeqTsFileResourceList,
- CompactionTaskManager.currentTaskNum);
- }
-
- @Override
- protected void doCompaction() {
- long totalUnseqFileSize = 0;
- for (TsFileResource resource : selectedUnSeqTsFileResourceList) {
- totalUnseqFileSize += resource.getTsFileSize();
- }
- long avgSizeAddToSeqFile = totalUnseqFileSize / selectedSeqTsFileResourceList.size();
- for (TsFileResource resource : selectedSeqTsFileResourceList) {
- ((FakedTsFileResource) resource)
- .setTsFileSize(resource.getTsFileSize() + avgSizeAddToSeqFile);
- }
- selectedSeqTsFileResourceList.clear();
- selectedUnSeqTsFileResourceList.clear();
- }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedInnerSpaceCompactionTask.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedInnerSpaceCompactionTask.java
deleted file mode 100644
index 8eafa19e75..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedInnerSpaceCompactionTask.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.compaction.task;
-
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.inner.sizetiered.SizeTieredCompactionTask;
-import org.apache.iotdb.db.engine.storagegroup.FakedTsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class FakedInnerSpaceCompactionTask extends SizeTieredCompactionTask {
-
- public FakedInnerSpaceCompactionTask(
- String logicalStorageGroupName,
- String virtualStorageGroupName,
- long timePartition,
- TsFileManager tsFileManager,
- List<TsFileResource> selectedTsFileResourceList,
- boolean sequence,
- AtomicInteger currentTaskNum) {
- super(
- logicalStorageGroupName,
- virtualStorageGroupName,
- timePartition,
- tsFileManager,
- selectedTsFileResourceList,
- sequence,
- currentTaskNum);
- }
-
- @Override
- protected void doCompaction() throws IOException {
- try {
- TsFileNameGenerator.TsFileName name =
- TsFileNameGenerator.getTsFileName(
- selectedTsFileResourceList.get(0).getTsFile().getName());
- String newName =
- TsFileNameGenerator.generateNewTsFileName(
- name.getTime(),
- name.getVersion(),
- name.getInnerCompactionCnt() + 1,
- name.getCrossCompactionCnt());
- FakedTsFileResource targetTsFileResource = new FakedTsFileResource(0, newName);
- long targetFileSize = 0;
- for (TsFileResource resource : selectedTsFileResourceList) {
- targetFileSize += resource.getTsFileSize();
- }
- targetTsFileResource.setTsFileSize(targetFileSize);
- this.tsFileResourceList.insertBefore(selectedTsFileResourceList.get(0), targetTsFileResource);
- for (TsFileResource tsFileResource : selectedTsFileResourceList) {
- this.tsFileResourceList.remove(tsFileResource);
- }
- } finally {
- CompactionTaskManager.getInstance().removeRunningTaskFromList(this);
- }
- }
-
- @Override
- public boolean equalsOtherTask(AbstractCompactionTask otherTask) {
- if (otherTask instanceof FakedInnerSpaceCompactionTask) {
- FakedInnerSpaceCompactionTask fakedOtherTask = (FakedInnerSpaceCompactionTask) otherTask;
- return this.selectedTsFileResourceList.equals(fakedOtherTask.selectedTsFileResourceList);
- }
- return false;
- }
-
- @Override
- public boolean checkValidAndSetMerging() {
- for (TsFileResource resource : selectedTsFileResourceList) {
- if (resource.isCompacting() || !resource.isClosed()) {
- return false;
- }
- }
- return true;
- }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedInnerSpaceCompactionTaskFactory.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedInnerSpaceCompactionTaskFactory.java
deleted file mode 100644
index a54fb6dffe..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/task/FakedInnerSpaceCompactionTaskFactory.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.compaction.task;
-
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTaskFactory;
-import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList;
-
-import java.util.List;
-
-public class FakedInnerSpaceCompactionTaskFactory extends InnerSpaceCompactionTaskFactory {
-
- public FakedInnerSpaceCompactionTaskFactory() {}
-
- public AbstractCompactionTask createTask(
- String logicalStorageGroupName,
- String virtualStorageGroupName,
- long timePartition,
- TsFileManager tsFileManager,
- TsFileResourceList tsFileResourceList,
- List<TsFileResource> selectedTsFileResourceList,
- boolean sequence) {
- return new FakedInnerSpaceCompactionTask(
- logicalStorageGroupName,
- virtualStorageGroupName,
- timePartition,
- tsFileManager,
- selectedTsFileResourceList,
- sequence,
- CompactionTaskManager.currentTaskNum);
- }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionClearUtils.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionClearUtils.java
index 0280c045ea..dc3bb0cca1 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionClearUtils.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionClearUtils.java
@@ -24,7 +24,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.cache.BloomFilterCache;
import org.apache.iotdb.db.engine.cache.ChunkCache;
import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.query.control.FileReaderManager;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionConfigRestorer.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionConfigRestorer.java
index 014ad1afd5..8820d198d7 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionConfigRestorer.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/utils/CompactionConfigRestorer.java
@@ -22,15 +22,16 @@ package org.apache.iotdb.db.engine.compaction.utils;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.constant.CompactionPriority;
-import org.apache.iotdb.db.engine.compaction.cross.CrossCompactionStrategy;
-import org.apache.iotdb.db.engine.compaction.inner.InnerCompactionStrategy;
+import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionSelector;
+import org.apache.iotdb.db.engine.compaction.constant.InnerSequenceCompactionSelector;
public class CompactionConfigRestorer {
private boolean enableSeqSpaceCompaction = true;
private boolean enableUnseqSpaceCompaction = false;
private boolean enableCrossSpaceCompaction = true;
- private CrossCompactionStrategy crossStrategy = CrossCompactionStrategy.REWRITE_COMPACTION;
- private InnerCompactionStrategy innerStrategy = InnerCompactionStrategy.SIZE_TIERED_COMPACTION;
+ private CrossCompactionSelector crossStrategy = CrossCompactionSelector.REWRITE;
+ private InnerSequenceCompactionSelector innerStrategy =
+ InnerSequenceCompactionSelector.SIZE_TIERED;
private CompactionPriority priority = CompactionPriority.BALANCE;
private long targetFileSize = 1073741824L;
private long targetChunkSize = 1048576L;
@@ -51,8 +52,8 @@ public class CompactionConfigRestorer {
config.setEnableSeqSpaceCompaction(enableSeqSpaceCompaction);
config.setEnableUnseqSpaceCompaction(enableUnseqSpaceCompaction);
config.setEnableCrossSpaceCompaction(enableCrossSpaceCompaction);
- config.setCrossCompactionStrategy(crossStrategy);
- config.setInnerCompactionStrategy(innerStrategy);
+ config.setCrossCompactionSelector(crossStrategy);
+ config.setInnerSequenceCompactionSelector(innerStrategy);
config.setCompactionPriority(priority);
config.setTargetCompactionFileSize(targetFileSize);
config.setTargetChunkSize(targetChunkSize);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
index 8ac22d8cc5..9c1d7bfee9 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
@@ -25,9 +25,10 @@ import org.apache.iotdb.db.constant.TestConstant;
import org.apache.iotdb.db.engine.MetadataManagerHelper;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.inner.sizetiered.SizeTieredCompactionTask;
+import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
+import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
+import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
-import org.apache.iotdb.db.engine.compaction.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.flush.FlushManager;
import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
@@ -721,14 +722,13 @@ public class StorageGroupProcessorTest {
processor.asyncCloseAllWorkingTsFileProcessors();
}
processor.syncCloseAllWorkingTsFileProcessors();
- SizeTieredCompactionTask task =
- new SizeTieredCompactionTask(
- storageGroup,
- "0",
+ InnerSpaceCompactionTask task =
+ new InnerSpaceCompactionTask(
0,
processor.getTsFileManager(),
processor.getSequenceFileList(),
true,
+ new ReadChunkCompactionPerformer(processor.getSequenceFileList()),
new AtomicInteger(0));
CompactionTaskManager.getInstance().submitTask(task);
Thread.sleep(20);