From 36125432efdfac0e1fd6065c0eadb52617f867bd Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Fri, 1 Apr 2022 17:27:20 +0800 Subject: [PATCH 01/17] first complete concurrent --- .../iotdb/tsfile/TsFileSequenceRead.java | 3 +- .../iotdb/commons/concurrent/ThreadName.java | 1 + .../resources/conf/iotdb-engine.properties | 6 + .../org/apache/iotdb/db/conf/IoTDBConfig.java | 15 ++- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 10 +- .../compaction/CompactionTaskManager.java | 19 ++- .../db/engine/compaction/CompactionUtils.java | 122 +++++++++++------- .../cross/rewrite/task/SubCompactionTask.java | 82 ++++++++++++ .../writer/AbstractCompactionWriter.java | 52 +++++--- .../writer/CrossSpaceCompactionWriter.java | 76 ++++++----- .../writer/InnerSpaceCompactionWriter.java | 24 ++-- 11 files changed, 290 insertions(+), 120 deletions(-) create mode 100644 server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/SubCompactionTask.java diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java index 6d2c2ff891784..8c019dc7f57be 100644 --- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java +++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java @@ -53,7 +53,8 @@ public class TsFileSequenceRead { "squid:S106" }) // Suppress high Cognitive Complexity and Standard outputs warning public static void main(String[] args) throws IOException { - String filename = "test.tsfile"; + String filename = + "C:\\IOTDB\\projects\\apache\\iotdb\\server\\target\\data\\sequence\\root.compactionTest\\0\\0\\1648799238944-0-1-0.inner"; if (args.length >= 1) { filename = args[0]; } diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index f6ea467f74c8a..4df1111ea7dcc 100644 --- a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -37,6 +37,7 @@ public enum ThreadName { FLUSH_SERVICE("Flush"), FLUSH_SUB_TASK_SERVICE("Flush-SubTask"), COMPACTION_SERVICE("Compaction"), + COMPACTION_SUB_SERVICE("Sub-Compaction"), COMPACTION_SCHEDULE("Compaction_Schedule"), WAL_DAEMON("WAL-Sync"), WAL_FORCE_DAEMON("WAL-Force"), diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties index d6598a5d1437c..dcfdce34821a1 100644 --- a/server/src/assembly/resources/conf/iotdb-engine.properties +++ b/server/src/assembly/resources/conf/iotdb-engine.properties @@ -505,6 +505,12 @@ timestamp_precision=ms # Datatype: int # query_timeout_threshold=60000 +# The number of sub compaction threads to be set up to perform compaction. +# Currently only works for cross space compaction and unseq inner space compaction. +# Set to 1 when less than or equal to 0. +# Datatype: int +# sub_compaction_thread_num=4 + #################### ### Metadata Cache Configuration #################### 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 580d8f75521d9..22a2506211e29 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 @@ -35,7 +35,6 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.fileSystem.FSType; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -436,6 +435,12 @@ public class IoTDBConfig { /** The interval of compaction task submission from queue in CompactionTaskMananger */ private long compactionSubmissionIntervalInMs = 60_000L; + /** + * The number of sub compaction threads to be set up to perform compaction. Currently only works + * for cross space compaction and unseq inner space compaction. + */ + private int subCompactionTaskNum = 4; + /** whether to cache meta data(ChunkMetaData and TsFileMetaData) or not. */ private boolean metaDataCacheEnable = true; @@ -2515,6 +2520,14 @@ public void setCompactionSubmissionIntervalInMs(long interval) { compactionSubmissionIntervalInMs = interval; } + public int getSubCompactionTaskNum() { + return subCompactionTaskNum; + } + + public void setSubCompactionTaskNum(int subCompactionTaskNum) { + this.subCompactionTaskNum = subCompactionTaskNum; + } + public String getDeviceIDTransformationMethod() { return deviceIDTransformationMethod; } 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 e384b5c2b1e4b..ac029f40cba3e 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 @@ -18,6 +18,7 @@ */ package org.apache.iotdb.db.conf; +import com.google.common.net.InetAddresses; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.db.conf.directories.DirectoryManager; import org.apache.iotdb.db.engine.StorageEngine; @@ -36,8 +37,6 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.fileSystem.FSType; import org.apache.iotdb.tsfile.utils.FilePathUtils; - -import com.google.common.net.InetAddresses; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -374,6 +373,13 @@ private void loadProps() { properties.getProperty( "compaction_priority", conf.getCompactionPriority().toString()))); + int subtaskNum = + Integer.parseInt( + properties.getProperty( + "sub_compaction_thread_num", Integer.toString(conf.getSubCompactionTaskNum()))); + subtaskNum = subtaskNum <= 0 ? 1 : subtaskNum; + conf.setSubCompactionTaskNum(subtaskNum); + conf.setQueryTimeoutThreshold( Integer.parseInt( properties.getProperty( 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 b0ecbe53f7dde..50d78d9c71c21 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 @@ -19,6 +19,7 @@ package org.apache.iotdb.db.engine.compaction; +import com.google.common.util.concurrent.RateLimiter; import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.concurrent.threadpool.WrappedScheduledExecutorService; @@ -30,8 +31,6 @@ 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; - -import com.google.common.util.concurrent.RateLimiter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,6 +58,9 @@ public class CompactionTaskManager implements IService { // The thread pool that executes the compaction task. The default number of threads for this pool // is 10. private WrappedScheduledExecutorService taskExecutionPool; + + private ScheduledExecutorService subCompactionTaskExecutionPool; + public static volatile AtomicInteger currentTaskNum = new AtomicInteger(0); private FixedPriorityBlockingQueue candidateCompactionTaskQueue = new FixedPriorityBlockingQueue<>(1024, new CompactionTaskComparator()); @@ -90,6 +92,11 @@ public void start() { IoTDBThreadPoolFactory.newScheduledThreadPool( IoTDBDescriptor.getInstance().getConfig().getConcurrentCompactionThread(), ThreadName.COMPACTION_SERVICE.getName()); + this.subCompactionTaskExecutionPool = + IoTDBThreadPoolFactory.newScheduledThreadPool( + IoTDBDescriptor.getInstance().getConfig().getConcurrentCompactionThread() + * IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum(), + ThreadName.COMPACTION_SUB_SERVICE.getName()); currentTaskNum = new AtomicInteger(0); compactionTaskSubmissionThreadPool = IoTDBThreadPoolFactory.newScheduledThreadPool(1, ThreadName.COMPACTION_SERVICE.getName()); @@ -303,6 +310,14 @@ public synchronized void submitTask(Callable compactionMergeTask) : "taskExecutionPool is terminated"); } + public synchronized Future submitSubTask(Callable subCompactionTask) { + if (subCompactionTaskExecutionPool != null && !subCompactionTaskExecutionPool.isTerminated()) { + Future future = subCompactionTaskExecutionPool.submit(subCompactionTask); + return future; + } + return null; + } + /** * Abort all compactions of a storage group. The caller must acquire the write lock of the * corresponding storage group. 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 6e9cf68291859..4828b5fede4d2 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,6 +20,7 @@ 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; @@ -50,18 +51,20 @@ 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; import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; 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; /** @@ -73,6 +76,8 @@ public class CompactionUtils { private static final Logger logger = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME); + private static final int subTaskNum = + IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum(); public static void compact( List seqFileResources, @@ -86,8 +91,11 @@ public static void compact( .getQueryFileManager() .addUsedFilesForQuery(queryId, queryDataSource); + List targetFileWriters = + getTsFileIOWriter(seqFileResources, unseqFileResources, targetFileResources); try (AbstractCompactionWriter compactionWriter = - getCompactionWriter(seqFileResources, unseqFileResources, targetFileResources)) { + getCompactionWriter( + seqFileResources, unseqFileResources, targetFileResources, targetFileWriters)) { // Do not close device iterator, because tsfile reader is managed by FileReaderManager. MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(seqFileResources, unseqFileResources); @@ -157,9 +165,9 @@ private static void compactAlignedSeries( if (dataBatchReader.hasNextBatch()) { // chunkgroup is serialized only when at least one timeseries under this device has data compactionWriter.startChunkGroup(device, true); - compactionWriter.startMeasurement(measurementSchemas); - writeWithReader(compactionWriter, dataBatchReader); - compactionWriter.endMeasurement(); + compactionWriter.startMeasurement(measurementSchemas, 0); + writeWithReader(compactionWriter, dataBatchReader, 0); + compactionWriter.endMeasurement(0); compactionWriter.endChunkGroup(); } } @@ -170,59 +178,53 @@ private static void compactNonAlignedSeries( AbstractCompactionWriter compactionWriter, QueryContext queryContext, QueryDataSource queryDataSource) - throws MetadataException, IOException { - boolean hasStartChunkGroup = false; + throws IOException { MultiTsFileDeviceIterator.MeasurementIterator measurementIterator = deviceIterator.iterateNotAlignedSeries(device, false); Set allMeasurements = measurementIterator.getAllMeasurements(); + int subTaskNums = Math.min(allMeasurements.size(), subTaskNum); + Set[] measurementsForEachSubTask = new HashSet[subTaskNums]; + int idx = 0; for (String measurement : allMeasurements) { - List measurementSchemas = new ArrayList<>(); - 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()); - continue; + if (measurementsForEachSubTask[idx % subTaskNums] == null) { + measurementsForEachSubTask[idx % subTaskNums] = new HashSet(); } - - IBatchReader dataBatchReader = - constructReader( - device, - Collections.singletonList(measurement), - measurementSchemas, - allMeasurements, - queryContext, - queryDataSource, - false); - - if (dataBatchReader.hasNextBatch()) { - if (!hasStartChunkGroup) { - // chunkgroup is serialized only when at least one timeseries under this device has - // data - compactionWriter.startChunkGroup(device, false); - hasStartChunkGroup = true; - } - compactionWriter.startMeasurement(measurementSchemas); - writeWithReader(compactionWriter, dataBatchReader); - compactionWriter.endMeasurement(); + measurementsForEachSubTask[idx++ % subTaskNums].add(measurement); + } + List> 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))); + } + for (int i = 0; i < subTaskNums; i++) { + try { + futures.get(i).get(); + } catch (InterruptedException e) { + logger.error("SubCompactionTask interrupted", e); + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + throw new IOException(e); } } - if (hasStartChunkGroup) { - compactionWriter.endChunkGroup(); - } + compactionWriter.endChunkGroup(); } - private static void writeWithReader(AbstractCompactionWriter writer, IBatchReader reader) - throws IOException { + 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()); + writer.write(batchData.currentTime(), batchData.currentValue(), subTaskId); batchData.next(); } } @@ -232,7 +234,7 @@ private static void writeWithReader(AbstractCompactionWriter writer, IBatchReade * @param measurementIds if device is aligned, then measurementIds contain all measurements. If * device is not aligned, then measurementIds only contain one measurement. */ - private static IBatchReader constructReader( + public static IBatchReader constructReader( String deviceId, List measurementIds, List measurementSchemas, @@ -254,17 +256,37 @@ private static IBatchReader constructReader( seriesPath, allSensors, tsDataType, queryContext, queryDataSource, null, null, null, true); } - private static AbstractCompactionWriter getCompactionWriter( + private static List getTsFileIOWriter( List seqFileResources, List unseqFileResources, List targetFileResources) throws IOException { + List targetFileWriters = new ArrayList<>(); + if (!seqFileResources.isEmpty() && !unseqFileResources.isEmpty()) { + // cross space + for (TsFileResource targetFileResource : targetFileResources) { + targetFileWriters.add(new TsFileIOWriter(targetFileResource.getTsFile())); + } + } else { + // inner space + targetFileWriters.add(new TsFileIOWriter(targetFileResources.get(0).getTsFile())); + } + return targetFileWriters; + } + + private static AbstractCompactionWriter getCompactionWriter( + List seqFileResources, + List unseqFileResources, + List targetFileResources, + List targetFileWriters) + throws IOException { if (!seqFileResources.isEmpty() && !unseqFileResources.isEmpty()) { // cross space - return new CrossSpaceCompactionWriter(targetFileResources, seqFileResources); + return new CrossSpaceCompactionWriter( + targetFileResources, seqFileResources, targetFileWriters); } else { // inner space - return new InnerSpaceCompactionWriter(targetFileResources.get(0)); + return new InnerSpaceCompactionWriter(targetFileResources.get(0), targetFileWriters.get(0)); } } 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/SubCompactionTask.java new file mode 100644 index 0000000000000..0b70be668d9d0 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/cross/rewrite/task/SubCompactionTask.java @@ -0,0 +1,82 @@ +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.writer.AbstractCompactionWriter; +import org.apache.iotdb.db.engine.querycontext.QueryDataSource; +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.context.QueryContext; +import org.apache.iotdb.db.service.IoTDB; +import org.apache.iotdb.tsfile.read.reader.IBatchReader; +import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; + +public class SubCompactionTask implements Callable { + private static final Logger logger = + LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME); + private final String device; + private final Set measurementList; + private final QueryContext queryContext; + private final QueryDataSource queryDataSource; + private final AbstractCompactionWriter compactionWriter; + private final int taskId; + + public SubCompactionTask( + String device, + Set measurementList, + QueryContext queryContext, + QueryDataSource queryDataSource, + AbstractCompactionWriter compactionWriter,int taskId) { + this.device = device; + this.measurementList = measurementList; + this.queryContext = queryContext; + this.queryDataSource = queryDataSource; + this.compactionWriter = compactionWriter; + this.taskId=taskId; + } + + @Override + public Void call() throws Exception { + for (String measurement : measurementList) { + List measurementSchemas = new ArrayList<>(); + 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()); + continue; + } + + IBatchReader dataBatchReader = + CompactionUtils.constructReader( + device, + Collections.singletonList(measurement), + measurementSchemas, + measurementList, + queryContext, + queryDataSource, + false); + + if (dataBatchReader.hasNextBatch()) { + compactionWriter.startMeasurement(measurementSchemas,taskId); + CompactionUtils.writeWithReader(compactionWriter, dataBatchReader,taskId); + compactionWriter.endMeasurement(taskId); + } + } + return null; + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index 053631a516a32..98598eb28a9e3 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -36,10 +36,13 @@ import java.io.IOException; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; public abstract class AbstractCompactionWriter implements AutoCloseable { - protected IChunkWriter chunkWriter; + // subTaskId -> IChunkWriter + protected Map chunkWriterMap = new ConcurrentHashMap<>(); protected boolean isAlign; @@ -50,24 +53,26 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { MetricConfigDescriptor.getInstance().getMetricConfig().getEnableMetric(); // point count in current measurment, which is used to check size - private int measurementPointCount; + private Map measurementPointCount = new ConcurrentHashMap<>(); public abstract void startChunkGroup(String deviceId, boolean isAlign) throws IOException; public abstract void endChunkGroup() throws IOException; - public void startMeasurement(List measurementSchemaList) { - measurementPointCount = 0; + public void startMeasurement(List measurementSchemaList, int subTaskId) { + measurementPointCount.computeIfAbsent(subTaskId, id -> 0); if (isAlign) { - chunkWriter = new AlignedChunkWriterImpl(measurementSchemaList); + // chunkWriter = new AlignedChunkWriterImpl(measurementSchemaList); + chunkWriterMap.put(subTaskId, new AlignedChunkWriterImpl(measurementSchemaList)); } else { - chunkWriter = new ChunkWriterImpl(measurementSchemaList.get(0), true); + // chunkWriter = new ChunkWriterImpl(measurementSchemaList.get(0), true); + chunkWriterMap.put(subTaskId, new ChunkWriterImpl(measurementSchemaList.get(0), true)); } } - public abstract void endMeasurement() throws IOException; + public abstract void endMeasurement(int subTaskId) throws IOException; - public abstract void write(long timestamp, Object value) throws IOException; + public abstract void write(long timestamp, Object value, int subTaskId) throws IOException; public abstract void write(long[] timestamps, Object values); @@ -75,9 +80,9 @@ public void startMeasurement(List measurementSchemaList) { public abstract void close() throws IOException; - protected void writeDataPoint(Long timestamp, Object value) { + protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { if (!isAlign) { - ChunkWriterImpl chunkWriter = (ChunkWriterImpl) this.chunkWriter; + ChunkWriterImpl chunkWriter = (ChunkWriterImpl) this.chunkWriterMap.get(subTaskId); switch (chunkWriter.getDataType()) { case TEXT: chunkWriter.write(timestamp, (Binary) value); @@ -101,7 +106,8 @@ protected void writeDataPoint(Long timestamp, Object value) { throw new UnsupportedOperationException("Unknown data type " + chunkWriter.getDataType()); } } else { - AlignedChunkWriterImpl chunkWriter = (AlignedChunkWriterImpl) this.chunkWriter; + AlignedChunkWriterImpl chunkWriter = + (AlignedChunkWriterImpl) this.chunkWriterMap.get(subTaskId); for (TsPrimitiveType val : (TsPrimitiveType[]) value) { if (val == null) { chunkWriter.write(timestamp, null, true); @@ -133,28 +139,32 @@ protected void writeDataPoint(Long timestamp, Object value) { } chunkWriter.write(timestamp); } - measurementPointCount++; + measurementPointCount.put(subTaskId, measurementPointCount.get(subTaskId) + 1); } - protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter) throws IOException { - if (measurementPointCount % 10 == 0 && checkChunkSize()) { - writeRateLimit(chunkWriter.estimateMaxSeriesMemSize()); + protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) + throws IOException { + if (measurementPointCount.get(subTaskId) % 10 == 0 && checkChunkSize(subTaskId)) { + writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); CompactionMetricsManager.recordWriteInfo( this instanceof CrossSpaceCompactionWriter ? CompactionType.CROSS_COMPACTION : CompactionType.INNER_UNSEQ_COMPACTION, ProcessChunkType.DESERIALIZE_CHUNK, this.isAlign, - chunkWriter.estimateMaxSeriesMemSize()); - chunkWriter.writeToFileWriter(fileWriter); + chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + synchronized (fileWriter) { + chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriter); + } } } - private boolean checkChunkSize() { - if (chunkWriter instanceof AlignedChunkWriterImpl) { - return ((AlignedChunkWriterImpl) chunkWriter).checkIsChunkSizeOverThreshold(targetChunkSize); + private boolean checkChunkSize(int subTaskId) { + if (chunkWriterMap.get(subTaskId) instanceof AlignedChunkWriterImpl) { + return ((AlignedChunkWriterImpl) chunkWriterMap.get(subTaskId)) + .checkIsChunkSizeOverThreshold(targetChunkSize); } else { - return chunkWriter.estimateMaxSeriesMemSize() > targetChunkSize; + return chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize() > targetChunkSize; } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index 2ebe22581d7ce..851c811f4c394 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -21,22 +21,21 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.query.control.FileReaderManager; import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata; -import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter; import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // target fileIOWriters - private List fileWriterList = new ArrayList<>(); + private List fileWriterList; // source tsfiles private List seqTsFileResources; - private int seqFileIndex; + private Map seqFileIndex; private final long[] currentDeviceEndTime; @@ -47,25 +46,28 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { private final List targetTsFileResources; public CrossSpaceCompactionWriter( - List targetResources, List seqFileResources) + List targetResources, + List seqFileResources, + List fileWriterList) throws IOException { currentDeviceEndTime = new long[seqFileResources.size()]; - isEmptyFile = new boolean[seqFileResources.size()]; hasTargetFileStartChunkGroup = new boolean[seqFileResources.size()]; + this.fileWriterList = fileWriterList; + isEmptyFile = new boolean[seqFileResources.size()]; for (int i = 0; i < targetResources.size(); i++) { - this.fileWriterList.add(new RestorableTsFileIOWriter(targetResources.get(i).getTsFile())); + // this.fileWriterList.add(new RestorableTsFileIOWriter(targetResources.get(i).getTsFile())); isEmptyFile[i] = true; } this.seqTsFileResources = seqFileResources; this.targetTsFileResources = targetResources; - seqFileIndex = 0; + this.seqFileIndex = new ConcurrentHashMap<>(); } @Override public void startChunkGroup(String deviceId, boolean isAlign) throws IOException { this.deviceId = deviceId; this.isAlign = isAlign; - this.seqFileIndex = 0; + this.seqFileIndex = new ConcurrentHashMap<>(); checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { hasTargetFileStartChunkGroup[i] = false; @@ -83,21 +85,25 @@ public void endChunkGroup() throws IOException { } @Override - public void endMeasurement() throws IOException { - writeRateLimit(chunkWriter.estimateMaxSeriesMemSize()); - chunkWriter.writeToFileWriter(fileWriterList.get(seqFileIndex)); - chunkWriter = null; - seqFileIndex = 0; + public void endMeasurement(int subTaskId) throws IOException { + writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + synchronized (fileWriterList.get(seqFileIndex.get(subTaskId))) { + chunkWriterMap + .get(subTaskId) + .writeToFileWriter(fileWriterList.get(seqFileIndex.get(subTaskId))); + } + // chunkWriterMap.get(subTaskId)=null; + seqFileIndex.put(subTaskId, 0); } @Override - public void write(long timestamp, Object value) throws IOException { - checkTimeAndMayFlushChunkToCurrentFile(timestamp); - checkAndMayStartChunkGroup(); - writeDataPoint(timestamp, value); - updateDeviceStartAndEndTime(targetTsFileResources.get(seqFileIndex), timestamp); - checkChunkSizeAndMayOpenANewChunk(fileWriterList.get(seqFileIndex)); - isEmptyFile[seqFileIndex] = false; + public void write(long timestamp, Object value, int subTaskId) throws IOException { + checkTimeAndMayFlushChunkToCurrentFile(timestamp, subTaskId); + checkAndMayStartChunkGroup(subTaskId); + writeDataPoint(timestamp, value, subTaskId); + updateDeviceStartAndEndTime(targetTsFileResources.get(seqFileIndex.get(subTaskId)), timestamp); + checkChunkSizeAndMayOpenANewChunk(fileWriterList.get(seqFileIndex.get(subTaskId)), subTaskId); + isEmptyFile[seqFileIndex.get(subTaskId)] = false; } @Override @@ -123,16 +129,21 @@ public void close() throws IOException { } fileWriterList = null; seqTsFileResources = null; - chunkWriter = null; + // chunkWriter = null; + chunkWriterMap.clear(); } - private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp) throws IOException { + private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskId) + throws IOException { + int fileIndex = seqFileIndex.computeIfAbsent(subTaskId, id -> 0); // if timestamp is later than the current source seq tsfile, than flush chunk writer - while (timestamp > currentDeviceEndTime[seqFileIndex]) { - if (seqFileIndex != seqTsFileResources.size() - 1) { - writeRateLimit(chunkWriter.estimateMaxSeriesMemSize()); - chunkWriter.writeToFileWriter(fileWriterList.get(seqFileIndex)); - seqFileIndex++; + while (timestamp > currentDeviceEndTime[fileIndex]) { + if (fileIndex != seqTsFileResources.size() - 1) { + writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + synchronized (fileWriterList.get(fileIndex)) { + chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriterList.get(fileIndex)); + } + seqFileIndex.put(subTaskId, ++fileIndex); } else { // If the seq file is deleted for various reasons, the following two situations may occur // when selecting the source files: (1) unseq files may have some devices or measurements @@ -169,10 +180,11 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { } } - private void checkAndMayStartChunkGroup() throws IOException { - if (!hasTargetFileStartChunkGroup[seqFileIndex]) { - fileWriterList.get(seqFileIndex).startChunkGroup(deviceId); - hasTargetFileStartChunkGroup[seqFileIndex] = true; + private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { + int fileIndex = seqFileIndex.computeIfAbsent(subTaskId, id -> 0); + if (!hasTargetFileStartChunkGroup[fileIndex]) { + fileWriterList.get(fileIndex).startChunkGroup(deviceId); + hasTargetFileStartChunkGroup[fileIndex] = true; } } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java index 7b0e31095dfd3..eea16db07efc9 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java @@ -19,7 +19,6 @@ package org.apache.iotdb.db.engine.compaction.writer; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; -import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter; import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; import java.io.IOException; @@ -31,8 +30,8 @@ public class InnerSpaceCompactionWriter extends AbstractCompactionWriter { private final TsFileResource targetTsFileResource; - public InnerSpaceCompactionWriter(TsFileResource targetFileResource) throws IOException { - fileWriter = new RestorableTsFileIOWriter(targetFileResource.getTsFile()); + public InnerSpaceCompactionWriter(TsFileResource targetFileResource, TsFileIOWriter fileWriter) { + this.fileWriter = fileWriter; isEmptyFile = true; this.targetTsFileResource = targetFileResource; } @@ -50,17 +49,19 @@ public void endChunkGroup() throws IOException { } @Override - public void endMeasurement() throws IOException { - writeRateLimit(chunkWriter.estimateMaxSeriesMemSize()); - chunkWriter.writeToFileWriter(fileWriter); - chunkWriter = null; + public void endMeasurement(int subTaskId) throws IOException { + writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + synchronized (fileWriter) { + chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriter); + } + // chunkWriter = null; } @Override - public void write(long timestamp, Object value) throws IOException { - writeDataPoint(timestamp, value); + public void write(long timestamp, Object value, int subTaskId) throws IOException { + writeDataPoint(timestamp, value, subTaskId); updateDeviceStartAndEndTime(targetTsFileResource, timestamp); - checkChunkSizeAndMayOpenANewChunk(fileWriter); + checkChunkSizeAndMayOpenANewChunk(fileWriter, subTaskId); isEmptyFile = false; } @@ -80,7 +81,8 @@ public void close() throws IOException { if (fileWriter != null && fileWriter.canWrite()) { fileWriter.close(); } - chunkWriter = null; + // chunkWriter = null; + chunkWriterMap.clear(); fileWriter = null; } } From ebbd04fe506c79b0b8b6c79d5de734c2f1ba6593 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Tue, 5 Apr 2022 15:34:50 +0800 Subject: [PATCH 02/17] update device start time and end time after compacting --- .../iotdb/tsfile/TsFileSequenceRead.java | 3 +- .../resources/conf/iotdb-engine.properties | 2 +- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 3 +- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 3 +- .../compaction/CompactionTaskManager.java | 4 +- .../db/engine/compaction/CompactionUtils.java | 63 ++++++++++--------- .../cross/rewrite/task/SubCompactionTask.java | 33 ++++++++-- .../writer/AbstractCompactionWriter.java | 16 ++--- .../writer/CrossSpaceCompactionWriter.java | 46 +++++++------- .../writer/InnerSpaceCompactionWriter.java | 15 +++-- .../compaction/CompactionUtilsTest.java | 14 +++++ 11 files changed, 123 insertions(+), 79 deletions(-) diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java index 8c019dc7f57be..6d2c2ff891784 100644 --- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java +++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java @@ -53,8 +53,7 @@ public class TsFileSequenceRead { "squid:S106" }) // Suppress high Cognitive Complexity and Standard outputs warning public static void main(String[] args) throws IOException { - String filename = - "C:\\IOTDB\\projects\\apache\\iotdb\\server\\target\\data\\sequence\\root.compactionTest\\0\\0\\1648799238944-0-1-0.inner"; + String filename = "test.tsfile"; if (args.length >= 1) { filename = args[0]; } diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties index dcfdce34821a1..d8cf5a2653f5d 100644 --- a/server/src/assembly/resources/conf/iotdb-engine.properties +++ b/server/src/assembly/resources/conf/iotdb-engine.properties @@ -506,7 +506,7 @@ timestamp_precision=ms # query_timeout_threshold=60000 # The number of sub compaction threads to be set up to perform compaction. -# Currently only works for cross space compaction and unseq inner space compaction. +# Currently only works for nonAligned data in cross space compaction and unseq inner space compaction. # Set to 1 when less than or equal to 0. # Datatype: int # sub_compaction_thread_num=4 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 22a2506211e29..987051887cd10 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 @@ -35,6 +35,7 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.fileSystem.FSType; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -437,7 +438,7 @@ public class IoTDBConfig { /** * The number of sub compaction threads to be set up to perform compaction. Currently only works - * for cross space compaction and unseq inner space compaction. + * for nonAligned data in cross space compaction and unseq inner space compaction. */ private int subCompactionTaskNum = 4; 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 ac029f40cba3e..d2fc333a18a61 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 @@ -18,7 +18,6 @@ */ package org.apache.iotdb.db.conf; -import com.google.common.net.InetAddresses; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.db.conf.directories.DirectoryManager; import org.apache.iotdb.db.engine.StorageEngine; @@ -37,6 +36,8 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.fileSystem.FSType; import org.apache.iotdb.tsfile.utils.FilePathUtils; + +import com.google.common.net.InetAddresses; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 50d78d9c71c21..8ed8ff4159b7c 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 @@ -19,7 +19,6 @@ package org.apache.iotdb.db.engine.compaction; -import com.google.common.util.concurrent.RateLimiter; import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.concurrent.threadpool.WrappedScheduledExecutorService; @@ -31,6 +30,8 @@ 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; + +import com.google.common.util.concurrent.RateLimiter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,6 +60,7 @@ public class CompactionTaskManager implements IService { // is 10. private WrappedScheduledExecutorService taskExecutionPool; + // The thread pool that executes the sub compaction task. private ScheduledExecutorService subCompactionTaskExecutionPool; public static volatile AtomicInteger currentTaskNum = new AtomicInteger(0); 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 4828b5fede4d2..1cb4a52304400 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 @@ -45,6 +45,7 @@ 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; @@ -52,6 +53,7 @@ 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; @@ -91,11 +93,8 @@ public static void compact( .getQueryFileManager() .addUsedFilesForQuery(queryId, queryDataSource); - List targetFileWriters = - getTsFileIOWriter(seqFileResources, unseqFileResources, targetFileResources); try (AbstractCompactionWriter compactionWriter = - getCompactionWriter( - seqFileResources, unseqFileResources, targetFileResources, targetFileWriters)) { + getCompactionWriter(seqFileResources, unseqFileResources, targetFileResources)) { // Do not close device iterator, because tsfile reader is managed by FileReaderManager. MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(seqFileResources, unseqFileResources); @@ -116,6 +115,7 @@ public static void compact( } compactionWriter.endFile(); + updateDeviceStartTimeAndEndTime(targetFileResources, compactionWriter); updatePlanIndexes(targetFileResources, seqFileResources, unseqFileResources); } finally { QueryResourceManager.getInstance().endQuery(queryId); @@ -183,6 +183,8 @@ private static void compactNonAlignedSeries( deviceIterator.iterateNotAlignedSeries(device, false); Set allMeasurements = measurementIterator.getAllMeasurements(); int subTaskNums = Math.min(allMeasurements.size(), subTaskNum); + + // assign all measurements to different sub tasks Set[] measurementsForEachSubTask = new HashSet[subTaskNums]; int idx = 0; for (String measurement : allMeasurements) { @@ -191,6 +193,8 @@ private static void compactNonAlignedSeries( } measurementsForEachSubTask[idx++ % subTaskNums].add(measurement); } + + // construct sub tasks and start compacting measurements in parallel List> futures = new ArrayList<>(); compactionWriter.startChunkGroup(device, false); for (int i = 0; i < subTaskNums; i++) { @@ -205,13 +209,13 @@ private static void compactNonAlignedSeries( compactionWriter, i))); } + + // wait for all sub tasks finish for (int i = 0; i < subTaskNums; i++) { try { futures.get(i).get(); - } catch (InterruptedException e) { - logger.error("SubCompactionTask interrupted", e); - Thread.currentThread().interrupt(); - } catch (ExecutionException e) { + } catch (InterruptedException | ExecutionException e) { + logger.error("SubCompactionTask meet errors ", e); throw new IOException(e); } } @@ -256,37 +260,38 @@ public static IBatchReader constructReader( seriesPath, allSensors, tsDataType, queryContext, queryDataSource, null, null, null, true); } - private static List getTsFileIOWriter( + private static AbstractCompactionWriter getCompactionWriter( List seqFileResources, List unseqFileResources, List targetFileResources) throws IOException { - List targetFileWriters = new ArrayList<>(); if (!seqFileResources.isEmpty() && !unseqFileResources.isEmpty()) { // cross space - for (TsFileResource targetFileResource : targetFileResources) { - targetFileWriters.add(new TsFileIOWriter(targetFileResource.getTsFile())); - } + return new CrossSpaceCompactionWriter(targetFileResources, seqFileResources); } else { // inner space - targetFileWriters.add(new TsFileIOWriter(targetFileResources.get(0).getTsFile())); + return new InnerSpaceCompactionWriter(targetFileResources.get(0)); } - return targetFileWriters; } - private static AbstractCompactionWriter getCompactionWriter( - List seqFileResources, - List unseqFileResources, - List targetFileResources, - List targetFileWriters) - throws IOException { - if (!seqFileResources.isEmpty() && !unseqFileResources.isEmpty()) { - // cross space - return new CrossSpaceCompactionWriter( - targetFileResources, seqFileResources, targetFileWriters); - } else { - // inner space - return new InnerSpaceCompactionWriter(targetFileResources.get(0), targetFileWriters.get(0)); + private static void updateDeviceStartTimeAndEndTime( + List targetResources, AbstractCompactionWriter compactionWriter) { + List targetFileWriters = compactionWriter.getFileIOWriter(); + for (int i = 0; i < targetFileWriters.size(); i++) { + TsFileIOWriter fileIOWriter = targetFileWriters.get(i); + TsFileResource fileResource = targetResources.get(i); + // skip the target file that has been deleted + if (!fileResource.getTsFile().exists()) { + continue; + } + for (Map.Entry> 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()); + } + } } } @@ -302,7 +307,7 @@ private static void updatePlanIndexes( // in the new file for (int i = 0; i < targetResources.size(); i++) { TsFileResource targetResource = targetResources.get(i); - // remove the target file been deleted from list + // remove the target file that has been deleted from list if (!targetResource.getTsFile().exists()) { targetResources.remove(i--); continue; 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/SubCompactionTask.java index 0b70be668d9d0..5a93442047f1d 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/SubCompactionTask.java @@ -1,3 +1,21 @@ +/* + * 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; @@ -12,6 +30,7 @@ import org.apache.iotdb.db.service.IoTDB; import org.apache.iotdb.tsfile.read.reader.IBatchReader; import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -21,6 +40,11 @@ import java.util.Set; import java.util.concurrent.Callable; +/** + * This class is used to implement reading the measurements and writing to the target files in + * 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 { private static final Logger logger = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME); @@ -36,13 +60,14 @@ public SubCompactionTask( Set measurementList, QueryContext queryContext, QueryDataSource queryDataSource, - AbstractCompactionWriter compactionWriter,int taskId) { + AbstractCompactionWriter compactionWriter, + int taskId) { this.device = device; this.measurementList = measurementList; this.queryContext = queryContext; this.queryDataSource = queryDataSource; this.compactionWriter = compactionWriter; - this.taskId=taskId; + this.taskId = taskId; } @Override @@ -72,8 +97,8 @@ public Void call() throws Exception { false); if (dataBatchReader.hasNextBatch()) { - compactionWriter.startMeasurement(measurementSchemas,taskId); - CompactionUtils.writeWithReader(compactionWriter, dataBatchReader,taskId); + compactionWriter.startMeasurement(measurementSchemas, taskId); + CompactionUtils.writeWithReader(compactionWriter, dataBatchReader, taskId); compactionWriter.endMeasurement(taskId); } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index 98598eb28a9e3..f017e62845e0c 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -23,7 +23,6 @@ import org.apache.iotdb.db.engine.compaction.CompactionTaskManager; import org.apache.iotdb.db.engine.compaction.constant.CompactionType; import org.apache.iotdb.db.engine.compaction.constant.ProcessChunkType; -import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.metrics.config.MetricConfigDescriptor; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.utils.Binary; @@ -53,19 +52,17 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { MetricConfigDescriptor.getInstance().getMetricConfig().getEnableMetric(); // point count in current measurment, which is used to check size - private Map measurementPointCount = new ConcurrentHashMap<>(); + private Map measurementPointCountMap = new ConcurrentHashMap<>(); public abstract void startChunkGroup(String deviceId, boolean isAlign) throws IOException; public abstract void endChunkGroup() throws IOException; public void startMeasurement(List measurementSchemaList, int subTaskId) { - measurementPointCount.computeIfAbsent(subTaskId, id -> 0); + measurementPointCountMap.put(subTaskId, 0); if (isAlign) { - // chunkWriter = new AlignedChunkWriterImpl(measurementSchemaList); chunkWriterMap.put(subTaskId, new AlignedChunkWriterImpl(measurementSchemaList)); } else { - // chunkWriter = new ChunkWriterImpl(measurementSchemaList.get(0), true); chunkWriterMap.put(subTaskId, new ChunkWriterImpl(measurementSchemaList.get(0), true)); } } @@ -139,12 +136,12 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { } chunkWriter.write(timestamp); } - measurementPointCount.put(subTaskId, measurementPointCount.get(subTaskId) + 1); + measurementPointCountMap.put(subTaskId, measurementPointCountMap.get(subTaskId) + 1); } protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) throws IOException { - if (measurementPointCount.get(subTaskId) % 10 == 0 && checkChunkSize(subTaskId)) { + if (measurementPointCountMap.get(subTaskId) % 10 == 0 && checkChunkSize(subTaskId)) { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); CompactionMetricsManager.recordWriteInfo( this instanceof CrossSpaceCompactionWriter @@ -173,8 +170,5 @@ protected void writeRateLimit(long bytesLength) { CompactionTaskManager.getInstance().getMergeWriteRateLimiter(), bytesLength); } - protected void updateDeviceStartAndEndTime(TsFileResource targetResource, long timestamp) { - targetResource.updateStartTime(deviceId, timestamp); - targetResource.updateEndTime(deviceId, timestamp); - } + public abstract List getFileIOWriter(); } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index 851c811f4c394..50f85a3794106 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -24,18 +24,20 @@ import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // target fileIOWriters - private List fileWriterList; + private List fileWriterList = new ArrayList<>(); // source tsfiles private List seqTsFileResources; - private Map seqFileIndex; + // subTaskId -> seqFileIndex + private Map seqFileIndexMap; private final long[] currentDeviceEndTime; @@ -43,31 +45,25 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { private final boolean[] hasTargetFileStartChunkGroup; - private final List targetTsFileResources; - public CrossSpaceCompactionWriter( - List targetResources, - List seqFileResources, - List fileWriterList) + List targetResources, List seqFileResources) throws IOException { currentDeviceEndTime = new long[seqFileResources.size()]; - hasTargetFileStartChunkGroup = new boolean[seqFileResources.size()]; - this.fileWriterList = fileWriterList; isEmptyFile = new boolean[seqFileResources.size()]; + hasTargetFileStartChunkGroup = new boolean[seqFileResources.size()]; for (int i = 0; i < targetResources.size(); i++) { - // this.fileWriterList.add(new RestorableTsFileIOWriter(targetResources.get(i).getTsFile())); + this.fileWriterList.add(new TsFileIOWriter(targetResources.get(i).getTsFile())); isEmptyFile[i] = true; } this.seqTsFileResources = seqFileResources; - this.targetTsFileResources = targetResources; - this.seqFileIndex = new ConcurrentHashMap<>(); + this.seqFileIndexMap = new ConcurrentHashMap<>(); } @Override public void startChunkGroup(String deviceId, boolean isAlign) throws IOException { this.deviceId = deviceId; this.isAlign = isAlign; - this.seqFileIndex = new ConcurrentHashMap<>(); + this.seqFileIndexMap.clear(); checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { hasTargetFileStartChunkGroup[i] = false; @@ -87,13 +83,13 @@ public void endChunkGroup() throws IOException { @Override public void endMeasurement(int subTaskId) throws IOException { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriterList.get(seqFileIndex.get(subTaskId))) { + synchronized (fileWriterList.get(seqFileIndexMap.get(subTaskId))) { chunkWriterMap .get(subTaskId) - .writeToFileWriter(fileWriterList.get(seqFileIndex.get(subTaskId))); + .writeToFileWriter(fileWriterList.get(seqFileIndexMap.get(subTaskId))); } // chunkWriterMap.get(subTaskId)=null; - seqFileIndex.put(subTaskId, 0); + seqFileIndexMap.put(subTaskId, 0); } @Override @@ -101,9 +97,9 @@ public void write(long timestamp, Object value, int subTaskId) throws IOExceptio checkTimeAndMayFlushChunkToCurrentFile(timestamp, subTaskId); checkAndMayStartChunkGroup(subTaskId); writeDataPoint(timestamp, value, subTaskId); - updateDeviceStartAndEndTime(targetTsFileResources.get(seqFileIndex.get(subTaskId)), timestamp); - checkChunkSizeAndMayOpenANewChunk(fileWriterList.get(seqFileIndex.get(subTaskId)), subTaskId); - isEmptyFile[seqFileIndex.get(subTaskId)] = false; + checkChunkSizeAndMayOpenANewChunk( + fileWriterList.get(seqFileIndexMap.get(subTaskId)), subTaskId); + isEmptyFile[seqFileIndexMap.get(subTaskId)] = false; } @Override @@ -129,13 +125,17 @@ public void close() throws IOException { } fileWriterList = null; seqTsFileResources = null; - // chunkWriter = null; chunkWriterMap.clear(); } + @Override + public List getFileIOWriter() { + return fileWriterList; + } + private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskId) throws IOException { - int fileIndex = seqFileIndex.computeIfAbsent(subTaskId, id -> 0); + int fileIndex = seqFileIndexMap.computeIfAbsent(subTaskId, id -> 0); // if timestamp is later than the current source seq tsfile, than flush chunk writer while (timestamp > currentDeviceEndTime[fileIndex]) { if (fileIndex != seqTsFileResources.size() - 1) { @@ -143,7 +143,7 @@ private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskI synchronized (fileWriterList.get(fileIndex)) { chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriterList.get(fileIndex)); } - seqFileIndex.put(subTaskId, ++fileIndex); + seqFileIndexMap.put(subTaskId, ++fileIndex); } else { // If the seq file is deleted for various reasons, the following two situations may occur // when selecting the source files: (1) unseq files may have some devices or measurements @@ -181,7 +181,7 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { } private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { - int fileIndex = seqFileIndex.computeIfAbsent(subTaskId, id -> 0); + int fileIndex = seqFileIndexMap.get(subTaskId); if (!hasTargetFileStartChunkGroup[fileIndex]) { fileWriterList.get(fileIndex).startChunkGroup(deviceId); hasTargetFileStartChunkGroup[fileIndex] = true; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java index eea16db07efc9..b0f543179d662 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java @@ -22,18 +22,17 @@ import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; import java.io.IOException; +import java.util.Collections; +import java.util.List; public class InnerSpaceCompactionWriter extends AbstractCompactionWriter { private TsFileIOWriter fileWriter; private boolean isEmptyFile; - private final TsFileResource targetTsFileResource; - - public InnerSpaceCompactionWriter(TsFileResource targetFileResource, TsFileIOWriter fileWriter) { - this.fileWriter = fileWriter; + public InnerSpaceCompactionWriter(TsFileResource targetFileResource) throws IOException { + this.fileWriter = new TsFileIOWriter(targetFileResource.getTsFile()); isEmptyFile = true; - this.targetTsFileResource = targetFileResource; } @Override @@ -60,7 +59,6 @@ public void endMeasurement(int subTaskId) throws IOException { @Override public void write(long timestamp, Object value, int subTaskId) throws IOException { writeDataPoint(timestamp, value, subTaskId); - updateDeviceStartAndEndTime(targetTsFileResource, timestamp); checkChunkSizeAndMayOpenANewChunk(fileWriter, subTaskId); isEmptyFile = false; } @@ -85,4 +83,9 @@ public void close() throws IOException { chunkWriterMap.clear(); fileWriter = null; } + + @Override + public List getFileIOWriter() { + return Collections.singletonList(fileWriter); + } } 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/CompactionUtilsTest.java index a6aab11e1c625..f64beb75f3e6d 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/CompactionUtilsTest.java @@ -202,6 +202,20 @@ public void testSeqInnerSpaceCompactionWithDifferentTimeseries() CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources(seqResources, true); CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, true, COMPACTION_TEST_SG); + assertEquals( + 0, targetResources.get(0).getStartTime(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + assertEquals( + 0, targetResources.get(0).getStartTime(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + assertEquals( + 250, targetResources.get(0).getStartTime(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + assertEquals( + 600, targetResources.get(0).getStartTime(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + assertEquals( + 600, targetResources.get(0).getStartTime(COMPACTION_TEST_SG + PATH_SEPARATOR + "d4")); + for (int i = 0; i < 5; i++) { + assertEquals( + 749, targetResources.get(0).getEndTime(COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i)); + } for (int i = 0; i < 5; i++) { for (int j = 0; j < 5; j++) { From 8787bb7fa274efeaefe01c23dda50237a7fca43f Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Tue, 5 Apr 2022 16:30:22 +0800 Subject: [PATCH 03/17] fix conflict --- .../iotdb/db/engine/compaction/CompactionTaskManager.java | 1 + .../engine/compaction/cross/rewrite/task/SubCompactionTask.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) 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 17720bda7798c..bf54c18b91324 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 @@ -39,6 +39,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; 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/SubCompactionTask.java index 5a93442047f1d..315122e3cf647 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/SubCompactionTask.java @@ -45,7 +45,7 @@ * 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 { +public class SubCompactionTask implements Callable { private static final Logger logger = LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME); private final String device; From 6254f868043531cf457d9f86235a1a37bd83ea3a Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Tue, 5 Apr 2022 19:05:45 +0800 Subject: [PATCH 04/17] fix chunkReaderTest and use AtomicInteger array instead of ConcurrentHashMap --- .../iotdb/tsfile/TsFileSequenceRead.java | 5 ++- .../db/engine/compaction/CompactionUtils.java | 7 +++- .../writer/AbstractCompactionWriter.java | 19 +++++++--- .../writer/CrossSpaceCompactionWriter.java | 37 ++++++++++--------- .../writer/InnerSpaceCompactionWriter.java | 2 - .../iotdb/tsfile/read/TsFileReaderTest.java | 4 ++ .../tsfile/utils/TsFileGeneratorForTest.java | 12 ++++++ 7 files changed, 57 insertions(+), 29 deletions(-) diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java index 6d2c2ff891784..2c1922b5406fc 100644 --- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java +++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java @@ -46,14 +46,15 @@ /** This tool is used to read TsFile sequentially, including nonAligned or aligned timeseries. */ public class TsFileSequenceRead { // if you wanna print detailed datas in pages, then turn it true. - private static boolean printDetail = false; + private static boolean printDetail = true; @SuppressWarnings({ "squid:S3776", "squid:S106" }) // Suppress high Cognitive Complexity and Standard outputs warning public static void main(String[] args) throws IOException { - String filename = "test.tsfile"; + String filename = + "C:\\IOTDB\\projects\\apache\\iotdb\\tsfile\\target\\data\\sequence\\root.testsg\\0\\0\\1649149773204-0-0-0.tsfile"; if (args.length >= 1) { filename = args[0]; } 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 1cb4a52304400..8e11db7e6b79f 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 @@ -73,7 +73,7 @@ * 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 inner space compaction. + * an seq inner space compaction. */ public class CompactionUtils { private static final Logger logger = @@ -216,6 +216,7 @@ private static void compactNonAlignedSeries( futures.get(i).get(); } catch (InterruptedException | ExecutionException e) { logger.error("SubCompactionTask meet errors ", e); + Thread.currentThread().interrupt(); throw new IOException(e); } } @@ -280,7 +281,9 @@ private static void updateDeviceStartTimeAndEndTime( for (int i = 0; i < targetFileWriters.size(); i++) { TsFileIOWriter fileIOWriter = targetFileWriters.get(i); TsFileResource fileResource = targetResources.get(i); - // skip the target file that has been deleted + // 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; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index f017e62845e0c..3a191956b9d1a 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -37,8 +37,11 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; public abstract class AbstractCompactionWriter implements AutoCloseable { + protected static final int subTaskNum = + IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum(); // subTaskId -> IChunkWriter protected Map chunkWriterMap = new ConcurrentHashMap<>(); @@ -51,15 +54,21 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { private final boolean enableMetrics = MetricConfigDescriptor.getInstance().getMetricConfig().getEnableMetric(); - // point count in current measurment, which is used to check size - private Map measurementPointCountMap = new ConcurrentHashMap<>(); + // point count of current measurment in each sub task, which is used to check size + private final AtomicInteger[] measurementPointCountMap = new AtomicInteger[subTaskNum]; + + public AbstractCompactionWriter() { + for (int i = 0; i < measurementPointCountMap.length; i++) { + measurementPointCountMap[i] = new AtomicInteger(0); + } + } public abstract void startChunkGroup(String deviceId, boolean isAlign) throws IOException; public abstract void endChunkGroup() throws IOException; public void startMeasurement(List measurementSchemaList, int subTaskId) { - measurementPointCountMap.put(subTaskId, 0); + measurementPointCountMap[subTaskId].set(0); if (isAlign) { chunkWriterMap.put(subTaskId, new AlignedChunkWriterImpl(measurementSchemaList)); } else { @@ -136,12 +145,12 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { } chunkWriter.write(timestamp); } - measurementPointCountMap.put(subTaskId, measurementPointCountMap.get(subTaskId) + 1); + measurementPointCountMap[subTaskId].getAndAdd(1); } protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) throws IOException { - if (measurementPointCountMap.get(subTaskId) % 10 == 0 && checkChunkSize(subTaskId)) { + if (measurementPointCountMap[subTaskId].get() % 10 == 0 && checkChunkSize(subTaskId)) { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); CompactionMetricsManager.recordWriteInfo( this instanceof CrossSpaceCompactionWriter diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index 50f85a3794106..9085b95761c7c 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -27,7 +27,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // target fileIOWriters @@ -36,8 +36,8 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // source tsfiles private List seqTsFileResources; - // subTaskId -> seqFileIndex - private Map seqFileIndexMap; + // each sub task has its corresponding seqFileIndex + private final AtomicInteger[] seqFileIndexArray = new AtomicInteger[subTaskNum]; private final long[] currentDeviceEndTime; @@ -55,15 +55,16 @@ public CrossSpaceCompactionWriter( this.fileWriterList.add(new TsFileIOWriter(targetResources.get(i).getTsFile())); isEmptyFile[i] = true; } + for (int i = 0; i < seqFileIndexArray.length; i++) { + seqFileIndexArray[i] = new AtomicInteger(0); + } this.seqTsFileResources = seqFileResources; - this.seqFileIndexMap = new ConcurrentHashMap<>(); } @Override public void startChunkGroup(String deviceId, boolean isAlign) throws IOException { this.deviceId = deviceId; this.isAlign = isAlign; - this.seqFileIndexMap.clear(); checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { hasTargetFileStartChunkGroup[i] = false; @@ -83,13 +84,12 @@ public void endChunkGroup() throws IOException { @Override public void endMeasurement(int subTaskId) throws IOException { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriterList.get(seqFileIndexMap.get(subTaskId))) { + synchronized (fileWriterList.get(seqFileIndexArray[subTaskId].get())) { chunkWriterMap .get(subTaskId) - .writeToFileWriter(fileWriterList.get(seqFileIndexMap.get(subTaskId))); + .writeToFileWriter(fileWriterList.get(seqFileIndexArray[subTaskId].get())); } - // chunkWriterMap.get(subTaskId)=null; - seqFileIndexMap.put(subTaskId, 0); + seqFileIndexArray[subTaskId].set(0); } @Override @@ -98,8 +98,8 @@ public void write(long timestamp, Object value, int subTaskId) throws IOExceptio checkAndMayStartChunkGroup(subTaskId); writeDataPoint(timestamp, value, subTaskId); checkChunkSizeAndMayOpenANewChunk( - fileWriterList.get(seqFileIndexMap.get(subTaskId)), subTaskId); - isEmptyFile[seqFileIndexMap.get(subTaskId)] = false; + fileWriterList.get(seqFileIndexArray[subTaskId].get()), subTaskId); + isEmptyFile[seqFileIndexArray[subTaskId].get()] = false; } @Override @@ -135,15 +135,16 @@ public List getFileIOWriter() { private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskId) throws IOException { - int fileIndex = seqFileIndexMap.computeIfAbsent(subTaskId, id -> 0); // if timestamp is later than the current source seq tsfile, than flush chunk writer - while (timestamp > currentDeviceEndTime[fileIndex]) { - if (fileIndex != seqTsFileResources.size() - 1) { + while (timestamp > currentDeviceEndTime[seqFileIndexArray[subTaskId].get()]) { + if (seqFileIndexArray[subTaskId].get() != seqTsFileResources.size() - 1) { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriterList.get(fileIndex)) { - chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriterList.get(fileIndex)); + synchronized (fileWriterList.get(seqFileIndexArray[subTaskId].get())) { + chunkWriterMap + .get(subTaskId) + .writeToFileWriter(fileWriterList.get(seqFileIndexArray[subTaskId].get())); } - seqFileIndexMap.put(subTaskId, ++fileIndex); + seqFileIndexArray[subTaskId].getAndAdd(1); } else { // If the seq file is deleted for various reasons, the following two situations may occur // when selecting the source files: (1) unseq files may have some devices or measurements @@ -181,7 +182,7 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { } private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { - int fileIndex = seqFileIndexMap.get(subTaskId); + int fileIndex = seqFileIndexArray[subTaskId].get(); if (!hasTargetFileStartChunkGroup[fileIndex]) { fileWriterList.get(fileIndex).startChunkGroup(deviceId); hasTargetFileStartChunkGroup[fileIndex] = true; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java index b0f543179d662..8abf2f980cf14 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java @@ -53,7 +53,6 @@ public void endMeasurement(int subTaskId) throws IOException { synchronized (fileWriter) { chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriter); } - // chunkWriter = null; } @Override @@ -79,7 +78,6 @@ public void close() throws IOException { if (fileWriter != null && fileWriter.canWrite()) { fileWriter.close(); } - // chunkWriter = null; chunkWriterMap.clear(); fileWriter = null; } diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/TsFileReaderTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/TsFileReaderTest.java index 5c9a85af8fc1b..92ec1c00f5223 100644 --- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/TsFileReaderTest.java +++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/TsFileReaderTest.java @@ -71,6 +71,8 @@ public void multiPagesTest() throws IOException, WriteProcessException { TSFileConfig tsFileConfig = TSFileDescriptor.getInstance().getConfig(); // make multi pages in one group + int oldPointNumInPage = tsFileConfig.getMaxNumberOfPointsInPage(); + int oldGroupSizeInByte = tsFileConfig.getGroupSizeInByte(); tsFileConfig.setMaxNumberOfPointsInPage(100); tsFileConfig.setGroupSizeInByte(100 * 1024 * 1024); TsFileWriter tsFileWriter = new TsFileWriter(file, new Schema(), tsFileConfig); @@ -120,6 +122,8 @@ public void multiPagesTest() throws IOException, WriteProcessException { tsFileReader.close(); file.delete(); + tsFileConfig.setGroupSizeInByte(oldGroupSizeInByte); + tsFileConfig.setMaxNumberOfPointsInPage(oldPointNumInPage); } @Test diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/TsFileGeneratorForTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/TsFileGeneratorForTest.java index 486131ffadbfe..52195ab823074 100755 --- a/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/TsFileGeneratorForTest.java +++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/TsFileGeneratorForTest.java @@ -176,6 +176,9 @@ public static void write() throws IOException { Schema schema = generateTestSchema(); + int oldGroupSizeInByte = TSFileDescriptor.getInstance().getConfig().getGroupSizeInByte(); + int oldMaxPointNumInPage = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); TSFileDescriptor.getInstance().getConfig().setGroupSizeInByte(chunkGroupSize); TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(pageSize); @@ -190,6 +193,9 @@ public static void write() throws IOException { } } catch (WriteProcessException e) { e.printStackTrace(); + } finally { + TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(oldMaxPointNumInPage); + TSFileDescriptor.getInstance().getConfig().setGroupSizeInByte(oldGroupSizeInByte); } } @@ -281,6 +287,9 @@ public static void generateAlignedTsFile(int rowCount, int chunkGroupSize, int p Assert.assertTrue(file.delete()); } file.getParentFile().mkdirs(); + int oldGroupSizeInByte = TSFileDescriptor.getInstance().getConfig().getGroupSizeInByte(); + int oldMaxPointNumInPage = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); TSFileDescriptor.getInstance().getConfig().setGroupSizeInByte(chunkGroupSize); TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(pageSize); try (TsFileWriter tsFileWriter = new TsFileWriter(file)) { @@ -309,6 +318,9 @@ public static void generateAlignedTsFile(int rowCount, int chunkGroupSize, int p } catch (IOException | WriteProcessException e) { e.printStackTrace(); + } finally { + TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(oldMaxPointNumInPage); + TSFileDescriptor.getInstance().getConfig().setGroupSizeInByte(oldGroupSizeInByte); } } From f3040318f740ec362dde19eea063be20faa24fc4 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Tue, 5 Apr 2022 19:12:10 +0800 Subject: [PATCH 05/17] add Thread.interrupted --- .../org/apache/iotdb/db/engine/compaction/CompactionUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8e11db7e6b79f..705b945756563 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 @@ -216,7 +216,7 @@ private static void compactNonAlignedSeries( futures.get(i).get(); } catch (InterruptedException | ExecutionException e) { logger.error("SubCompactionTask meet errors ", e); - Thread.currentThread().interrupt(); + Thread.interrupted(); throw new IOException(e); } } From 448558709c4e95c4212ef83055c2cb545f56ab07 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Wed, 6 Apr 2022 18:42:41 +0800 Subject: [PATCH 06/17] fix hasStartChunk param in CrossSpaceCompactionWriter safe in concurrent thread --- .../apache/iotdb/tsfile/TsFileSequenceRead.java | 2 +- .../db/engine/compaction/CompactionUtils.java | 4 ++-- .../writer/CrossSpaceCompactionWriter.java | 16 ++++++++++------ .../db/engine/storagegroup/TsFileResource.java | 3 +-- .../cross/CrossSpaceCompactionTest.java | 4 +++- 5 files changed, 17 insertions(+), 12 deletions(-) diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java index 2c1922b5406fc..e9563c423cf48 100644 --- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java +++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java @@ -54,7 +54,7 @@ public class TsFileSequenceRead { }) // Suppress high Cognitive Complexity and Standard outputs warning public static void main(String[] args) throws IOException { String filename = - "C:\\IOTDB\\projects\\apache\\iotdb\\tsfile\\target\\data\\sequence\\root.testsg\\0\\0\\1649149773204-0-0-0.tsfile"; + "C:\\IOTDB\\projects\\apache\\iotdb\\server\\target\\data\\sequence\\root.compactionTest\\0\\0\\1649239225422-3-0-1.tsfile"; if (args.length >= 1) { filename = args[0]; } 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 705b945756563..a4328449cac50 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 @@ -178,7 +178,7 @@ private static void compactNonAlignedSeries( AbstractCompactionWriter compactionWriter, QueryContext queryContext, QueryDataSource queryDataSource) - throws IOException { + throws IOException, InterruptedException { MultiTsFileDeviceIterator.MeasurementIterator measurementIterator = deviceIterator.iterateNotAlignedSeries(device, false); Set allMeasurements = measurementIterator.getAllMeasurements(); @@ -217,7 +217,7 @@ private static void compactNonAlignedSeries( } catch (InterruptedException | ExecutionException e) { logger.error("SubCompactionTask meet errors ", e); Thread.interrupted(); - throw new IOException(e); + throw new InterruptedException(); } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index 9085b95761c7c..cd0c59c19e257 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -27,6 +27,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { @@ -43,16 +44,19 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { private final boolean[] isEmptyFile; - private final boolean[] hasTargetFileStartChunkGroup; + // private final boolean[] hasTargetFileStartChunkGroup; + private final AtomicBoolean[] hasTargetFileStartChunkGroup; public CrossSpaceCompactionWriter( List targetResources, List seqFileResources) throws IOException { currentDeviceEndTime = new long[seqFileResources.size()]; isEmptyFile = new boolean[seqFileResources.size()]; - hasTargetFileStartChunkGroup = new boolean[seqFileResources.size()]; + // hasTargetFileStartChunkGroup = new boolean[seqFileResources.size()]; + hasTargetFileStartChunkGroup = new AtomicBoolean[seqFileResources.size()]; for (int i = 0; i < targetResources.size(); i++) { this.fileWriterList.add(new TsFileIOWriter(targetResources.get(i).getTsFile())); + this.hasTargetFileStartChunkGroup[i] = new AtomicBoolean(false); isEmptyFile[i] = true; } for (int i = 0; i < seqFileIndexArray.length; i++) { @@ -67,14 +71,15 @@ public void startChunkGroup(String deviceId, boolean isAlign) throws IOException this.isAlign = isAlign; checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { - hasTargetFileStartChunkGroup[i] = false; + // hasTargetFileStartChunkGroup[i] = false; + hasTargetFileStartChunkGroup[i].set(false); } } @Override public void endChunkGroup() throws IOException { for (int i = 0; i < seqTsFileResources.size(); i++) { - if (hasTargetFileStartChunkGroup[i]) { + if (hasTargetFileStartChunkGroup[i].get()) { fileWriterList.get(i).endChunkGroup(); } } @@ -183,9 +188,8 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { int fileIndex = seqFileIndexArray[subTaskId].get(); - if (!hasTargetFileStartChunkGroup[fileIndex]) { + if (hasTargetFileStartChunkGroup[fileIndex].compareAndSet(false, true)) { fileWriterList.get(fileIndex).startChunkGroup(deviceId); - hasTargetFileStartChunkGroup[fileIndex] = true; } } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 6c9e83c8b5f78..e3198d91177f2 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -43,7 +43,6 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter; import org.apache.iotdb.tsfile.utils.FilePathUtils; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -550,7 +549,7 @@ void moveTo(File targetDir) { @Override public String toString() { - return String.format("file is %s, status: ", file.toString(), status); + return String.format("file is %s, status: %s", file.toString(), status); } @Override 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 a413419f32576..43e78bbd90dc5 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 @@ -21,6 +21,7 @@ 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.selector.ICrossSpaceMergeFileSelector; import org.apache.iotdb.db.engine.compaction.cross.rewrite.selector.RewriteCompactionFileSelector; @@ -45,7 +46,6 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.read.TimeValuePair; import org.apache.iotdb.tsfile.utils.Pair; - import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -108,6 +108,7 @@ public void setUp() throws MetadataException { TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap()); } + CompactionTaskManager.getInstance().start(); Thread.currentThread().setName("pool-1-IoTDB-Compaction-1"); } @@ -118,6 +119,7 @@ public void tearDown() throws IOException, StorageEngineException { ChunkCache.getInstance().clear(); TimeSeriesMetadataCache.getInstance().clear(); IoTDB.configManager.clear(); + CompactionTaskManager.getInstance().stop(); EnvironmentUtils.cleanAllDir(); Thread.currentThread().setName(oldThreadName); new CompactionConfigRestorer().restoreCompactionConfig(); From 3c26be131785fbe543f8b9e5afd4c5c99887e5a7 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Thu, 7 Apr 2022 11:24:12 +0800 Subject: [PATCH 07/17] change AtomicInteger[] to ConcurrentHashMap --- .../writer/AbstractCompactionWriter.java | 17 +++------- .../writer/CrossSpaceCompactionWriter.java | 34 +++++++++---------- 2 files changed, 22 insertions(+), 29 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index 3a191956b9d1a..85320c4e4f88e 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -37,7 +37,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; public abstract class AbstractCompactionWriter implements AutoCloseable { protected static final int subTaskNum = @@ -54,21 +53,15 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { private final boolean enableMetrics = MetricConfigDescriptor.getInstance().getMetricConfig().getEnableMetric(); - // point count of current measurment in each sub task, which is used to check size - private final AtomicInteger[] measurementPointCountMap = new AtomicInteger[subTaskNum]; - - public AbstractCompactionWriter() { - for (int i = 0; i < measurementPointCountMap.length; i++) { - measurementPointCountMap[i] = new AtomicInteger(0); - } - } + // point count in current measurment, which is used to check size + private Map measurementPointCountMap = new ConcurrentHashMap<>(); public abstract void startChunkGroup(String deviceId, boolean isAlign) throws IOException; public abstract void endChunkGroup() throws IOException; public void startMeasurement(List measurementSchemaList, int subTaskId) { - measurementPointCountMap[subTaskId].set(0); + measurementPointCountMap.put(subTaskId, 0); if (isAlign) { chunkWriterMap.put(subTaskId, new AlignedChunkWriterImpl(measurementSchemaList)); } else { @@ -145,12 +138,12 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { } chunkWriter.write(timestamp); } - measurementPointCountMap[subTaskId].getAndAdd(1); + measurementPointCountMap.put(subTaskId, measurementPointCountMap.get(subTaskId) + 1); } protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) throws IOException { - if (measurementPointCountMap[subTaskId].get() % 10 == 0 && checkChunkSize(subTaskId)) { + if (measurementPointCountMap.get(subTaskId) % 10 == 0 && checkChunkSize(subTaskId)) { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); CompactionMetricsManager.recordWriteInfo( this instanceof CrossSpaceCompactionWriter diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index cd0c59c19e257..e5e1977da12de 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -27,8 +27,8 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // target fileIOWriters @@ -37,8 +37,8 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // source tsfiles private List seqTsFileResources; - // each sub task has its corresponding seqFileIndex - private final AtomicInteger[] seqFileIndexArray = new AtomicInteger[subTaskNum]; + // subTaskId -> seqFileIndex + private final Map seqFileIndexMap; private final long[] currentDeviceEndTime; @@ -59,16 +59,15 @@ public CrossSpaceCompactionWriter( this.hasTargetFileStartChunkGroup[i] = new AtomicBoolean(false); isEmptyFile[i] = true; } - for (int i = 0; i < seqFileIndexArray.length; i++) { - seqFileIndexArray[i] = new AtomicInteger(0); - } this.seqTsFileResources = seqFileResources; + this.seqFileIndexMap = new ConcurrentHashMap<>(); } @Override public void startChunkGroup(String deviceId, boolean isAlign) throws IOException { this.deviceId = deviceId; this.isAlign = isAlign; + this.seqFileIndexMap.clear(); checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { // hasTargetFileStartChunkGroup[i] = false; @@ -89,12 +88,13 @@ public void endChunkGroup() throws IOException { @Override public void endMeasurement(int subTaskId) throws IOException { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriterList.get(seqFileIndexArray[subTaskId].get())) { + synchronized (fileWriterList.get(seqFileIndexMap.get(subTaskId))) { chunkWriterMap .get(subTaskId) - .writeToFileWriter(fileWriterList.get(seqFileIndexArray[subTaskId].get())); + .writeToFileWriter(fileWriterList.get(seqFileIndexMap.get(subTaskId))); } - seqFileIndexArray[subTaskId].set(0); + // chunkWriterMap.get(subTaskId)=null; + seqFileIndexMap.put(subTaskId, 0); } @Override @@ -103,8 +103,8 @@ public void write(long timestamp, Object value, int subTaskId) throws IOExceptio checkAndMayStartChunkGroup(subTaskId); writeDataPoint(timestamp, value, subTaskId); checkChunkSizeAndMayOpenANewChunk( - fileWriterList.get(seqFileIndexArray[subTaskId].get()), subTaskId); - isEmptyFile[seqFileIndexArray[subTaskId].get()] = false; + fileWriterList.get(seqFileIndexMap.get(subTaskId)), subTaskId); + isEmptyFile[seqFileIndexMap.get(subTaskId)] = false; } @Override @@ -141,15 +141,15 @@ public List getFileIOWriter() { private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskId) throws IOException { // if timestamp is later than the current source seq tsfile, than flush chunk writer - while (timestamp > currentDeviceEndTime[seqFileIndexArray[subTaskId].get()]) { - if (seqFileIndexArray[subTaskId].get() != seqTsFileResources.size() - 1) { + while (timestamp > currentDeviceEndTime[seqFileIndexMap.get(subTaskId)]) { + if (seqFileIndexMap.get(subTaskId) != seqTsFileResources.size() - 1) { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriterList.get(seqFileIndexArray[subTaskId].get())) { + synchronized (fileWriterList.get(seqFileIndexMap.get(subTaskId))) { chunkWriterMap .get(subTaskId) - .writeToFileWriter(fileWriterList.get(seqFileIndexArray[subTaskId].get())); + .writeToFileWriter(fileWriterList.get(seqFileIndexMap.get(subTaskId))); } - seqFileIndexArray[subTaskId].getAndAdd(1); + seqFileIndexMap.put(subTaskId, seqFileIndexMap.get(subTaskId) + 1); } else { // If the seq file is deleted for various reasons, the following two situations may occur // when selecting the source files: (1) unseq files may have some devices or measurements @@ -187,7 +187,7 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { } private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { - int fileIndex = seqFileIndexArray[subTaskId].get(); + int fileIndex = seqFileIndexMap.get(subTaskId); if (hasTargetFileStartChunkGroup[fileIndex].compareAndSet(false, true)) { fileWriterList.get(fileIndex).startChunkGroup(deviceId); } From 386479ace3858740e70b7eac9499ca59305de056 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Thu, 7 Apr 2022 16:46:21 +0800 Subject: [PATCH 08/17] fix concurrency bug: The first thread has not flushed ChunkGroupHeader, and the following threads will directly flush Chunk --- .../iotdb/tsfile/TsFileSequenceRead.java | 3 +- .../writer/AbstractCompactionWriter.java | 13 +- .../writer/CrossSpaceCompactionWriter.java | 69 ++- .../compaction/CompactionUtilsTest.java | 478 +++++++++++------- 4 files changed, 356 insertions(+), 207 deletions(-) diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java index e9563c423cf48..825896c3030b1 100644 --- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java +++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java @@ -53,8 +53,7 @@ public class TsFileSequenceRead { "squid:S106" }) // Suppress high Cognitive Complexity and Standard outputs warning public static void main(String[] args) throws IOException { - String filename = - "C:\\IOTDB\\projects\\apache\\iotdb\\server\\target\\data\\sequence\\root.compactionTest\\0\\0\\1649239225422-3-0-1.tsfile"; + String filename = "C:\\Users\\BensonChou\\Desktop\\TestData\\3\\1649311240706-3-0-1.tsfile"; if (args.length >= 1) { filename = args[0]; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index 85320c4e4f88e..b4395aaebd20b 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -54,7 +54,14 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { MetricConfigDescriptor.getInstance().getMetricConfig().getEnableMetric(); // point count in current measurment, which is used to check size - private Map measurementPointCountMap = new ConcurrentHashMap<>(); + protected Map measurementPointCountMap = new ConcurrentHashMap<>(); + // private final AtomicInteger[] measurementPointCountMap = new AtomicInteger[subTaskNum]; + // + // public AbstractCompactionWriter() { + // for (int i = 0; i < measurementPointCountMap.length; i++) { + // measurementPointCountMap[i] = new AtomicInteger(0); + // } + // } public abstract void startChunkGroup(String deviceId, boolean isAlign) throws IOException; @@ -62,6 +69,7 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { public void startMeasurement(List measurementSchemaList, int subTaskId) { measurementPointCountMap.put(subTaskId, 0); + // measurementPointCountMap[subTaskId].set(0); if (isAlign) { chunkWriterMap.put(subTaskId, new AlignedChunkWriterImpl(measurementSchemaList)); } else { @@ -139,6 +147,7 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { chunkWriter.write(timestamp); } measurementPointCountMap.put(subTaskId, measurementPointCountMap.get(subTaskId) + 1); + // measurementPointCountMap[subTaskId].getAndAdd(1); } protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) @@ -158,7 +167,7 @@ protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int } } - private boolean checkChunkSize(int subTaskId) { + protected boolean checkChunkSize(int subTaskId) { if (chunkWriterMap.get(subTaskId) instanceof AlignedChunkWriterImpl) { return ((AlignedChunkWriterImpl) chunkWriterMap.get(subTaskId)) .checkIsChunkSizeOverThreshold(targetChunkSize); diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index e5e1977da12de..6ef7731a72d11 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -18,6 +18,9 @@ */ package org.apache.iotdb.db.engine.compaction.writer; +import org.apache.iotdb.db.engine.compaction.CompactionMetricsManager; +import org.apache.iotdb.db.engine.compaction.constant.CompactionType; +import org.apache.iotdb.db.engine.compaction.constant.ProcessChunkType; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.query.control.FileReaderManager; import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata; @@ -28,7 +31,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // target fileIOWriters @@ -45,7 +48,11 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { private final boolean[] isEmptyFile; // private final boolean[] hasTargetFileStartChunkGroup; - private final AtomicBoolean[] hasTargetFileStartChunkGroup; + // it has 3 values, which is + // 0: has not start chunk group yet + // 1: start flushing chunk group header + // 2: finish flushing chunk group header + private final AtomicInteger[] hasTargetFileStartChunkGroup; public CrossSpaceCompactionWriter( List targetResources, List seqFileResources) @@ -53,10 +60,10 @@ public CrossSpaceCompactionWriter( currentDeviceEndTime = new long[seqFileResources.size()]; isEmptyFile = new boolean[seqFileResources.size()]; // hasTargetFileStartChunkGroup = new boolean[seqFileResources.size()]; - hasTargetFileStartChunkGroup = new AtomicBoolean[seqFileResources.size()]; + hasTargetFileStartChunkGroup = new AtomicInteger[seqFileResources.size()]; for (int i = 0; i < targetResources.size(); i++) { this.fileWriterList.add(new TsFileIOWriter(targetResources.get(i).getTsFile())); - this.hasTargetFileStartChunkGroup[i] = new AtomicBoolean(false); + this.hasTargetFileStartChunkGroup[i] = new AtomicInteger(0); isEmptyFile[i] = true; } this.seqTsFileResources = seqFileResources; @@ -71,14 +78,14 @@ public void startChunkGroup(String deviceId, boolean isAlign) throws IOException checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { // hasTargetFileStartChunkGroup[i] = false; - hasTargetFileStartChunkGroup[i].set(false); + hasTargetFileStartChunkGroup[i].set(0); } } @Override public void endChunkGroup() throws IOException { for (int i = 0; i < seqTsFileResources.size(); i++) { - if (hasTargetFileStartChunkGroup[i].get()) { + if (hasTargetFileStartChunkGroup[i].get() == 2) { fileWriterList.get(i).endChunkGroup(); } } @@ -87,12 +94,7 @@ public void endChunkGroup() throws IOException { @Override public void endMeasurement(int subTaskId) throws IOException { - writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriterList.get(seqFileIndexMap.get(subTaskId))) { - chunkWriterMap - .get(subTaskId) - .writeToFileWriter(fileWriterList.get(seqFileIndexMap.get(subTaskId))); - } + flushChunkToFileWriter(subTaskId); // chunkWriterMap.get(subTaskId)=null; seqFileIndexMap.put(subTaskId, 0); } @@ -140,16 +142,12 @@ public List getFileIOWriter() { private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskId) throws IOException { + int fileIndex = seqFileIndexMap.computeIfAbsent(subTaskId, id -> 0); // if timestamp is later than the current source seq tsfile, than flush chunk writer - while (timestamp > currentDeviceEndTime[seqFileIndexMap.get(subTaskId)]) { - if (seqFileIndexMap.get(subTaskId) != seqTsFileResources.size() - 1) { - writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriterList.get(seqFileIndexMap.get(subTaskId))) { - chunkWriterMap - .get(subTaskId) - .writeToFileWriter(fileWriterList.get(seqFileIndexMap.get(subTaskId))); - } - seqFileIndexMap.put(subTaskId, seqFileIndexMap.get(subTaskId) + 1); + while (timestamp > currentDeviceEndTime[fileIndex]) { + if (fileIndex != seqTsFileResources.size() - 1) { + flushChunkToFileWriter(subTaskId); + seqFileIndexMap.put(subTaskId, ++fileIndex); } else { // If the seq file is deleted for various reasons, the following two situations may occur // when selecting the source files: (1) unseq files may have some devices or measurements @@ -188,8 +186,35 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { int fileIndex = seqFileIndexMap.get(subTaskId); - if (hasTargetFileStartChunkGroup[fileIndex].compareAndSet(false, true)) { + if (hasTargetFileStartChunkGroup[fileIndex].compareAndSet(0, 1)) { fileWriterList.get(fileIndex).startChunkGroup(deviceId); + hasTargetFileStartChunkGroup[fileIndex].set(2); + } + } + + private void flushChunkToFileWriter(int subTaskId) throws IOException { + writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + while (hasTargetFileStartChunkGroup[seqFileIndexMap.get(subTaskId)].get() == 1) { + // wait until the target file has finished flushing chunk group header + } + synchronized (fileWriterList.get(seqFileIndexMap.get(subTaskId))) { + chunkWriterMap + .get(subTaskId) + .writeToFileWriter(fileWriterList.get(seqFileIndexMap.get(subTaskId))); + } + } + + protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) + throws IOException { + if (measurementPointCountMap.get(subTaskId) % 10 == 0 && checkChunkSize(subTaskId)) { + flushChunkToFileWriter(subTaskId); + CompactionMetricsManager.recordWriteInfo( + this instanceof CrossSpaceCompactionWriter + ? CompactionType.CROSS_COMPACTION + : CompactionType.INNER_UNSEQ_COMPACTION, + ProcessChunkType.DESERIALIZE_CHUNK, + this.isAlign, + chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); } } } 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/CompactionUtilsTest.java index f64beb75f3e6d..c5947d4ffa7d5 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/CompactionUtilsTest.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.IllegalPathException; import org.apache.iotdb.db.exception.metadata.MetadataException; @@ -40,12 +41,13 @@ import org.apache.iotdb.tsfile.utils.TsPrimitiveType; import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; - import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -1873,46 +1875,52 @@ public void testCrossSpaceCompactionWithDifferentTimeseries() null, true); int count = 0; - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); + try { + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); + } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - count++; - batchData.next(); } - } - tsFilesReader.close(); - if (i < 2 && j < 3) { - assertEquals(1280, count); - } else if (i < 1 && j < 4) { - assertEquals(1230, count); - } else if (i == 0) { - assertEquals(800, count); - } else if ((i == 1 && j == 4)) { - assertEquals(600, count); - } else if (i < 3 && j < 4) { - assertEquals(1200, count); - } else { - assertEquals(600, count); + tsFilesReader.close(); + if (i < 2 && j < 3) { + assertEquals(1280, count); + } else if (i < 1 && j < 4) { + assertEquals(1230, count); + } else if (i == 0) { + assertEquals(800, count); + } else if ((i == 1 && j == 4)) { + assertEquals(600, count); + } else if (i < 3 && j < 4) { + assertEquals(1200, count); + } else { + assertEquals(600, count); + } + + } catch (Throwable e) { + e.printStackTrace(); + Thread.currentThread().stop(); } } } @@ -2038,48 +2046,57 @@ public void testCrossSpaceCompactionWithAllDataDeletedInTimeseries() null, true); int count = 0; - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); + try { + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); + } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - count++; - batchData.next(); } - } - tsFilesReader.close(); - if ((i == 0 && j == 0) || (i == 0 && j == 1) || (i == 2 && j == 4) || (i == 3 && j == 4)) { - assertEquals(0, count); - } else if (i < 2 && j < 3) { - assertEquals(1280, count); - } else if (i < 1 && j < 4) { - assertEquals(1230, count); - } else if (i == 0) { - assertEquals(800, count); - } else if ((i == 1 && j == 4)) { - assertEquals(600, count); - } else if (i < 3) { - assertEquals(1200, count); - } else { - assertEquals(600, count); + tsFilesReader.close(); + if ((i == 0 && j == 0) + || (i == 0 && j == 1) + || (i == 2 && j == 4) + || (i == 3 && j == 4)) { + assertEquals(0, count); + } else if (i < 2 && j < 3) { + assertEquals(1280, count); + } else if (i < 1 && j < 4) { + assertEquals(1230, count); + } else if (i == 0) { + assertEquals(800, count); + } else if ((i == 1 && j == 4)) { + assertEquals(600, count); + } else if (i < 3) { + assertEquals(1200, count); + } else { + assertEquals(600, count); + } + + } catch (Throwable e) { + e.printStackTrace(); + Thread.currentThread().stop(); } } } @@ -2201,44 +2218,50 @@ public void testCrossSpaceCompactionWithAllDataDeletedInDevice() null, true); int count = 0; - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); + try { + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); + } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - count++; - batchData.next(); } - } - tsFilesReader.close(); - if (i == 0 || i == 2) { - assertEquals(0, count); - } else if (i < 2 && j < 3) { - assertEquals(1280, count); - } else if ((i == 1 && j == 4)) { - assertEquals(600, count); - } else if (i < 3) { - assertEquals(1200, count); - } else { - assertEquals(600, count); + tsFilesReader.close(); + if (i == 0 || i == 2) { + assertEquals(0, count); + } else if (i < 2 && j < 3) { + assertEquals(1280, count); + } else if ((i == 1 && j == 4)) { + assertEquals(600, count); + } else if (i < 3) { + assertEquals(1200, count); + } else { + assertEquals(600, count); + } + + } catch (Throwable e) { + e.printStackTrace(); + Thread.currentThread().stop(); } } } @@ -2356,38 +2379,44 @@ public void testCrossSpaceCompactionWithAllDataDeletedInOneTargetFile() null, true); int count = 0; - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); + try { + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); + } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - count++; - batchData.next(); } - } - tsFilesReader.close(); - if (i == 0 || i == 1 || i == 2) { - assertEquals(0, count); - } else { - assertEquals(600, count); + tsFilesReader.close(); + if (i == 0 || i == 1 || i == 2) { + assertEquals(0, count); + } else { + assertEquals(600, count); + } + + } catch (Throwable e) { + e.printStackTrace(); + Thread.currentThread().stop(); } } } @@ -2513,48 +2542,54 @@ public void testCrossSpaceCompactionWithAllDataDeletedInDeviceInSeqFiles() null, true); int count = 0; - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); + try { + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); + } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } + tsFilesReader.close(); + if (i < 1) { + if (j < 4) { + assertEquals(630, count); } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); + assertEquals(200, count); + } + } else if (i < 3) { + if (j < 4) { + assertEquals(600, count); + } else { + assertEquals(0, count); } - count++; - batchData.next(); - } - } - tsFilesReader.close(); - if (i < 1) { - if (j < 4) { - assertEquals(630, count); } else { - assertEquals(200, count); - } - } else if (i < 3) { - if (j < 4) { assertEquals(600, count); - } else { - assertEquals(0, count); } - } else { - assertEquals(600, count); + + } catch (Throwable e) { + e.printStackTrace(); + Thread.currentThread().stop(); } } } @@ -3518,6 +3553,87 @@ public void testCrossSpaceCompactionWithDeviceMaxTimeLaterInUnseqFile() { } } + @Ignore + @Test + public void testReadWrongFile() throws IllegalPathException, IOException { + List targetResources = new ArrayList<>(); + TsFileResource resource = + new TsFileResource( + new File("C:\\Users\\BensonChou\\Desktop\\TestData\\3\\1649311240699-2-0-1.tsfile")); + resource.deserialize(); + resource.setStatus(TsFileResourceStatus.CLOSED); + targetResources.add(resource); + resource = + new TsFileResource( + new File("C:\\Users\\BensonChou\\Desktop\\TestData\\3\\1649311240706-3-0-1.tsfile")); + resource.deserialize(); + resource.setStatus(TsFileResourceStatus.CLOSED); + targetResources.add(resource); + Map measurementMaxTime = new HashMap<>(); + for (int i = 0; i < 4; i++) { + for (int j = 0; j < 5; j++) { + measurementMaxTime.putIfAbsent( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + Long.MIN_VALUE); + PartialPath path = + new MeasurementPath( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i, + "s" + j, + new MeasurementSchema("s" + j, TSDataType.INT64)); + IBatchReader tsFilesReader = + new SeriesRawDataBatchReader( + path, + TSDataType.INT64, + EnvironmentUtils.TEST_QUERY_CONTEXT, + targetResources, + new ArrayList<>(), + null, + null, + true); + int count = 0; + try { + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); + } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); + } + } + tsFilesReader.close(); + if (i == 0 || i == 1 || i == 2) { + assertEquals(0, count); + } else { + assertEquals(600, count); + } + + } catch (Throwable e) { + e.printStackTrace(); + Thread.currentThread().stop(); + } + } + } + } + private void generateModsFile( List seriesPaths, List resources, long startValue, long endValue) throws IllegalPathException, IOException { From 144775cf0daf76e61544ba2dc530b316fd95cefe Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Thu, 7 Apr 2022 17:09:41 +0800 Subject: [PATCH 09/17] change concurrenctHashMap to int[] array --- .../writer/AbstractCompactionWriter.java | 20 +++------ .../writer/CrossSpaceCompactionWriter.java | 41 ++++++++----------- .../engine/storagegroup/TsFileResource.java | 1 + .../compaction/CompactionUtilsTest.java | 1 + .../cross/CrossSpaceCompactionTest.java | 1 + 5 files changed, 26 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index b4395aaebd20b..a3304856199ee 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -53,23 +53,16 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { private final boolean enableMetrics = MetricConfigDescriptor.getInstance().getMetricConfig().getEnableMetric(); - // point count in current measurment, which is used to check size - protected Map measurementPointCountMap = new ConcurrentHashMap<>(); - // private final AtomicInteger[] measurementPointCountMap = new AtomicInteger[subTaskNum]; - // - // public AbstractCompactionWriter() { - // for (int i = 0; i < measurementPointCountMap.length; i++) { - // measurementPointCountMap[i] = new AtomicInteger(0); - // } - // } + // Each sub task has point count in current measurment, which is used to check size. + // The index of the array corresponds to subTaskId. + protected int[] measurementPointCountMap = new int[subTaskNum]; public abstract void startChunkGroup(String deviceId, boolean isAlign) throws IOException; public abstract void endChunkGroup() throws IOException; public void startMeasurement(List measurementSchemaList, int subTaskId) { - measurementPointCountMap.put(subTaskId, 0); - // measurementPointCountMap[subTaskId].set(0); + measurementPointCountMap[subTaskId] = 0; if (isAlign) { chunkWriterMap.put(subTaskId, new AlignedChunkWriterImpl(measurementSchemaList)); } else { @@ -146,13 +139,12 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { } chunkWriter.write(timestamp); } - measurementPointCountMap.put(subTaskId, measurementPointCountMap.get(subTaskId) + 1); - // measurementPointCountMap[subTaskId].getAndAdd(1); + measurementPointCountMap[subTaskId] += 1; } protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) throws IOException { - if (measurementPointCountMap.get(subTaskId) % 10 == 0 && checkChunkSize(subTaskId)) { + if (measurementPointCountMap[subTaskId] % 10 == 0 && checkChunkSize(subTaskId)) { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); CompactionMetricsManager.recordWriteInfo( this instanceof CrossSpaceCompactionWriter diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index 6ef7731a72d11..e95577a10345f 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -30,7 +30,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { @@ -40,15 +39,15 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // source tsfiles private List seqTsFileResources; - // subTaskId -> seqFileIndex - private final Map seqFileIndexMap; + // Each sub task has its corresponding seq file index. + // The index of the array corresponds to subTaskId. + private int[] seqFileIndexMap = new int[subTaskNum]; private final long[] currentDeviceEndTime; private final boolean[] isEmptyFile; - // private final boolean[] hasTargetFileStartChunkGroup; - // it has 3 values, which is + // This variable has three values, which is // 0: has not start chunk group yet // 1: start flushing chunk group header // 2: finish flushing chunk group header @@ -59,7 +58,6 @@ public CrossSpaceCompactionWriter( throws IOException { currentDeviceEndTime = new long[seqFileResources.size()]; isEmptyFile = new boolean[seqFileResources.size()]; - // hasTargetFileStartChunkGroup = new boolean[seqFileResources.size()]; hasTargetFileStartChunkGroup = new AtomicInteger[seqFileResources.size()]; for (int i = 0; i < targetResources.size(); i++) { this.fileWriterList.add(new TsFileIOWriter(targetResources.get(i).getTsFile())); @@ -67,14 +65,13 @@ public CrossSpaceCompactionWriter( isEmptyFile[i] = true; } this.seqTsFileResources = seqFileResources; - this.seqFileIndexMap = new ConcurrentHashMap<>(); } @Override public void startChunkGroup(String deviceId, boolean isAlign) throws IOException { this.deviceId = deviceId; this.isAlign = isAlign; - this.seqFileIndexMap.clear(); + this.seqFileIndexMap = new int[subTaskNum]; checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { // hasTargetFileStartChunkGroup[i] = false; @@ -96,7 +93,7 @@ public void endChunkGroup() throws IOException { public void endMeasurement(int subTaskId) throws IOException { flushChunkToFileWriter(subTaskId); // chunkWriterMap.get(subTaskId)=null; - seqFileIndexMap.put(subTaskId, 0); + seqFileIndexMap[subTaskId] = 0; } @Override @@ -104,9 +101,8 @@ public void write(long timestamp, Object value, int subTaskId) throws IOExceptio checkTimeAndMayFlushChunkToCurrentFile(timestamp, subTaskId); checkAndMayStartChunkGroup(subTaskId); writeDataPoint(timestamp, value, subTaskId); - checkChunkSizeAndMayOpenANewChunk( - fileWriterList.get(seqFileIndexMap.get(subTaskId)), subTaskId); - isEmptyFile[seqFileIndexMap.get(subTaskId)] = false; + checkChunkSizeAndMayOpenANewChunk(fileWriterList.get(seqFileIndexMap[subTaskId]), subTaskId); + isEmptyFile[seqFileIndexMap[subTaskId]] = false; } @Override @@ -142,12 +138,12 @@ public List getFileIOWriter() { private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskId) throws IOException { - int fileIndex = seqFileIndexMap.computeIfAbsent(subTaskId, id -> 0); + int fileIndex = seqFileIndexMap[subTaskId]; // if timestamp is later than the current source seq tsfile, than flush chunk writer while (timestamp > currentDeviceEndTime[fileIndex]) { if (fileIndex != seqTsFileResources.size() - 1) { flushChunkToFileWriter(subTaskId); - seqFileIndexMap.put(subTaskId, ++fileIndex); + seqFileIndexMap[subTaskId] = ++fileIndex; } else { // If the seq file is deleted for various reasons, the following two situations may occur // when selecting the source files: (1) unseq files may have some devices or measurements @@ -185,7 +181,7 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { } private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { - int fileIndex = seqFileIndexMap.get(subTaskId); + int fileIndex = seqFileIndexMap[subTaskId]; if (hasTargetFileStartChunkGroup[fileIndex].compareAndSet(0, 1)) { fileWriterList.get(fileIndex).startChunkGroup(deviceId); hasTargetFileStartChunkGroup[fileIndex].set(2); @@ -193,25 +189,22 @@ private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { } private void flushChunkToFileWriter(int subTaskId) throws IOException { + int fileIndex = seqFileIndexMap[subTaskId]; writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - while (hasTargetFileStartChunkGroup[seqFileIndexMap.get(subTaskId)].get() == 1) { + while (hasTargetFileStartChunkGroup[fileIndex].get() == 1) { // wait until the target file has finished flushing chunk group header } - synchronized (fileWriterList.get(seqFileIndexMap.get(subTaskId))) { - chunkWriterMap - .get(subTaskId) - .writeToFileWriter(fileWriterList.get(seqFileIndexMap.get(subTaskId))); + synchronized (fileWriterList.get(fileIndex)) { + chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriterList.get(fileIndex)); } } protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) throws IOException { - if (measurementPointCountMap.get(subTaskId) % 10 == 0 && checkChunkSize(subTaskId)) { + if (measurementPointCountMap[subTaskId] % 10 == 0 && checkChunkSize(subTaskId)) { flushChunkToFileWriter(subTaskId); CompactionMetricsManager.recordWriteInfo( - this instanceof CrossSpaceCompactionWriter - ? CompactionType.CROSS_COMPACTION - : CompactionType.INNER_UNSEQ_COMPACTION, + CompactionType.CROSS_COMPACTION, ProcessChunkType.DESERIALIZE_CHUNK, this.isAlign, chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index e3198d91177f2..d7dd49f49a878 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -43,6 +43,7 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter; import org.apache.iotdb.tsfile.utils.FilePathUtils; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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/CompactionUtilsTest.java index c5947d4ffa7d5..2d1c489889387 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/CompactionUtilsTest.java @@ -41,6 +41,7 @@ import org.apache.iotdb.tsfile.utils.TsPrimitiveType; import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; + import org.junit.After; import org.junit.Assert; import org.junit.Before; 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 43e78bbd90dc5..5e0824911e72c 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 @@ -46,6 +46,7 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.read.TimeValuePair; import org.apache.iotdb.tsfile.utils.Pair; + import org.junit.After; import org.junit.Before; import org.junit.Test; From 0053bf015819c2ccee745c530571ca7545483c61 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Thu, 7 Apr 2022 17:22:59 +0800 Subject: [PATCH 10/17] merge master --- .../iotdb/tsfile/TsFileSequenceRead.java | 4 +- .../writer/CrossSpaceCompactionWriter.java | 4 +- .../compaction/CompactionUtilsTest.java | 477 +++++++----------- 3 files changed, 184 insertions(+), 301 deletions(-) diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java index 825896c3030b1..6d2c2ff891784 100644 --- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java +++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java @@ -46,14 +46,14 @@ /** This tool is used to read TsFile sequentially, including nonAligned or aligned timeseries. */ public class TsFileSequenceRead { // if you wanna print detailed datas in pages, then turn it true. - private static boolean printDetail = true; + private static boolean printDetail = false; @SuppressWarnings({ "squid:S3776", "squid:S106" }) // Suppress high Cognitive Complexity and Standard outputs warning public static void main(String[] args) throws IOException { - String filename = "C:\\Users\\BensonChou\\Desktop\\TestData\\3\\1649311240706-3-0-1.tsfile"; + String filename = "test.tsfile"; if (args.length >= 1) { filename = args[0]; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index e95577a10345f..c08b674bd3d95 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -43,8 +43,10 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // The index of the array corresponds to subTaskId. private int[] seqFileIndexMap = new int[subTaskNum]; + // device end time in each source seq file private final long[] currentDeviceEndTime; + // whether each target file is empty or not private final boolean[] isEmptyFile; // This variable has three values, which is @@ -74,7 +76,6 @@ public void startChunkGroup(String deviceId, boolean isAlign) throws IOException this.seqFileIndexMap = new int[subTaskNum]; checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { - // hasTargetFileStartChunkGroup[i] = false; hasTargetFileStartChunkGroup[i].set(0); } } @@ -92,7 +93,6 @@ public void endChunkGroup() throws IOException { @Override public void endMeasurement(int subTaskId) throws IOException { flushChunkToFileWriter(subTaskId); - // chunkWriterMap.get(subTaskId)=null; seqFileIndexMap[subTaskId] = 0; } 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/CompactionUtilsTest.java index 2d1c489889387..f64beb75f3e6d 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/CompactionUtilsTest.java @@ -21,7 +21,6 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.engine.compaction.utils.CompactionFileGeneratorUtils; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; -import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.IllegalPathException; import org.apache.iotdb.db.exception.metadata.MetadataException; @@ -45,10 +44,8 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -1876,52 +1873,46 @@ public void testCrossSpaceCompactionWithDifferentTimeseries() null, true); int count = 0; - try { - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); - } - count++; - batchData.next(); + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - tsFilesReader.close(); - if (i < 2 && j < 3) { - assertEquals(1280, count); - } else if (i < 1 && j < 4) { - assertEquals(1230, count); - } else if (i == 0) { - assertEquals(800, count); - } else if ((i == 1 && j == 4)) { - assertEquals(600, count); - } else if (i < 3 && j < 4) { - assertEquals(1200, count); - } else { - assertEquals(600, count); - } - - } catch (Throwable e) { - e.printStackTrace(); - Thread.currentThread().stop(); + } + tsFilesReader.close(); + if (i < 2 && j < 3) { + assertEquals(1280, count); + } else if (i < 1 && j < 4) { + assertEquals(1230, count); + } else if (i == 0) { + assertEquals(800, count); + } else if ((i == 1 && j == 4)) { + assertEquals(600, count); + } else if (i < 3 && j < 4) { + assertEquals(1200, count); + } else { + assertEquals(600, count); } } } @@ -2047,57 +2038,48 @@ public void testCrossSpaceCompactionWithAllDataDeletedInTimeseries() null, true); int count = 0; - try { - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); - } - count++; - batchData.next(); + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - tsFilesReader.close(); - if ((i == 0 && j == 0) - || (i == 0 && j == 1) - || (i == 2 && j == 4) - || (i == 3 && j == 4)) { - assertEquals(0, count); - } else if (i < 2 && j < 3) { - assertEquals(1280, count); - } else if (i < 1 && j < 4) { - assertEquals(1230, count); - } else if (i == 0) { - assertEquals(800, count); - } else if ((i == 1 && j == 4)) { - assertEquals(600, count); - } else if (i < 3) { - assertEquals(1200, count); - } else { - assertEquals(600, count); - } - - } catch (Throwable e) { - e.printStackTrace(); - Thread.currentThread().stop(); + } + tsFilesReader.close(); + if ((i == 0 && j == 0) || (i == 0 && j == 1) || (i == 2 && j == 4) || (i == 3 && j == 4)) { + assertEquals(0, count); + } else if (i < 2 && j < 3) { + assertEquals(1280, count); + } else if (i < 1 && j < 4) { + assertEquals(1230, count); + } else if (i == 0) { + assertEquals(800, count); + } else if ((i == 1 && j == 4)) { + assertEquals(600, count); + } else if (i < 3) { + assertEquals(1200, count); + } else { + assertEquals(600, count); } } } @@ -2219,50 +2201,44 @@ public void testCrossSpaceCompactionWithAllDataDeletedInDevice() null, true); int count = 0; - try { - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); - } - count++; - batchData.next(); + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - tsFilesReader.close(); - if (i == 0 || i == 2) { - assertEquals(0, count); - } else if (i < 2 && j < 3) { - assertEquals(1280, count); - } else if ((i == 1 && j == 4)) { - assertEquals(600, count); - } else if (i < 3) { - assertEquals(1200, count); - } else { - assertEquals(600, count); - } - - } catch (Throwable e) { - e.printStackTrace(); - Thread.currentThread().stop(); + } + tsFilesReader.close(); + if (i == 0 || i == 2) { + assertEquals(0, count); + } else if (i < 2 && j < 3) { + assertEquals(1280, count); + } else if ((i == 1 && j == 4)) { + assertEquals(600, count); + } else if (i < 3) { + assertEquals(1200, count); + } else { + assertEquals(600, count); } } } @@ -2380,44 +2356,38 @@ public void testCrossSpaceCompactionWithAllDataDeletedInOneTargetFile() null, true); int count = 0; - try { - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); - } - count++; - batchData.next(); + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); } + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); + } else { + assertEquals(batchData.currentTime(), batchData.currentValue()); + } + count++; + batchData.next(); } - tsFilesReader.close(); - if (i == 0 || i == 1 || i == 2) { - assertEquals(0, count); - } else { - assertEquals(600, count); - } - - } catch (Throwable e) { - e.printStackTrace(); - Thread.currentThread().stop(); + } + tsFilesReader.close(); + if (i == 0 || i == 1 || i == 2) { + assertEquals(0, count); + } else { + assertEquals(600, count); } } } @@ -2543,54 +2513,48 @@ public void testCrossSpaceCompactionWithAllDataDeletedInDeviceInSeqFiles() null, true); int count = 0; - try { - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); - } - count++; - batchData.next(); - } - } - tsFilesReader.close(); - if (i < 1) { - if (j < 4) { - assertEquals(630, count); - } else { - assertEquals(200, count); + while (tsFilesReader.hasNextBatch()) { + BatchData batchData = tsFilesReader.nextBatch(); + while (batchData.hasCurrent()) { + if (measurementMaxTime.get( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) + >= batchData.currentTime()) { + Assert.fail(); } - } else if (i < 3) { - if (j < 4) { - assertEquals(600, count); + measurementMaxTime.put( + COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, + batchData.currentTime()); + if (i == 0 + && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) + || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { + assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); + } else if ((i < 3 && j < 4) + && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) + || (250 <= batchData.currentTime() && batchData.currentTime() < 450) + || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { + assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); } else { - assertEquals(0, count); + assertEquals(batchData.currentTime(), batchData.currentValue()); } + count++; + batchData.next(); + } + } + tsFilesReader.close(); + if (i < 1) { + if (j < 4) { + assertEquals(630, count); } else { + assertEquals(200, count); + } + } else if (i < 3) { + if (j < 4) { assertEquals(600, count); + } else { + assertEquals(0, count); } - - } catch (Throwable e) { - e.printStackTrace(); - Thread.currentThread().stop(); + } else { + assertEquals(600, count); } } } @@ -3554,87 +3518,6 @@ public void testCrossSpaceCompactionWithDeviceMaxTimeLaterInUnseqFile() { } } - @Ignore - @Test - public void testReadWrongFile() throws IllegalPathException, IOException { - List targetResources = new ArrayList<>(); - TsFileResource resource = - new TsFileResource( - new File("C:\\Users\\BensonChou\\Desktop\\TestData\\3\\1649311240699-2-0-1.tsfile")); - resource.deserialize(); - resource.setStatus(TsFileResourceStatus.CLOSED); - targetResources.add(resource); - resource = - new TsFileResource( - new File("C:\\Users\\BensonChou\\Desktop\\TestData\\3\\1649311240706-3-0-1.tsfile")); - resource.deserialize(); - resource.setStatus(TsFileResourceStatus.CLOSED); - targetResources.add(resource); - Map measurementMaxTime = new HashMap<>(); - for (int i = 0; i < 4; i++) { - for (int j = 0; j < 5; j++) { - measurementMaxTime.putIfAbsent( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - Long.MIN_VALUE); - PartialPath path = - new MeasurementPath( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i, - "s" + j, - new MeasurementSchema("s" + j, TSDataType.INT64)); - IBatchReader tsFilesReader = - new SeriesRawDataBatchReader( - path, - TSDataType.INT64, - EnvironmentUtils.TEST_QUERY_CONTEXT, - targetResources, - new ArrayList<>(), - null, - null, - true); - int count = 0; - try { - while (tsFilesReader.hasNextBatch()) { - BatchData batchData = tsFilesReader.nextBatch(); - while (batchData.hasCurrent()) { - if (measurementMaxTime.get( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j) - >= batchData.currentTime()) { - Assert.fail(); - } - measurementMaxTime.put( - COMPACTION_TEST_SG + PATH_SEPARATOR + "d" + i + PATH_SEPARATOR + "s" + j, - batchData.currentTime()); - if (i == 0 - && ((450 <= batchData.currentTime() && batchData.currentTime() < 550) - || (550 <= batchData.currentTime() && batchData.currentTime() < 650))) { - assertEquals(batchData.currentTime() + 20000, batchData.currentValue()); - } else if ((i < 3 && j < 4) - && ((20 <= batchData.currentTime() && batchData.currentTime() < 220) - || (250 <= batchData.currentTime() && batchData.currentTime() < 450) - || (480 <= batchData.currentTime() && batchData.currentTime() < 680))) { - assertEquals(batchData.currentTime() + 10000, batchData.currentValue()); - } else { - assertEquals(batchData.currentTime(), batchData.currentValue()); - } - count++; - batchData.next(); - } - } - tsFilesReader.close(); - if (i == 0 || i == 1 || i == 2) { - assertEquals(0, count); - } else { - assertEquals(600, count); - } - - } catch (Throwable e) { - e.printStackTrace(); - Thread.currentThread().stop(); - } - } - } - } - private void generateModsFile( List seriesPaths, List resources, long startValue, long endValue) throws IllegalPathException, IOException { From 3295ac788328a5d86c080868a022dce1fd8db794 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Thu, 7 Apr 2022 17:29:06 +0800 Subject: [PATCH 11/17] rename map to array --- .../writer/AbstractCompactionWriter.java | 8 +++---- .../writer/CrossSpaceCompactionWriter.java | 21 ++++++++++--------- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index a3304856199ee..409956b1a6914 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -55,14 +55,14 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { // Each sub task has point count in current measurment, which is used to check size. // The index of the array corresponds to subTaskId. - protected int[] measurementPointCountMap = new int[subTaskNum]; + protected int[] measurementPointCountArray = new int[subTaskNum]; public abstract void startChunkGroup(String deviceId, boolean isAlign) throws IOException; public abstract void endChunkGroup() throws IOException; public void startMeasurement(List measurementSchemaList, int subTaskId) { - measurementPointCountMap[subTaskId] = 0; + measurementPointCountArray[subTaskId] = 0; if (isAlign) { chunkWriterMap.put(subTaskId, new AlignedChunkWriterImpl(measurementSchemaList)); } else { @@ -139,12 +139,12 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { } chunkWriter.write(timestamp); } - measurementPointCountMap[subTaskId] += 1; + measurementPointCountArray[subTaskId] += 1; } protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) throws IOException { - if (measurementPointCountMap[subTaskId] % 10 == 0 && checkChunkSize(subTaskId)) { + if (measurementPointCountArray[subTaskId] % 10 == 0 && checkChunkSize(subTaskId)) { writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); CompactionMetricsManager.recordWriteInfo( this instanceof CrossSpaceCompactionWriter diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index c08b674bd3d95..c0ac70ab03e93 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -41,7 +41,7 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // Each sub task has its corresponding seq file index. // The index of the array corresponds to subTaskId. - private int[] seqFileIndexMap = new int[subTaskNum]; + private int[] seqFileIndexArray = new int[subTaskNum]; // device end time in each source seq file private final long[] currentDeviceEndTime; @@ -53,6 +53,7 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // 0: has not start chunk group yet // 1: start flushing chunk group header // 2: finish flushing chunk group header + // The index of the array corresponds to each target file. private final AtomicInteger[] hasTargetFileStartChunkGroup; public CrossSpaceCompactionWriter( @@ -73,7 +74,7 @@ public CrossSpaceCompactionWriter( public void startChunkGroup(String deviceId, boolean isAlign) throws IOException { this.deviceId = deviceId; this.isAlign = isAlign; - this.seqFileIndexMap = new int[subTaskNum]; + this.seqFileIndexArray = new int[subTaskNum]; checkIsDeviceExistAndGetDeviceEndTime(); for (int i = 0; i < seqTsFileResources.size(); i++) { hasTargetFileStartChunkGroup[i].set(0); @@ -93,7 +94,7 @@ public void endChunkGroup() throws IOException { @Override public void endMeasurement(int subTaskId) throws IOException { flushChunkToFileWriter(subTaskId); - seqFileIndexMap[subTaskId] = 0; + seqFileIndexArray[subTaskId] = 0; } @Override @@ -101,8 +102,8 @@ public void write(long timestamp, Object value, int subTaskId) throws IOExceptio checkTimeAndMayFlushChunkToCurrentFile(timestamp, subTaskId); checkAndMayStartChunkGroup(subTaskId); writeDataPoint(timestamp, value, subTaskId); - checkChunkSizeAndMayOpenANewChunk(fileWriterList.get(seqFileIndexMap[subTaskId]), subTaskId); - isEmptyFile[seqFileIndexMap[subTaskId]] = false; + checkChunkSizeAndMayOpenANewChunk(fileWriterList.get(seqFileIndexArray[subTaskId]), subTaskId); + isEmptyFile[seqFileIndexArray[subTaskId]] = false; } @Override @@ -138,12 +139,12 @@ public List getFileIOWriter() { private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskId) throws IOException { - int fileIndex = seqFileIndexMap[subTaskId]; + int fileIndex = seqFileIndexArray[subTaskId]; // if timestamp is later than the current source seq tsfile, than flush chunk writer while (timestamp > currentDeviceEndTime[fileIndex]) { if (fileIndex != seqTsFileResources.size() - 1) { flushChunkToFileWriter(subTaskId); - seqFileIndexMap[subTaskId] = ++fileIndex; + seqFileIndexArray[subTaskId] = ++fileIndex; } else { // If the seq file is deleted for various reasons, the following two situations may occur // when selecting the source files: (1) unseq files may have some devices or measurements @@ -181,7 +182,7 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { } private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { - int fileIndex = seqFileIndexMap[subTaskId]; + int fileIndex = seqFileIndexArray[subTaskId]; if (hasTargetFileStartChunkGroup[fileIndex].compareAndSet(0, 1)) { fileWriterList.get(fileIndex).startChunkGroup(deviceId); hasTargetFileStartChunkGroup[fileIndex].set(2); @@ -189,7 +190,7 @@ private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { } private void flushChunkToFileWriter(int subTaskId) throws IOException { - int fileIndex = seqFileIndexMap[subTaskId]; + int fileIndex = seqFileIndexArray[subTaskId]; writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); while (hasTargetFileStartChunkGroup[fileIndex].get() == 1) { // wait until the target file has finished flushing chunk group header @@ -201,7 +202,7 @@ private void flushChunkToFileWriter(int subTaskId) throws IOException { protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) throws IOException { - if (measurementPointCountMap[subTaskId] % 10 == 0 && checkChunkSize(subTaskId)) { + if (measurementPointCountArray[subTaskId] % 10 == 0 && checkChunkSize(subTaskId)) { flushChunkToFileWriter(subTaskId); CompactionMetricsManager.recordWriteInfo( CompactionType.CROSS_COMPACTION, From fe782770cca128485d898bacb8f868d863c62d43 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Thu, 7 Apr 2022 22:38:45 +0800 Subject: [PATCH 12/17] remove status in TsFileResource --- .../db/engine/compaction/writer/CrossSpaceCompactionWriter.java | 2 +- .../org/apache/iotdb/db/engine/storagegroup/TsFileResource.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index c0ac70ab03e93..a853411af4003 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -50,7 +50,7 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { private final boolean[] isEmptyFile; // This variable has three values, which is - // 0: has not start chunk group yet + // 0: has not flushed chunk group header yet // 1: start flushing chunk group header // 2: finish flushing chunk group header // The index of the array corresponds to each target file. diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index d7dd49f49a878..6c9e83c8b5f78 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -550,7 +550,7 @@ void moveTo(File targetDir) { @Override public String toString() { - return String.format("file is %s, status: %s", file.toString(), status); + return String.format("file is %s, status: ", file.toString(), status); } @Override From 9d77ab77ae358325b1c5dd5e04f4ded3fb9c39aa Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Fri, 8 Apr 2022 15:21:01 +0800 Subject: [PATCH 13/17] start chunkGroup in all target files and clean empty chunkGroup at last --- .../writer/CrossSpaceCompactionWriter.java | 39 +++++++------------ .../tsfile/write/writer/TsFileIOWriter.java | 9 +++-- 2 files changed, 20 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index a853411af4003..ef1bd25b5036f 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -30,7 +30,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // target fileIOWriters @@ -49,22 +48,19 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // whether each target file is empty or not private final boolean[] isEmptyFile; - // This variable has three values, which is - // 0: has not flushed chunk group header yet - // 1: start flushing chunk group header - // 2: finish flushing chunk group header - // The index of the array corresponds to each target file. - private final AtomicInteger[] hasTargetFileStartChunkGroup; + // whether each target file has device data + private final boolean[] isDeviceExistedInTargetFiles; + + private int chunkGroupHeaderSize; public CrossSpaceCompactionWriter( List targetResources, List seqFileResources) throws IOException { currentDeviceEndTime = new long[seqFileResources.size()]; isEmptyFile = new boolean[seqFileResources.size()]; - hasTargetFileStartChunkGroup = new AtomicInteger[seqFileResources.size()]; + isDeviceExistedInTargetFiles = new boolean[targetResources.size()]; for (int i = 0; i < targetResources.size(); i++) { this.fileWriterList.add(new TsFileIOWriter(targetResources.get(i).getTsFile())); - this.hasTargetFileStartChunkGroup[i] = new AtomicInteger(0); isEmptyFile[i] = true; } this.seqTsFileResources = seqFileResources; @@ -76,17 +72,21 @@ public void startChunkGroup(String deviceId, boolean isAlign) throws IOException this.isAlign = isAlign; this.seqFileIndexArray = new int[subTaskNum]; checkIsDeviceExistAndGetDeviceEndTime(); - for (int i = 0; i < seqTsFileResources.size(); i++) { - hasTargetFileStartChunkGroup[i].set(0); + for (int i = 0; i < fileWriterList.size(); i++) { + chunkGroupHeaderSize = fileWriterList.get(i).startChunkGroup(deviceId); } } @Override public void endChunkGroup() throws IOException { for (int i = 0; i < seqTsFileResources.size(); i++) { - if (hasTargetFileStartChunkGroup[i].get() == 2) { - fileWriterList.get(i).endChunkGroup(); + TsFileIOWriter targetFileWriter = fileWriterList.get(i); + if (isDeviceExistedInTargetFiles[i]) { + targetFileWriter.endChunkGroup(); + } else { + targetFileWriter.truncate(targetFileWriter.getPos() - chunkGroupHeaderSize); } + isDeviceExistedInTargetFiles[i] = false; } deviceId = null; } @@ -100,9 +100,9 @@ public void endMeasurement(int subTaskId) throws IOException { @Override public void write(long timestamp, Object value, int subTaskId) throws IOException { checkTimeAndMayFlushChunkToCurrentFile(timestamp, subTaskId); - checkAndMayStartChunkGroup(subTaskId); writeDataPoint(timestamp, value, subTaskId); checkChunkSizeAndMayOpenANewChunk(fileWriterList.get(seqFileIndexArray[subTaskId]), subTaskId); + isDeviceExistedInTargetFiles[seqFileIndexArray[subTaskId]] = true; isEmptyFile[seqFileIndexArray[subTaskId]] = false; } @@ -181,20 +181,9 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { } } - private void checkAndMayStartChunkGroup(int subTaskId) throws IOException { - int fileIndex = seqFileIndexArray[subTaskId]; - if (hasTargetFileStartChunkGroup[fileIndex].compareAndSet(0, 1)) { - fileWriterList.get(fileIndex).startChunkGroup(deviceId); - hasTargetFileStartChunkGroup[fileIndex].set(2); - } - } - private void flushChunkToFileWriter(int subTaskId) throws IOException { int fileIndex = seqFileIndexArray[subTaskId]; writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - while (hasTargetFileStartChunkGroup[fileIndex].get() == 1) { - // wait until the target file has finished flushing chunk group header - } synchronized (fileWriterList.get(fileIndex)) { chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriterList.get(fileIndex)); } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java index 7eb9a95110778..568f4e6715eb1 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java @@ -40,7 +40,6 @@ import org.apache.iotdb.tsfile.utils.BytesUtils; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -142,14 +141,14 @@ protected void startFile() throws IOException { out.write(VERSION_NUMBER_BYTE); } - public void startChunkGroup(String deviceId) throws IOException { + public int startChunkGroup(String deviceId) throws IOException { this.currentChunkGroupDeviceId = deviceId; if (logger.isDebugEnabled()) { logger.debug("start chunk group:{}, file position {}", deviceId, out.getPosition()); } chunkMetadataList = new ArrayList<>(); ChunkGroupHeader chunkGroupHeader = new ChunkGroupHeader(currentChunkGroupDeviceId); - chunkGroupHeader.serializeTo(out.wrapAsStream()); + return chunkGroupHeader.serializeTo(out.wrapAsStream()); } /** @@ -458,6 +457,10 @@ public void writePlanIndices() throws IOException { out.flush(); } + public void truncate(long truncateSize) throws IOException { + out.truncate(truncateSize); + } + /** * this function is only for Test. * From 3eeafb90d7092bf42e151151c03314c443ec3c73 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Fri, 8 Apr 2022 16:59:53 +0800 Subject: [PATCH 14/17] spotless --- .../org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java | 1 + 1 file changed, 1 insertion(+) diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java index 568f4e6715eb1..6073fb3f13a9a 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java @@ -40,6 +40,7 @@ import org.apache.iotdb.tsfile.utils.BytesUtils; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; From d16416f7d8d0592eea4d0123e5af6d494ad14fd7 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Fri, 8 Apr 2022 20:34:31 +0800 Subject: [PATCH 15/17] add flushChunkToWriter method --- .../writer/AbstractCompactionWriter.java | 13 ++++++--- .../writer/CrossSpaceCompactionWriter.java | 27 ++----------------- 2 files changed, 11 insertions(+), 29 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index 409956b1a6914..9ca7f19c4c326 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -142,10 +142,18 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { measurementPointCountArray[subTaskId] += 1; } + protected void flushChunkToFileWriter(TsFileIOWriter targetWriter, int subTaskId) + throws IOException { + writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + synchronized (targetWriter) { + chunkWriterMap.get(subTaskId).writeToFileWriter(targetWriter); + } + } + protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) throws IOException { if (measurementPointCountArray[subTaskId] % 10 == 0 && checkChunkSize(subTaskId)) { - writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + flushChunkToFileWriter(fileWriter, subTaskId); CompactionMetricsManager.recordWriteInfo( this instanceof CrossSpaceCompactionWriter ? CompactionType.CROSS_COMPACTION @@ -153,9 +161,6 @@ protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int ProcessChunkType.DESERIALIZE_CHUNK, this.isAlign, chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriter) { - chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriter); - } } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index ef1bd25b5036f..db3303d1bd4b3 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -18,9 +18,6 @@ */ package org.apache.iotdb.db.engine.compaction.writer; -import org.apache.iotdb.db.engine.compaction.CompactionMetricsManager; -import org.apache.iotdb.db.engine.compaction.constant.CompactionType; -import org.apache.iotdb.db.engine.compaction.constant.ProcessChunkType; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.query.control.FileReaderManager; import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata; @@ -93,7 +90,7 @@ public void endChunkGroup() throws IOException { @Override public void endMeasurement(int subTaskId) throws IOException { - flushChunkToFileWriter(subTaskId); + flushChunkToFileWriter(fileWriterList.get(seqFileIndexArray[subTaskId]), subTaskId); seqFileIndexArray[subTaskId] = 0; } @@ -143,7 +140,7 @@ private void checkTimeAndMayFlushChunkToCurrentFile(long timestamp, int subTaskI // if timestamp is later than the current source seq tsfile, than flush chunk writer while (timestamp > currentDeviceEndTime[fileIndex]) { if (fileIndex != seqTsFileResources.size() - 1) { - flushChunkToFileWriter(subTaskId); + flushChunkToFileWriter(fileWriterList.get(fileIndex), subTaskId); seqFileIndexArray[subTaskId] = ++fileIndex; } else { // If the seq file is deleted for various reasons, the following two situations may occur @@ -180,24 +177,4 @@ private void checkIsDeviceExistAndGetDeviceEndTime() throws IOException { fileIndex++; } } - - private void flushChunkToFileWriter(int subTaskId) throws IOException { - int fileIndex = seqFileIndexArray[subTaskId]; - writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriterList.get(fileIndex)) { - chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriterList.get(fileIndex)); - } - } - - protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int subTaskId) - throws IOException { - if (measurementPointCountArray[subTaskId] % 10 == 0 && checkChunkSize(subTaskId)) { - flushChunkToFileWriter(subTaskId); - CompactionMetricsManager.recordWriteInfo( - CompactionType.CROSS_COMPACTION, - ProcessChunkType.DESERIALIZE_CHUNK, - this.isAlign, - chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - } - } } From c7966bf54faf4966c1ec0fb9b3452bb44701d620 Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Fri, 8 Apr 2022 20:48:18 +0800 Subject: [PATCH 16/17] define ChunkWriters as array instead of concurrenctHashMap --- .../writer/AbstractCompactionWriter.java | 28 +++++++++---------- .../writer/CrossSpaceCompactionWriter.java | 1 - .../writer/InnerSpaceCompactionWriter.java | 6 +--- 3 files changed, 14 insertions(+), 21 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java index 9ca7f19c4c326..5c1460230dc0a 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/AbstractCompactionWriter.java @@ -35,15 +35,14 @@ import java.io.IOException; import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; public abstract class AbstractCompactionWriter implements AutoCloseable { protected static final int subTaskNum = IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum(); - // subTaskId -> IChunkWriter - protected Map chunkWriterMap = new ConcurrentHashMap<>(); + // Each sub task has its own chunk writer. + // The index of the array corresponds to subTaskId. + protected IChunkWriter[] chunkWriters = new IChunkWriter[subTaskNum]; protected boolean isAlign; @@ -64,9 +63,9 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { public void startMeasurement(List measurementSchemaList, int subTaskId) { measurementPointCountArray[subTaskId] = 0; if (isAlign) { - chunkWriterMap.put(subTaskId, new AlignedChunkWriterImpl(measurementSchemaList)); + chunkWriters[subTaskId] = new AlignedChunkWriterImpl(measurementSchemaList); } else { - chunkWriterMap.put(subTaskId, new ChunkWriterImpl(measurementSchemaList.get(0), true)); + chunkWriters[subTaskId] = new ChunkWriterImpl(measurementSchemaList.get(0), true); } } @@ -82,7 +81,7 @@ public void startMeasurement(List measurementSchemaList, int protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { if (!isAlign) { - ChunkWriterImpl chunkWriter = (ChunkWriterImpl) this.chunkWriterMap.get(subTaskId); + ChunkWriterImpl chunkWriter = (ChunkWriterImpl) this.chunkWriters[subTaskId]; switch (chunkWriter.getDataType()) { case TEXT: chunkWriter.write(timestamp, (Binary) value); @@ -106,8 +105,7 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { throw new UnsupportedOperationException("Unknown data type " + chunkWriter.getDataType()); } } else { - AlignedChunkWriterImpl chunkWriter = - (AlignedChunkWriterImpl) this.chunkWriterMap.get(subTaskId); + AlignedChunkWriterImpl chunkWriter = (AlignedChunkWriterImpl) this.chunkWriters[subTaskId]; for (TsPrimitiveType val : (TsPrimitiveType[]) value) { if (val == null) { chunkWriter.write(timestamp, null, true); @@ -144,9 +142,9 @@ protected void writeDataPoint(Long timestamp, Object value, int subTaskId) { protected void flushChunkToFileWriter(TsFileIOWriter targetWriter, int subTaskId) throws IOException { - writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + writeRateLimit(chunkWriters[subTaskId].estimateMaxSeriesMemSize()); synchronized (targetWriter) { - chunkWriterMap.get(subTaskId).writeToFileWriter(targetWriter); + chunkWriters[subTaskId].writeToFileWriter(targetWriter); } } @@ -160,16 +158,16 @@ protected void checkChunkSizeAndMayOpenANewChunk(TsFileIOWriter fileWriter, int : CompactionType.INNER_UNSEQ_COMPACTION, ProcessChunkType.DESERIALIZE_CHUNK, this.isAlign, - chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); + chunkWriters[subTaskId].estimateMaxSeriesMemSize()); } } protected boolean checkChunkSize(int subTaskId) { - if (chunkWriterMap.get(subTaskId) instanceof AlignedChunkWriterImpl) { - return ((AlignedChunkWriterImpl) chunkWriterMap.get(subTaskId)) + if (chunkWriters[subTaskId] instanceof AlignedChunkWriterImpl) { + return ((AlignedChunkWriterImpl) chunkWriters[subTaskId]) .checkIsChunkSizeOverThreshold(targetChunkSize); } else { - return chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize() > targetChunkSize; + return chunkWriters[subTaskId].estimateMaxSeriesMemSize() > targetChunkSize; } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index db3303d1bd4b3..d721bf496d8ed 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -126,7 +126,6 @@ public void close() throws IOException { } fileWriterList = null; seqTsFileResources = null; - chunkWriterMap.clear(); } @Override diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java index 8abf2f980cf14..af2cc53c67ed4 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/InnerSpaceCompactionWriter.java @@ -49,10 +49,7 @@ public void endChunkGroup() throws IOException { @Override public void endMeasurement(int subTaskId) throws IOException { - writeRateLimit(chunkWriterMap.get(subTaskId).estimateMaxSeriesMemSize()); - synchronized (fileWriter) { - chunkWriterMap.get(subTaskId).writeToFileWriter(fileWriter); - } + flushChunkToFileWriter(fileWriter, subTaskId); } @Override @@ -78,7 +75,6 @@ public void close() throws IOException { if (fileWriter != null && fileWriter.canWrite()) { fileWriter.close(); } - chunkWriterMap.clear(); fileWriter = null; } From 65bccce8719f8918dd45b4dd955162329b33d93a Mon Sep 17 00:00:00 2001 From: BensonChou <2027178562@qq.com> Date: Sat, 9 Apr 2022 15:59:25 +0800 Subject: [PATCH 17/17] add tests to test whether cross compaction target file contain empty chunk group --- .../writer/CrossSpaceCompactionWriter.java | 3 +- .../compaction/CompactionUtilsTest.java | 307 +++++++++++++++++- .../tsfile/write/writer/TsFileIOWriter.java | 4 +- 3 files changed, 304 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java index d721bf496d8ed..3e245cfc35a13 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/writer/CrossSpaceCompactionWriter.java @@ -45,9 +45,10 @@ public class CrossSpaceCompactionWriter extends AbstractCompactionWriter { // whether each target file is empty or not private final boolean[] isEmptyFile; - // whether each target file has device data + // whether each target file has device data or not private final boolean[] isDeviceExistedInTargetFiles; + // current chunk group header size private int chunkGroupHeaderSize; public CrossSpaceCompactionWriter( 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/CompactionUtilsTest.java index f64beb75f3e6d..69ff88dee05a0 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/CompactionUtilsTest.java @@ -30,9 +30,14 @@ import org.apache.iotdb.db.query.control.FileReaderManager; import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader; import org.apache.iotdb.db.utils.EnvironmentUtils; +import org.apache.iotdb.tsfile.common.conf.TSFileConfig; import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; import org.apache.iotdb.tsfile.exception.write.WriteProcessException; +import org.apache.iotdb.tsfile.file.MetaMarker; +import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader; +import org.apache.iotdb.tsfile.file.header.ChunkHeader; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; import org.apache.iotdb.tsfile.read.common.BatchData; import org.apache.iotdb.tsfile.read.reader.IBatchReader; import org.apache.iotdb.tsfile.utils.Pair; @@ -1766,7 +1771,7 @@ public void testCrossSpaceCompactionWithSameTimeseries() } /** - * Total 5 seq files and 5 unseq files, each file has different nonAligned timeseries. + * Total 4 seq files and 5 unseq files, each file has different nonAligned timeseries. * *

Seq files
* first and second file has d0 ~ d1 and s0 ~ s2, time range is 0 ~ 299 and 350 ~ 649, value range @@ -1850,6 +1855,34 @@ public void testCrossSpaceCompactionWithDifferentTimeseries() CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG); + List deviceIdList = new ArrayList<>(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1"); + for (int i = 0; i < 2; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + for (int i = 2; i < 4; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + Map measurementMaxTime = new HashMap<>(); for (int i = 0; i < 4; i++) { @@ -1919,7 +1952,7 @@ public void testCrossSpaceCompactionWithDifferentTimeseries() } /** - * Total 5 seq files and 5 unseq files, each file has different nonAligned timeseries. + * Total 4 seq files and 5 unseq files, each file has different nonAligned timeseries. * *

Seq files
* first and second file has d0 ~ d1 and s0 ~ s2, time range is 0 ~ 299 and 350 ~ 649, value range @@ -2016,6 +2049,34 @@ public void testCrossSpaceCompactionWithAllDataDeletedInTimeseries() CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG); + List deviceIdList = new ArrayList<>(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1"); + for (int i = 0; i < 2; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + for (int i = 2; i < 4; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + Map measurementMaxTime = new HashMap<>(); for (int i = 0; i < 4; i++) { for (int j = 0; j < 5; j++) { @@ -2086,7 +2147,7 @@ public void testCrossSpaceCompactionWithAllDataDeletedInTimeseries() } /** - * Total 5 seq files and 5 unseq files, each file has different nonAligned timeseries. + * Total 4 seq files and 5 unseq files, each file has different nonAligned timeseries. * *

Seq files
* first and second file has d0 ~ d1 and s0 ~ s2, time range is 0 ~ 299 and 350 ~ 649, value range @@ -2179,6 +2240,32 @@ public void testCrossSpaceCompactionWithAllDataDeletedInDevice() CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG); + List deviceIdList = new ArrayList<>(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1"); + for (int i = 0; i < 2; i++) { + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + for (int i = 2; i < 4; i++) { + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + Map measurementMaxTime = new HashMap<>(); for (int i = 0; i < 4; i++) { for (int j = 0; j < 5; j++) { @@ -2245,7 +2332,7 @@ public void testCrossSpaceCompactionWithAllDataDeletedInDevice() } /** - * Total 5 seq files and 5 unseq files, each file has different nonAligned timeseries. + * Total 4 seq files and 5 unseq files, each file has different nonAligned timeseries. * *

Seq files
* first and second file has d0 ~ d1 and s0 ~ s2, time range is 0 ~ 299 and 350 ~ 649, value range @@ -2259,8 +2346,8 @@ public void testCrossSpaceCompactionWithAllDataDeletedInDevice() * forth and fifth file has d0 and s0 ~ s4, time range is 450 ~ 549 and 550 ~ 649, value range is * 20450 ~ 20549 and 20550 ~ 20649. * - *

The data of d0, d1 and d2 is deleted in each file. Data in the first target file is all - * deleted. + *

The data of d0, d1 and d2 is deleted in each file. Data in the first and second target file + * is all deleted. */ @Test public void testCrossSpaceCompactionWithAllDataDeletedInOneTargetFile() @@ -2334,6 +2421,21 @@ public void testCrossSpaceCompactionWithAllDataDeletedInOneTargetFile() CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG); + Assert.assertEquals(2, targetResources.size()); + List deviceIdList = new ArrayList<>(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + for (int i = 0; i < 2; i++) { + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + Map measurementMaxTime = new HashMap<>(); for (int i = 0; i < 4; i++) { for (int j = 0; j < 5; j++) { @@ -2394,7 +2496,7 @@ public void testCrossSpaceCompactionWithAllDataDeletedInOneTargetFile() } /** - * Total 5 seq files and 5 unseq files, each file has different nonAligned timeseries. + * Total 4 seq files and 5 unseq files, each file has different nonAligned timeseries. * *

Seq files
* first and second file has d0 ~ d1 and s0 ~ s2, time range is 0 ~ 299 and 350 ~ 649, value range @@ -2491,6 +2593,48 @@ public void testCrossSpaceCompactionWithAllDataDeletedInDeviceInSeqFiles() CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG); + Assert.assertEquals(4, targetResources.size()); + List deviceIdList = new ArrayList<>(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1"); + for (int i = 0; i < 2; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + for (int i = 2; i < 3; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.clear(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + for (int i = 3; i < 4; i++) { + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + Map measurementMaxTime = new HashMap<>(); for (int i = 0; i < 4; i++) { for (int j = 0; j < 5; j++) { @@ -2947,6 +3091,35 @@ public void testAlignedCrossSpaceCompactionWithAllDataDeletedInTimeseries() CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG); + Assert.assertEquals(4, targetResources.size()); + List deviceIdList = new ArrayList<>(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10000"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10001"); + for (int i = 0; i < 2; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10000")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10001")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10002")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10003")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10002"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10003"); + for (int i = 2; i < 4; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10000")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10001")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10002")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d10003")); + check(targetResources.get(i), deviceIdList); + } + for (int i = TsFileGeneratorUtils.getAlignDeviceOffset(); i < TsFileGeneratorUtils.getAlignDeviceOffset() + 4; i++) { @@ -3447,6 +3620,54 @@ public void testCrossSpaceCompactionWithNewDeviceInUnseqFile() { CompactionFileGeneratorUtils.getCrossCompactionTargetTsFileResources(seqResources); CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG); + + Assert.assertEquals(4, targetResources.size()); + List deviceIdList = new ArrayList<>(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1"); + for (int i = 0; i < 2; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + for (int i = 2; i < 3; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d4"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d5"); + for (int i = 3; i < 4; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d4")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d5")); + check(targetResources.get(i), deviceIdList); + } } catch (MetadataException | IOException | WriteProcessException @@ -3470,6 +3691,35 @@ public void testCrossSpaceCompactionWithDeviceMaxTimeLaterInUnseqFile() { CompactionUtils.compact(seqResources, unseqResources, targetResources); CompactionUtils.moveTargetFile(targetResources, false, COMPACTION_TEST_SG); + Assert.assertEquals(2, targetResources.size()); + List deviceIdList = new ArrayList<>(); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1"); + for (int i = 0; i < 1; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertFalse( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2"); + deviceIdList.add(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3"); + for (int i = 1; i < 2; i++) { + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d0")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d2")); + Assert.assertTrue( + targetResources.get(i).isDeviceIdExist(COMPACTION_TEST_SG + PATH_SEPARATOR + "d3")); + check(targetResources.get(i), deviceIdList); + } + for (int i = 0; i < 4; i++) { for (int j = 0; j < 4; j++) { PartialPath path = @@ -3529,4 +3779,47 @@ private void generateModsFile( CompactionFileGeneratorUtils.generateMods(deleteMap, resource, false); } } + + /** + * Check whether target file contain empty chunk group or not. Assert fail if it contains empty + * chunk group whose deviceID is not in the deviceIdList. + */ + public void check(TsFileResource targetResource, List deviceIdList) throws IOException { + byte marker; + try (TsFileSequenceReader reader = + new TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) { + reader.position((long) TSFileConfig.MAGIC_STRING.getBytes().length + 1); + while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) { + switch (marker) { + case MetaMarker.CHUNK_HEADER: + case MetaMarker.TIME_CHUNK_HEADER: + case MetaMarker.VALUE_CHUNK_HEADER: + case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER: + case MetaMarker.ONLY_ONE_PAGE_TIME_CHUNK_HEADER: + case MetaMarker.ONLY_ONE_PAGE_VALUE_CHUNK_HEADER: + ChunkHeader header = reader.readChunkHeader(marker); + int dataSize = header.getDataSize(); + reader.position(reader.position() + dataSize); + break; + case MetaMarker.CHUNK_GROUP_HEADER: + ChunkGroupHeader chunkGroupHeader = reader.readChunkGroupHeader(); + String deviceID = chunkGroupHeader.getDeviceID(); + if (!deviceIdList.contains(deviceID)) { + Assert.fail( + "Target file " + + targetResource.getTsFile().getPath() + + " contains empty chunk group " + + deviceID); + } + break; + case MetaMarker.OPERATION_INDEX_RANGE: + reader.readPlanIndex(); + break; + default: + // the disk file is corrupted, using this file may be dangerous + throw new IOException("Unexpected marker " + marker); + } + } + } + } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java index 6073fb3f13a9a..2f865f297f081 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java @@ -458,8 +458,8 @@ public void writePlanIndices() throws IOException { out.flush(); } - public void truncate(long truncateSize) throws IOException { - out.truncate(truncateSize); + public void truncate(long offset) throws IOException { + out.truncate(offset); } /**