From 727d667cf96ef5811f72590c19238fba3fc4bfdb Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sat, 16 Nov 2024 00:07:42 +0800 Subject: [PATCH 01/42] Pipe: Sink supports table model TSFile format --- .../batch/PipeTabletEventTsFileBatch.java | 364 ++++-------------- .../request/PipeTransferTabletRawReq.java | 4 +- .../request/PipeTransferTabletRawReqV2.java | 4 +- .../protocol/opcua/OpcUaNameSpace.java | 4 +- .../async/IoTDBDataRegionAsyncConnector.java | 8 +- .../sync/IoTDBDataRegionSyncConnector.java | 8 +- .../util/PipeTableModeTsFileBuilder.java | 240 ++++++++++++ .../util/PipeTableModelTabletEventSorter.java | 229 +++++++++++ .../util/PipeTreeModelTSFileBuilder.java | 247 ++++++++++++ ...va => PipeTreeModelTabletEventSorter.java} | 4 +- .../connector/util/PipeTsFileBuilder.java | 145 +++++++ .../SubscriptionPipeTsFileEventBatch.java | 9 +- .../connector/PipeTabletEventSorterTest.java | 8 +- 13 files changed, 957 insertions(+), 317 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/{PipeTabletEventSorter.java => PipeTreeModelTabletEventSorter.java} (98%) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index bf8bd5223693..012aeec64ae4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -19,45 +19,34 @@ package org.apache.iotdb.db.pipe.connector.payload.evolvable.batch; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; -import org.apache.iotdb.db.pipe.connector.util.PipeTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.PipeTableModeTsFileBuilder; +import org.apache.iotdb.db.pipe.connector.util.PipeTableModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTSFileBuilder; +import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.PipeTsFileBuilder; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager; -import org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; -import org.apache.iotdb.pipe.api.exception.PipeException; -import org.apache.commons.io.FileUtils; -import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.exception.write.WriteProcessException; -import org.apache.tsfile.read.common.Path; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.Pair; -import org.apache.tsfile.write.TsFileWriter; import org.apache.tsfile.write.record.Tablet; -import org.apache.tsfile.write.schema.IMeasurementSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; public class PipeTabletEventTsFileBatch extends PipeTabletEventBatch { @@ -66,67 +55,20 @@ public class PipeTabletEventTsFileBatch extends PipeTabletEventBatch { private static final AtomicReference FOLDER_MANAGER = new AtomicReference<>(); private static final AtomicLong BATCH_ID_GENERATOR = new AtomicLong(0); private final AtomicLong currentBatchId = new AtomicLong(BATCH_ID_GENERATOR.incrementAndGet()); - private final File batchFileBaseDir; - - private static final String TS_FILE_PREFIX = "tb"; // tb means tablet batch - private final AtomicLong tsFileIdGenerator = new AtomicLong(0); private final long maxSizeInBytes; + private final PipeTsFileBuilder treeModeTsFileBuilder; + private final PipeTsFileBuilder tableModeTsFileBuilder; private final Map, Double> pipeName2WeightMap = new HashMap<>(); - private final List tabletList = new ArrayList<>(); - private final List isTabletAlignedList = new ArrayList<>(); - - @SuppressWarnings("java:S3077") - private volatile TsFileWriter fileWriter; - public PipeTabletEventTsFileBatch(final int maxDelayInMs, final long requestMaxBatchSizeInBytes) { super(maxDelayInMs); this.maxSizeInBytes = requestMaxBatchSizeInBytes; - try { - this.batchFileBaseDir = getNextBaseDir(); - } catch (final Exception e) { - throw new PipeException( - String.format("Failed to create file dir for batch: %s", e.getMessage())); - } - } - - private File getNextBaseDir() throws DiskSpaceInsufficientException { - if (FOLDER_MANAGER.get() == null) { - synchronized (FOLDER_MANAGER) { - if (FOLDER_MANAGER.get() == null) { - FOLDER_MANAGER.set( - new FolderManager( - Arrays.stream(IoTDBDescriptor.getInstance().getConfig().getPipeReceiverFileDirs()) - .map(fileDir -> fileDir + File.separator + ".batch") - .collect(Collectors.toList()), - DirectoryStrategyType.SEQUENCE_STRATEGY)); - } - } - } - - final File baseDir = - new File(FOLDER_MANAGER.get().getNextFolder(), Long.toString(currentBatchId.get())); - if (baseDir.exists()) { - FileUtils.deleteQuietly(baseDir); - } - if (!baseDir.exists() && !baseDir.mkdirs()) { - LOGGER.warn( - "Batch id = {}: Failed to create batch file dir {}.", - currentBatchId.get(), - baseDir.getPath()); - throw new PipeException( - String.format( - "Failed to create batch file dir %s. (Batch id = %s)", - baseDir.getPath(), currentBatchId.get())); - } - LOGGER.info( - "Batch id = {}: Create batch dir successfully, batch file dir = {}.", - currentBatchId.get(), - baseDir.getPath()); - return baseDir; + AtomicLong tsFileIdGenerator = new AtomicLong(0); + treeModeTsFileBuilder = new PipeTreeModelTSFileBuilder(currentBatchId, tsFileIdGenerator); + tableModeTsFileBuilder = new PipeTableModeTsFileBuilder(currentBatchId, tsFileIdGenerator); } @Override @@ -134,14 +76,25 @@ protected boolean constructBatch(final TabletInsertionEvent event) { if (event instanceof PipeInsertNodeTabletInsertionEvent) { final PipeInsertNodeTabletInsertionEvent insertNodeTabletInsertionEvent = (PipeInsertNodeTabletInsertionEvent) event; - // TODO: for table model insertion, we need to get the database name + final boolean isTableModel = insertNodeTabletInsertionEvent.isTableModelEvent(); final List tablets = insertNodeTabletInsertionEvent.convertToTablets(); for (int i = 0; i < tablets.size(); ++i) { final Tablet tablet = tablets.get(i); if (tablet.rowSize == 0) { continue; } - bufferTablet( + // table Model + if (isTableModel) { + bufferTableModelTablet( + insertNodeTabletInsertionEvent.getPipeName(), + insertNodeTabletInsertionEvent.getCreationTime(), + tablet, + insertNodeTabletInsertionEvent.getTableModelDatabaseName()); + continue; + } + + // tree Model + bufferTreeModelTablet( insertNodeTabletInsertionEvent.getPipeName(), insertNodeTabletInsertionEvent.getCreationTime(), tablet, @@ -154,7 +107,17 @@ protected boolean constructBatch(final TabletInsertionEvent event) { if (tablet.rowSize == 0) { return true; } - bufferTablet( + // table Model + if (rawTabletInsertionEvent.isTableModelEvent()) { + bufferTableModelTablet( + rawTabletInsertionEvent.getPipeName(), + rawTabletInsertionEvent.getCreationTime(), + tablet, + rawTabletInsertionEvent.getTableModelDatabaseName()); + return true; + } + // tree Model + bufferTreeModelTablet( rawTabletInsertionEvent.getPipeName(), rawTabletInsertionEvent.getCreationTime(), tablet, @@ -169,12 +132,12 @@ protected boolean constructBatch(final TabletInsertionEvent event) { return true; } - private void bufferTablet( + private void bufferTreeModelTablet( final String pipeName, final long creationTime, final Tablet tablet, final boolean isAligned) { - new PipeTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); totalBufferSize += PipeMemoryWeightUtil.calculateTabletSizeInBytes(tablet); @@ -182,8 +145,25 @@ private void bufferTablet( new Pair<>(pipeName, creationTime), (pipe, weight) -> Objects.nonNull(weight) ? ++weight : 1); - tabletList.add(tablet); - isTabletAlignedList.add(isAligned); + treeModeTsFileBuilder.bufferTreeModelTablet(tablet, isAligned); + } + + private void bufferTableModelTablet( + final String pipeName, final long creationTime, final Tablet tablet, final String dataBase) { + final Map> deviceID2TimeRange = + new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + + if (deviceID2TimeRange == null) { + return; + } + + totalBufferSize += PipeMemoryWeightUtil.calculateTabletSizeInBytes(tablet); + + pipeName2WeightMap.compute( + new Pair<>(pipeName, creationTime), + (pipe, weight) -> Objects.nonNull(weight) ? ++weight : 1); + + tableModeTsFileBuilder.bufferTableModelTablet(dataBase, tablet, deviceID2TimeRange); } public Map, Double> deepCopyPipe2WeightMap() { @@ -195,193 +175,21 @@ public Map, Double> deepCopyPipe2WeightMap() { return new HashMap<>(pipeName2WeightMap); } - public synchronized List sealTsFiles() throws IOException, WriteProcessException { - return isClosed ? Collections.emptyList() : writeTabletsToTsFiles(); - } - - private List writeTabletsToTsFiles() throws IOException, WriteProcessException { - final Map> device2Tablets = new HashMap<>(); - final Map device2Aligned = new HashMap<>(); - - // Sort the tablets by device id - for (int i = 0, size = tabletList.size(); i < size; ++i) { - final Tablet tablet = tabletList.get(i); - final String deviceId = tablet.getDeviceId(); - device2Tablets.computeIfAbsent(deviceId, k -> new ArrayList<>()).add(tablet); - device2Aligned.put(deviceId, isTabletAlignedList.get(i)); - } - - // Sort the tablets by start time in each device - for (final List tablets : device2Tablets.values()) { - tablets.sort( - // Each tablet has at least one timestamp - Comparator.comparingLong(tablet -> tablet.timestamps[0])); - } - - // Sort the devices by device id - final List devices = new ArrayList<>(device2Tablets.keySet()); - devices.sort(Comparator.naturalOrder()); - - // Replace ArrayList with LinkedList to improve performance - final LinkedHashMap> device2TabletsLinkedList = - new LinkedHashMap<>(); - for (final String device : devices) { - device2TabletsLinkedList.put(device, new LinkedList<>(device2Tablets.get(device))); + public synchronized List> sealTsFiles() + throws IOException, WriteProcessException { + if (isClosed) { + return Collections.emptyList(); } - // Help GC - devices.clear(); - device2Tablets.clear(); - - // Write the tablets to the tsfile device by device, and the tablets - // in the same device are written in order of start time. Tablets in - // the same device should not be written if their time ranges overlap. - // If overlapped, we try to write the tablets whose device id is not - // the same as the previous one. For the tablets not written in the - // previous round, we write them in a new tsfile. - final List sealedFiles = new ArrayList<>(); - - // Try making the tsfile size as large as possible - while (!device2TabletsLinkedList.isEmpty()) { - if (Objects.isNull(fileWriter)) { - fileWriter = - new TsFileWriter( - new File( - batchFileBaseDir, - TS_FILE_PREFIX - + "_" - + IoTDBDescriptor.getInstance().getConfig().getDataNodeId() - + "_" - + currentBatchId.get() - + "_" - + tsFileIdGenerator.getAndIncrement() - + TsFileConstant.TSFILE_SUFFIX)); - } - - try { - tryBestToWriteTabletsIntoOneFile(device2TabletsLinkedList, device2Aligned); - } catch (final Exception e) { - LOGGER.warn( - "Batch id = {}: Failed to write tablets into tsfile, because {}", - currentBatchId.get(), - e.getMessage(), - e); - - try { - fileWriter.close(); - } catch (final Exception closeException) { - LOGGER.warn( - "Batch id = {}: Failed to close the tsfile {} after failed to write tablets into, because {}", - currentBatchId.get(), - fileWriter.getIOWriter().getFile().getPath(), - closeException.getMessage(), - closeException); - } finally { - // Add current writing file to the list and delete the file - sealedFiles.add(fileWriter.getIOWriter().getFile()); - } - - for (final File sealedFile : sealedFiles) { - final boolean deleteSuccess = FileUtils.deleteQuietly(sealedFile); - LOGGER.warn( - "Batch id = {}: {} delete the tsfile {} after failed to write tablets into {}. {}", - currentBatchId.get(), - deleteSuccess ? "Successfully" : "Failed to", - sealedFile.getPath(), - fileWriter.getIOWriter().getFile().getPath(), - deleteSuccess ? "" : "Maybe the tsfile needs to be deleted manually."); - } - sealedFiles.clear(); - - fileWriter = null; - - throw e; - } - - fileWriter.close(); - final File sealedFile = fileWriter.getIOWriter().getFile(); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug( - "Batch id = {}: Seal tsfile {} successfully.", - currentBatchId.get(), - sealedFile.getPath()); - } - sealedFiles.add(sealedFile); - fileWriter = null; + List> list = new ArrayList<>(); + if (!treeModeTsFileBuilder.isEmpty()) { + list.addAll(treeModeTsFileBuilder.sealTsFiles()); } - return sealedFiles; - } - - private void tryBestToWriteTabletsIntoOneFile( - final LinkedHashMap> device2TabletsLinkedList, - final Map device2Aligned) - throws IOException, WriteProcessException { - final Iterator>> iterator = - device2TabletsLinkedList.entrySet().iterator(); - - while (iterator.hasNext()) { - final Map.Entry> entry = iterator.next(); - final String deviceId = entry.getKey(); - final LinkedList tablets = entry.getValue(); - - final List tabletsToWrite = new ArrayList<>(); - - Tablet lastTablet = null; - while (!tablets.isEmpty()) { - final Tablet tablet = tablets.peekFirst(); - if (Objects.isNull(lastTablet) - // lastTablet.rowSize is not 0 - || lastTablet.timestamps[lastTablet.rowSize - 1] < tablet.timestamps[0]) { - tabletsToWrite.add(tablet); - lastTablet = tablet; - tablets.pollFirst(); - } else { - break; - } - } - - if (tablets.isEmpty()) { - iterator.remove(); - } - - final boolean isAligned = device2Aligned.get(deviceId); - if (isAligned) { - final Map> deviceId2MeasurementSchemas = new HashMap<>(); - tabletsToWrite.forEach( - tablet -> - deviceId2MeasurementSchemas.compute( - tablet.getDeviceId(), - (k, v) -> { - if (Objects.isNull(v)) { - return new ArrayList<>(tablet.getSchemas()); - } - v.addAll(tablet.getSchemas()); - return v; - })); - for (final Entry> deviceIdWithMeasurementSchemas : - deviceId2MeasurementSchemas.entrySet()) { - fileWriter.registerAlignedTimeseries( - new Path(deviceIdWithMeasurementSchemas.getKey()), - deviceIdWithMeasurementSchemas.getValue()); - } - for (final Tablet tablet : tabletsToWrite) { - fileWriter.writeAligned(tablet); - } - } else { - for (final Tablet tablet : tabletsToWrite) { - for (final IMeasurementSchema schema : tablet.getSchemas()) { - try { - fileWriter.registerTimeseries(new Path(tablet.getDeviceId()), schema); - } catch (final WriteProcessException ignore) { - // Do nothing if the timeSeries has been registered - } - } - - fileWriter.write(tablet); - } - } + if (!tableModeTsFileBuilder.isEmpty()) { + list.addAll(tableModeTsFileBuilder.sealTsFiles()); } + return list; } @Override @@ -394,13 +202,8 @@ public synchronized void onSuccess() { super.onSuccess(); pipeName2WeightMap.clear(); - - tabletList.clear(); - isTabletAlignedList.clear(); - - // We don't need to delete the tsFile here, because the tsFile - // will be deleted after the file is transferred. - fileWriter = null; + tableModeTsFileBuilder.onSuccess(); + treeModeTsFileBuilder.onSuccess(); } @Override @@ -408,34 +211,7 @@ public synchronized void close() { super.close(); pipeName2WeightMap.clear(); - - tabletList.clear(); - isTabletAlignedList.clear(); - - if (Objects.nonNull(fileWriter)) { - try { - fileWriter.close(); - } catch (final Exception e) { - LOGGER.info( - "Batch id = {}: Failed to close the tsfile {} when trying to close batch, because {}", - currentBatchId.get(), - fileWriter.getIOWriter().getFile().getPath(), - e.getMessage(), - e); - } - - try { - FileUtils.delete(fileWriter.getIOWriter().getFile()); - } catch (final Exception e) { - LOGGER.info( - "Batch id = {}: Failed to delete the tsfile {} when trying to close batch, because {}", - currentBatchId.get(), - fileWriter.getIOWriter().getFile().getPath(), - e.getMessage(), - e); - } - - fileWriter = null; - } + tableModeTsFileBuilder.close(); + treeModeTsFileBuilder.close(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java index 15bd70cf4701..bc2da259fcdd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java @@ -23,7 +23,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.IoTDBConnectorRequestVersion; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeRequestType; import org.apache.iotdb.commons.utils.PathUtils; -import org.apache.iotdb.db.pipe.connector.util.PipeTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.queryengine.plan.parser.StatementGenerator; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; @@ -60,7 +60,7 @@ public boolean getIsAligned() { } public InsertTabletStatement constructStatement() { - new PipeTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); try { if (isTabletEmpty(tablet)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java index 0700acab1992..b30d716e5732 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java @@ -23,7 +23,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.IoTDBConnectorRequestVersion; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeRequestType; import org.apache.iotdb.commons.utils.PathUtils; -import org.apache.iotdb.db.pipe.connector.util.PipeTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.queryengine.plan.parser.StatementGenerator; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; @@ -57,7 +57,7 @@ public String getDataBaseName() { @Override public InsertTabletStatement constructStatement() { - new PipeTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); try { if (isTabletEmpty(tablet)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java index cc2f7d57aa1a..2aa4f4cada68 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java @@ -21,7 +21,7 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; -import org.apache.iotdb.db.pipe.connector.util.PipeTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.utils.DateTimeUtils; import org.apache.iotdb.db.utils.TimestampPrecisionUtils; import org.apache.iotdb.pipe.api.event.Event; @@ -101,7 +101,7 @@ void transfer(final Tablet tablet) throws UaException { } private void transferTabletForClientServerModel(final Tablet tablet) { - new PipeTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); final String[] segments = tablet.getDeviceId().split("\\."); if (segments.length == 0) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index e473017a489c..56bcd036b8db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -178,13 +178,13 @@ private void transferInBatchWithoutCheck( new PipeTransferTabletBatchEventHandler((PipeTabletEventPlainBatch) batch, this)); } else if (batch instanceof PipeTabletEventTsFileBatch) { final PipeTabletEventTsFileBatch tsFileBatch = (PipeTabletEventTsFileBatch) batch; - final List sealedFiles = tsFileBatch.sealTsFiles(); + final List> sealedFiles = tsFileBatch.sealTsFiles(); final Map, Double> pipe2WeightMap = tsFileBatch.deepCopyPipe2WeightMap(); final List events = tsFileBatch.deepCopyEvents(); final AtomicInteger eventsReferenceCount = new AtomicInteger(sealedFiles.size()); final AtomicBoolean eventsHadBeenAddedToRetryQueue = new AtomicBoolean(false); - for (final File sealedFile : sealedFiles) { + for (final Pair sealedFile : sealedFiles) { transfer( new PipeTransferTsFileHandler( this, @@ -192,10 +192,10 @@ private void transferInBatchWithoutCheck( events, eventsReferenceCount, eventsHadBeenAddedToRetryQueue, - sealedFile, + sealedFile.right, null, false, - null)); + sealedFile.left)); } } else { LOGGER.warn( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index 14f2ff5fd070..da28bbd46f75 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -306,13 +306,13 @@ private void doTransfer( private void doTransfer(final PipeTabletEventTsFileBatch batchToTransfer) throws IOException, WriteProcessException { - final List sealedFiles = batchToTransfer.sealTsFiles(); + final List> sealedFiles = batchToTransfer.sealTsFiles(); final Map, Double> pipe2WeightMap = batchToTransfer.deepCopyPipe2WeightMap(); - for (final File tsFile : sealedFiles) { - doTransfer(pipe2WeightMap, tsFile, null, null); + for (final Pair tsFile : sealedFiles) { + doTransfer(pipe2WeightMap, tsFile.right, null, tsFile.left); try { - FileUtils.delete(tsFile); + FileUtils.delete(tsFile.right); } catch (final NoSuchFileException e) { LOGGER.info("The file {} is not found, may already be deleted.", tsFile); } catch (final Exception e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java new file mode 100644 index 000000000000..a849779b43ed --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -0,0 +1,240 @@ +package org.apache.iotdb.db.pipe.connector.util; + +import org.apache.iotdb.pipe.api.exception.PipeException; + +import org.apache.commons.io.FileUtils; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.TableSchema; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.write.record.Tablet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; + +public class PipeTableModeTsFileBuilder extends PipeTsFileBuilder { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTableModeTsFileBuilder.class); + + private final Map> dataBase2TabletList = new HashMap<>(); + private final Map>>> tabletDeviceIdTimeRange = + new HashMap<>(); + + public PipeTableModeTsFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileIdGenerator) { + super(currentBatchId, tsFileIdGenerator); + } + + @Override + public void bufferTableModelTablet( + String dataBase, Tablet tablet, Map> deviceID2TimeRange) { + dataBase2TabletList.computeIfAbsent(dataBase, db -> new ArrayList<>()).add(tablet); + tabletDeviceIdTimeRange + .computeIfAbsent(dataBase, db -> new ArrayList<>()) + .add(deviceID2TimeRange); + } + + @Override + public void bufferTreeModelTablet(Tablet tablet, Boolean isAligned) {} + + @Override + public List> sealTsFiles() throws IOException, WriteProcessException { + if (dataBase2TabletList.isEmpty()) { + return new ArrayList<>(0); + } + List> pairList = new ArrayList<>(); + for (Map.Entry> entry : dataBase2TabletList.entrySet()) { + pairList.addAll( + writeTableModelTabletsToTsFiles( + entry.getValue(), tabletDeviceIdTimeRange.get(entry.getKey()), entry.getKey())); + } + return pairList; + } + + @Override + public boolean isEmpty() { + return dataBase2TabletList.isEmpty(); + } + + @Override + public synchronized void onSuccess() { + super.onSuccess(); + dataBase2TabletList.clear(); + } + + @Override + public synchronized void close() { + super.close(); + dataBase2TabletList.clear(); + } + + private List> writeTableModelTabletsToTsFiles( + List tabletList, + List>> deviceIDPairMap, + String dataBase) + throws IOException, WriteProcessException { + + final Map>>>> tableName2Tablets = + new HashMap<>(); + + // Sort the tablets by dataBaseName + for (int i = 0; i < tabletList.size(); i++) { + final Tablet tablet = tabletList.get(i); + tableName2Tablets + .computeIfAbsent(tablet.getTableName(), k -> new ArrayList<>()) + .add(new Pair<>(tablet, deviceIDPairMap.get(i))); + } + + // Sort the tablets by start time in each device + for (final List>>> tablets : + tableName2Tablets.values()) { + + tablets.sort( + // Each tablet has at least one timestamp + Comparator.comparingLong(pair -> pair.left.timestamps[0])); + } + + // Sort the devices by tableName + final List tables = new ArrayList<>(tableName2Tablets.keySet()); + tables.sort(Comparator.naturalOrder()); + + // Replace ArrayList with LinkedList to improve performance + final LinkedHashMap>>>> + table2TabletsLinkedList = new LinkedHashMap<>(); + for (final String tableName : tables) { + table2TabletsLinkedList.put(tableName, new LinkedList<>(tableName2Tablets.get(tableName))); + } + + // Help GC + tables.clear(); + tableName2Tablets.clear(); + + final List> sealedFiles = new ArrayList<>(); + + // Try making the tsfile size as large as possible + while (!table2TabletsLinkedList.isEmpty()) { + if (Objects.isNull(fileWriter)) { + createFileWriter(); + } + + try { + tryBestToWriteTabletsIntoOneFile(table2TabletsLinkedList); + } catch (final Exception e) { + LOGGER.warn( + "Batch id = {}: Failed to write tablets into tsfile, because {}", + currentBatchId.get(), + e.getMessage(), + e); + + try { + fileWriter.close(); + } catch (final Exception closeException) { + LOGGER.warn( + "Batch id = {}: Failed to close the tsfile {} after failed to write tablets into, because {}", + currentBatchId.get(), + fileWriter.getIOWriter().getFile().getPath(), + closeException.getMessage(), + closeException); + } finally { + // Add current writing file to the list and delete the file + sealedFiles.add(new Pair<>(dataBase, fileWriter.getIOWriter().getFile())); + } + + for (final Pair sealedFile : sealedFiles) { + final boolean deleteSuccess = FileUtils.deleteQuietly(sealedFile.right); + LOGGER.warn( + "Batch id = {}: {} delete the tsfile {} after failed to write tablets into {}. {}", + currentBatchId.get(), + deleteSuccess ? "Successfully" : "Failed to", + sealedFile.right.getPath(), + fileWriter.getIOWriter().getFile().getPath(), + deleteSuccess ? "" : "Maybe the tsfile needs to be deleted manually."); + } + sealedFiles.clear(); + + fileWriter = null; + + throw e; + } + + fileWriter.close(); + final File sealedFile = fileWriter.getIOWriter().getFile(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "Batch id = {}: Seal tsfile {} successfully.", + currentBatchId.get(), + sealedFile.getPath()); + } + sealedFiles.add(new Pair<>(dataBase, sealedFile)); + fileWriter = null; + } + + return sealedFiles; + } + + private void tryBestToWriteTabletsIntoOneFile( + final LinkedHashMap>>>> + device2TabletsLinkedList) + throws IOException, WriteProcessException { + final Iterator>>>>> + iterator = device2TabletsLinkedList.entrySet().iterator(); + + while (iterator.hasNext()) { + final Map.Entry>>>> entry = + iterator.next(); + final String tableName = entry.getKey(); + final LinkedList>>> tablets = entry.getValue(); + + final List tabletsToWrite = new ArrayList<>(); + + Pair>> lastTablet = null; + while (!tablets.isEmpty()) { + final Pair>> pair = tablets.peekFirst(); + if (Objects.isNull(lastTablet)) { + tabletsToWrite.add(pair.left); + lastTablet = pair; + tablets.pollFirst(); + } else { + break; + } + } + + if (tablets.isEmpty()) { + iterator.remove(); + } + boolean schemaNotRegistered = true; + for (final Tablet tablet : tabletsToWrite) { + if (schemaNotRegistered) { + fileWriter.registerTableSchema( + new TableSchema(tableName, tablet.getSchemas(), tablet.getColumnTypes())); + schemaNotRegistered = false; + } + try { + fileWriter.writeTable(tablet); + } catch (WriteProcessException e) { + throw new PipeException(""); + } + } + } + } + + // private boolean isOver(Map> lastPair,Map> + // currPair){ + // for(Map.Entry> entry : lastPair.entrySet()){ + // Pair pair = currPair.get(entry.getKey()); + // long lastPairL= + // if(pair.left>entry.getValue().right) + // } + // } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java new file mode 100644 index 000000000000..90469c7511c2 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -0,0 +1,229 @@ +/* + * 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.pipe.connector.util; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.write.UnSupportedDataTypeException; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.IMeasurementSchema; + +import java.time.LocalDate; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +public class PipeTableModelTabletEventSorter { + + private final Tablet tablet; + + private final Map> deviceID2TimeRange = new HashMap<>(); + + private Integer[] index; + private boolean isSorted; + private boolean isDeduplicated; + private int deduplicatedSize; + + public PipeTableModelTabletEventSorter(final Tablet tablet) { + this.tablet = tablet; + deduplicatedSize = tablet == null ? 0 : tablet.rowSize; + } + + public Map> deduplicateAndSortTimestampsIfNecessary() { + if (tablet == null || tablet.rowSize == 0) { + return null; + } + + for (int i = 1, size = tablet.rowSize; i < size; ++i) { + final IDeviceID deviceID = tablet.getDeviceID(i); + final Long currentTimestamp = tablet.timestamps[i]; + final Pair timeRange = deviceID2TimeRange.get(deviceID); + + if (timeRange == null) { + deviceID2TimeRange.put(deviceID, new Pair<>(currentTimestamp, currentTimestamp)); + continue; + } + if (timeRange.right > currentTimestamp) { + isSorted = false; + } else { + timeRange.right = currentTimestamp; + } + + if (timeRange.right.equals(currentTimestamp)) { + isDeduplicated = false; + } + + if (!isSorted && !isDeduplicated) { + break; + } + } + + if (isSorted && isDeduplicated) { + return deviceID2TimeRange; + } + deviceID2TimeRange.clear(); + index = new Integer[tablet.rowSize]; + for (int i = 0, size = tablet.rowSize; i < size; i++) { + index[i] = i; + } + + if (!isSorted) { + sortTimestamps(); + + // Do deduplicate anyway. + // isDeduplicated may be false positive when isSorted is false. + deduplicateTimestamps(); + isDeduplicated = true; + } + + if (!isDeduplicated) { + deduplicateTimestamps(); + } + + sortAndDeduplicateValuesAndBitMaps(); + return deviceID2TimeRange; + } + + private void sortTimestamps() { + Arrays.sort(index, Comparator.comparingLong(i -> tablet.timestamps[i])); + Arrays.sort(tablet.timestamps, 0, tablet.rowSize); + } + + private void deduplicateTimestamps() { + deduplicatedSize = 0; + Map lastIDeviceID = new HashMap<>(); + for (int i = 0, size = tablet.rowSize; i < size; i++) { + final IDeviceID deviceID = tablet.getDeviceID(index[i]); + final Pair timeRange = deviceID2TimeRange.get(deviceID); + final Long currTime = tablet.timestamps[i]; + if (timeRange != null && timeRange.right.equals(currTime)) { + final int lastIndex = lastIDeviceID.get(deviceID); + index[lastIndex] = index[i]; + tablet.timestamps[lastIndex] = tablet.timestamps[i]; + } else { + index[deduplicatedSize] = index[i]; + tablet.timestamps[deduplicatedSize] = tablet.timestamps[i]; + lastIDeviceID.put(deviceID, deduplicatedSize); + ++deduplicatedSize; + if (timeRange == null) { + deviceID2TimeRange.put(deviceID, new Pair<>(currTime, currTime)); + } else { + timeRange.right = currTime; + } + } + } + tablet.rowSize = deduplicatedSize; + } + + private void sortAndDeduplicateValuesAndBitMaps() { + int columnIndex = 0; + for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) { + final IMeasurementSchema schema = tablet.getSchemas().get(i); + if (schema != null) { + tablet.values[columnIndex] = + reorderValueList(deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); + if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) { + tablet.bitMaps[columnIndex] = + reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); + } + columnIndex++; + } + } + } + + private static Object reorderValueList( + int deduplicatedSize, + final Object valueList, + final TSDataType dataType, + final Integer[] index) { + switch (dataType) { + case BOOLEAN: + final boolean[] boolValues = (boolean[]) valueList; + final boolean[] deduplicatedBoolValues = new boolean[boolValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedBoolValues[i] = boolValues[index[i]]; + } + return deduplicatedBoolValues; + case INT32: + final int[] intValues = (int[]) valueList; + final int[] deduplicatedIntValues = new int[intValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedIntValues[i] = intValues[index[i]]; + } + return deduplicatedIntValues; + case DATE: + final LocalDate[] dateValues = (LocalDate[]) valueList; + final LocalDate[] deduplicatedDateValues = new LocalDate[dateValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedDateValues[i] = dateValues[index[i]]; + } + return deduplicatedDateValues; + case INT64: + case TIMESTAMP: + final long[] longValues = (long[]) valueList; + final long[] deduplicatedLongValues = new long[longValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedLongValues[i] = longValues[index[i]]; + } + return deduplicatedLongValues; + case FLOAT: + final float[] floatValues = (float[]) valueList; + final float[] deduplicatedFloatValues = new float[floatValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedFloatValues[i] = floatValues[index[i]]; + } + return deduplicatedFloatValues; + case DOUBLE: + final double[] doubleValues = (double[]) valueList; + final double[] deduplicatedDoubleValues = new double[doubleValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedDoubleValues[i] = doubleValues[index[i]]; + } + return deduplicatedDoubleValues; + case TEXT: + case BLOB: + case STRING: + final Binary[] binaryValues = (Binary[]) valueList; + final Binary[] deduplicatedBinaryValues = new Binary[binaryValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedBinaryValues[i] = binaryValues[index[i]]; + } + return deduplicatedBinaryValues; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataType)); + } + } + + private static BitMap reorderBitMap( + int deduplicatedSize, final BitMap bitMap, final Integer[] index) { + final BitMap deduplicatedBitMap = new BitMap(bitMap.getSize()); + for (int i = 0; i < deduplicatedSize; i++) { + if (bitMap.isMarked(index[i])) { + deduplicatedBitMap.mark(i); + } + } + return deduplicatedBitMap; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java new file mode 100644 index 000000000000..330094c8faed --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java @@ -0,0 +1,247 @@ +package org.apache.iotdb.db.pipe.connector.util; + +import org.apache.commons.io.FileUtils; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.common.Path; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; + +public class PipeTreeModelTSFileBuilder extends PipeTsFileBuilder { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTreeModelTSFileBuilder.class); + + private final List tabletList = new ArrayList<>(); + private final List isTabletAlignedList = new ArrayList<>(); + + public PipeTreeModelTSFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileIdGenerator) { + super(currentBatchId, tsFileIdGenerator); + } + + @Override + public void bufferTableModelTablet( + String dataBase, Tablet tablet, Map> deviceID2TimeRange) { + throw new UnsupportedOperationException(""); + } + + @Override + public void bufferTreeModelTablet(Tablet tablet, Boolean isAligned) { + tabletList.add(tablet); + isTabletAlignedList.add(isAligned); + } + + @Override + public List> sealTsFiles() throws IOException, WriteProcessException { + return writeTabletsToTsFiles(); + } + + @Override + public boolean isEmpty() { + return tabletList.isEmpty(); + } + + @Override + public void onSuccess() { + super.onSuccess(); + tabletList.clear(); + isTabletAlignedList.clear(); + } + + @Override + public synchronized void close() { + super.close(); + tabletList.clear(); + isTabletAlignedList.clear(); + } + + private List> writeTabletsToTsFiles() + throws IOException, WriteProcessException { + final Map> device2Tablets = new HashMap<>(); + final Map device2Aligned = new HashMap<>(); + + // Sort the tablets by device id + for (int i = 0, size = tabletList.size(); i < size; ++i) { + final Tablet tablet = tabletList.get(i); + final String deviceId = tablet.getDeviceId(); + device2Tablets.computeIfAbsent(deviceId, k -> new ArrayList<>()).add(tablet); + device2Aligned.put(deviceId, isTabletAlignedList.get(i)); + } + + // Sort the tablets by start time in each device + for (final List tablets : device2Tablets.values()) { + tablets.sort( + // Each tablet has at least one timestamp + Comparator.comparingLong(tablet -> tablet.timestamps[0])); + } + + // Sort the devices by device id + final List devices = new ArrayList<>(device2Tablets.keySet()); + devices.sort(Comparator.naturalOrder()); + + // Replace ArrayList with LinkedList to improve performance + final LinkedHashMap> device2TabletsLinkedList = + new LinkedHashMap<>(); + for (final String device : devices) { + device2TabletsLinkedList.put(device, new LinkedList<>(device2Tablets.get(device))); + } + + // Help GC + devices.clear(); + device2Tablets.clear(); + + // Write the tablets to the tsfile device by device, and the tablets + // in the same device are written in order of start time. Tablets in + // the same device should not be written if their time ranges overlap. + // If overlapped, we try to write the tablets whose device id is not + // the same as the previous one. For the tablets not written in the + // previous round, we write them in a new tsfile. + final List> sealedFiles = new ArrayList<>(); + + // Try making the tsfile size as large as possible + while (!device2TabletsLinkedList.isEmpty()) { + if (Objects.isNull(fileWriter)) { + createFileWriter(); + } + try { + tryBestToWriteTabletsIntoOneFile(device2TabletsLinkedList, device2Aligned); + } catch (final Exception e) { + LOGGER.warn( + "Batch id = {}: Failed to write tablets into tsfile, because {}", + currentBatchId.get(), + e.getMessage(), + e); + + try { + fileWriter.close(); + } catch (final Exception closeException) { + LOGGER.warn( + "Batch id = {}: Failed to close the tsfile {} after failed to write tablets into, because {}", + currentBatchId.get(), + fileWriter.getIOWriter().getFile().getPath(), + closeException.getMessage(), + closeException); + } finally { + // Add current writing file to the list and delete the file + sealedFiles.add(new Pair<>(null, fileWriter.getIOWriter().getFile())); + } + + for (final Pair sealedFile : sealedFiles) { + final boolean deleteSuccess = FileUtils.deleteQuietly(sealedFile.right); + LOGGER.warn( + "Batch id = {}: {} delete the tsfile {} after failed to write tablets into {}. {}", + currentBatchId.get(), + deleteSuccess ? "Successfully" : "Failed to", + sealedFile.right.getPath(), + fileWriter.getIOWriter().getFile().getPath(), + deleteSuccess ? "" : "Maybe the tsfile needs to be deleted manually."); + } + sealedFiles.clear(); + + fileWriter = null; + + throw e; + } + + fileWriter.close(); + final File sealedFile = fileWriter.getIOWriter().getFile(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "Batch id = {}: Seal tsfile {} successfully.", + currentBatchId.get(), + sealedFile.getPath()); + } + sealedFiles.add(new Pair<>(null, sealedFile)); + fileWriter = null; + } + + return sealedFiles; + } + + private void tryBestToWriteTabletsIntoOneFile( + final LinkedHashMap> device2TabletsLinkedList, + final Map device2Aligned) + throws IOException, WriteProcessException { + final Iterator>> iterator = + device2TabletsLinkedList.entrySet().iterator(); + + while (iterator.hasNext()) { + final Map.Entry> entry = iterator.next(); + final String deviceId = entry.getKey(); + final LinkedList tablets = entry.getValue(); + + final List tabletsToWrite = new ArrayList<>(); + + Tablet lastTablet = null; + while (!tablets.isEmpty()) { + final Tablet tablet = tablets.peekFirst(); + if (Objects.isNull(lastTablet) + // lastTablet.rowSize is not 0 + || lastTablet.timestamps[lastTablet.rowSize - 1] < tablet.timestamps[0]) { + tabletsToWrite.add(tablet); + lastTablet = tablet; + tablets.pollFirst(); + } else { + break; + } + } + + if (tablets.isEmpty()) { + iterator.remove(); + } + + final boolean isAligned = device2Aligned.get(deviceId); + if (isAligned) { + final Map> deviceId2MeasurementSchemas = new HashMap<>(); + tabletsToWrite.forEach( + tablet -> + deviceId2MeasurementSchemas.compute( + tablet.getDeviceId(), + (k, v) -> { + if (Objects.isNull(v)) { + return new ArrayList<>(tablet.getSchemas()); + } + v.addAll(tablet.getSchemas()); + return v; + })); + for (final Map.Entry> deviceIdWithMeasurementSchemas : + deviceId2MeasurementSchemas.entrySet()) { + fileWriter.registerAlignedTimeseries( + new Path(deviceIdWithMeasurementSchemas.getKey()), + deviceIdWithMeasurementSchemas.getValue()); + } + for (final Tablet tablet : tabletsToWrite) { + fileWriter.writeAligned(tablet); + } + } else { + for (final Tablet tablet : tabletsToWrite) { + for (final IMeasurementSchema schema : tablet.getSchemas()) { + try { + fileWriter.registerTimeseries( + IDeviceID.Factory.DEFAULT_FACTORY.create(tablet.getDeviceId()), schema); + } catch (final WriteProcessException ignore) { + // Do nothing if the timeSeries has been registered + } + } + + fileWriter.write(tablet); + } + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java similarity index 98% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTabletEventSorter.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java index 2a5e8769b59c..2fe45324d346 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java @@ -30,7 +30,7 @@ import java.util.Arrays; import java.util.Comparator; -public class PipeTabletEventSorter { +public class PipeTreeModelTabletEventSorter { private final Tablet tablet; @@ -40,7 +40,7 @@ public class PipeTabletEventSorter { private Integer[] index; private int deduplicatedSize; - public PipeTabletEventSorter(final Tablet tablet) { + public PipeTreeModelTabletEventSorter(final Tablet tablet) { this.tablet = tablet; deduplicatedSize = tablet == null ? 0 : tablet.rowSize; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java new file mode 100644 index 000000000000..be8f1518ec50 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java @@ -0,0 +1,145 @@ +package org.apache.iotdb.db.pipe.connector.util; + +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; +import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager; +import org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType; +import org.apache.iotdb.pipe.api.exception.PipeException; + +import org.apache.commons.io.FileUtils; +import org.apache.tsfile.common.constant.TsFileConstant; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.write.TsFileWriter; +import org.apache.tsfile.write.record.Tablet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +public abstract class PipeTsFileBuilder { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileBuilder.class); + + private static final AtomicReference FOLDER_MANAGER = new AtomicReference<>(); + protected final AtomicLong currentBatchId; + private final File batchFileBaseDir; + + private static final String TS_FILE_PREFIX = "tb"; // tb means tablet batch + private final AtomicLong tsFileIdGenerator; + + @SuppressWarnings("java:S3077") + protected volatile TsFileWriter fileWriter; + + public PipeTsFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileIdGenerator) { + this.currentBatchId = currentBatchId; + this.tsFileIdGenerator = tsFileIdGenerator; + try { + this.batchFileBaseDir = getNextBaseDir(); + } catch (final Exception e) { + throw new PipeException( + String.format("Failed to create file dir for batch: %s", e.getMessage())); + } + } + + private File getNextBaseDir() throws DiskSpaceInsufficientException { + if (FOLDER_MANAGER.get() == null) { + synchronized (FOLDER_MANAGER) { + if (FOLDER_MANAGER.get() == null) { + FOLDER_MANAGER.set( + new FolderManager( + Arrays.stream(IoTDBDescriptor.getInstance().getConfig().getPipeReceiverFileDirs()) + .map(fileDir -> fileDir + File.separator + ".batch") + .collect(Collectors.toList()), + DirectoryStrategyType.SEQUENCE_STRATEGY)); + } + } + } + + final File baseDir = + new File(FOLDER_MANAGER.get().getNextFolder(), Long.toString(currentBatchId.get())); + if (baseDir.exists()) { + FileUtils.deleteQuietly(baseDir); + } + if (!baseDir.exists() && !baseDir.mkdirs()) { + LOGGER.warn( + "Batch id = {}: Failed to create batch file dir {}.", + currentBatchId.get(), + baseDir.getPath()); + throw new PipeException( + String.format( + "Failed to create batch file dir %s. (Batch id = %s)", + baseDir.getPath(), currentBatchId.get())); + } + LOGGER.info( + "Batch id = {}: Create batch dir successfully, batch file dir = {}.", + currentBatchId.get(), + baseDir.getPath()); + return baseDir; + } + + public abstract void bufferTableModelTablet( + String dataBase, Tablet tablet, Map> deviceID2TimeRange); + + public abstract void bufferTreeModelTablet(Tablet tablet, Boolean isAligned); + + public abstract List> sealTsFiles() throws IOException, WriteProcessException; + + public abstract boolean isEmpty(); + + public synchronized void onSuccess() { + fileWriter = null; + } + + public synchronized void close() { + if (Objects.nonNull(fileWriter)) { + try { + fileWriter.close(); + } catch (final Exception e) { + LOGGER.info( + "Batch id = {}: Failed to close the tsfile {} when trying to close batch, because {}", + currentBatchId.get(), + fileWriter.getIOWriter().getFile().getPath(), + e.getMessage(), + e); + } + + try { + FileUtils.delete(fileWriter.getIOWriter().getFile()); + } catch (final Exception e) { + LOGGER.info( + "Batch id = {}: Failed to delete the tsfile {} when trying to close batch, because {}", + currentBatchId.get(), + fileWriter.getIOWriter().getFile().getPath(), + e.getMessage(), + e); + } + + fileWriter = null; + } + } + + protected void createFileWriter() throws IOException { + fileWriter = + new TsFileWriter( + new File( + batchFileBaseDir, + TS_FILE_PREFIX + + "_" + + IoTDBDescriptor.getInstance().getConfig().getDataNodeId() + + "_" + + currentBatchId.get() + + "_" + + tsFileIdGenerator.getAndIncrement() + + TsFileConstant.TSFILE_SUFFIX)); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/batch/SubscriptionPipeTsFileEventBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/batch/SubscriptionPipeTsFileEventBatch.java index 514795db23a3..d397a35cf20c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/batch/SubscriptionPipeTsFileEventBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/batch/SubscriptionPipeTsFileEventBatch.java @@ -28,6 +28,7 @@ import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; +import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,14 +96,16 @@ protected List generateSubscriptionEvents() throws Exception } final List events = new ArrayList<>(); - final List tsFiles = batch.sealTsFiles(); + final List> tsFiles = batch.sealTsFiles(); final AtomicInteger referenceCount = new AtomicInteger(tsFiles.size()); - for (final File tsFile : tsFiles) { + for (final Pair tsFile : tsFiles) { final SubscriptionCommitContext commitContext = prefetchingQueue.generateSubscriptionCommitContext(); events.add( new SubscriptionEvent( - new SubscriptionPipeTsFileBatchEvents(this, referenceCount), tsFile, commitContext)); + new SubscriptionPipeTsFileBatchEvents(this, referenceCount), + tsFile.right, + commitContext)); } return events; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index e6977c7384c6..8a89f0cffa28 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -19,7 +19,7 @@ package org.apache.iotdb.db.pipe.connector; -import org.apache.iotdb.db.pipe.connector.util.PipeTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.write.record.Tablet; @@ -82,7 +82,7 @@ public void testDeduplicateAndSort() { Assert.assertFalse(checkSorted(tablet)); - new PipeTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); Assert.assertTrue(checkSorted(tablet)); @@ -127,7 +127,7 @@ public void testDeduplicate() { Assert.assertTrue(checkSorted(tablet)); - new PipeTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); Assert.assertTrue(checkSorted(tablet)); @@ -196,7 +196,7 @@ public void testSort() { } } - new PipeTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); Assert.assertTrue(checkSorted(tablet)); From 205eb0e286dcca78bd0cdb96920af6d47a644fca Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 18 Nov 2024 11:58:40 +0800 Subject: [PATCH 02/42] Modify the sorting algorithm --- .../batch/PipeTabletEventTsFileBatch.java | 6 +- .../util/PipeTableModeTsFileBuilder.java | 108 ++++++++---- .../util/PipeTableModelTabletEventSorter.java | 156 +++++++++++------- .../util/PipeTreeModelTabletEventSorter.java | 5 +- .../connector/util/PipeTsFileBuilder.java | 3 +- 5 files changed, 183 insertions(+), 95 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index 012aeec64ae4..41677e0b95f8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -150,10 +150,10 @@ private void bufferTreeModelTablet( private void bufferTableModelTablet( final String pipeName, final long creationTime, final Tablet tablet, final String dataBase) { - final Map> deviceID2TimeRange = + final List> deviceID2Index = new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); - if (deviceID2TimeRange == null) { + if (deviceID2Index == null) { return; } @@ -163,7 +163,7 @@ private void bufferTableModelTablet( new Pair<>(pipeName, creationTime), (pipe, weight) -> Objects.nonNull(weight) ? ++weight : 1); - tableModeTsFileBuilder.bufferTableModelTablet(dataBase, tablet, deviceID2TimeRange); + tableModeTsFileBuilder.bufferTableModelTablet(dataBase, tablet, deviceID2Index); } public Map, Double> deepCopyPipe2WeightMap() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index a849779b43ed..51fdd8776038 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -29,7 +29,7 @@ public class PipeTableModeTsFileBuilder extends PipeTsFileBuilder { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTableModeTsFileBuilder.class); private final Map> dataBase2TabletList = new HashMap<>(); - private final Map>>> tabletDeviceIdTimeRange = + private final Map>>> tabletDeviceIdTimeIndex = new HashMap<>(); public PipeTableModeTsFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileIdGenerator) { @@ -38,11 +38,9 @@ public PipeTableModeTsFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileId @Override public void bufferTableModelTablet( - String dataBase, Tablet tablet, Map> deviceID2TimeRange) { + String dataBase, Tablet tablet, List> deviceID2Index) { dataBase2TabletList.computeIfAbsent(dataBase, db -> new ArrayList<>()).add(tablet); - tabletDeviceIdTimeRange - .computeIfAbsent(dataBase, db -> new ArrayList<>()) - .add(deviceID2TimeRange); + tabletDeviceIdTimeIndex.computeIfAbsent(dataBase, db -> new ArrayList<>()).add(deviceID2Index); } @Override @@ -57,7 +55,7 @@ public List> sealTsFiles() throws IOException, WriteProcessEx for (Map.Entry> entry : dataBase2TabletList.entrySet()) { pairList.addAll( writeTableModelTabletsToTsFiles( - entry.getValue(), tabletDeviceIdTimeRange.get(entry.getKey()), entry.getKey())); + entry.getValue(), tabletDeviceIdTimeIndex.get(entry.getKey()), entry.getKey())); } return pairList; } @@ -81,11 +79,11 @@ public synchronized void close() { private List> writeTableModelTabletsToTsFiles( List tabletList, - List>> deviceIDPairMap, + List>> deviceIDPairMap, String dataBase) throws IOException, WriteProcessException { - final Map>>>> tableName2Tablets = + final Map>>>> tableName2Tablets = new HashMap<>(); // Sort the tablets by dataBaseName @@ -97,12 +95,10 @@ private List> writeTableModelTabletsToTsFiles( } // Sort the tablets by start time in each device - for (final List>>> tablets : + for (final List>>> tablets : tableName2Tablets.values()) { - tablets.sort( - // Each tablet has at least one timestamp - Comparator.comparingLong(pair -> pair.left.timestamps[0])); + tablets.sort(this::comparePairs); } // Sort the devices by tableName @@ -110,7 +106,7 @@ private List> writeTableModelTabletsToTsFiles( tables.sort(Comparator.naturalOrder()); // Replace ArrayList with LinkedList to improve performance - final LinkedHashMap>>>> + final LinkedHashMap>>>> table2TabletsLinkedList = new LinkedHashMap<>(); for (final String tableName : tables) { table2TabletsLinkedList.put(tableName, new LinkedList<>(tableName2Tablets.get(tableName))); @@ -184,24 +180,25 @@ private List> writeTableModelTabletsToTsFiles( } private void tryBestToWriteTabletsIntoOneFile( - final LinkedHashMap>>>> + final LinkedHashMap>>>> device2TabletsLinkedList) - throws IOException, WriteProcessException { - final Iterator>>>>> + throws IOException { + final Iterator>>>>> iterator = device2TabletsLinkedList.entrySet().iterator(); while (iterator.hasNext()) { - final Map.Entry>>>> entry = + final Map.Entry>>>> entry = iterator.next(); final String tableName = entry.getKey(); - final LinkedList>>> tablets = entry.getValue(); + final LinkedList>>> tablets = entry.getValue(); final List tabletsToWrite = new ArrayList<>(); - Pair>> lastTablet = null; + Pair>> lastTablet = null; + Map deviceLastTimestampMap = new HashMap<>(); while (!tablets.isEmpty()) { - final Pair>> pair = tablets.peekFirst(); - if (Objects.isNull(lastTablet)) { + final Pair>> pair = tablets.peekFirst(); + if (Objects.isNull(lastTablet) || hasNoTimestampOverlaps(pair, deviceLastTimestampMap)) { tabletsToWrite.add(pair.left); lastTablet = pair; tablets.pollFirst(); @@ -229,12 +226,65 @@ private void tryBestToWriteTabletsIntoOneFile( } } - // private boolean isOver(Map> lastPair,Map> - // currPair){ - // for(Map.Entry> entry : lastPair.entrySet()){ - // Pair pair = currPair.get(entry.getKey()); - // long lastPairL= - // if(pair.left>entry.getValue().right) - // } - // } + private boolean hasNoTimestampOverlaps( + final Pair>> tabletPair, + final Map deviceLastTimestampMap) { + int currentTimestampIndex = 0; + for (Pair deviceTimestampIndexPair : tabletPair.right) { + final Long lastDeviceTimestamp = deviceLastTimestampMap.get(deviceTimestampIndexPair.left); + if (lastDeviceTimestamp != null + && lastDeviceTimestamp >= tabletPair.left.timestamps[currentTimestampIndex]) { + return false; + } + currentTimestampIndex = deviceTimestampIndexPair.right; + deviceLastTimestampMap.put( + deviceTimestampIndexPair.left, tabletPair.left.timestamps[currentTimestampIndex - 1]); + } + + return true; + } + + private int comparePairs( + final Pair>> pairA, + final Pair>> pairB) { + int aCount = 0; + int bCount = 0; + int aIndex = 0; + int bIndex = 0; + int aLastTimeIndex = 0; + int bLastTimeIndex = 0; + final List> listA = pairA.right; + final List> listB = pairB.right; + while (aIndex < listA.size() && bIndex < listB.size()) { + int comparisonResult = listA.get(aIndex).left.compareTo(listB.get(bIndex).left); + if (comparisonResult == 0) { + long aTime = pairA.left.timestamps[aLastTimeIndex]; + long bTime = pairB.left.timestamps[bLastTimeIndex]; + if (aTime > bTime) { + aCount++; + } else if (aTime < bTime) { + bCount++; + } + aLastTimeIndex = listA.get(aIndex).right; + bLastTimeIndex = listB.get(bIndex).right; + aIndex++; + bIndex++; + continue; + } + + if (comparisonResult > 0) { + bLastTimeIndex = listB.get(bIndex).right; + bIndex++; + aCount++; + continue; + } + + aLastTimeIndex = listA.get(aIndex).right; + aIndex++; + bCount++; + } + bCount += listB.size() - bIndex; + aCount += listA.size() - aIndex; + return Integer.compare(bCount, aCount); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index 90469c7511c2..d50ba82a2d15 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -29,75 +29,88 @@ import org.apache.tsfile.write.schema.IMeasurementSchema; import java.time.LocalDate; +import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Map; +import java.util.HashSet; +import java.util.List; +import java.util.Set; public class PipeTableModelTabletEventSorter { private final Tablet tablet; - private final Map> deviceID2TimeRange = new HashMap<>(); - private Integer[] index; - private boolean isSorted; - private boolean isDeduplicated; + private boolean isUnSorted = false; + private boolean isHasDeduplicated = false; private int deduplicatedSize; + private List> deviceID2TimeRange; public PipeTableModelTabletEventSorter(final Tablet tablet) { this.tablet = tablet; deduplicatedSize = tablet == null ? 0 : tablet.rowSize; } - public Map> deduplicateAndSortTimestampsIfNecessary() { - if (tablet == null || tablet.rowSize == 0) { + /** + * For the sorting and deduplication needs of the table model tablet, it is done according to the + * {@link IDeviceID}. For sorting, it is necessary to sort the {@link IDeviceID} first, and then + * sort by time. Deduplication is to remove the same timestamp in the same {@link IDeviceID}, and + * the same timestamp in different {@link IDeviceID} will not be processed. + * + * @return A list of pairs, each containing an instance of {@link IDeviceID} and the corresponding + * last index in the Tablet. + */ + public List> deduplicateAndSortTimestampsIfNecessary() { + if (tablet == null || tablet.rowSize < 1) { return null; } + deviceID2TimeRange = new ArrayList<>(); + final Set deviceIDSet = new HashSet<>(); + final long[] timestamps = tablet.timestamps; + + IDeviceID lastDevice = tablet.getDeviceID(0); for (int i = 1, size = tablet.rowSize; i < size; ++i) { final IDeviceID deviceID = tablet.getDeviceID(i); - final Long currentTimestamp = tablet.timestamps[i]; - final Pair timeRange = deviceID2TimeRange.get(deviceID); - - if (timeRange == null) { - deviceID2TimeRange.put(deviceID, new Pair<>(currentTimestamp, currentTimestamp)); + final long currentTimestamp = timestamps[i]; + final int deviceComparison = deviceID.compareTo(lastDevice); + if (deviceComparison == 0) { + final long previousTimestamp = tablet.timestamps[i - 1]; + if (previousTimestamp > currentTimestamp) { + isUnSorted = true; + break; + } + if (previousTimestamp == currentTimestamp) { + isHasDeduplicated = true; + } continue; } - if (timeRange.right > currentTimestamp) { - isSorted = false; - } else { - timeRange.right = currentTimestamp; - } - if (timeRange.right.equals(currentTimestamp)) { - isDeduplicated = false; - } - - if (!isSorted && !isDeduplicated) { + if (deviceComparison < 0 || deviceIDSet.contains(deviceID)) { + isUnSorted = true; break; } + deviceIDSet.add(deviceID); + deviceID2TimeRange.add(new Pair<>(deviceID, i - 1)); + lastDevice = deviceID; } - if (isSorted && isDeduplicated) { + if (!isUnSorted && !isHasDeduplicated) { + deviceID2TimeRange.add(new Pair<>(lastDevice, tablet.rowSize - 1)); return deviceID2TimeRange; } + deviceID2TimeRange.clear(); index = new Integer[tablet.rowSize]; for (int i = 0, size = tablet.rowSize; i < size; i++) { index[i] = i; } - if (!isSorted) { - sortTimestamps(); - - // Do deduplicate anyway. - // isDeduplicated may be false positive when isSorted is false. - deduplicateTimestamps(); - isDeduplicated = true; + if (!isUnSorted) { + sortAndDeduplicateTimestamps(); + isHasDeduplicated = true; } - if (!isDeduplicated) { + if (!isHasDeduplicated) { deduplicateTimestamps(); } @@ -105,35 +118,64 @@ public Map> deduplicateAndSortTimestampsIfNecessary( return deviceID2TimeRange; } - private void sortTimestamps() { - Arrays.sort(index, Comparator.comparingLong(i -> tablet.timestamps[i])); - Arrays.sort(tablet.timestamps, 0, tablet.rowSize); + private void sortAndDeduplicateTimestamps() { + Arrays.sort( + index, + (a, b) -> { + final int deviceComparison = tablet.getDeviceID(a).compareTo(tablet.getDeviceID(b)); + if (deviceComparison == 0) { + return Long.compare(tablet.timestamps[a], tablet.timestamps[b]); + } + return deviceComparison; + }); + final long[] timestamps = new long[tablet.rowSize]; + final long[] tabletTimestamps = tablet.timestamps; + timestamps[0] = tabletTimestamps[index[0]]; + IDeviceID lastDevice = tablet.getDeviceID(0); + deduplicatedSize = 1; + + for (int i = 1; i < timestamps.length; i++) { + final int timeIndex = index[i]; + final IDeviceID deviceId = tablet.getDeviceID(timeIndex); + if (!lastDevice.equals(deviceId)) { + timestamps[deduplicatedSize] = tabletTimestamps[timeIndex]; + lastDevice = deviceId; + deviceID2TimeRange.add(new Pair<>(deviceId, i)); + deduplicatedSize++; + continue; + } + + if (timestamps[deduplicatedSize - 1] != timestamps[deduplicatedSize]) { + timestamps[deduplicatedSize] = tabletTimestamps[timeIndex]; + lastDevice = deviceId; + deduplicatedSize++; + } + } + deviceID2TimeRange.add(new Pair<>(lastDevice, tablet.rowSize)); + tablet.timestamps = timestamps; } private void deduplicateTimestamps() { - deduplicatedSize = 0; - Map lastIDeviceID = new HashMap<>(); - for (int i = 0, size = tablet.rowSize; i < size; i++) { - final IDeviceID deviceID = tablet.getDeviceID(index[i]); - final Pair timeRange = deviceID2TimeRange.get(deviceID); - final Long currTime = tablet.timestamps[i]; - if (timeRange != null && timeRange.right.equals(currTime)) { - final int lastIndex = lastIDeviceID.get(deviceID); - index[lastIndex] = index[i]; - tablet.timestamps[lastIndex] = tablet.timestamps[i]; - } else { - index[deduplicatedSize] = index[i]; - tablet.timestamps[deduplicatedSize] = tablet.timestamps[i]; - lastIDeviceID.put(deviceID, deduplicatedSize); - ++deduplicatedSize; - if (timeRange == null) { - deviceID2TimeRange.put(deviceID, new Pair<>(currTime, currTime)); - } else { - timeRange.right = currTime; - } + final long[] timestamps = tablet.timestamps; + IDeviceID lastDevice = tablet.getDeviceID(0); + deduplicatedSize = 1; + for (int i = 1; i < timestamps.length; i++) { + final IDeviceID deviceId = tablet.getDeviceID(i); + if (!lastDevice.equals(deviceId)) { + timestamps[deduplicatedSize] = timestamps[i]; + lastDevice = deviceId; + deviceID2TimeRange.add(new Pair<>(deviceId, i)); + deduplicatedSize++; + continue; + } + + if (timestamps[deduplicatedSize - 1] != timestamps[deduplicatedSize]) { + timestamps[deduplicatedSize] = timestamps[i]; + lastDevice = deviceId; + deduplicatedSize++; } } - tablet.rowSize = deduplicatedSize; + deviceID2TimeRange.add(new Pair<>(lastDevice, tablet.rowSize)); } private void sortAndDeduplicateValuesAndBitMaps() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java index 2fe45324d346..fe98ff93ebdb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java @@ -56,14 +56,11 @@ public void deduplicateAndSortTimestampsIfNecessary() { if (currentTimestamp < previousTimestamp) { isSorted = false; + break; } if (currentTimestamp == previousTimestamp) { isDeduplicated = false; } - - if (!isSorted && !isDeduplicated) { - break; - } } if (isSorted && isDeduplicated) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java index be8f1518ec50..051fed1547fa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -88,7 +87,7 @@ private File getNextBaseDir() throws DiskSpaceInsufficientException { } public abstract void bufferTableModelTablet( - String dataBase, Tablet tablet, Map> deviceID2TimeRange); + String dataBase, Tablet tablet, List> deviceID2Index); public abstract void bufferTreeModelTablet(Tablet tablet, Boolean isAligned); From c40cc9d9f7e0e5c1f59aac75e7006cb0546cd155 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 18 Nov 2024 15:48:26 +0800 Subject: [PATCH 03/42] Modify code format --- .../util/PipeTableModeTsFileBuilder.java | 22 ++++++++++++++----- .../util/PipeTreeModelTSFileBuilder.java | 5 +++-- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index 51fdd8776038..dcd63297415b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -44,7 +44,10 @@ public void bufferTableModelTablet( } @Override - public void bufferTreeModelTablet(Tablet tablet, Boolean isAligned) {} + public void bufferTreeModelTablet(Tablet tablet, Boolean isAligned) { + throw new UnsupportedOperationException( + "PipeTableModeTsFileBuilder does not support tree model tablet to build TSFile"); + } @Override public List> sealTsFiles() throws IOException, WriteProcessException { @@ -69,19 +72,21 @@ public boolean isEmpty() { public synchronized void onSuccess() { super.onSuccess(); dataBase2TabletList.clear(); + tabletDeviceIdTimeIndex.clear(); } @Override public synchronized void close() { super.close(); dataBase2TabletList.clear(); + tabletDeviceIdTimeIndex.clear(); } private List> writeTableModelTabletsToTsFiles( List tabletList, List>> deviceIDPairMap, String dataBase) - throws IOException, WriteProcessException { + throws IOException { final Map>>>> tableName2Tablets = new HashMap<>(); @@ -94,10 +99,16 @@ private List> writeTableModelTabletsToTsFiles( .add(new Pair<>(tablet, deviceIDPairMap.get(i))); } - // Sort the tablets by start time in each device for (final List>>> tablets : tableName2Tablets.values()) { - + // Let's make an assumption that as long as the tablets with non-duplicate timestamps are + // sorted from small to large, assuming that there are N tablets that overlap with other + // tablets, then (n+1) TSFiles must be generated. Therefore, the Builder only needs to ensure + // that the tablets with non-overlapping times are sorted in order. + // A[ (c,3),(d,2)], device a [A],[B,C,D], + // B[(a,2),(b,1),(c,4), ],------->device b [D,A],[B,C],-------> [A->B->C],[D] + // C[(a,3),(b,2), (d,4)], device c [C],[D,A,B], + // D[(a,4), (c,2),(d,1)] device d [B],[D,A,C], tablets.sort(this::comparePairs); } @@ -220,7 +231,8 @@ private void tryBestToWriteTabletsIntoOneFile( try { fileWriter.writeTable(tablet); } catch (WriteProcessException e) { - throw new PipeException(""); + throw new PipeException( + "The Schema in the Tablet is inconsistent with the registered TableSchema"); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java index 330094c8faed..68221db0b9d5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java @@ -36,8 +36,9 @@ public PipeTreeModelTSFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileId @Override public void bufferTableModelTablet( - String dataBase, Tablet tablet, Map> deviceID2TimeRange) { - throw new UnsupportedOperationException(""); + String dataBase, Tablet tablet, List> deviceID2Index) { + throw new UnsupportedOperationException( + "PipeTreeModelTSFileBuilder does not support table model tablet to build TSFile"); } @Override From 7a1c7d84def9011f3919ee1437c32b9ee962826f Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 18 Nov 2024 16:00:10 +0800 Subject: [PATCH 04/42] Modify code format --- .../util/PipeTableModelTabletEventSorter.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index d50ba82a2d15..0461f0365245 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -41,7 +41,7 @@ public class PipeTableModelTabletEventSorter { private Integer[] index; private boolean isUnSorted = false; - private boolean isHasDeduplicated = false; + private boolean hasDuplicates = false; private int deduplicatedSize; private List> deviceID2TimeRange; @@ -80,7 +80,7 @@ public List> deduplicateAndSortTimestampsIfNecessary() break; } if (previousTimestamp == currentTimestamp) { - isHasDeduplicated = true; + hasDuplicates = true; } continue; } @@ -94,7 +94,7 @@ public List> deduplicateAndSortTimestampsIfNecessary() lastDevice = deviceID; } - if (!isUnSorted && !isHasDeduplicated) { + if (!isUnSorted && !hasDuplicates) { deviceID2TimeRange.add(new Pair<>(lastDevice, tablet.rowSize - 1)); return deviceID2TimeRange; } @@ -107,10 +107,10 @@ public List> deduplicateAndSortTimestampsIfNecessary() if (!isUnSorted) { sortAndDeduplicateTimestamps(); - isHasDeduplicated = true; + hasDuplicates = true; } - if (!isHasDeduplicated) { + if (!hasDuplicates) { deduplicateTimestamps(); } From ad580d6cb46920a67161b2776b7692e81adb1a05 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 18 Nov 2024 16:14:43 +0800 Subject: [PATCH 05/42] add license --- .../util/PipeTableModeTsFileBuilder.java | 19 +++++++++++++++++++ .../util/PipeTreeModelTSFileBuilder.java | 19 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index dcd63297415b..1ef9e52fe04e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -1,3 +1,22 @@ +/* + * 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.pipe.connector.util; import org.apache.iotdb.pipe.api.exception.PipeException; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java index 68221db0b9d5..b48d864e388b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java @@ -1,3 +1,22 @@ +/* + * 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.pipe.connector.util; import org.apache.commons.io.FileUtils; From a1152d5e72c20df38a42c61daa553ae455837219 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 18 Nov 2024 18:57:37 +0800 Subject: [PATCH 06/42] add license --- .../connector/util/PipeTsFileBuilder.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java index 051fed1547fa..076deee2a52f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java @@ -1,3 +1,22 @@ +/* + * 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.pipe.connector.util; import org.apache.iotdb.db.conf.IoTDBDescriptor; From bf01dcee9021a420c8a133093c636aad073a062c Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 18 Nov 2024 22:06:59 +0800 Subject: [PATCH 07/42] add test --- .../it/tablemodel/IoTDBPipeDataSinkIT.java | 2 - .../util/PipeTableModeTsFileBuilder.java | 63 ++++++++++--------- 2 files changed, 34 insertions(+), 31 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 5c9360417436..ff1231cb4484 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -110,8 +110,6 @@ public void testSinkTabletFormat() throws Exception { testSinkFormat("tablet"); } - // table model not support - @Ignore @Test public void testSinkTsFileFormat() throws Exception { testSinkFormat("tsfile"); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index 1ef9e52fe04e..ea5bf6b84df6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -113,22 +113,9 @@ private List> writeTableModelTabletsToTsFiles( // Sort the tablets by dataBaseName for (int i = 0; i < tabletList.size(); i++) { final Tablet tablet = tabletList.get(i); - tableName2Tablets - .computeIfAbsent(tablet.getTableName(), k -> new ArrayList<>()) - .add(new Pair<>(tablet, deviceIDPairMap.get(i))); - } - - for (final List>>> tablets : - tableName2Tablets.values()) { - // Let's make an assumption that as long as the tablets with non-duplicate timestamps are - // sorted from small to large, assuming that there are N tablets that overlap with other - // tablets, then (n+1) TSFiles must be generated. Therefore, the Builder only needs to ensure - // that the tablets with non-overlapping times are sorted in order. - // A[ (c,3),(d,2)], device a [A],[B,C,D], - // B[(a,2),(b,1),(c,4), ],------->device b [D,A],[B,C],-------> [A->B->C],[D] - // C[(a,3),(b,2), (d,4)], device c [C],[D,A,B], - // D[(a,4), (c,2),(d,1)] device d [B],[D,A,C], - tablets.sort(this::comparePairs); + insertPairs( + tableName2Tablets.computeIfAbsent(tablet.getTableName(), k -> new ArrayList<>()), + new Pair<>(tablet, deviceIDPairMap.get(i))); } // Sort the devices by tableName @@ -222,14 +209,14 @@ private void tryBestToWriteTabletsIntoOneFile( final String tableName = entry.getKey(); final LinkedList>>> tablets = entry.getValue(); - final List tabletsToWrite = new ArrayList<>(); + final List>>> tabletsToWrite = new ArrayList<>(); Pair>> lastTablet = null; Map deviceLastTimestampMap = new HashMap<>(); while (!tablets.isEmpty()) { final Pair>> pair = tablets.peekFirst(); if (Objects.isNull(lastTablet) || hasNoTimestampOverlaps(pair, deviceLastTimestampMap)) { - tabletsToWrite.add(pair.left); + tabletsToWrite.add(pair); lastTablet = pair; tablets.pollFirst(); } else { @@ -241,14 +228,15 @@ private void tryBestToWriteTabletsIntoOneFile( iterator.remove(); } boolean schemaNotRegistered = true; - for (final Tablet tablet : tabletsToWrite) { + for (final Pair>> pair : tabletsToWrite) { + final Tablet tablet = pair.left; if (schemaNotRegistered) { fileWriter.registerTableSchema( new TableSchema(tableName, tablet.getSchemas(), tablet.getColumnTypes())); schemaNotRegistered = false; } try { - fileWriter.writeTable(tablet); + fileWriter.writeTable(tablet, pair.right); } catch (WriteProcessException e) { throw new PipeException( "The Schema in the Tablet is inconsistent with the registered TableSchema"); @@ -275,10 +263,33 @@ private boolean hasNoTimestampOverlaps( return true; } + private void insertPairs( + List>>> list, + Pair>> pair) { + if (list.isEmpty()) { + list.add(pair); + return; + } + int lastCompare = Integer.MAX_VALUE; + for (int i = 0; i < list.size(); i++) { + final int result = comparePairs(list.get(i), pair); + if (result == 0) { + lastCompare = result; + continue; + } + + if (lastCompare == 0) { + list.add(pair); + return; + } + lastCompare = result; + } + list.add(pair); + } + private int comparePairs( final Pair>> pairA, final Pair>> pairB) { - int aCount = 0; int bCount = 0; int aIndex = 0; int bIndex = 0; @@ -291,9 +302,7 @@ private int comparePairs( if (comparisonResult == 0) { long aTime = pairA.left.timestamps[aLastTimeIndex]; long bTime = pairB.left.timestamps[bLastTimeIndex]; - if (aTime > bTime) { - aCount++; - } else if (aTime < bTime) { + if (aTime < bTime) { bCount++; } aLastTimeIndex = listA.get(aIndex).right; @@ -306,16 +315,12 @@ private int comparePairs( if (comparisonResult > 0) { bLastTimeIndex = listB.get(bIndex).right; bIndex++; - aCount++; continue; } aLastTimeIndex = listA.get(aIndex).right; aIndex++; - bCount++; } - bCount += listB.size() - bIndex; - aCount += listA.size() - aIndex; - return Integer.compare(bCount, aCount); + return bCount; } } From e2482539c9bee8d4ade0cd6864a0e82e8c4bbea9 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 18 Nov 2024 22:14:18 +0800 Subject: [PATCH 08/42] update PipeTableModeTsFileBuilder --- .../connector/util/PipeTableModeTsFileBuilder.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index ea5bf6b84df6..87489f9735a8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -270,19 +270,13 @@ private void insertPairs( list.add(pair); return; } - int lastCompare = Integer.MAX_VALUE; + for (int i = 0; i < list.size(); i++) { final int result = comparePairs(list.get(i), pair); if (result == 0) { - lastCompare = result; - continue; - } - - if (lastCompare == 0) { - list.add(pair); + list.add(i, pair); return; } - lastCompare = result; } list.add(pair); } From f3d5671cf21424760942ce38fdaa3bd7d3c95acd Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 18 Nov 2024 22:27:04 +0800 Subject: [PATCH 09/42] update IoTDBPipeDataSinkIT --- .../org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index ff1231cb4484..456f4f78aff3 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -29,7 +29,6 @@ import org.apache.iotdb.rpc.TSStatusCode; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; From 3676d6a315e6a1d97ef5c388c3eedcbeadf93fad Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 19 Nov 2024 14:59:16 +0800 Subject: [PATCH 10/42] Fix array out of bounds --- .../util/PipeTableModeTsFileBuilder.java | 21 ++++++++++++++-- .../util/PipeTableModelTabletEventSorter.java | 24 ++++++++++--------- 2 files changed, 32 insertions(+), 13 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index 87489f9735a8..371bd96f0785 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -220,6 +220,8 @@ private void tryBestToWriteTabletsIntoOneFile( lastTablet = pair; tablets.pollFirst(); } else { + // help GC + deviceLastTimestampMap = null; break; } } @@ -245,6 +247,13 @@ private void tryBestToWriteTabletsIntoOneFile( } } + /** + * A Map is used to record the maximum time each {@link IDeviceID} is written. {@link Pair} + * records the Index+1 of the maximum timestamp of IDevice in each {@link Tablet}. + * + * @return If true, the tablet overlaps with the previous tablet; if false, there is no time + * overlap. + */ private boolean hasNoTimestampOverlaps( final Pair>> tabletPair, final Map deviceLastTimestampMap) { @@ -263,9 +272,16 @@ private boolean hasNoTimestampOverlaps( return true; } + /** + * Add the Tablet to the List and compare the IDevice minimum timestamp with each Tablet from the + * beginning. If all the IDevice minimum timestamps of the current Tablet are smaller than the + * IDevice minimum timestamps of a certain Tablet in the List, put the current Tablet in this + * position. + */ private void insertPairs( List>>> list, Pair>> pair) { + int lastResult = Integer.MAX_VALUE; if (list.isEmpty()) { list.add(pair); return; @@ -273,10 +289,11 @@ private void insertPairs( for (int i = 0; i < list.size(); i++) { final int result = comparePairs(list.get(i), pair); - if (result == 0) { - list.add(i, pair); + if (lastResult == 0 && result != 0) { + list.add(i - 1, pair); return; } + lastResult = result; } list.add(pair); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index 0461f0365245..5b6ebc844c8d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -43,7 +43,7 @@ public class PipeTableModelTabletEventSorter { private boolean isUnSorted = false; private boolean hasDuplicates = false; private int deduplicatedSize; - private List> deviceID2TimeRange; + private List> deviceID2TimeIndex; public PipeTableModelTabletEventSorter(final Tablet tablet) { this.tablet = tablet; @@ -64,11 +64,13 @@ public List> deduplicateAndSortTimestampsIfNecessary() return null; } - deviceID2TimeRange = new ArrayList<>(); + deviceID2TimeIndex = new ArrayList<>(); final Set deviceIDSet = new HashSet<>(); final long[] timestamps = tablet.timestamps; IDeviceID lastDevice = tablet.getDeviceID(0); + deviceIDSet.add(lastDevice); + for (int i = 1, size = tablet.rowSize; i < size; ++i) { final IDeviceID deviceID = tablet.getDeviceID(i); final long currentTimestamp = timestamps[i]; @@ -90,16 +92,16 @@ public List> deduplicateAndSortTimestampsIfNecessary() break; } deviceIDSet.add(deviceID); - deviceID2TimeRange.add(new Pair<>(deviceID, i - 1)); + deviceID2TimeIndex.add(new Pair<>(deviceID, i)); lastDevice = deviceID; } if (!isUnSorted && !hasDuplicates) { - deviceID2TimeRange.add(new Pair<>(lastDevice, tablet.rowSize - 1)); - return deviceID2TimeRange; + deviceID2TimeIndex.add(new Pair<>(lastDevice, tablet.rowSize - 1)); + return deviceID2TimeIndex; } - deviceID2TimeRange.clear(); + deviceID2TimeIndex.clear(); index = new Integer[tablet.rowSize]; for (int i = 0, size = tablet.rowSize; i < size; i++) { index[i] = i; @@ -115,7 +117,7 @@ public List> deduplicateAndSortTimestampsIfNecessary() } sortAndDeduplicateValuesAndBitMaps(); - return deviceID2TimeRange; + return deviceID2TimeIndex; } private void sortAndDeduplicateTimestamps() { @@ -140,7 +142,7 @@ private void sortAndDeduplicateTimestamps() { if (!lastDevice.equals(deviceId)) { timestamps[deduplicatedSize] = tabletTimestamps[timeIndex]; lastDevice = deviceId; - deviceID2TimeRange.add(new Pair<>(deviceId, i)); + deviceID2TimeIndex.add(new Pair<>(deviceId, i)); deduplicatedSize++; continue; } @@ -151,7 +153,7 @@ private void sortAndDeduplicateTimestamps() { deduplicatedSize++; } } - deviceID2TimeRange.add(new Pair<>(lastDevice, tablet.rowSize)); + deviceID2TimeIndex.add(new Pair<>(lastDevice, tablet.rowSize)); tablet.timestamps = timestamps; } @@ -164,7 +166,7 @@ private void deduplicateTimestamps() { if (!lastDevice.equals(deviceId)) { timestamps[deduplicatedSize] = timestamps[i]; lastDevice = deviceId; - deviceID2TimeRange.add(new Pair<>(deviceId, i)); + deviceID2TimeIndex.add(new Pair<>(deviceId, i)); deduplicatedSize++; continue; } @@ -175,7 +177,7 @@ private void deduplicateTimestamps() { deduplicatedSize++; } } - deviceID2TimeRange.add(new Pair<>(lastDevice, tablet.rowSize)); + deviceID2TimeIndex.add(new Pair<>(lastDevice, tablet.rowSize)); } private void sortAndDeduplicateValuesAndBitMaps() { From add5ad6c4ea05013ea7616f9d0e475663d707039 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 19 Nov 2024 17:04:16 +0800 Subject: [PATCH 11/42] fix PipeTableModelTabletEventSorter --- .../it/tablemodel/IoTDBPipeDataSinkIT.java | 5 +++- .../util/PipeTableModelTabletEventSorter.java | 25 +++++++++++-------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 456f4f78aff3..9589e05ecd7d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -202,6 +202,9 @@ private void testSinkFormat(final String format) throws Exception { } TableModelUtils.insertData("test", "test", 150, 200, senderEnv, true); + TableModelUtils.insertDataByTablet("test", "test", 200, 250, senderEnv, true); + TableModelUtils.insertDataByTablet("test", "test", 250, 300, senderEnv, true); + TableModelUtils.insertDataByTablet("test", "test", 300, 350, senderEnv, true); TestUtils.assertDataEventuallyOnEnv( receiverEnv, @@ -210,7 +213,7 @@ private void testSinkFormat(final String format) throws Exception { Collections.unmodifiableSet( new HashSet<>(Arrays.asList("0,1.0,", "1,1.0,", "2,1.0,", "3,1.0,", "4,1.0,")))); - TableModelUtils.assertCountData("test", "test", 150, receiverEnv); + TableModelUtils.assertCountData("test", "test", 300, receiverEnv); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index 5b6ebc844c8d..210cedf253ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -92,12 +92,12 @@ public List> deduplicateAndSortTimestampsIfNecessary() break; } deviceIDSet.add(deviceID); - deviceID2TimeIndex.add(new Pair<>(deviceID, i)); + deviceID2TimeIndex.add(new Pair<>(lastDevice, i)); lastDevice = deviceID; } if (!isUnSorted && !hasDuplicates) { - deviceID2TimeIndex.add(new Pair<>(lastDevice, tablet.rowSize - 1)); + deviceID2TimeIndex.add(new Pair<>(lastDevice, tablet.rowSize)); return deviceID2TimeIndex; } @@ -125,15 +125,14 @@ private void sortAndDeduplicateTimestamps() { index, (a, b) -> { final int deviceComparison = tablet.getDeviceID(a).compareTo(tablet.getDeviceID(b)); - if (deviceComparison == 0) { - return Long.compare(tablet.timestamps[a], tablet.timestamps[b]); - } - return deviceComparison; + return deviceComparison == 0 + ? Long.compare(tablet.timestamps[a], tablet.timestamps[b]) + : deviceComparison; }); final long[] timestamps = new long[tablet.rowSize]; final long[] tabletTimestamps = tablet.timestamps; timestamps[0] = tabletTimestamps[index[0]]; - IDeviceID lastDevice = tablet.getDeviceID(0); + IDeviceID lastDevice = tablet.getDeviceID(index[0]); deduplicatedSize = 1; for (int i = 1; i < timestamps.length; i++) { @@ -141,14 +140,16 @@ private void sortAndDeduplicateTimestamps() { final IDeviceID deviceId = tablet.getDeviceID(timeIndex); if (!lastDevice.equals(deviceId)) { timestamps[deduplicatedSize] = tabletTimestamps[timeIndex]; + index[deduplicatedSize] = timeIndex; + deviceID2TimeIndex.add(new Pair<>(lastDevice, deduplicatedSize)); lastDevice = deviceId; - deviceID2TimeIndex.add(new Pair<>(deviceId, i)); deduplicatedSize++; continue; } - if (timestamps[deduplicatedSize - 1] != timestamps[deduplicatedSize]) { + if (timestamps[deduplicatedSize - 1] != timestamps[timeIndex]) { timestamps[deduplicatedSize] = tabletTimestamps[timeIndex]; + index[deduplicatedSize] = timeIndex; lastDevice = deviceId; deduplicatedSize++; } @@ -165,14 +166,16 @@ private void deduplicateTimestamps() { final IDeviceID deviceId = tablet.getDeviceID(i); if (!lastDevice.equals(deviceId)) { timestamps[deduplicatedSize] = timestamps[i]; + index[deduplicatedSize] = i; + deviceID2TimeIndex.add(new Pair<>(lastDevice, i)); lastDevice = deviceId; - deviceID2TimeIndex.add(new Pair<>(deviceId, i)); deduplicatedSize++; continue; } - if (timestamps[deduplicatedSize - 1] != timestamps[deduplicatedSize]) { + if (timestamps[deduplicatedSize - 1] != timestamps[i]) { timestamps[deduplicatedSize] = timestamps[i]; + index[deduplicatedSize] = i; lastDevice = deviceId; deduplicatedSize++; } From 189e086eceb5304253ec35a79424ef511dbd388f Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 19 Nov 2024 20:01:16 +0800 Subject: [PATCH 12/42] add UT --- .../util/PipeTableModeTsFileBuilder.java | 18 +-- .../util/PipeTableModelTabletEventSorter.java | 19 +-- .../util/PipeTreeModelTSFileBuilder.java | 9 +- .../util/PipeTreeModelTabletEventSorter.java | 4 +- .../connector/util/PipeTsFileBuilder.java | 2 +- .../connector/PipeTabletEventSorterTest.java | 135 +++++++++++++++++- 6 files changed, 161 insertions(+), 26 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index 371bd96f0785..2c9afd9e24b3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -102,9 +102,9 @@ public synchronized void close() { } private List> writeTableModelTabletsToTsFiles( - List tabletList, - List>> deviceIDPairMap, - String dataBase) + final List tabletList, + final List>> deviceIDPairMap, + final String dataBase) throws IOException { final Map>>>> tableName2Tablets = @@ -220,8 +220,6 @@ private void tryBestToWriteTabletsIntoOneFile( lastTablet = pair; tablets.pollFirst(); } else { - // help GC - deviceLastTimestampMap = null; break; } } @@ -240,8 +238,12 @@ private void tryBestToWriteTabletsIntoOneFile( try { fileWriter.writeTable(tablet, pair.right); } catch (WriteProcessException e) { + LOGGER.warn( + "Batch id = {}: Failed to build the table model TSFile. Please check whether the written Tablet has time overlap and whether the Table Schema is correct.", + currentBatchId.get(), + e); throw new PipeException( - "The Schema in the Tablet is inconsistent with the registered TableSchema"); + "The written Tablet time may overlap or the Schema may be incorrect"); } } } @@ -279,8 +281,8 @@ private boolean hasNoTimestampOverlaps( * position. */ private void insertPairs( - List>>> list, - Pair>> pair) { + final List>>> list, + final Pair>> pair) { int lastResult = Integer.MAX_VALUE; if (list.isEmpty()) { list.add(pair); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index 210cedf253ce..482e2dabeebb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -107,12 +107,12 @@ public List> deduplicateAndSortTimestampsIfNecessary() index[i] = i; } - if (!isUnSorted) { + if (isUnSorted) { sortAndDeduplicateTimestamps(); - hasDuplicates = true; + hasDuplicates = false; } - if (!hasDuplicates) { + if (hasDuplicates) { deduplicateTimestamps(); } @@ -147,14 +147,14 @@ private void sortAndDeduplicateTimestamps() { continue; } - if (timestamps[deduplicatedSize - 1] != timestamps[timeIndex]) { + if (timestamps[deduplicatedSize - 1] != tabletTimestamps[timeIndex]) { timestamps[deduplicatedSize] = tabletTimestamps[timeIndex]; index[deduplicatedSize] = timeIndex; lastDevice = deviceId; deduplicatedSize++; } } - deviceID2TimeIndex.add(new Pair<>(lastDevice, tablet.rowSize)); + deviceID2TimeIndex.add(new Pair<>(lastDevice, deduplicatedSize)); tablet.timestamps = timestamps; } @@ -167,7 +167,7 @@ private void deduplicateTimestamps() { if (!lastDevice.equals(deviceId)) { timestamps[deduplicatedSize] = timestamps[i]; index[deduplicatedSize] = i; - deviceID2TimeIndex.add(new Pair<>(lastDevice, i)); + deviceID2TimeIndex.add(new Pair<>(lastDevice, deduplicatedSize)); lastDevice = deviceId; deduplicatedSize++; continue; @@ -180,7 +180,7 @@ private void deduplicateTimestamps() { deduplicatedSize++; } } - deviceID2TimeIndex.add(new Pair<>(lastDevice, tablet.rowSize)); + deviceID2TimeIndex.add(new Pair<>(lastDevice, deduplicatedSize)); } private void sortAndDeduplicateValuesAndBitMaps() { @@ -197,10 +197,11 @@ private void sortAndDeduplicateValuesAndBitMaps() { columnIndex++; } } + tablet.rowSize = deduplicatedSize; } private static Object reorderValueList( - int deduplicatedSize, + final int deduplicatedSize, final Object valueList, final TSDataType dataType, final Integer[] index) { @@ -264,7 +265,7 @@ private static Object reorderValueList( } private static BitMap reorderBitMap( - int deduplicatedSize, final BitMap bitMap, final Integer[] index) { + final int deduplicatedSize, final BitMap bitMap, final Integer[] index) { final BitMap deduplicatedBitMap = new BitMap(bitMap.getSize()); for (int i = 0; i < deduplicatedSize; i++) { if (bitMap.isMarked(index[i])) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java index b48d864e388b..080c31d8868c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java @@ -49,19 +49,22 @@ public class PipeTreeModelTSFileBuilder extends PipeTsFileBuilder { private final List tabletList = new ArrayList<>(); private final List isTabletAlignedList = new ArrayList<>(); - public PipeTreeModelTSFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileIdGenerator) { + public PipeTreeModelTSFileBuilder( + final AtomicLong currentBatchId, final AtomicLong tsFileIdGenerator) { super(currentBatchId, tsFileIdGenerator); } @Override public void bufferTableModelTablet( - String dataBase, Tablet tablet, List> deviceID2Index) { + final String dataBase, + final Tablet tablet, + final List> deviceID2Index) { throw new UnsupportedOperationException( "PipeTreeModelTSFileBuilder does not support table model tablet to build TSFile"); } @Override - public void bufferTreeModelTablet(Tablet tablet, Boolean isAligned) { + public void bufferTreeModelTablet(final Tablet tablet, final Boolean isAligned) { tabletList.add(tablet); isTabletAlignedList.add(isAligned); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java index fe98ff93ebdb..614e4f032e71 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java @@ -123,7 +123,7 @@ private void sortAndDeduplicateValuesAndBitMaps() { } private static Object reorderValueList( - int deduplicatedSize, + final int deduplicatedSize, final Object valueList, final TSDataType dataType, final Integer[] index) { @@ -187,7 +187,7 @@ private static Object reorderValueList( } private static BitMap reorderBitMap( - int deduplicatedSize, final BitMap bitMap, final Integer[] index) { + final int deduplicatedSize, final BitMap bitMap, final Integer[] index) { final BitMap deduplicatedBitMap = new BitMap(bitMap.getSize()); for (int i = 0; i < deduplicatedSize; i++) { if (bitMap.isMarked(index[i])) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java index 076deee2a52f..da145e54c0cd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java @@ -58,7 +58,7 @@ public abstract class PipeTsFileBuilder { @SuppressWarnings("java:S3077") protected volatile TsFileWriter fileWriter; - public PipeTsFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileIdGenerator) { + public PipeTsFileBuilder(final AtomicLong currentBatchId, final AtomicLong tsFileIdGenerator) { this.currentBatchId = currentBatchId; this.tsFileIdGenerator = tsFileIdGenerator; try { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index 8a89f0cffa28..6274c1695149 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -19,17 +19,27 @@ package org.apache.iotdb.db.pipe.connector; +import org.apache.iotdb.db.pipe.connector.util.PipeTableModelTabletEventSorter; import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.DateUtils; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.WriteUtils; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.MeasurementSchema; import org.junit.Assert; import org.junit.Test; +import java.nio.charset.StandardCharsets; +import java.text.SimpleDateFormat; +import java.time.LocalDate; import java.util.ArrayList; import java.util.Arrays; +import java.util.Date; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -46,7 +56,7 @@ private static boolean checkSorted(final Tablet tablet) { } @Test - public void testDeduplicateAndSort() { + public void testTreeModelDeduplicateAndSort() { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); schemaList.add(new MeasurementSchema("s2", TSDataType.INT64)); @@ -103,7 +113,7 @@ public void testDeduplicateAndSort() { } @Test - public void testDeduplicate() { + public void testTreeModelDeduplicate() { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); schemaList.add(new MeasurementSchema("s2", TSDataType.INT64)); @@ -148,7 +158,7 @@ public void testDeduplicate() { } @Test - public void testSort() { + public void testTreeModelSort() { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); schemaList.add(new MeasurementSchema("s2", TSDataType.INT64)); @@ -215,4 +225,123 @@ public void testSort() { } } } + + @Test + public void testTableModelDeduplicateAndSort() { + doTableModelTest(true, true); + } + + @Test + public void testTableModelDeduplicate() { + doTableModelTest(true, false); + } + + @Test + public void testTableModelSort() { + doTableModelTest(false, true); + } + + public void doTableModelTest(final boolean isDeduplicated, final boolean isUnSorted) { + Tablet tablet = generateTablet("test", 10, isDeduplicated, isUnSorted); + + List> list = + new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + List> list1 = WriteUtils.splitTabletByDevice(tablet); + Assert.assertEquals(list1, list); + for (int i = 1; i < tablet.rowSize; i++) { + long time = tablet.timestamps[i]; + Assert.assertTrue(time > tablet.timestamps[i - 1]); + Assert.assertEquals( + tablet.getValue(i, 0), + new Binary(String.valueOf(i / 100).getBytes(StandardCharsets.UTF_8))); + Assert.assertEquals(tablet.getValue(i, 1), (long) i); + Assert.assertEquals(tablet.getValue(i, 2), i * 1.0f); + Assert.assertEquals( + tablet.getValue(i, 3), new Binary(String.valueOf(i).getBytes(StandardCharsets.UTF_8))); + Assert.assertEquals(tablet.getValue(i, 4), (long) i); + Assert.assertEquals(tablet.getValue(i, 5), i); + Assert.assertEquals(tablet.getValue(i, 6), i * 0.1); + Assert.assertEquals(tablet.getValue(i, 7), getDate(i)); + Assert.assertEquals( + tablet.getValue(i, 8), new Binary(String.valueOf(i).getBytes(StandardCharsets.UTF_8))); + } + } + + private Tablet generateTablet( + final String tableName, + final int deviceIDNum, + final boolean isDeduplicated, + final boolean isUnSorted) { + List schemaList = new ArrayList<>(); + schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); + schemaList.add(new MeasurementSchema("s2", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s3", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s4", TSDataType.TIMESTAMP)); + schemaList.add(new MeasurementSchema("s5", TSDataType.INT32)); + schemaList.add(new MeasurementSchema("s6", TSDataType.DOUBLE)); + schemaList.add(new MeasurementSchema("s7", TSDataType.DATE)); + schemaList.add(new MeasurementSchema("s8", TSDataType.TEXT)); + + final List columnTypes = + Arrays.asList( + Tablet.ColumnType.ID, + Tablet.ColumnType.MEASUREMENT, + Tablet.ColumnType.MEASUREMENT, + Tablet.ColumnType.MEASUREMENT, + Tablet.ColumnType.MEASUREMENT, + Tablet.ColumnType.MEASUREMENT, + Tablet.ColumnType.MEASUREMENT, + Tablet.ColumnType.MEASUREMENT, + Tablet.ColumnType.MEASUREMENT); + Tablet tablet = new Tablet(tableName, schemaList, columnTypes, deviceIDNum * 1000); + tablet.initBitMaps(); + + // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text + int rowIndex = 0; + for (long row = 0; row < deviceIDNum; row++) { + + for (int i = 0; i < 100; i++) { + final long value; + if (isUnSorted) { + value = (row + 1) * 100 - i - 1; + } else { + value = (row) * 100 + i; + } + for (int j = 0; j < 10; j++) { + tablet.addTimestamp(rowIndex, value); + tablet.addValue( + "s0", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s1", rowIndex, value); + tablet.addValue("s2", rowIndex, (value * 1.0f)); + tablet.addValue( + "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s4", rowIndex, value); + tablet.addValue("s5", rowIndex, (int) value); + tablet.addValue("s6", rowIndex, value * 0.1); + tablet.addValue("s7", rowIndex, getDate((int) value)); + tablet.addValue( + "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + rowIndex++; + tablet.rowSize++; + if (!isDeduplicated) { + break; + } + } + } + } + + return tablet; + } + + public static LocalDate getDate(int value) { + Date date = new Date(value); + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); + try { + return DateUtils.parseIntToLocalDate( + DateUtils.parseDateExpressionToInt(dateFormat.format(date))); + } catch (Exception e) { + return DateUtils.parseIntToLocalDate(DateUtils.parseDateExpressionToInt("1970-01-01")); + } + } } From 4c764053ce01120d25394223c5cf0d5bf71ffff0 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 19 Nov 2024 22:10:57 +0800 Subject: [PATCH 13/42] fix ci --- .../iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java | 4 ++-- .../connector/util/PipeTableModeTsFileBuilder.java | 11 ++++------- .../util/PipeTableModelTabletEventSorter.java | 4 +++- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 9589e05ecd7d..3e27febec0e0 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -164,7 +164,7 @@ private void testSinkFormat(final String format) throws Exception { Assert.assertEquals( TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("testPipe").getCode()); - TableModelUtils.insertData("test", "test", 100, 150, senderEnv, true); + TableModelUtils.insertData("test", "test", 50, 150, senderEnv, true); if (!TestUtils.tryExecuteNonQueriesWithRetry( senderEnv, @@ -213,7 +213,7 @@ private void testSinkFormat(final String format) throws Exception { Collections.unmodifiableSet( new HashSet<>(Arrays.asList("0,1.0,", "1,1.0,", "2,1.0,", "3,1.0,", "4,1.0,")))); - TableModelUtils.assertCountData("test", "test", 300, receiverEnv); + TableModelUtils.assertCountData("test", "test", 350, receiverEnv); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index 2c9afd9e24b3..61169b264910 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -210,18 +210,15 @@ private void tryBestToWriteTabletsIntoOneFile( final LinkedList>>> tablets = entry.getValue(); final List>>> tabletsToWrite = new ArrayList<>(); - - Pair>> lastTablet = null; - Map deviceLastTimestampMap = new HashMap<>(); + final Map deviceLastTimestampMap = new HashMap<>(); while (!tablets.isEmpty()) { final Pair>> pair = tablets.peekFirst(); - if (Objects.isNull(lastTablet) || hasNoTimestampOverlaps(pair, deviceLastTimestampMap)) { + if (hasNoTimestampOverlaps(pair, deviceLastTimestampMap)) { tabletsToWrite.add(pair); - lastTablet = pair; tablets.pollFirst(); - } else { - break; + continue; } + break; } if (tablets.isEmpty()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index 482e2dabeebb..6afd55e7325a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -65,7 +65,7 @@ public List> deduplicateAndSortTimestampsIfNecessary() } deviceID2TimeIndex = new ArrayList<>(); - final Set deviceIDSet = new HashSet<>(); + Set deviceIDSet = new HashSet<>(); final long[] timestamps = tablet.timestamps; IDeviceID lastDevice = tablet.getDeviceID(0); @@ -79,6 +79,7 @@ public List> deduplicateAndSortTimestampsIfNecessary() final long previousTimestamp = tablet.timestamps[i - 1]; if (previousTimestamp > currentTimestamp) { isUnSorted = true; + deviceIDSet = null; break; } if (previousTimestamp == currentTimestamp) { @@ -89,6 +90,7 @@ public List> deduplicateAndSortTimestampsIfNecessary() if (deviceComparison < 0 || deviceIDSet.contains(deviceID)) { isUnSorted = true; + deviceIDSet = null; break; } deviceIDSet.add(deviceID); From 211beacb44ce3f3f53b128e161a3a79380f8ae13 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 21 Nov 2024 10:22:11 +0800 Subject: [PATCH 14/42] modify comment --- .../db/pipe/connector/util/PipeTableModeTsFileBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index 61169b264910..4f927bcd0a52 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -250,7 +250,7 @@ private void tryBestToWriteTabletsIntoOneFile( * A Map is used to record the maximum time each {@link IDeviceID} is written. {@link Pair} * records the Index+1 of the maximum timestamp of IDevice in each {@link Tablet}. * - * @return If true, the tablet overlaps with the previous tablet; if false, there is no time + * @return If false, the tablet overlaps with the previous tablet; if true, there is no time * overlap. */ private boolean hasNoTimestampOverlaps( From 3108ccf122934d90c78c55699780d0435ca27a00 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 21 Nov 2024 12:25:19 +0800 Subject: [PATCH 15/42] fix PipeTransferTabletRawReqV2 --- .../evolvable/request/PipeTransferTabletRawReqV2.java | 6 +++++- .../db/pipe/connector/PipeTabletEventSorterTest.java | 8 ++++---- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java index b30d716e5732..e7e6ee2cbd82 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java @@ -57,7 +57,11 @@ public String getDataBaseName() { @Override public InsertTabletStatement constructStatement() { - new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + if (Objects.isNull(dataBaseName)) { + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + } else { + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + } try { if (isTabletEmpty(tablet)) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index 6274c1695149..c58f8c322c56 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -242,7 +242,7 @@ public void testTableModelSort() { } public void doTableModelTest(final boolean isDeduplicated, final boolean isUnSorted) { - Tablet tablet = generateTablet("test", 10, isDeduplicated, isUnSorted); + final Tablet tablet = generateTablet("test", 10, isDeduplicated, isUnSorted); List> list = new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); @@ -272,7 +272,7 @@ private Tablet generateTablet( final int deviceIDNum, final boolean isDeduplicated, final boolean isUnSorted) { - List schemaList = new ArrayList<>(); + final List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); schemaList.add(new MeasurementSchema("s2", TSDataType.FLOAT)); @@ -294,7 +294,7 @@ private Tablet generateTablet( Tablet.ColumnType.MEASUREMENT, Tablet.ColumnType.MEASUREMENT, Tablet.ColumnType.MEASUREMENT); - Tablet tablet = new Tablet(tableName, schemaList, columnTypes, deviceIDNum * 1000); + final Tablet tablet = new Tablet(tableName, schemaList, columnTypes, deviceIDNum * 1000); tablet.initBitMaps(); // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text @@ -334,7 +334,7 @@ private Tablet generateTablet( return tablet; } - public static LocalDate getDate(int value) { + public LocalDate getDate(final int value) { Date date = new Date(value); SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); try { From 68640c5ddc5ba131dda076601dd33417a3f61ebb Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 26 Nov 2024 22:57:05 +0800 Subject: [PATCH 16/42] update PipeTableModelTabletEventSorter --- .../util/PipeTableModelTabletEventSorter.java | 180 +++++++++++------- .../connector/PipeTabletEventSorterTest.java | 11 +- 2 files changed, 113 insertions(+), 78 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index 6afd55e7325a..602ebbae9544 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -31,9 +31,10 @@ import java.time.LocalDate; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashSet; +import java.util.Comparator; +import java.util.HashMap; import java.util.List; -import java.util.Set; +import java.util.Objects; public class PipeTableModelTabletEventSorter { @@ -43,7 +44,8 @@ public class PipeTableModelTabletEventSorter { private boolean isUnSorted = false; private boolean hasDuplicates = false; private int deduplicatedSize; - private List> deviceID2TimeIndex; + + private List> deviceIDTimeIndexList; public PipeTableModelTabletEventSorter(final Tablet tablet) { this.tablet = tablet; @@ -64,125 +66,121 @@ public List> deduplicateAndSortTimestampsIfNecessary() return null; } - deviceID2TimeIndex = new ArrayList<>(); - Set deviceIDSet = new HashSet<>(); + deviceIDTimeIndexList = new ArrayList<>(); + HashMap deviceIDToIndexMap = new HashMap<>(); final long[] timestamps = tablet.timestamps; IDeviceID lastDevice = tablet.getDeviceID(0); - deviceIDSet.add(lastDevice); - + long previousTimestamp = tablet.timestamps[0]; + int lasIndex = 0; for (int i = 1, size = tablet.rowSize; i < size; ++i) { final IDeviceID deviceID = tablet.getDeviceID(i); final long currentTimestamp = timestamps[i]; final int deviceComparison = deviceID.compareTo(lastDevice); if (deviceComparison == 0) { - final long previousTimestamp = tablet.timestamps[i - 1]; if (previousTimestamp > currentTimestamp) { isUnSorted = true; - deviceIDSet = null; - break; + continue; } if (previousTimestamp == currentTimestamp) { hasDuplicates = true; } + previousTimestamp = currentTimestamp; continue; } - - if (deviceComparison < 0 || deviceIDSet.contains(deviceID)) { + if (deviceComparison < 0) { isUnSorted = true; - deviceIDSet = null; - break; } - deviceIDSet.add(deviceID); - deviceID2TimeIndex.add(new Pair<>(lastDevice, i)); + updateDeviceIDIndex(deviceIDToIndexMap, lastDevice, lasIndex, i); lastDevice = deviceID; + lasIndex = i; + previousTimestamp = currentTimestamp; } + updateDeviceIDIndex(deviceIDToIndexMap, lastDevice, lasIndex, tablet.rowSize); if (!isUnSorted && !hasDuplicates) { - deviceID2TimeIndex.add(new Pair<>(lastDevice, tablet.rowSize)); - return deviceID2TimeIndex; + return deviceIDTimeIndexList; } - deviceID2TimeIndex.clear(); - index = new Integer[tablet.rowSize]; - for (int i = 0, size = tablet.rowSize; i < size; i++) { - index[i] = i; - } + initIndex(); if (isUnSorted) { sortAndDeduplicateTimestamps(); hasDuplicates = false; + isUnSorted = false; } if (hasDuplicates) { deduplicateTimestamps(); + hasDuplicates = false; } sortAndDeduplicateValuesAndBitMaps(); - return deviceID2TimeIndex; + return deviceIDTimeIndexList; } + // This function sorts the tablets. It sorts the time under each IDeviceID first, then sorts each + // IDevice, and then removes duplicates. private void sortAndDeduplicateTimestamps() { - Arrays.sort( - index, - (a, b) -> { - final int deviceComparison = tablet.getDeviceID(a).compareTo(tablet.getDeviceID(b)); - return deviceComparison == 0 - ? Long.compare(tablet.timestamps[a], tablet.timestamps[b]) - : deviceComparison; - }); + // Sorting the time of the same IDevice + int startIndex = 0; + final Comparator comparator = Comparator.comparingLong(i -> tablet.timestamps[i]); + List>> deviceIndexRange = + new ArrayList<>(deviceIDTimeIndexList.size()); + for (Pair pair : deviceIDTimeIndexList) { + Arrays.sort(this.index, startIndex, pair.right, comparator); + deviceIndexRange.add(new Pair<>(pair.left, new Pair<>(startIndex, pair.right - 1))); + startIndex = pair.right; + } + + // Sort IDevices + deviceIDTimeIndexList.clear(); + deviceIndexRange.sort(Comparator.comparing(a -> a.left)); + + // Deduplication and update Index array final long[] timestamps = new long[tablet.rowSize]; final long[] tabletTimestamps = tablet.timestamps; - timestamps[0] = tabletTimestamps[index[0]]; - IDeviceID lastDevice = tablet.getDeviceID(index[0]); - deduplicatedSize = 1; - - for (int i = 1; i < timestamps.length; i++) { - final int timeIndex = index[i]; - final IDeviceID deviceId = tablet.getDeviceID(timeIndex); - if (!lastDevice.equals(deviceId)) { - timestamps[deduplicatedSize] = tabletTimestamps[timeIndex]; - index[deduplicatedSize] = timeIndex; - deviceID2TimeIndex.add(new Pair<>(lastDevice, deduplicatedSize)); - lastDevice = deviceId; - deduplicatedSize++; - continue; - } + final Integer[] copyIndex = new Integer[index.length]; - if (timestamps[deduplicatedSize - 1] != tabletTimestamps[timeIndex]) { - timestamps[deduplicatedSize] = tabletTimestamps[timeIndex]; - index[deduplicatedSize] = timeIndex; - lastDevice = deviceId; - deduplicatedSize++; + deduplicatedSize = 0; + for (Pair> deviceRange : deviceIndexRange) { + startIndex = deviceRange.right.left; + long lastTimestamps = timestamps[deduplicatedSize] = tabletTimestamps[index[startIndex]]; + copyIndex[deduplicatedSize++] = index[startIndex++]; + for (final int end = deviceRange.right.right; startIndex <= end; startIndex++) { + final long curTimestamps = tabletTimestamps[index[startIndex]]; + if (lastTimestamps == curTimestamps) { + continue; + } + lastTimestamps = timestamps[deduplicatedSize] = curTimestamps; + copyIndex[deduplicatedSize++] = index[startIndex]; } + deviceIDTimeIndexList.add(new Pair<>(deviceRange.left, deduplicatedSize)); } - deviceID2TimeIndex.add(new Pair<>(lastDevice, deduplicatedSize)); + index = copyIndex; tablet.timestamps = timestamps; } private void deduplicateTimestamps() { + int startIndex = 0; + deduplicatedSize = 0; final long[] timestamps = tablet.timestamps; - IDeviceID lastDevice = tablet.getDeviceID(0); - deduplicatedSize = 1; - for (int i = 1; i < timestamps.length; i++) { - final IDeviceID deviceId = tablet.getDeviceID(i); - if (!lastDevice.equals(deviceId)) { - timestamps[deduplicatedSize] = timestamps[i]; - index[deduplicatedSize] = i; - deviceID2TimeIndex.add(new Pair<>(lastDevice, deduplicatedSize)); - lastDevice = deviceId; - deduplicatedSize++; - continue; - } - - if (timestamps[deduplicatedSize - 1] != timestamps[i]) { - timestamps[deduplicatedSize] = timestamps[i]; - index[deduplicatedSize] = i; - lastDevice = deviceId; - deduplicatedSize++; + for (Pair pair : deviceIDTimeIndexList) { + final int endIndex = pair.right; + long lastTimestamps = timestamps[startIndex]; + timestamps[deduplicatedSize] = lastTimestamps; + index[deduplicatedSize++] = index[startIndex++]; + for (; startIndex < endIndex; startIndex++) { + final long curTimestamp = timestamps[startIndex]; + if (curTimestamp == lastTimestamps) { + continue; + } + index[deduplicatedSize] = index[startIndex]; + timestamps[deduplicatedSize++] = curTimestamp; + lastTimestamps = curTimestamp; } + pair.right = deduplicatedSize; } - deviceID2TimeIndex.add(new Pair<>(lastDevice, deduplicatedSize)); } private void sortAndDeduplicateValuesAndBitMaps() { @@ -276,4 +274,42 @@ private static BitMap reorderBitMap( } return deduplicatedBitMap; } + + private void updateDeviceIDIndex( + HashMap deviceID2ListIndex, IDeviceID device, int start, int end) { + final Integer index = deviceID2ListIndex.putIfAbsent(device, deviceIDTimeIndexList.size()); + if (Objects.isNull(index)) { + deviceIDTimeIndexList.add(new Pair<>(device, end)); + return; + } + final Pair pair = deviceIDTimeIndexList.get(index); + initIndex(); + shiftElements(this.index, start, end, pair.right); + final int length = end - start; + for (int j = index; j < deviceIDTimeIndexList.size(); j++) { + final Pair var = deviceIDTimeIndexList.get(j); + var.right = var.right + length; + } + isUnSorted = true; + } + + private static void shiftElements(Integer[] arr, int start, int end, int index) { + final int moveCount = end - start; + Integer[] temp = new Integer[moveCount]; + System.arraycopy(arr, start, temp, 0, moveCount); + for (int j = end - 1, first = index + moveCount; j >= first; j--) { + arr[j] = arr[j - moveCount]; + } + System.arraycopy(temp, 0, arr, index, moveCount); + } + + private void initIndex() { + if (Objects.nonNull(index)) { + return; + } + index = new Integer[tablet.rowSize]; + for (int i = 0, size = tablet.rowSize; i < size; i++) { + index[i] = i; + } + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index c58f8c322c56..6cf013bb3047 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -241,13 +241,12 @@ public void testTableModelSort() { doTableModelTest(false, true); } - public void doTableModelTest(final boolean isDeduplicated, final boolean isUnSorted) { - final Tablet tablet = generateTablet("test", 10, isDeduplicated, isUnSorted); + public void doTableModelTest(final boolean hasDuplicates, final boolean isUnSorted) { + final Tablet tablet = generateTablet("test", 10, hasDuplicates, isUnSorted); List> list = new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); - List> list1 = WriteUtils.splitTabletByDevice(tablet); - Assert.assertEquals(list1, list); + Assert.assertEquals(WriteUtils.splitTabletByDevice(tablet), list); for (int i = 1; i < tablet.rowSize; i++) { long time = tablet.timestamps[i]; Assert.assertTrue(time > tablet.timestamps[i - 1]); @@ -270,7 +269,7 @@ public void doTableModelTest(final boolean isDeduplicated, final boolean isUnSor private Tablet generateTablet( final String tableName, final int deviceIDNum, - final boolean isDeduplicated, + final boolean hasDuplicates, final boolean isUnSorted) { final List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); @@ -324,7 +323,7 @@ private Tablet generateTablet( "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); rowIndex++; tablet.rowSize++; - if (!isDeduplicated) { + if (!hasDuplicates) { break; } } From c869757583e7b43165e79c5cdf618685407015b6 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Wed, 27 Nov 2024 15:26:37 +0800 Subject: [PATCH 17/42] Refactor --- .../batch/PipeTabletEventTsFileBatch.java | 7 +-- .../util/PipeTableModeTsFileBuilder.java | 7 ++- .../util/PipeTreeModelTSFileBuilder.java | 3 +- .../connector/util/PipeTsFileBuilder.java | 3 +- .../connector/PipeTabletEventSorterTest.java | 61 ++++++++++--------- 5 files changed, 43 insertions(+), 38 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index 41677e0b95f8..1320beff3d58 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -27,7 +27,6 @@ import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; -import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.tsfile.exception.write.WriteProcessException; @@ -46,13 +45,11 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; public class PipeTabletEventTsFileBatch extends PipeTabletEventBatch { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTabletEventTsFileBatch.class); - private static final AtomicReference FOLDER_MANAGER = new AtomicReference<>(); private static final AtomicLong BATCH_ID_GENERATOR = new AtomicLong(0); private final AtomicLong currentBatchId = new AtomicLong(BATCH_ID_GENERATOR.incrementAndGet()); @@ -183,11 +180,11 @@ public synchronized List> sealTsFiles() List> list = new ArrayList<>(); if (!treeModeTsFileBuilder.isEmpty()) { - list.addAll(treeModeTsFileBuilder.sealTsFiles()); + list.addAll(treeModeTsFileBuilder.convertTabletToTSFileWithDBInfo()); } if (!tableModeTsFileBuilder.isEmpty()) { - list.addAll(tableModeTsFileBuilder.sealTsFiles()); + list.addAll(tableModeTsFileBuilder.convertTabletToTSFileWithDBInfo()); } return list; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index 4f927bcd0a52..378f1784b4f0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -69,7 +69,7 @@ public void bufferTreeModelTablet(Tablet tablet, Boolean isAligned) { } @Override - public List> sealTsFiles() throws IOException, WriteProcessException { + public List> convertTabletToTSFileWithDBInfo() throws IOException { if (dataBase2TabletList.isEmpty()) { return new ArrayList<>(0); } @@ -141,6 +141,7 @@ private List> writeTableModelTabletsToTsFiles( createFileWriter(); } + // try { tryBestToWriteTabletsIntoOneFile(table2TabletsLinkedList); } catch (final Exception e) { @@ -213,7 +214,7 @@ private void tryBestToWriteTabletsIntoOneFile( final Map deviceLastTimestampMap = new HashMap<>(); while (!tablets.isEmpty()) { final Pair>> pair = tablets.peekFirst(); - if (hasNoTimestampOverlaps(pair, deviceLastTimestampMap)) { + if (timestampsAreNonOverlapping(pair, deviceLastTimestampMap)) { tabletsToWrite.add(pair); tablets.pollFirst(); continue; @@ -253,7 +254,7 @@ private void tryBestToWriteTabletsIntoOneFile( * @return If false, the tablet overlaps with the previous tablet; if true, there is no time * overlap. */ - private boolean hasNoTimestampOverlaps( + private boolean timestampsAreNonOverlapping( final Pair>> tabletPair, final Map deviceLastTimestampMap) { int currentTimestampIndex = 0; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java index 080c31d8868c..04a048aea19a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java @@ -70,7 +70,8 @@ public void bufferTreeModelTablet(final Tablet tablet, final Boolean isAligned) } @Override - public List> sealTsFiles() throws IOException, WriteProcessException { + public List> convertTabletToTSFileWithDBInfo() + throws IOException, WriteProcessException { return writeTabletsToTsFiles(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java index da145e54c0cd..a8ec4525e8ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java @@ -110,7 +110,8 @@ public abstract void bufferTableModelTablet( public abstract void bufferTreeModelTablet(Tablet tablet, Boolean isAligned); - public abstract List> sealTsFiles() throws IOException, WriteProcessException; + public abstract List> convertTabletToTSFileWithDBInfo() + throws IOException, WriteProcessException; public abstract boolean isEmpty(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index 6cf013bb3047..d0bf5777575a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -298,37 +298,42 @@ private Tablet generateTablet( // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text int rowIndex = 0; - for (long row = 0; row < deviceIDNum; row++) { - - for (int i = 0; i < 100; i++) { - final long value; - if (isUnSorted) { - value = (row + 1) * 100 - i - 1; - } else { - value = (row) * 100 + i; - } - for (int j = 0; j < 10; j++) { - tablet.addTimestamp(rowIndex, value); - tablet.addValue( - "s0", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); - tablet.addValue("s1", rowIndex, value); - tablet.addValue("s2", rowIndex, (value * 1.0f)); - tablet.addValue( - "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); - tablet.addValue("s4", rowIndex, value); - tablet.addValue("s5", rowIndex, (int) value); - tablet.addValue("s6", rowIndex, value * 0.1); - tablet.addValue("s7", rowIndex, getDate((int) value)); - tablet.addValue( - "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); - rowIndex++; - tablet.rowSize++; - if (!hasDuplicates) { - break; + boolean hasWrite = false; + do { + + for (long row = 0; row < deviceIDNum; row++) { + for (int i = hasWrite ? 50 : 0; i < (isUnSorted && !hasWrite ? 50 : 100); i++) { + + final long value; + if (isUnSorted) { + value = (row + 1) * 100 - i - 1; + } else { + value = (row) * 100 + i; + } + for (int j = 0; j < 10; j++) { + tablet.addTimestamp(rowIndex, value); + tablet.addValue( + "s0", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s1", rowIndex, value); + tablet.addValue("s2", rowIndex, (value * 1.0f)); + tablet.addValue( + "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s4", rowIndex, value); + tablet.addValue("s5", rowIndex, (int) value); + tablet.addValue("s6", rowIndex, value * 0.1); + tablet.addValue("s7", rowIndex, getDate((int) value)); + tablet.addValue( + "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + rowIndex++; + tablet.rowSize++; + if (!hasDuplicates) { + break; + } } } } - } + hasWrite = !hasWrite; + } while (isUnSorted && hasWrite); return tablet; } From 051c1f8bcec9f43a078f67c237ed957b8fbfefa5 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 3 Dec 2024 15:59:52 +0800 Subject: [PATCH 18/42] fix --- .../batch/PipeTabletEventTsFileBatch.java | 8 ++ .../request/PipeTransferTabletRawReqV2.java | 3 +- .../async/IoTDBDataRegionAsyncConnector.java | 6 +- .../sync/IoTDBDataRegionSyncConnector.java | 4 +- .../util/PipeTableModelTabletEventSorter.java | 102 ++++------------- .../util/PipeTreeModelTabletEventSorter.java | 85 +------------- .../pipe/connector/util/TabletSortUtil.java | 105 ++++++++++++++++++ .../SubscriptionPipeTsFileEventBatch.java | 6 +- 8 files changed, 146 insertions(+), 173 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/TabletSortUtil.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index 1320beff3d58..f099d2772591 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -172,6 +172,14 @@ public Map, Double> deepCopyPipe2WeightMap() { return new HashMap<>(pipeName2WeightMap); } + /** + * Converts a Tablet to a TSFile and returns the generated TSFile along with its corresponding + * database name. + * + * @return a list of pairs containing the database name and the generated TSFile + * @throws IOException if an I/O error occurs during the conversion process + * @throws WriteProcessException if an error occurs during the write process + */ public synchronized List> sealTsFiles() throws IOException, WriteProcessException { if (isClosed) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java index e7e6ee2cbd82..69009b37789e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.IoTDBConnectorRequestVersion; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeRequestType; import org.apache.iotdb.commons.utils.PathUtils; +import org.apache.iotdb.db.pipe.connector.util.PipeTableModelTabletEventSorter; import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.queryengine.plan.parser.StatementGenerator; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; @@ -60,7 +61,7 @@ public InsertTabletStatement constructStatement() { if (Objects.isNull(dataBaseName)) { new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); } else { - new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); } try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 56bcd036b8db..6ea39379c5d7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -178,13 +178,13 @@ private void transferInBatchWithoutCheck( new PipeTransferTabletBatchEventHandler((PipeTabletEventPlainBatch) batch, this)); } else if (batch instanceof PipeTabletEventTsFileBatch) { final PipeTabletEventTsFileBatch tsFileBatch = (PipeTabletEventTsFileBatch) batch; - final List> sealedFiles = tsFileBatch.sealTsFiles(); + final List> dbTsFilePairs = tsFileBatch.sealTsFiles(); final Map, Double> pipe2WeightMap = tsFileBatch.deepCopyPipe2WeightMap(); final List events = tsFileBatch.deepCopyEvents(); - final AtomicInteger eventsReferenceCount = new AtomicInteger(sealedFiles.size()); + final AtomicInteger eventsReferenceCount = new AtomicInteger(dbTsFilePairs.size()); final AtomicBoolean eventsHadBeenAddedToRetryQueue = new AtomicBoolean(false); - for (final Pair sealedFile : sealedFiles) { + for (final Pair sealedFile : dbTsFilePairs) { transfer( new PipeTransferTsFileHandler( this, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index da28bbd46f75..a6f02cbf92ea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -306,10 +306,10 @@ private void doTransfer( private void doTransfer(final PipeTabletEventTsFileBatch batchToTransfer) throws IOException, WriteProcessException { - final List> sealedFiles = batchToTransfer.sealTsFiles(); + final List> dbTsFilePairs = batchToTransfer.sealTsFiles(); final Map, Double> pipe2WeightMap = batchToTransfer.deepCopyPipe2WeightMap(); - for (final Pair tsFile : sealedFiles) { + for (final Pair tsFile : dbTsFilePairs) { doTransfer(pipe2WeightMap, tsFile.right, null, tsFile.left); try { FileUtils.delete(tsFile.right); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index 602ebbae9544..9026eaa00322 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -19,16 +19,11 @@ package org.apache.iotdb.db.pipe.connector.util; -import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.Pair; -import org.apache.tsfile.write.UnSupportedDataTypeException; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; -import java.time.LocalDate; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; @@ -66,13 +61,30 @@ public List> deduplicateAndSortTimestampsIfNecessary() return null; } + // The deviceIDTimeIndexList stores the last displacement of each DeviceID + 1, such as + // [A1,A1,B1,B1,A2,A2,A3], then the deviceIDTimeIndexList is [(A1,2),(B1,4),(A2,6),(A3,7)] deviceIDTimeIndexList = new ArrayList<>(); + + // The deviceIDToIndexMap stores the index of each DeviceID value in the List. HashMap deviceIDToIndexMap = new HashMap<>(); final long[] timestamps = tablet.timestamps; IDeviceID lastDevice = tablet.getDeviceID(0); long previousTimestamp = tablet.timestamps[0]; int lasIndex = 0; + // It is necessary to determine whether reordering and deduplication are required, and after the + // loop is completed, it is necessary to ensure that the displacement of each DeviceID is + // continuous on the Index. For example, the Index array obtained from + // [A1,A2,A3,A4,A2,A2,A4,A4],[0,1,2,3,4,5,6,7] becomes + // [A1,A2,A2,A2,A3,A4,A4,A4],[0,1,4,5,2,3,6,7], + // + // Tablets need to be sorted under the following conditions: 1. The timestamps of the same + // DeviceID are out of order 2. The same DeviceID is not continuous 3. The DeviceID is out of + // order + // + // Tablets need to be deduplicated under the following conditions: + // If Tablets need to be sorted, they must be deduplicated. The same DeviceID with the same time + // needs to be deduplicated for (int i = 1, size = tablet.rowSize; i < size; ++i) { final IDeviceID deviceID = tablet.getDeviceID(i); final long currentTimestamp = timestamps[i]; @@ -189,10 +201,11 @@ private void sortAndDeduplicateValuesAndBitMaps() { final IMeasurementSchema schema = tablet.getSchemas().get(i); if (schema != null) { tablet.values[columnIndex] = - reorderValueList(deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); + TabletSortUtil.reorderValueList( + deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) { tablet.bitMaps[columnIndex] = - reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); + TabletSortUtil.reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); } columnIndex++; } @@ -200,81 +213,6 @@ private void sortAndDeduplicateValuesAndBitMaps() { tablet.rowSize = deduplicatedSize; } - private static Object reorderValueList( - final int deduplicatedSize, - final Object valueList, - final TSDataType dataType, - final Integer[] index) { - switch (dataType) { - case BOOLEAN: - final boolean[] boolValues = (boolean[]) valueList; - final boolean[] deduplicatedBoolValues = new boolean[boolValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedBoolValues[i] = boolValues[index[i]]; - } - return deduplicatedBoolValues; - case INT32: - final int[] intValues = (int[]) valueList; - final int[] deduplicatedIntValues = new int[intValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedIntValues[i] = intValues[index[i]]; - } - return deduplicatedIntValues; - case DATE: - final LocalDate[] dateValues = (LocalDate[]) valueList; - final LocalDate[] deduplicatedDateValues = new LocalDate[dateValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedDateValues[i] = dateValues[index[i]]; - } - return deduplicatedDateValues; - case INT64: - case TIMESTAMP: - final long[] longValues = (long[]) valueList; - final long[] deduplicatedLongValues = new long[longValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedLongValues[i] = longValues[index[i]]; - } - return deduplicatedLongValues; - case FLOAT: - final float[] floatValues = (float[]) valueList; - final float[] deduplicatedFloatValues = new float[floatValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedFloatValues[i] = floatValues[index[i]]; - } - return deduplicatedFloatValues; - case DOUBLE: - final double[] doubleValues = (double[]) valueList; - final double[] deduplicatedDoubleValues = new double[doubleValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedDoubleValues[i] = doubleValues[index[i]]; - } - return deduplicatedDoubleValues; - case TEXT: - case BLOB: - case STRING: - final Binary[] binaryValues = (Binary[]) valueList; - final Binary[] deduplicatedBinaryValues = new Binary[binaryValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedBinaryValues[i] = binaryValues[index[i]]; - } - return deduplicatedBinaryValues; - default: - throw new UnSupportedDataTypeException( - String.format("Data type %s is not supported.", dataType)); - } - } - - private static BitMap reorderBitMap( - final int deduplicatedSize, final BitMap bitMap, final Integer[] index) { - final BitMap deduplicatedBitMap = new BitMap(bitMap.getSize()); - for (int i = 0; i < deduplicatedSize; i++) { - if (bitMap.isMarked(index[i])) { - deduplicatedBitMap.mark(i); - } - } - return deduplicatedBitMap; - } - private void updateDeviceIDIndex( HashMap deviceID2ListIndex, IDeviceID device, int start, int end) { final Integer index = deviceID2ListIndex.putIfAbsent(device, deviceIDTimeIndexList.size()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java index 614e4f032e71..39a2c592cc2b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java @@ -19,14 +19,9 @@ package org.apache.iotdb.db.pipe.connector.util; -import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.utils.BitMap; -import org.apache.tsfile.write.UnSupportedDataTypeException; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; -import java.time.LocalDate; import java.util.Arrays; import java.util.Comparator; @@ -112,88 +107,14 @@ private void sortAndDeduplicateValuesAndBitMaps() { final IMeasurementSchema schema = tablet.getSchemas().get(i); if (schema != null) { tablet.values[columnIndex] = - reorderValueList(deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); + TabletSortUtil.reorderValueList( + deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) { tablet.bitMaps[columnIndex] = - reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); + TabletSortUtil.reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); } columnIndex++; } } } - - private static Object reorderValueList( - final int deduplicatedSize, - final Object valueList, - final TSDataType dataType, - final Integer[] index) { - switch (dataType) { - case BOOLEAN: - final boolean[] boolValues = (boolean[]) valueList; - final boolean[] deduplicatedBoolValues = new boolean[boolValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedBoolValues[i] = boolValues[index[i]]; - } - return deduplicatedBoolValues; - case INT32: - final int[] intValues = (int[]) valueList; - final int[] deduplicatedIntValues = new int[intValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedIntValues[i] = intValues[index[i]]; - } - return deduplicatedIntValues; - case DATE: - final LocalDate[] dateValues = (LocalDate[]) valueList; - final LocalDate[] deduplicatedDateValues = new LocalDate[dateValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedDateValues[i] = dateValues[index[i]]; - } - return deduplicatedDateValues; - case INT64: - case TIMESTAMP: - final long[] longValues = (long[]) valueList; - final long[] deduplicatedLongValues = new long[longValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedLongValues[i] = longValues[index[i]]; - } - return deduplicatedLongValues; - case FLOAT: - final float[] floatValues = (float[]) valueList; - final float[] deduplicatedFloatValues = new float[floatValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedFloatValues[i] = floatValues[index[i]]; - } - return deduplicatedFloatValues; - case DOUBLE: - final double[] doubleValues = (double[]) valueList; - final double[] deduplicatedDoubleValues = new double[doubleValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedDoubleValues[i] = doubleValues[index[i]]; - } - return deduplicatedDoubleValues; - case TEXT: - case BLOB: - case STRING: - final Binary[] binaryValues = (Binary[]) valueList; - final Binary[] deduplicatedBinaryValues = new Binary[binaryValues.length]; - for (int i = 0; i < deduplicatedSize; i++) { - deduplicatedBinaryValues[i] = binaryValues[index[i]]; - } - return deduplicatedBinaryValues; - default: - throw new UnSupportedDataTypeException( - String.format("Data type %s is not supported.", dataType)); - } - } - - private static BitMap reorderBitMap( - final int deduplicatedSize, final BitMap bitMap, final Integer[] index) { - final BitMap deduplicatedBitMap = new BitMap(bitMap.getSize()); - for (int i = 0; i < deduplicatedSize; i++) { - if (bitMap.isMarked(index[i])) { - deduplicatedBitMap.mark(i); - } - } - return deduplicatedBitMap; - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/TabletSortUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/TabletSortUtil.java new file mode 100644 index 000000000000..070fa31339cb --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/TabletSortUtil.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.connector.util; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.write.UnSupportedDataTypeException; + +import java.time.LocalDate; + +public class TabletSortUtil { + + public static Object reorderValueList( + final int deduplicatedSize, + final Object valueList, + final TSDataType dataType, + final Integer[] index) { + switch (dataType) { + case BOOLEAN: + final boolean[] boolValues = (boolean[]) valueList; + final boolean[] deduplicatedBoolValues = new boolean[boolValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedBoolValues[i] = boolValues[index[i]]; + } + return deduplicatedBoolValues; + case INT32: + final int[] intValues = (int[]) valueList; + final int[] deduplicatedIntValues = new int[intValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedIntValues[i] = intValues[index[i]]; + } + return deduplicatedIntValues; + case DATE: + final LocalDate[] dateValues = (LocalDate[]) valueList; + final LocalDate[] deduplicatedDateValues = new LocalDate[dateValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedDateValues[i] = dateValues[index[i]]; + } + return deduplicatedDateValues; + case INT64: + case TIMESTAMP: + final long[] longValues = (long[]) valueList; + final long[] deduplicatedLongValues = new long[longValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedLongValues[i] = longValues[index[i]]; + } + return deduplicatedLongValues; + case FLOAT: + final float[] floatValues = (float[]) valueList; + final float[] deduplicatedFloatValues = new float[floatValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedFloatValues[i] = floatValues[index[i]]; + } + return deduplicatedFloatValues; + case DOUBLE: + final double[] doubleValues = (double[]) valueList; + final double[] deduplicatedDoubleValues = new double[doubleValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedDoubleValues[i] = doubleValues[index[i]]; + } + return deduplicatedDoubleValues; + case TEXT: + case BLOB: + case STRING: + final Binary[] binaryValues = (Binary[]) valueList; + final Binary[] deduplicatedBinaryValues = new Binary[binaryValues.length]; + for (int i = 0; i < deduplicatedSize; i++) { + deduplicatedBinaryValues[i] = binaryValues[index[i]]; + } + return deduplicatedBinaryValues; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataType)); + } + } + + public static BitMap reorderBitMap( + final int deduplicatedSize, final BitMap bitMap, final Integer[] index) { + final BitMap deduplicatedBitMap = new BitMap(bitMap.getSize()); + for (int i = 0; i < deduplicatedSize; i++) { + if (bitMap.isMarked(index[i])) { + deduplicatedBitMap.mark(i); + } + } + return deduplicatedBitMap; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/batch/SubscriptionPipeTsFileEventBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/batch/SubscriptionPipeTsFileEventBatch.java index d397a35cf20c..2781350db5d6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/batch/SubscriptionPipeTsFileEventBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/batch/SubscriptionPipeTsFileEventBatch.java @@ -96,9 +96,9 @@ protected List generateSubscriptionEvents() throws Exception } final List events = new ArrayList<>(); - final List> tsFiles = batch.sealTsFiles(); - final AtomicInteger referenceCount = new AtomicInteger(tsFiles.size()); - for (final Pair tsFile : tsFiles) { + final List> dbTsFilePairs = batch.sealTsFiles(); + final AtomicInteger referenceCount = new AtomicInteger(dbTsFilePairs.size()); + for (final Pair tsFile : dbTsFilePairs) { final SubscriptionCommitContext commitContext = prefetchingQueue.generateSubscriptionCommitContext(); events.add( From 715c4dcbcfbe3cfd7338325a2f8a1d0d048d1101 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 3 Dec 2024 16:53:12 +0800 Subject: [PATCH 19/42] update TSFileBuilder --- .../batch/PipeTabletEventTsFileBatch.java | 4 ++-- .../util/PipeTableModelTabletEventSorter.java | 16 ++++++++-------- .../util/PipeTreeModelTSFileBuilder.java | 4 ++-- .../util/PipeTreeModelTabletEventSorter.java | 16 ++++++++-------- 4 files changed, 20 insertions(+), 20 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index f099d2772591..b17e5fb1f8b5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -77,7 +77,7 @@ protected boolean constructBatch(final TabletInsertionEvent event) { final List tablets = insertNodeTabletInsertionEvent.convertToTablets(); for (int i = 0; i < tablets.size(); ++i) { final Tablet tablet = tablets.get(i); - if (tablet.rowSize == 0) { + if (tablet.getRowSize() == 0) { continue; } // table Model @@ -101,7 +101,7 @@ protected boolean constructBatch(final TabletInsertionEvent event) { final PipeRawTabletInsertionEvent rawTabletInsertionEvent = (PipeRawTabletInsertionEvent) event; final Tablet tablet = rawTabletInsertionEvent.convertToTablet(); - if (tablet.rowSize == 0) { + if (tablet.getRowSize() == 0) { return true; } // table Model diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index 9026eaa00322..e0597cab28c5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -44,7 +44,7 @@ public class PipeTableModelTabletEventSorter { public PipeTableModelTabletEventSorter(final Tablet tablet) { this.tablet = tablet; - deduplicatedSize = tablet == null ? 0 : tablet.rowSize; + deduplicatedSize = tablet == null ? 0 : tablet.getRowSize(); } /** @@ -57,7 +57,7 @@ public PipeTableModelTabletEventSorter(final Tablet tablet) { * last index in the Tablet. */ public List> deduplicateAndSortTimestampsIfNecessary() { - if (tablet == null || tablet.rowSize < 1) { + if (tablet == null || tablet.getRowSize() < 1) { return null; } @@ -85,7 +85,7 @@ public List> deduplicateAndSortTimestampsIfNecessary() // Tablets need to be deduplicated under the following conditions: // If Tablets need to be sorted, they must be deduplicated. The same DeviceID with the same time // needs to be deduplicated - for (int i = 1, size = tablet.rowSize; i < size; ++i) { + for (int i = 1, size = tablet.getRowSize(); i < size; ++i) { final IDeviceID deviceID = tablet.getDeviceID(i); final long currentTimestamp = timestamps[i]; final int deviceComparison = deviceID.compareTo(lastDevice); @@ -108,7 +108,7 @@ public List> deduplicateAndSortTimestampsIfNecessary() lasIndex = i; previousTimestamp = currentTimestamp; } - updateDeviceIDIndex(deviceIDToIndexMap, lastDevice, lasIndex, tablet.rowSize); + updateDeviceIDIndex(deviceIDToIndexMap, lastDevice, lasIndex, tablet.getRowSize()); if (!isUnSorted && !hasDuplicates) { return deviceIDTimeIndexList; @@ -150,7 +150,7 @@ private void sortAndDeduplicateTimestamps() { deviceIndexRange.sort(Comparator.comparing(a -> a.left)); // Deduplication and update Index array - final long[] timestamps = new long[tablet.rowSize]; + final long[] timestamps = new long[tablet.getRowSize()]; final long[] tabletTimestamps = tablet.timestamps; final Integer[] copyIndex = new Integer[index.length]; @@ -210,7 +210,7 @@ private void sortAndDeduplicateValuesAndBitMaps() { columnIndex++; } } - tablet.rowSize = deduplicatedSize; + tablet.setRowSize(deduplicatedSize); } private void updateDeviceIDIndex( @@ -245,8 +245,8 @@ private void initIndex() { if (Objects.nonNull(index)) { return; } - index = new Integer[tablet.rowSize]; - for (int i = 0, size = tablet.rowSize; i < size; i++) { + index = new Integer[tablet.getRowSize()]; + for (int i = 0, size = tablet.getRowSize(); i < size; i++) { index[i] = i; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java index 04a048aea19a..8671abe237c4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java @@ -216,7 +216,7 @@ private void tryBestToWriteTabletsIntoOneFile( final Tablet tablet = tablets.peekFirst(); if (Objects.isNull(lastTablet) // lastTablet.rowSize is not 0 - || lastTablet.timestamps[lastTablet.rowSize - 1] < tablet.timestamps[0]) { + || lastTablet.timestamps[lastTablet.getRowSize() - 1] < tablet.timestamps[0]) { tabletsToWrite.add(tablet); lastTablet = tablet; tablets.pollFirst(); @@ -263,7 +263,7 @@ private void tryBestToWriteTabletsIntoOneFile( } } - fileWriter.write(tablet); + fileWriter.writeTree(tablet); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java index 39a2c592cc2b..63a38c65eac8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java @@ -37,15 +37,15 @@ public class PipeTreeModelTabletEventSorter { public PipeTreeModelTabletEventSorter(final Tablet tablet) { this.tablet = tablet; - deduplicatedSize = tablet == null ? 0 : tablet.rowSize; + deduplicatedSize = tablet == null ? 0 : tablet.getRowSize(); } public void deduplicateAndSortTimestampsIfNecessary() { - if (tablet == null || tablet.rowSize == 0) { + if (tablet == null || tablet.getRowSize() == 0) { return; } - for (int i = 1, size = tablet.rowSize; i < size; ++i) { + for (int i = 1, size = tablet.getRowSize(); i < size; ++i) { final long currentTimestamp = tablet.timestamps[i]; final long previousTimestamp = tablet.timestamps[i - 1]; @@ -62,8 +62,8 @@ public void deduplicateAndSortTimestampsIfNecessary() { return; } - index = new Integer[tablet.rowSize]; - for (int i = 0, size = tablet.rowSize; i < size; i++) { + index = new Integer[tablet.getRowSize()]; + for (int i = 0, size = tablet.getRowSize(); i < size; i++) { index[i] = i; } @@ -85,12 +85,12 @@ public void deduplicateAndSortTimestampsIfNecessary() { private void sortTimestamps() { Arrays.sort(index, Comparator.comparingLong(i -> tablet.timestamps[i])); - Arrays.sort(tablet.timestamps, 0, tablet.rowSize); + Arrays.sort(tablet.timestamps, 0, tablet.getRowSize()); } private void deduplicateTimestamps() { deduplicatedSize = 1; - for (int i = 1, size = tablet.rowSize; i < size; i++) { + for (int i = 1, size = tablet.getRowSize(); i < size; i++) { if (tablet.timestamps[i] != tablet.timestamps[i - 1]) { index[deduplicatedSize] = index[i]; tablet.timestamps[deduplicatedSize] = tablet.timestamps[i]; @@ -98,7 +98,7 @@ private void deduplicateTimestamps() { ++deduplicatedSize; } } - tablet.rowSize = deduplicatedSize; + tablet.setRowSize(deduplicatedSize); } private void sortAndDeduplicateValuesAndBitMaps() { From 207e56d89da2632c26190dd6d097d4684d9feccc Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 3 Dec 2024 17:05:00 +0800 Subject: [PATCH 20/42] update PipeTabletEventSorterTest --- .../connector/PipeTabletEventSorterTest.java | 32 +++++++++++-------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index ad0031af94b8..48610ea362e5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -247,7 +247,7 @@ public void doTableModelTest(final boolean hasDuplicates, final boolean isUnSort List> list = new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); Assert.assertEquals(WriteUtils.splitTabletByDevice(tablet), list); - for (int i = 1; i < tablet.rowSize; i++) { + for (int i = 1; i < tablet.getRowSize(); i++) { long time = tablet.timestamps[i]; Assert.assertTrue(time > tablet.timestamps[i - 1]); Assert.assertEquals( @@ -282,18 +282,24 @@ private Tablet generateTablet( schemaList.add(new MeasurementSchema("s7", TSDataType.DATE)); schemaList.add(new MeasurementSchema("s8", TSDataType.TEXT)); - final List columnTypes = + final List columnTypes = Arrays.asList( - Tablet.ColumnType.ID, - Tablet.ColumnType.MEASUREMENT, - Tablet.ColumnType.MEASUREMENT, - Tablet.ColumnType.MEASUREMENT, - Tablet.ColumnType.MEASUREMENT, - Tablet.ColumnType.MEASUREMENT, - Tablet.ColumnType.MEASUREMENT, - Tablet.ColumnType.MEASUREMENT, - Tablet.ColumnType.MEASUREMENT); - final Tablet tablet = new Tablet(tableName, schemaList, columnTypes, deviceIDNum * 1000); + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT); + Tablet tablet = + new Tablet( + tableName, + IMeasurementSchema.getMeasurementNameList(schemaList), + IMeasurementSchema.getDataTypeList(schemaList), + columnTypes, + deviceIDNum * 1000); tablet.initBitMaps(); // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text @@ -325,7 +331,7 @@ private Tablet generateTablet( tablet.addValue( "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); rowIndex++; - tablet.rowSize++; + tablet.setRowSize(tablet.getRowSize() + 1); if (!hasDuplicates) { break; } From 806f6fba37aa8d6c02f649ec4e4a76cccafcae0a Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 3 Dec 2024 21:19:00 +0800 Subject: [PATCH 21/42] fix --- .../iotdb/db/pipe/connector/PipeTabletEventSorterTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index 48610ea362e5..f20be0c046b9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -331,7 +331,7 @@ private Tablet generateTablet( tablet.addValue( "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); rowIndex++; - tablet.setRowSize(tablet.getRowSize() + 1); + tablet.setRowSize(rowIndex); if (!hasDuplicates) { break; } From bf8ac8290ceb9cc6e454e2c0c7f24ea51e466a0e Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 5 Dec 2024 14:56:50 +0800 Subject: [PATCH 22/42] update OpcUaNameSpace --- .../protocol/opcua/OpcUaNameSpace.java | 43 +++++++++++-------- 1 file changed, 25 insertions(+), 18 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java index 402122970274..f82f6f574a80 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java @@ -21,18 +21,18 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; - import org.apache.iotdb.commons.utils.PathUtils; -import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.pipe.connector.util.PipeTableModelTabletEventSorter; - +import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.utils.DateTimeUtils; import org.apache.iotdb.db.utils.TimestampPrecisionUtils; import org.apache.iotdb.pipe.api.event.Event; import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.UnSupportedDataTypeException; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -121,7 +121,7 @@ private void transferTabletForClientServerModel(final Tablet tablet, final boole final List schemas = tablet.getSchemas(); final List newSchemas = new ArrayList<>(); if (!isTableModel) { - new PipeTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); final List timestamps = new ArrayList<>(); final List values = new ArrayList<>(); @@ -142,15 +142,20 @@ private void transferTabletForClientServerModel(final Tablet tablet, final boole transferTabletRowForClientServerModel( tablet.getDeviceId().split("\\."), newSchemas, timestamps, values); } else { + final List> deviceIndex = + new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); final List columnIndexes = new ArrayList<>(); + for (int i = 0; i < schemas.size(); ++i) { if (tablet.getColumnTypes().get(i) == Tablet.ColumnCategory.MEASUREMENT) { columnIndexes.add(i); newSchemas.add(schemas.get(i)); } } - for (int i = 0; i < tablet.getRowSize(); ++i) { - final Object[] segments = tablet.getDeviceID(i).getSegments(); + + int lastRowIndex = 0; + for (Pair pair : deviceIndex) { + final Object[] segments = pair.left.getSegments(); final String[] folderSegments = new String[segments.length + 2]; folderSegments[0] = "root"; folderSegments[1] = unQualifiedDatabaseName; @@ -159,18 +164,20 @@ private void transferTabletForClientServerModel(final Tablet tablet, final boole folderSegments[j + 2] = Objects.isNull(segments[j]) ? placeHolder : (String) segments[j]; } - - final int finalI = i; - transferTabletRowForClientServerModel( - folderSegments, - newSchemas, - Collections.singletonList(tablet.timestamps[i]), - columnIndexes.stream() - .map( - index -> - getTabletObjectValue4Opc( - tablet.values[index], finalI, schemas.get(index).getType())) - .collect(Collectors.toList())); + for (int j = lastRowIndex; j < pair.right; j++) { + final int finalJ = j; + transferTabletRowForClientServerModel( + folderSegments, + newSchemas, + Collections.singletonList(tablet.timestamps[finalJ]), + columnIndexes.stream() + .map( + index -> + getTabletObjectValue4Opc( + tablet.values[index], finalJ, schemas.get(index).getType())) + .collect(Collectors.toList())); + } + lastRowIndex = pair.right; } } } From f4a7ed5425be1784da087a6fe07df13acec25a63 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 5 Dec 2024 21:52:49 +0800 Subject: [PATCH 23/42] update PipeTableModelTabletEventSorter --- .../payload/evolvable/batch/PipeTabletEventTsFileBatch.java | 2 +- .../payload/evolvable/request/PipeTransferTabletRawReqV2.java | 2 +- .../iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java | 2 +- .../db/pipe/connector/util/PipeTableModelTabletEventSorter.java | 2 +- .../iotdb/db/pipe/connector/PipeTabletEventSorterTest.java | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index b17e5fb1f8b5..8ad75007829c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -148,7 +148,7 @@ private void bufferTreeModelTablet( private void bufferTableModelTablet( final String pipeName, final long creationTime, final Tablet tablet, final String dataBase) { final List> deviceID2Index = - new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); if (deviceID2Index == null) { return; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java index e5d73cc81b52..ffbad5bd9ab0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java @@ -61,7 +61,7 @@ public InsertTabletStatement constructStatement() { if (Objects.isNull(dataBaseName)) { new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); } else { - new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); } try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java index f82f6f574a80..e870f2337d22 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java @@ -143,7 +143,7 @@ private void transferTabletForClientServerModel(final Tablet tablet, final boole tablet.getDeviceId().split("\\."), newSchemas, timestamps, values); } else { final List> deviceIndex = - new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); final List columnIndexes = new ArrayList<>(); for (int i = 0; i < schemas.size(); ++i) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index e0597cab28c5..f3b7d724ed5a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -56,7 +56,7 @@ public PipeTableModelTabletEventSorter(final Tablet tablet) { * @return A list of pairs, each containing an instance of {@link IDeviceID} and the corresponding * last index in the Tablet. */ - public List> deduplicateAndSortTimestampsIfNecessary() { + public List> sortAndDeduplicateByDevIdTimestamp() { if (tablet == null || tablet.getRowSize() < 1) { return null; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index f20be0c046b9..90a3039297a0 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -245,7 +245,7 @@ public void doTableModelTest(final boolean hasDuplicates, final boolean isUnSort final Tablet tablet = generateTablet("test", 10, hasDuplicates, isUnSorted); List> list = - new PipeTableModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); Assert.assertEquals(WriteUtils.splitTabletByDevice(tablet), list); for (int i = 1; i < tablet.getRowSize(); i++) { long time = tablet.timestamps[i]; From 2b6c91143c6e370526a3a6472d11f02b8fab1061 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 10 Dec 2024 21:06:10 +0800 Subject: [PATCH 24/42] fix --- .../batch/PipeTabletEventTsFileBatch.java | 10 +- .../protocol/opcua/OpcUaNameSpace.java | 35 ++- .../util/PipeTableModeTsFileBuilder.java | 108 +++++---- .../util/PipeTableModelTabletEventSorter.java | 206 ++++++------------ .../util/PipeTreeModelTSFileBuilder.java | 5 +- .../connector/util/PipeTsFileBuilder.java | 4 +- .../connector/PipeTabletEventSorterTest.java | 96 ++++---- 7 files changed, 193 insertions(+), 271 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index 8ad75007829c..0ba8f98d6233 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -30,7 +30,6 @@ import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.tsfile.exception.write.WriteProcessException; -import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.record.Tablet; import org.slf4j.Logger; @@ -147,12 +146,7 @@ private void bufferTreeModelTablet( private void bufferTableModelTablet( final String pipeName, final long creationTime, final Tablet tablet, final String dataBase) { - final List> deviceID2Index = - new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); - - if (deviceID2Index == null) { - return; - } + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); totalBufferSize += PipeMemoryWeightUtil.calculateTabletSizeInBytes(tablet); @@ -160,7 +154,7 @@ private void bufferTableModelTablet( new Pair<>(pipeName, creationTime), (pipe, weight) -> Objects.nonNull(weight) ? ++weight : 1); - tableModeTsFileBuilder.bufferTableModelTablet(dataBase, tablet, deviceID2Index); + tableModeTsFileBuilder.bufferTableModelTablet(dataBase, tablet); } public Map, Double> deepCopyPipe2WeightMap() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java index e870f2337d22..3053a09c1b55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java @@ -30,9 +30,7 @@ import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.UnSupportedDataTypeException; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -142,8 +140,7 @@ private void transferTabletForClientServerModel(final Tablet tablet, final boole transferTabletRowForClientServerModel( tablet.getDeviceId().split("\\."), newSchemas, timestamps, values); } else { - final List> deviceIndex = - new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); final List columnIndexes = new ArrayList<>(); for (int i = 0; i < schemas.size(); ++i) { @@ -153,9 +150,8 @@ private void transferTabletForClientServerModel(final Tablet tablet, final boole } } - int lastRowIndex = 0; - for (Pair pair : deviceIndex) { - final Object[] segments = pair.left.getSegments(); + for (int i = 0; i < tablet.getRowSize(); ++i) { + final Object[] segments = tablet.getDeviceID(i).getSegments(); final String[] folderSegments = new String[segments.length + 2]; folderSegments[0] = "root"; folderSegments[1] = unQualifiedDatabaseName; @@ -164,20 +160,17 @@ private void transferTabletForClientServerModel(final Tablet tablet, final boole folderSegments[j + 2] = Objects.isNull(segments[j]) ? placeHolder : (String) segments[j]; } - for (int j = lastRowIndex; j < pair.right; j++) { - final int finalJ = j; - transferTabletRowForClientServerModel( - folderSegments, - newSchemas, - Collections.singletonList(tablet.timestamps[finalJ]), - columnIndexes.stream() - .map( - index -> - getTabletObjectValue4Opc( - tablet.values[index], finalJ, schemas.get(index).getType())) - .collect(Collectors.toList())); - } - lastRowIndex = pair.right; + final int finalI = i; + transferTabletRowForClientServerModel( + folderSegments, + newSchemas, + Collections.singletonList(tablet.timestamps[i]), + columnIndexes.stream() + .map( + index -> + getTabletObjectValue4Opc( + tablet.values[index], finalI, schemas.get(index).getType())) + .collect(Collectors.toList())); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index 378f1784b4f0..fd76cee143f7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -26,6 +26,7 @@ import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.TableSchema; import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.WriteUtils; import org.apache.tsfile.write.record.Tablet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,9 +37,10 @@ import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; -import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.LinkedList; import java.util.List; +import java.util.ListIterator; import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; @@ -48,18 +50,14 @@ public class PipeTableModeTsFileBuilder extends PipeTsFileBuilder { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTableModeTsFileBuilder.class); private final Map> dataBase2TabletList = new HashMap<>(); - private final Map>>> tabletDeviceIdTimeIndex = - new HashMap<>(); public PipeTableModeTsFileBuilder(AtomicLong currentBatchId, AtomicLong tsFileIdGenerator) { super(currentBatchId, tsFileIdGenerator); } @Override - public void bufferTableModelTablet( - String dataBase, Tablet tablet, List> deviceID2Index) { + public void bufferTableModelTablet(String dataBase, Tablet tablet) { dataBase2TabletList.computeIfAbsent(dataBase, db -> new ArrayList<>()).add(tablet); - tabletDeviceIdTimeIndex.computeIfAbsent(dataBase, db -> new ArrayList<>()).add(deviceID2Index); } @Override @@ -75,9 +73,7 @@ public List> convertTabletToTSFileWithDBInfo() throws IOExcep } List> pairList = new ArrayList<>(); for (Map.Entry> entry : dataBase2TabletList.entrySet()) { - pairList.addAll( - writeTableModelTabletsToTsFiles( - entry.getValue(), tabletDeviceIdTimeIndex.get(entry.getKey()), entry.getKey())); + pairList.addAll(writeTableModelTabletsToTsFiles(entry.getValue(), entry.getKey())); } return pairList; } @@ -91,59 +87,52 @@ public boolean isEmpty() { public synchronized void onSuccess() { super.onSuccess(); dataBase2TabletList.clear(); - tabletDeviceIdTimeIndex.clear(); } @Override public synchronized void close() { super.close(); dataBase2TabletList.clear(); - tabletDeviceIdTimeIndex.clear(); } private List> writeTableModelTabletsToTsFiles( - final List tabletList, - final List>> deviceIDPairMap, - final String dataBase) - throws IOException { + final List tabletList, final String dataBase) throws IOException { - final Map>>>> tableName2Tablets = - new HashMap<>(); + final Map> tableName2Tablets = new HashMap<>(); // Sort the tablets by dataBaseName - for (int i = 0; i < tabletList.size(); i++) { - final Tablet tablet = tabletList.get(i); - insertPairs( - tableName2Tablets.computeIfAbsent(tablet.getTableName(), k -> new ArrayList<>()), - new Pair<>(tablet, deviceIDPairMap.get(i))); + for (final Tablet tablet : tabletList) { + tableName2Tablets.computeIfAbsent(tablet.getTableName(), k -> new ArrayList<>()).add(tablet); } - // Sort the devices by tableName - final List tables = new ArrayList<>(tableName2Tablets.keySet()); - tables.sort(Comparator.naturalOrder()); - // Replace ArrayList with LinkedList to improve performance - final LinkedHashMap>>>> - table2TabletsLinkedList = new LinkedHashMap<>(); - for (final String tableName : tables) { - table2TabletsLinkedList.put(tableName, new LinkedList<>(tableName2Tablets.get(tableName))); - } + final LinkedHashSet>>>> table2Tablets = + new LinkedHashSet<>(); + + tableName2Tablets.entrySet().stream() + .sorted(Map.Entry.comparingByKey(Comparator.naturalOrder())) + .forEach( + entry -> { + LinkedList>>> list = new LinkedList<>(); + for (final Tablet tablet : entry.getValue()) { + writerPairToList(list, new Pair<>(tablet, WriteUtils.splitTabletByDevice(tablet))); + } + table2Tablets.add(list); + }); // Help GC - tables.clear(); tableName2Tablets.clear(); final List> sealedFiles = new ArrayList<>(); // Try making the tsfile size as large as possible - while (!table2TabletsLinkedList.isEmpty()) { + while (!table2Tablets.isEmpty()) { if (Objects.isNull(fileWriter)) { createFileWriter(); } - // try { - tryBestToWriteTabletsIntoOneFile(table2TabletsLinkedList); + tryBestToWriteTabletsIntoOneFile(table2Tablets); } catch (final Exception e) { LOGGER.warn( "Batch id = {}: Failed to write tablets into tsfile, because {}", @@ -198,17 +187,14 @@ private List> writeTableModelTabletsToTsFiles( } private void tryBestToWriteTabletsIntoOneFile( - final LinkedHashMap>>>> + final LinkedHashSet>>>> device2TabletsLinkedList) throws IOException { - final Iterator>>>>> - iterator = device2TabletsLinkedList.entrySet().iterator(); + final Iterator>>>> iterator = + device2TabletsLinkedList.iterator(); while (iterator.hasNext()) { - final Map.Entry>>>> entry = - iterator.next(); - final String tableName = entry.getKey(); - final LinkedList>>> tablets = entry.getValue(); + final LinkedList>>> tablets = iterator.next(); final List>>> tabletsToWrite = new ArrayList<>(); final Map deviceLastTimestampMap = new HashMap<>(); @@ -230,7 +216,7 @@ private void tryBestToWriteTabletsIntoOneFile( final Tablet tablet = pair.left; if (schemaNotRegistered) { fileWriter.registerTableSchema( - new TableSchema(tableName, tablet.getSchemas(), tablet.getColumnTypes())); + new TableSchema(tablet.getTableName(), tablet.getSchemas(), tablet.getColumnTypes())); schemaNotRegistered = false; } try { @@ -278,19 +264,20 @@ private boolean timestampsAreNonOverlapping( * IDevice minimum timestamps of a certain Tablet in the List, put the current Tablet in this * position. */ - private void insertPairs( - final List>>> list, + private void writerPairToList( + final LinkedList>>> list, final Pair>> pair) { int lastResult = Integer.MAX_VALUE; if (list.isEmpty()) { list.add(pair); return; } - - for (int i = 0; i < list.size(); i++) { - final int result = comparePairs(list.get(i), pair); + ListIterator>>> iterator = list.listIterator(); + while (iterator.hasNext()) { + final Pair>> pair2 = iterator.next(); + final int result = compareDeviceID(pair2, pair); if (lastResult == 0 && result != 0) { - list.add(i - 1, pair); + iterator.add(pair); return; } lastResult = result; @@ -298,21 +285,30 @@ private void insertPairs( list.add(pair); } - private int comparePairs( - final Pair>> pairA, - final Pair>> pairB) { + /** + * Compares the time differences of the same DeviceID in two device ID lists. If the time of the + * same DeviceID in the second device list is greater than in the first, then a positive number is + * returned; if there is no such DeviceID, then 0 is returned. + * + * @param firstDeviceList The first device ID list and its associated times + * @param secondDeviceList The second device ID list and its associated times + * @return The comparison result + */ + private int compareDeviceID( + final Pair>> firstDeviceList, + final Pair>> secondDeviceList) { int bCount = 0; int aIndex = 0; int bIndex = 0; int aLastTimeIndex = 0; int bLastTimeIndex = 0; - final List> listA = pairA.right; - final List> listB = pairB.right; + final List> listA = firstDeviceList.right; + final List> listB = secondDeviceList.right; while (aIndex < listA.size() && bIndex < listB.size()) { int comparisonResult = listA.get(aIndex).left.compareTo(listB.get(bIndex).left); if (comparisonResult == 0) { - long aTime = pairA.left.timestamps[aLastTimeIndex]; - long bTime = pairB.left.timestamps[bLastTimeIndex]; + long aTime = firstDeviceList.left.timestamps[aLastTimeIndex]; + long bTime = secondDeviceList.left.timestamps[bLastTimeIndex]; if (aTime < bTime) { bCount++; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index f3b7d724ed5a..ea9315498a46 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.connector.util; +import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.record.Tablet; @@ -29,7 +30,7 @@ import java.util.Comparator; import java.util.HashMap; import java.util.List; -import java.util.Objects; +import java.util.Map; public class PipeTableModelTabletEventSorter { @@ -39,8 +40,7 @@ public class PipeTableModelTabletEventSorter { private boolean isUnSorted = false; private boolean hasDuplicates = false; private int deduplicatedSize; - - private List> deviceIDTimeIndexList; + private int initIndexSize; public PipeTableModelTabletEventSorter(final Tablet tablet) { this.tablet = tablet; @@ -53,49 +53,29 @@ public PipeTableModelTabletEventSorter(final Tablet tablet) { * sort by time. Deduplication is to remove the same timestamp in the same {@link IDeviceID}, and * the same timestamp in different {@link IDeviceID} will not be processed. * - * @return A list of pairs, each containing an instance of {@link IDeviceID} and the corresponding - * last index in the Tablet. */ - public List> sortAndDeduplicateByDevIdTimestamp() { + public void sortAndDeduplicateByDevIdTimestamp() { if (tablet == null || tablet.getRowSize() < 1) { - return null; + return; } - // The deviceIDTimeIndexList stores the last displacement of each DeviceID + 1, such as - // [A1,A1,B1,B1,A2,A2,A3], then the deviceIDTimeIndexList is [(A1,2),(B1,4),(A2,6),(A3,7)] - deviceIDTimeIndexList = new ArrayList<>(); - - // The deviceIDToIndexMap stores the index of each DeviceID value in the List. - HashMap deviceIDToIndexMap = new HashMap<>(); + HashMap>> deviceIDToIndexMap = new HashMap<>(); final long[] timestamps = tablet.timestamps; IDeviceID lastDevice = tablet.getDeviceID(0); long previousTimestamp = tablet.timestamps[0]; int lasIndex = 0; - // It is necessary to determine whether reordering and deduplication are required, and after the - // loop is completed, it is necessary to ensure that the displacement of each DeviceID is - // continuous on the Index. For example, the Index array obtained from - // [A1,A2,A3,A4,A2,A2,A4,A4],[0,1,2,3,4,5,6,7] becomes - // [A1,A2,A2,A2,A3,A4,A4,A4],[0,1,4,5,2,3,6,7], - // - // Tablets need to be sorted under the following conditions: 1. The timestamps of the same - // DeviceID are out of order 2. The same DeviceID is not continuous 3. The DeviceID is out of - // order - // - // Tablets need to be deduplicated under the following conditions: - // If Tablets need to be sorted, they must be deduplicated. The same DeviceID with the same time - // needs to be deduplicated for (int i = 1, size = tablet.getRowSize(); i < size; ++i) { final IDeviceID deviceID = tablet.getDeviceID(i); final long currentTimestamp = timestamps[i]; final int deviceComparison = deviceID.compareTo(lastDevice); if (deviceComparison == 0) { - if (previousTimestamp > currentTimestamp) { - isUnSorted = true; - continue; - } if (previousTimestamp == currentTimestamp) { hasDuplicates = true; + continue; + } + if (previousTimestamp > currentTimestamp) { + isUnSorted = true; } previousTimestamp = currentTimestamp; continue; @@ -103,100 +83,81 @@ public List> sortAndDeduplicateByDevIdTimestamp() { if (deviceComparison < 0) { isUnSorted = true; } - updateDeviceIDIndex(deviceIDToIndexMap, lastDevice, lasIndex, i); + + List> list = + deviceIDToIndexMap.computeIfAbsent(lastDevice, k -> new ArrayList<>()); + + if (!list.isEmpty()) { + isUnSorted = true; + } + list.add(new Pair<>(lasIndex, i)); lastDevice = deviceID; lasIndex = i; previousTimestamp = currentTimestamp; } - updateDeviceIDIndex(deviceIDToIndexMap, lastDevice, lasIndex, tablet.getRowSize()); - if (!isUnSorted && !hasDuplicates) { - return deviceIDTimeIndexList; + List> list = + deviceIDToIndexMap.computeIfAbsent(lastDevice, k -> new ArrayList<>()); + if (!list.isEmpty()) { + isUnSorted = true; } + list.add(new Pair<>(lasIndex, tablet.getRowSize())); - initIndex(); - - if (isUnSorted) { - sortAndDeduplicateTimestamps(); - hasDuplicates = false; - isUnSorted = false; + if (!isUnSorted && !hasDuplicates) { + return; } - if (hasDuplicates) { - deduplicateTimestamps(); - hasDuplicates = false; - } + initIndexSize = 0; + deduplicatedSize = 0; + index = new Integer[tablet.getRowSize()]; + deviceIDToIndexMap.entrySet().stream() + .sorted(Map.Entry.comparingByKey()) + .forEach( + entry -> { + final int start = initIndexSize; + int i = initIndexSize; + for (Pair pair : entry.getValue()) { + for (int j = pair.left; j < pair.right; j++) { + index[i++] = j; + } + } + if (isUnSorted) { + sortTimestamps(start, i); + deduplicateTimestamps(start, i); + initIndexSize = i; + return; + } + + if (hasDuplicates) { + deduplicateTimestamps(start, i); + } + initIndexSize = i; + }); sortAndDeduplicateValuesAndBitMaps(); - return deviceIDTimeIndexList; } - // This function sorts the tablets. It sorts the time under each IDeviceID first, then sorts each - // IDevice, and then removes duplicates. - private void sortAndDeduplicateTimestamps() { - // Sorting the time of the same IDevice - int startIndex = 0; - final Comparator comparator = Comparator.comparingLong(i -> tablet.timestamps[i]); - List>> deviceIndexRange = - new ArrayList<>(deviceIDTimeIndexList.size()); - for (Pair pair : deviceIDTimeIndexList) { - Arrays.sort(this.index, startIndex, pair.right, comparator); - deviceIndexRange.add(new Pair<>(pair.left, new Pair<>(startIndex, pair.right - 1))); - startIndex = pair.right; - } - - // Sort IDevices - deviceIDTimeIndexList.clear(); - deviceIndexRange.sort(Comparator.comparing(a -> a.left)); - - // Deduplication and update Index array - final long[] timestamps = new long[tablet.getRowSize()]; - final long[] tabletTimestamps = tablet.timestamps; - final Integer[] copyIndex = new Integer[index.length]; - - deduplicatedSize = 0; - for (Pair> deviceRange : deviceIndexRange) { - startIndex = deviceRange.right.left; - long lastTimestamps = timestamps[deduplicatedSize] = tabletTimestamps[index[startIndex]]; - copyIndex[deduplicatedSize++] = index[startIndex++]; - for (final int end = deviceRange.right.right; startIndex <= end; startIndex++) { - final long curTimestamps = tabletTimestamps[index[startIndex]]; - if (lastTimestamps == curTimestamps) { - continue; - } - lastTimestamps = timestamps[deduplicatedSize] = curTimestamps; - copyIndex[deduplicatedSize++] = index[startIndex]; - } - deviceIDTimeIndexList.add(new Pair<>(deviceRange.left, deduplicatedSize)); - } - index = copyIndex; - tablet.timestamps = timestamps; + private void sortTimestamps(int startIndex, int endIndex) { + Arrays.sort( + this.index, startIndex, endIndex, Comparator.comparingLong(i -> tablet.timestamps[i])); } - private void deduplicateTimestamps() { - int startIndex = 0; - deduplicatedSize = 0; - final long[] timestamps = tablet.timestamps; - for (Pair pair : deviceIDTimeIndexList) { - final int endIndex = pair.right; - long lastTimestamps = timestamps[startIndex]; - timestamps[deduplicatedSize] = lastTimestamps; - index[deduplicatedSize++] = index[startIndex++]; - for (; startIndex < endIndex; startIndex++) { - final long curTimestamp = timestamps[startIndex]; - if (curTimestamp == lastTimestamps) { - continue; - } - index[deduplicatedSize] = index[startIndex]; - timestamps[deduplicatedSize++] = curTimestamp; - lastTimestamps = curTimestamp; + private void deduplicateTimestamps(int startIndex, int endIndex) { + long lastTime = tablet.timestamps[index[startIndex]]; + index[deduplicatedSize++] = index[startIndex]; + for (int i = startIndex + 1; i < endIndex; i++) { + if (lastTime != (lastTime = tablet.timestamps[index[i]])) { + index[deduplicatedSize++] = index[i]; } - pair.right = deduplicatedSize; } } private void sortAndDeduplicateValuesAndBitMaps() { int columnIndex = 0; + tablet.timestamps = + (long[]) + TabletSortUtil.reorderValueList( + deduplicatedSize, tablet.timestamps, TSDataType.TIMESTAMP, index); for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) { final IMeasurementSchema schema = tablet.getSchemas().get(i); if (schema != null) { @@ -210,44 +171,7 @@ private void sortAndDeduplicateValuesAndBitMaps() { columnIndex++; } } - tablet.setRowSize(deduplicatedSize); - } - private void updateDeviceIDIndex( - HashMap deviceID2ListIndex, IDeviceID device, int start, int end) { - final Integer index = deviceID2ListIndex.putIfAbsent(device, deviceIDTimeIndexList.size()); - if (Objects.isNull(index)) { - deviceIDTimeIndexList.add(new Pair<>(device, end)); - return; - } - final Pair pair = deviceIDTimeIndexList.get(index); - initIndex(); - shiftElements(this.index, start, end, pair.right); - final int length = end - start; - for (int j = index; j < deviceIDTimeIndexList.size(); j++) { - final Pair var = deviceIDTimeIndexList.get(j); - var.right = var.right + length; - } - isUnSorted = true; - } - - private static void shiftElements(Integer[] arr, int start, int end, int index) { - final int moveCount = end - start; - Integer[] temp = new Integer[moveCount]; - System.arraycopy(arr, start, temp, 0, moveCount); - for (int j = end - 1, first = index + moveCount; j >= first; j--) { - arr[j] = arr[j - moveCount]; - } - System.arraycopy(temp, 0, arr, index, moveCount); - } - - private void initIndex() { - if (Objects.nonNull(index)) { - return; - } - index = new Integer[tablet.getRowSize()]; - for (int i = 0, size = tablet.getRowSize(); i < size; i++) { - index[i] = i; - } + tablet.setRowSize(deduplicatedSize); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java index 8671abe237c4..5b944ffd213b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java @@ -55,10 +55,7 @@ public PipeTreeModelTSFileBuilder( } @Override - public void bufferTableModelTablet( - final String dataBase, - final Tablet tablet, - final List> deviceID2Index) { + public void bufferTableModelTablet(final String dataBase, final Tablet tablet) { throw new UnsupportedOperationException( "PipeTreeModelTSFileBuilder does not support table model tablet to build TSFile"); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java index a8ec4525e8ce..3c1082fbb409 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java @@ -28,7 +28,6 @@ import org.apache.commons.io.FileUtils; import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.exception.write.WriteProcessException; -import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.TsFileWriter; import org.apache.tsfile.write.record.Tablet; @@ -105,8 +104,7 @@ private File getNextBaseDir() throws DiskSpaceInsufficientException { return baseDir; } - public abstract void bufferTableModelTablet( - String dataBase, Tablet tablet, List> deviceID2Index); + public abstract void bufferTableModelTablet(String dataBase, Tablet tablet); public abstract void bufferTreeModelTablet(Tablet tablet, Boolean isAligned); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index 90a3039297a0..c789e76a49c9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -23,11 +23,8 @@ import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.DateUtils; -import org.apache.tsfile.utils.Pair; -import org.apache.tsfile.utils.WriteUtils; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.MeasurementSchema; @@ -243,10 +240,7 @@ public void testTableModelSort() { public void doTableModelTest(final boolean hasDuplicates, final boolean isUnSorted) { final Tablet tablet = generateTablet("test", 10, hasDuplicates, isUnSorted); - - List> list = - new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); - Assert.assertEquals(WriteUtils.splitTabletByDevice(tablet), list); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); for (int i = 1; i < tablet.getRowSize(); i++) { long time = tablet.timestamps[i]; Assert.assertTrue(time > tablet.timestamps[i - 1]); @@ -304,43 +298,69 @@ private Tablet generateTablet( // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text int rowIndex = 0; - boolean hasWrite = false; - do { - for (long row = 0; row < deviceIDNum; row++) { - for (int i = hasWrite ? 50 : 0; i < (isUnSorted && !hasWrite ? 50 : 100); i++) { + for (long row = 0; row < deviceIDNum; row++) { + for (int i = 0; i < (isUnSorted ? 50 : 100); i++) { - final long value; - if (isUnSorted) { - value = (row + 1) * 100 - i - 1; - } else { - value = (row) * 100 + i; + final long value; + if (isUnSorted) { + value = (row + 1) * 100 - i - 1; + } else { + value = (row) * 100 + i; + } + for (int j = 0; j < 10; j++) { + tablet.addTimestamp(rowIndex, value); + tablet.addValue( + "s0", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s1", rowIndex, value); + tablet.addValue("s2", rowIndex, (value * 1.0f)); + tablet.addValue( + "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s4", rowIndex, value); + tablet.addValue("s5", rowIndex, (int) value); + tablet.addValue("s6", rowIndex, value * 0.1); + tablet.addValue("s7", rowIndex, getDate((int) value)); + tablet.addValue( + "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + rowIndex++; + tablet.setRowSize(rowIndex); + if (!hasDuplicates) { + break; } - for (int j = 0; j < 10; j++) { - tablet.addTimestamp(rowIndex, value); - tablet.addValue( - "s0", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); - tablet.addValue("s1", rowIndex, value); - tablet.addValue("s2", rowIndex, (value * 1.0f)); - tablet.addValue( - "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); - tablet.addValue("s4", rowIndex, value); - tablet.addValue("s5", rowIndex, (int) value); - tablet.addValue("s6", rowIndex, value * 0.1); - tablet.addValue("s7", rowIndex, getDate((int) value)); - tablet.addValue( - "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); - rowIndex++; - tablet.setRowSize(rowIndex); - if (!hasDuplicates) { - break; - } + } + } + } + if (!isUnSorted) { + return tablet; + } + for (long row = 0; row < deviceIDNum; row++) { + for (int i = 50; i < 100; i++) { + + final long value; + value = (row + 1) * 100 - i - 1; + + for (int j = 0; j < 10; j++) { + tablet.addTimestamp(rowIndex, value); + tablet.addValue( + "s0", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s1", rowIndex, value); + tablet.addValue("s2", rowIndex, (value * 1.0f)); + tablet.addValue( + "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s4", rowIndex, value); + tablet.addValue("s5", rowIndex, (int) value); + tablet.addValue("s6", rowIndex, value * 0.1); + tablet.addValue("s7", rowIndex, getDate((int) value)); + tablet.addValue( + "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + rowIndex++; + tablet.setRowSize(rowIndex); + if (!hasDuplicates) { + break; } } } - hasWrite = !hasWrite; - } while (isUnSorted && hasWrite); - + } return tablet; } From 08f4c8ab9d2b74475607887b296bdaf97acc1430 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 10 Dec 2024 22:19:59 +0800 Subject: [PATCH 25/42] modify code format --- .../db/pipe/connector/util/PipeTableModelTabletEventSorter.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index ea9315498a46..aa7f162e4f09 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -52,7 +52,6 @@ public PipeTableModelTabletEventSorter(final Tablet tablet) { * {@link IDeviceID}. For sorting, it is necessary to sort the {@link IDeviceID} first, and then * sort by time. Deduplication is to remove the same timestamp in the same {@link IDeviceID}, and * the same timestamp in different {@link IDeviceID} will not be processed. - * */ public void sortAndDeduplicateByDevIdTimestamp() { if (tablet == null || tablet.getRowSize() < 1) { From a10f4e7ea40dd1a33e92d60110ca780fc0a19706 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sat, 14 Dec 2024 18:59:02 +0800 Subject: [PATCH 26/42] add IT --- .../it/tablemodel/IoTDBPipeDataSinkIT.java | 6 +- .../it/tablemodel/IoTDBPipeNullValueIT.java | 4 +- .../pipe/it/tablemodel/TableModelUtils.java | 151 +++++++++++++++++- 3 files changed, 155 insertions(+), 6 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index a57757ef7c7b..5c5f215f3258 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -217,9 +217,9 @@ private void testSinkFormat(final String format) throws Exception { } TableModelUtils.insertData("test", "test", 150, 200, senderEnv, true); - TableModelUtils.insertDataByTablet("test", "test", 200, 250, senderEnv, true); - TableModelUtils.insertDataByTablet("test", "test", 250, 300, senderEnv, true); - TableModelUtils.insertDataByTablet("test", "test", 300, 350, senderEnv, true); + TableModelUtils.insertTablet("test", "test", 200, 250, senderEnv, true); + TableModelUtils.insertTablet("test", "test", 250, 300, senderEnv, true); + TableModelUtils.insertTablet("test", "test", 300, 350, senderEnv, true); TestUtils.assertDataEventuallyOnEnv( receiverEnv, diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeNullValueIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeNullValueIT.java index c801cd25c622..aae3626ee531 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeNullValueIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeNullValueIT.java @@ -69,7 +69,7 @@ private void testInsertNullValueTemplate( TableModelUtils.createDataBaseAndTable(senderEnv, "test", "test"); if (insertType == InsertType.SESSION_INSERT_TABLET) { - TableModelUtils.insertDataByTablet("test", "test", 0, 200, senderEnv, true); + TableModelUtils.insertTablet("test", "test", 0, 200, senderEnv, true); } else if (insertType == InsertType.SQL_INSERT) { TableModelUtils.insertData("test", "test", 0, 200, senderEnv, true); } @@ -96,7 +96,7 @@ private void testInsertNullValueTemplate( } if (insertType == InsertType.SESSION_INSERT_TABLET) { - TableModelUtils.insertDataByTablet("test", "test", 200, 400, senderEnv, true); + TableModelUtils.insertTablet("test", "test", 200, 400, senderEnv, true); } else if (insertType == InsertType.SQL_INSERT) { TableModelUtils.insertData("test", "test", 200, 400, senderEnv, true); } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index ce33dc3e6a53..f97226fafe36 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -175,7 +175,7 @@ public static boolean insertData( return true; } - public static boolean insertDataByTablet( + public static boolean insertTablet( String dataBaseName, String tableName, int start, @@ -195,6 +195,19 @@ public static boolean insertDataByTablet( } } + public static boolean insertTablet(String dataBaseName, Tablet tablet, BaseEnv baseEnv) { + ITableSessionPool tableSessionPool = baseEnv.getTableSessionPool(1); + try (final ITableSession session = tableSessionPool.getSession()) { + session.executeNonQueryStatement("use " + dataBaseName); + session.insert(tablet); + session.executeNonQueryStatement("flush"); + return true; + } catch (Exception e) { + e.printStackTrace(); + return false; + } + } + public static void deleteData( String dataBaseName, String tableName, int start, int end, BaseEnv baseEnv) { List list = new ArrayList<>(end - start + 1); @@ -426,4 +439,140 @@ public static Tablet generateTablet( return tablet; } + + public static Tablet generateTablet( + String tableName, + int deviceStartIndex, + int deviceEndIndex, + int start, + int end, + boolean allowNullValue) { + List schemaList = new ArrayList<>(); + schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); + schemaList.add(new MeasurementSchema("s2", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s3", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s4", TSDataType.TIMESTAMP)); + schemaList.add(new MeasurementSchema("s5", TSDataType.INT32)); + schemaList.add(new MeasurementSchema("s6", TSDataType.DOUBLE)); + schemaList.add(new MeasurementSchema("s7", TSDataType.DATE)); + schemaList.add(new MeasurementSchema("s8", TSDataType.TEXT)); + + final List columnTypes = + Arrays.asList( + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT); + Tablet tablet = + new Tablet( + tableName, + IMeasurementSchema.getMeasurementNameList(schemaList), + IMeasurementSchema.getDataTypeList(schemaList), + columnTypes, + (deviceEndIndex - deviceStartIndex) * (end - start)); + tablet.initBitMaps(); + final Random random = new Random(); + + for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { + // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text + for (long row = start; row < end - start; row++) { + int randomNumber = allowNullValue ? random.nextInt(9) : 9; + int rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, row); + tablet.addValue( + "s0", + rowIndex, + new Binary(String.valueOf(deviceIndex).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s1", rowIndex, row); + tablet.addValue("s2", rowIndex, (row * 1.0f)); + tablet.addValue( + "s3", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s4", rowIndex, row); + tablet.addValue("s5", rowIndex, (int) row); + tablet.addValue("s6", rowIndex, row * 0.1); + tablet.addValue("s7", rowIndex, getDate((int) row)); + tablet.addValue( + "s8", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + if (randomNumber < 9) { + tablet.addValue("s" + randomNumber, rowIndex, null); + } + } + } + + return tablet; + } + + public static Tablet generateTablet( + String tableName, + int deviceStartIndex, + int deviceEndIndex, + int deviceDataSize, + boolean allowNullValue) { + List schemaList = new ArrayList<>(); + schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); + schemaList.add(new MeasurementSchema("s2", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s3", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s4", TSDataType.TIMESTAMP)); + schemaList.add(new MeasurementSchema("s5", TSDataType.INT32)); + schemaList.add(new MeasurementSchema("s6", TSDataType.DOUBLE)); + schemaList.add(new MeasurementSchema("s7", TSDataType.DATE)); + schemaList.add(new MeasurementSchema("s8", TSDataType.TEXT)); + + final List columnTypes = + Arrays.asList( + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT); + Tablet tablet = + new Tablet( + tableName, + IMeasurementSchema.getMeasurementNameList(schemaList), + IMeasurementSchema.getDataTypeList(schemaList), + columnTypes, + (deviceEndIndex - deviceStartIndex) * deviceDataSize); + tablet.initBitMaps(); + final Random random = new Random(); + + for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { + // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text + for (long row = 0; row < deviceDataSize; row++) { + int randomNumber = allowNullValue ? random.nextInt(9) : 9; + int rowIndex = tablet.getRowSize(); + int value = random.nextInt(); + tablet.addTimestamp(rowIndex, row); + tablet.addValue( + "s0", + rowIndex, + new Binary(String.valueOf(deviceIndex).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s1", rowIndex, value); + tablet.addValue("s2", rowIndex, (value * 1.0f)); + tablet.addValue( + "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s4", rowIndex, value); + tablet.addValue("s5", rowIndex, value); + tablet.addValue("s6", rowIndex, value * 0.1); + tablet.addValue("s7", rowIndex, getDate(value)); + tablet.addValue( + "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + if (randomNumber < 9) { + tablet.addValue("s" + randomNumber, rowIndex, null); + } + } + } + + return tablet; + } } From 527eaa4f1ec48d5ca8fc5f6f4ed362839785b385 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sat, 14 Dec 2024 19:05:19 +0800 Subject: [PATCH 27/42] spotless --- .../apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 5c5f215f3258..666770d5d532 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -183,7 +183,7 @@ private void testSinkFormat(final String format) throws Exception { Arrays.asList("insert into root.vehicle.d0(time, s1) values (2, 1)", "flush"))) { return; } - + TableModelUtils.assertCountData("test", "test", 100, receiverEnv); TestUtils.assertDataEventuallyOnEnv( From b0a112f7126bd23e105a6d8f83e59fb6032c97b9 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sat, 14 Dec 2024 20:18:17 +0800 Subject: [PATCH 28/42] spotless --- .../pipe/it/tablemodel/TableModelUtils.java | 184 +++++++++++------- 1 file changed, 112 insertions(+), 72 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index f97226fafe36..47845a5a4cc5 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -64,7 +64,8 @@ */ public class TableModelUtils { - public static void createDataBaseAndTable(BaseEnv baseEnv, String table, String database) { + public static void createDataBaseAndTable( + final BaseEnv baseEnv, final String table, final String database) { try (Connection connection = baseEnv.getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { statement.execute("create database if not exists " + database); @@ -72,13 +73,13 @@ public static void createDataBaseAndTable(BaseEnv baseEnv, String table, String statement.execute( "CREATE TABLE " + table - + "(s0 string id, s1 int64 measurement, s2 float measurement, s3 string measurement, s4 timestamp measurement, s5 int32 measurement, s6 double measurement, s7 date measurement, s8 text measurement )"); + + "(s0 string id, s1 int64 id, s2 float id, s3 string id, s4 timestamp measurement, s5 int32 measurement, s6 double measurement, s7 date measurement, s8 text measurement )"); } catch (Exception e) { fail(e.getMessage()); } } - public static void createDataBase(BaseEnv baseEnv, String database) { + public static void createDataBase(final BaseEnv baseEnv, final String database) { try (Connection connection = baseEnv.getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { statement.execute("create database if not exists " + database); @@ -88,7 +89,11 @@ public static void createDataBase(BaseEnv baseEnv, String database) { } public static boolean insertData( - String dataBaseName, String tableName, int start, int end, BaseEnv baseEnv) { + final String dataBaseName, + final String tableName, + final int start, + final int end, + final BaseEnv baseEnv) { List list = new ArrayList<>(end - start + 1); for (int i = start; i < end; ++i) { list.add( @@ -105,12 +110,12 @@ public static boolean insertData( } public static boolean insertData( - String dataBaseName, - String tableName, - int start, - int end, - BaseEnv baseEnv, - boolean allowNullValue) { + final String dataBaseName, + final String tableName, + final int start, + final int end, + final BaseEnv baseEnv, + final boolean allowNullValue) { List list = new ArrayList<>(end - start + 1); Object[] values = new Object[9]; Random random = new Random(); @@ -140,7 +145,11 @@ public static boolean insertData( } public static boolean insertDataNotThrowError( - String dataBaseName, String tableName, int start, int end, BaseEnv baseEnv) { + final String dataBaseName, + final String tableName, + final int start, + final int end, + final BaseEnv baseEnv) { List list = new ArrayList<>(end - start + 1); for (int i = start; i < end; ++i) { list.add( @@ -153,12 +162,12 @@ public static boolean insertDataNotThrowError( } public static boolean insertData( - String dataBaseName, - String tableName, - int start, - int end, - BaseEnv baseEnv, - DataNodeWrapper wrapper) { + final String dataBaseName, + final String tableName, + final int start, + final int end, + final BaseEnv baseEnv, + final DataNodeWrapper wrapper) { List list = new ArrayList<>(end - start + 1); for (int i = start; i < end; ++i) { list.add( @@ -176,13 +185,13 @@ public static boolean insertData( } public static boolean insertTablet( - String dataBaseName, - String tableName, - int start, - int end, - BaseEnv baseEnv, - boolean allowNullValue) { - final Tablet tablet = generateTablet(tableName, start, end, allowNullValue); + final String dataBaseName, + final String tableName, + final int start, + final int end, + final BaseEnv baseEnv, + final boolean allowNullValue) { + final Tablet tablet = generateTablet(tableName, start, end, allowNullValue, true); ITableSessionPool tableSessionPool = baseEnv.getTableSessionPool(1); try (final ITableSession session = tableSessionPool.getSession()) { session.executeNonQueryStatement("use " + dataBaseName); @@ -195,7 +204,8 @@ public static boolean insertTablet( } } - public static boolean insertTablet(String dataBaseName, Tablet tablet, BaseEnv baseEnv) { + public static boolean insertTablet( + final String dataBaseName, final Tablet tablet, final BaseEnv baseEnv) { ITableSessionPool tableSessionPool = baseEnv.getTableSessionPool(1); try (final ITableSession session = tableSessionPool.getSession()) { session.executeNonQueryStatement("use " + dataBaseName); @@ -209,7 +219,11 @@ public static boolean insertTablet(String dataBaseName, Tablet tablet, BaseEnv b } public static void deleteData( - String dataBaseName, String tableName, int start, int end, BaseEnv baseEnv) { + final String dataBaseName, + final String tableName, + final int start, + final int end, + final BaseEnv baseEnv) { List list = new ArrayList<>(end - start + 1); list.add( String.format("delete from %s where time >= %s and time <= %s", tableName, start, end)); @@ -219,7 +233,7 @@ public static void deleteData( } } - public static Set generateExpectedResults(int start, int end) { + public static Set generateExpectedResults(final int start, final int end) { Set expectedResSet = new HashSet<>(); for (int i = start; i < end; ++i) { final String time = RpcUtils.formatDatetime("default", "ms", i, ZoneOffset.UTC); @@ -231,7 +245,7 @@ public static Set generateExpectedResults(int start, int end) { return expectedResSet; } - public static Set generateExpectedResults(Tablet tablet) { + public static Set generateExpectedResults(final Tablet tablet) { Set expectedResSet = new HashSet<>(); List schemas = tablet.getSchemas(); for (int i = 0; i < tablet.getRowSize(); i++) { @@ -294,16 +308,20 @@ public static String generateHeaderResults() { return "s0,s3,s2,s1,s4,s5,s6,s7,s8,time,"; } - public static String getQuerySql(String table) { + public static String getQuerySql(final String table) { return "select s0,s3,s2,s1,s4,s5,s6,s7,s8,time from " + table; } - public static String getQueryCountSql(String table) { + public static String getQueryCountSql(final String table) { return "select count(*) from " + table; } public static void assertData( - String database, String table, int start, int end, BaseEnv baseEnv) { + final String database, + final String table, + final int start, + final int end, + final BaseEnv baseEnv) { TestUtils.assertDataEventuallyOnEnv( baseEnv, TableModelUtils.getQuerySql(table), @@ -313,12 +331,12 @@ public static void assertData( } public static void assertData( - String database, - String table, - int start, - int end, - BaseEnv baseEnv, - Consumer handleFailure) { + final String database, + final String table, + final int start, + final int end, + final BaseEnv baseEnv, + final Consumer handleFailure) { TestUtils.assertDataEventuallyOnEnv( baseEnv, TableModelUtils.getQuerySql(table), @@ -328,7 +346,8 @@ public static void assertData( handleFailure); } - public static void assertData(String database, String table, Tablet tablet, BaseEnv baseEnv) { + public static void assertData( + final String database, final String table, final Tablet tablet, final BaseEnv baseEnv) { TestUtils.assertDataEventuallyOnEnv( baseEnv, TableModelUtils.getQuerySql(table), @@ -337,18 +356,23 @@ public static void assertData(String database, String table, Tablet tablet, Base database); } - public static boolean hasDataBase(String database, BaseEnv baseEnv) { + public static boolean hasDataBase(final String database, final BaseEnv baseEnv) { TestUtils.assertDataEventuallyOnEnv(baseEnv, "", "", Collections.emptySet(), database); return true; } - public static void assertCountData(String database, String table, int count, BaseEnv baseEnv) { + public static void assertCountData( + final String database, final String table, final int count, final BaseEnv baseEnv) { TestUtils.assertDataEventuallyOnEnv( baseEnv, getQueryCountSql(table), "_col0,", Collections.singleton(count + ","), database); } public static void assertCountData( - String database, String table, int count, BaseEnv baseEnv, Consumer handleFailure) { + final String database, + final String table, + final int count, + final BaseEnv baseEnv, + final Consumer handleFailure) { TestUtils.executeNonQueryWithRetry(baseEnv, "flush"); TestUtils.assertDataEventuallyOnEnv( baseEnv, @@ -359,7 +383,7 @@ public static void assertCountData( handleFailure); } - public static String getDateStr(int value) { + public static String getDateStr(final int value) { Date date = new Date(value); SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); try { @@ -369,7 +393,7 @@ public static String getDateStr(int value) { } } - public static LocalDate getDate(int value) { + public static LocalDate getDate(final int value) { Date date = new Date(value); SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); try { @@ -381,7 +405,11 @@ public static LocalDate getDate(int value) { } public static Tablet generateTablet( - String tableName, int start, int end, boolean allowNullValue) { + final String tableName, + final int start, + final int end, + final boolean allowNullValue, + final boolean allowNullDeviceColumn) { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); @@ -396,9 +424,9 @@ public static Tablet generateTablet( final List columnTypes = Arrays.asList( Tablet.ColumnCategory.ID, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, @@ -413,6 +441,7 @@ public static Tablet generateTablet( end - start); tablet.initBitMaps(); Random random = new Random(); + int nullDeviceIndex = allowNullDeviceColumn ? random.nextInt(4) : 4; // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text for (long row = 0; row < end - start; row++) { @@ -421,7 +450,7 @@ public static Tablet generateTablet( int rowIndex = tablet.getRowSize(); tablet.addTimestamp(rowIndex, value); tablet.addValue( - "s0", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + "s0", rowIndex, new Binary(String.format("t%s", value).getBytes(StandardCharsets.UTF_8))); tablet.addValue("s1", rowIndex, value); tablet.addValue("s2", rowIndex, (value * 1.0f)); tablet.addValue( @@ -435,18 +464,22 @@ public static Tablet generateTablet( if (randomNumber < 9) { tablet.addValue("s" + randomNumber, rowIndex, null); } + if (nullDeviceIndex < 4) { + tablet.addValue("s" + nullDeviceIndex, rowIndex, null); + } } return tablet; } public static Tablet generateTablet( - String tableName, - int deviceStartIndex, - int deviceEndIndex, - int start, - int end, - boolean allowNullValue) { + final String tableName, + final int deviceStartIndex, + final int deviceEndIndex, + final int start, + final int end, + final boolean allowNullValue, + final boolean allowNullDeviceColumn) { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); @@ -461,9 +494,9 @@ public static Tablet generateTablet( final List columnTypes = Arrays.asList( Tablet.ColumnCategory.ID, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, @@ -478,17 +511,16 @@ public static Tablet generateTablet( (deviceEndIndex - deviceStartIndex) * (end - start)); tablet.initBitMaps(); final Random random = new Random(); + int nullDeviceIndex = allowNullDeviceColumn ? random.nextInt(4) : 4; for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text - for (long row = start; row < end - start; row++) { + for (long row = start; row < end; row++) { int randomNumber = allowNullValue ? random.nextInt(9) : 9; int rowIndex = tablet.getRowSize(); tablet.addTimestamp(rowIndex, row); tablet.addValue( - "s0", - rowIndex, - new Binary(String.valueOf(deviceIndex).getBytes(StandardCharsets.UTF_8))); + "s0", rowIndex, new Binary(String.format("t%s", row).getBytes(StandardCharsets.UTF_8))); tablet.addValue("s1", rowIndex, row); tablet.addValue("s2", rowIndex, (row * 1.0f)); tablet.addValue( @@ -502,6 +534,9 @@ public static Tablet generateTablet( if (randomNumber < 9) { tablet.addValue("s" + randomNumber, rowIndex, null); } + if (nullDeviceIndex < 4) { + tablet.addValue("s" + nullDeviceIndex, rowIndex, null); + } } } @@ -509,11 +544,12 @@ public static Tablet generateTablet( } public static Tablet generateTablet( - String tableName, - int deviceStartIndex, - int deviceEndIndex, - int deviceDataSize, - boolean allowNullValue) { + final String tableName, + final int deviceStartIndex, + final int deviceEndIndex, + final int deviceDataSize, + final boolean allowNullValue, + final boolean allowNullDeviceColumn) { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); @@ -528,9 +564,9 @@ public static Tablet generateTablet( final List columnTypes = Arrays.asList( Tablet.ColumnCategory.ID, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, @@ -545,6 +581,7 @@ public static Tablet generateTablet( (deviceEndIndex - deviceStartIndex) * deviceDataSize); tablet.initBitMaps(); final Random random = new Random(); + int nullDeviceIndex = allowNullDeviceColumn ? random.nextInt(4) : 4; for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text @@ -556,20 +593,23 @@ public static Tablet generateTablet( tablet.addValue( "s0", rowIndex, - new Binary(String.valueOf(deviceIndex).getBytes(StandardCharsets.UTF_8))); + new Binary(String.format("t%s", value).getBytes(StandardCharsets.UTF_8))); tablet.addValue("s1", rowIndex, value); tablet.addValue("s2", rowIndex, (value * 1.0f)); tablet.addValue( "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); tablet.addValue("s4", rowIndex, value); - tablet.addValue("s5", rowIndex, value); + tablet.addValue("s5", rowIndex, (int) value); tablet.addValue("s6", rowIndex, value * 0.1); - tablet.addValue("s7", rowIndex, getDate(value)); + tablet.addValue("s7", rowIndex, getDate((int) value)); tablet.addValue( "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); if (randomNumber < 9) { tablet.addValue("s" + randomNumber, rowIndex, null); } + if (nullDeviceIndex < 4) { + tablet.addValue("s" + nullDeviceIndex, rowIndex, null); + } } } From 1051ce433f7d0d2d8b437687df78fc859e9ca670 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 16 Dec 2024 00:03:37 +0800 Subject: [PATCH 29/42] fix IT --- .../it/tablemodel/IoTDBPipeDataSinkIT.java | 254 +++++++++++++++++ .../pipe/it/tablemodel/TableModelUtils.java | 258 ++++++++++++------ 2 files changed, 422 insertions(+), 90 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 666770d5d532..0b760ffd6187 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -28,16 +28,21 @@ import org.apache.iotdb.itbase.category.MultiClusterIT2TableModel; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.tsfile.write.record.Tablet; import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Random; +import java.util.Set; import java.util.function.Consumer; @RunWith(IoTDBTestRunner.class) @@ -296,4 +301,253 @@ public void testWriteBackSink() throws Exception { TableModelUtils.assertCountData("test1", "test", 100, receiverEnv, handleFailure); } } + + @Test + public void testSinkTsFileFormat2() throws Exception { + final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); + + final String receiverIp = receiverDataNode.getIp(); + final int receiverPort = receiverDataNode.getPort(); + final Consumer handleFailure = + o -> { + TestUtils.executeNonQueryWithRetry(senderEnv, "flush"); + TestUtils.executeNonQueryWithRetry(receiverEnv, "flush"); + }; + + Map> testResult = new HashMap<>(); + Map> test1Result = new HashMap<>(); + + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + + for (int i = 0; i < 5; i++) { + TableModelUtils.createDataBaseAndTable(senderEnv, "test" + i, "test0"); + TableModelUtils.createDataBaseAndTable(senderEnv, "test" + i, "test1"); + } + + if (!TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList("insert into root.vehicle.d0(time, s1) values (1, 1)", "flush"))) { + return; + } + + final Map extractorAttributes = new HashMap<>(); + final Map processorAttributes = new HashMap<>(); + final Map connectorAttributes = new HashMap<>(); + + extractorAttributes.put("capture.table", "true"); + extractorAttributes.put("capture.tree", "true"); + extractorAttributes.put("extractor.database-name", "test.*"); + extractorAttributes.put("extractor.table-name", "test.*"); + + connectorAttributes.put("connector", "iotdb-thrift-connector"); + connectorAttributes.put("connector.ip", receiverIp); + connectorAttributes.put("connector.port", Integer.toString(receiverPort)); + connectorAttributes.put("connector.format", "tsfile"); + connectorAttributes.put("connector.realtime-first", "true"); + + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), + client + .createPipe( + new TCreatePipeReq("testPipe", connectorAttributes) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes)) + .getCode()); + + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("testPipe").getCode()); + + if (!TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList("insert into root.vehicle.d0(time, s1) values (2, 1)", "flush"))) { + return; + } + + insertTablet(testResult, test1Result); + + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select * from root.**", + "Time,root.vehicle.d0.s1,", + Collections.unmodifiableSet(new HashSet<>(Arrays.asList("1,1.0,", "2,1.0,"))), + handleFailure); + + if (!TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList( + "insert into root.vehicle.d0(time, s1) values (4, 1)", + "insert into root.vehicle.d0(time, s1) values (3, 1), (0, 1)", + "flush"))) { + return; + } + + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select * from root.**", + "Time,root.vehicle.d0.s1,", + Collections.unmodifiableSet( + new HashSet<>(Arrays.asList("0,1.0,", "1,1.0,", "2,1.0,", "3,1.0,", "4,1.0,"))), + handleFailure); + } + + for (Map.Entry> entry : testResult.entrySet()) { + final Set set = new HashSet<>(); + entry + .getValue() + .forEach( + tablet -> { + set.addAll(TableModelUtils.generateExpectedResults(tablet)); + }); + TableModelUtils.assertData("test0", entry.getKey(), set, receiverEnv, handleFailure); + } + + for (Map.Entry> entry : test1Result.entrySet()) { + final Set set = new HashSet<>(); + entry + .getValue() + .forEach( + tablet -> { + set.addAll(TableModelUtils.generateExpectedResults(tablet)); + }); + TableModelUtils.assertData("test1", entry.getKey(), set, receiverEnv, handleFailure); + } + } + + private void insertTablet( + final Map> testResult, final Map> test1Result) { + + final Random random = new Random(); + int deviceIDStartIndex = 0; + int deviceIDEndIndex = 100; + + for (int i = 0; i < 10; i++) { + final String tableName = "test" + i; + for (int j = 0; j < 10; j++) { + final String dataBaseName = "test" + j % 2; + Tablet tablet = + TableModelUtils.generateTablet( + tableName, deviceIDStartIndex, deviceIDEndIndex, 0, 10, false, true); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + deviceIDStartIndex += 25; + deviceIDEndIndex += 25; + } + } + + deviceIDStartIndex = 0; + deviceIDEndIndex = 100; + + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; + for (int j = 0; j < 10; j++) { + final String dataBaseName = "test" + j % 2; + Tablet tablet = + TableModelUtils.generateTablet( + tableName, deviceIDStartIndex, deviceIDEndIndex, 10, false, true); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + deviceIDStartIndex += 25; + deviceIDEndIndex += 25; + } + } + + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; + for (int j = 0; j < 10; j++) { + final String dataBaseName = "test" + j % 2; + deviceIDStartIndex = random.nextInt(1 << 16) - 25; + deviceIDEndIndex = deviceIDStartIndex + 25; + Tablet tablet = + TableModelUtils.generateTablet( + tableName, + deviceIDStartIndex, + deviceIDEndIndex, + deviceIDStartIndex, + deviceIDEndIndex, + false, + true); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + } + } + + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; + for (int j = 0; j < 10; j++) { + final String dataBaseName = "test" + j % 2; + deviceIDStartIndex = random.nextInt(1 << 16) - 25; + deviceIDEndIndex = deviceIDStartIndex + 25; + Tablet tablet = + TableModelUtils.generateTablet( + tableName, + deviceIDStartIndex, + deviceIDEndIndex, + deviceIDStartIndex, + deviceIDEndIndex, + false, + true); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + } + } + + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; + for (int j = 0; j < 10; j++) { + final String dataBaseName = "test" + j % 2; + deviceIDStartIndex = random.nextInt(1 << 16) - 25; + deviceIDEndIndex = deviceIDStartIndex + 25; + Tablet tablet = + TableModelUtils.generateTablet( + tableName, + deviceIDStartIndex, + deviceIDEndIndex, + deviceIDStartIndex, + deviceIDEndIndex, + false, + true); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + } + } + + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; + for (int j = 0; j < 10; j++) { + final String dataBaseName = "test" + j % 2; + deviceIDStartIndex = random.nextInt(1 << 16) - 25; + deviceIDEndIndex = deviceIDStartIndex + 25; + Tablet tablet = + TableModelUtils.generateTablet( + tableName, deviceIDStartIndex, deviceIDEndIndex, 100, 110, false, true); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + } + } + + deviceIDStartIndex = random.nextInt(1 << 16) - 300; + deviceIDEndIndex = deviceIDStartIndex + 25; + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; + for (int j = 0; j < 10; j++) { + final String dataBaseName = "test" + j % 2; + Tablet tablet = + TableModelUtils.generateTablet( + tableName, deviceIDStartIndex, deviceIDEndIndex, 100, 110, false, true); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + deviceIDStartIndex += 25; + deviceIDEndIndex += 25; + } + } + } } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index 47845a5a4cc5..824446345eb6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -71,9 +71,9 @@ public static void createDataBaseAndTable( statement.execute("create database if not exists " + database); statement.execute("use " + database); statement.execute( - "CREATE TABLE " - + table - + "(s0 string id, s1 int64 id, s2 float id, s3 string id, s4 timestamp measurement, s5 int32 measurement, s6 double measurement, s7 date measurement, s8 text measurement )"); + String.format( + "CREATE TABLE %s(s0 string id, s1 string id, s2 string id, s3 string id,s4 int64 measurement, s5 float measurement, s6 string measurement, s7 timestamp measurement, s8 int32 measurement, s9 double measurement, s10 date measurement, s11 text measurement )", + table)); } catch (Exception e) { fail(e.getMessage()); } @@ -98,15 +98,12 @@ public static boolean insertData( for (int i = start; i < end; ++i) { list.add( String.format( - "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, time) values ('t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", - tableName, i, i, i, i, i, i, i, getDateStr(i), i, i)); + "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11 time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", + tableName, i, i, i, i, i, i, i, i, i, i, getDateStr(i), i, i)); } list.add("flush"); - if (!TestUtils.tryExecuteNonQueriesWithRetry( - dataBaseName, BaseEnv.TABLE_SQL_DIALECT, baseEnv, list)) { - return false; - } - return true; + return TestUtils.tryExecuteNonQueriesWithRetry( + dataBaseName, BaseEnv.TABLE_SQL_DIALECT, baseEnv, list); } public static boolean insertData( @@ -117,31 +114,47 @@ public static boolean insertData( final BaseEnv baseEnv, final boolean allowNullValue) { List list = new ArrayList<>(end - start + 1); - Object[] values = new Object[9]; + Object[] values = new Object[12]; Random random = new Random(); - // s0 string, s1 int64, s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text + // s0 string, s1 int64, s2 float, s3 string, s4 int64, s5 float, s6 string s7 timestamp, s8 + // int32, s9 double, s10 date, s11 text for (int i = start; i < end; ++i) { Arrays.fill(values, i); values[0] = String.format("'t%s'", i); - values[2] = String.format("%s.0", i); - values[3] = String.format("%s", i); - values[6] = String.format("%s.0", i); - values[7] = String.format("'%s'", getDateStr(i)); - values[8] = String.format("'%s'", i); + values[1] = String.format("'t%s'", i); + values[2] = String.format("'t%s'", i); + values[3] = String.format("'t%s'", i); + values[4] = String.format("%s", i); + values[5] = String.format("%s.0", i); + values[6] = String.format("%s", i); + values[7] = String.format("%s", i); + values[8] = String.format("%s", i); + values[9] = String.format("%s.0", i); + values[10] = String.format("'%s'", getDateStr(i)); + values[11] = String.format("'%s'", i); if (allowNullValue) { values[random.nextInt(9)] = "null"; } list.add( String.format( - "insert into %s (s0, s1, s2, s3, s4, s5, s6, s7, s8, time) values (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s)", - tableName, values[0], values[1], values[2], values[3], values[4], values[5], - values[6], values[7], values[8], i)); + "insert into %s (s0, s1, s2, s3, s4, s5, s6, s7, s8,s9, s10, s11, time) values (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s)", + tableName, + values[0], + values[1], + values[2], + values[3], + values[4], + values[5], + values[6], + values[7], + values[8], + values[9], + values[10], + values[11], + i)); } - if (!TestUtils.tryExecuteNonQueriesWithRetry( - dataBaseName, BaseEnv.TABLE_SQL_DIALECT, baseEnv, list)) { - return false; - } - return true; + return TestUtils.tryExecuteNonQueriesWithRetry( + dataBaseName, BaseEnv.TABLE_SQL_DIALECT, baseEnv, list); } public static boolean insertDataNotThrowError( @@ -154,8 +167,8 @@ public static boolean insertDataNotThrowError( for (int i = start; i < end; ++i) { list.add( String.format( - "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, time) values ('t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", - tableName, i, i, i, i, i, i, i, getDateStr(i), i, i)); + "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11 time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", + tableName, i, i, i, i, i, i, i, i, i, i, getDateStr(i), i, i)); } return TestUtils.tryExecuteNonQueriesWithRetry( dataBaseName, BaseEnv.TABLE_SQL_DIALECT, baseEnv, list); @@ -172,16 +185,12 @@ public static boolean insertData( for (int i = start; i < end; ++i) { list.add( String.format( - "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, time) values ('t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", - tableName, i, i, i, i, i, i, i, getDateStr(i), i, i)); + "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11 time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", + tableName, i, i, i, i, i, i, i, i, i, i, getDateStr(i), i, i)); } list.add("flush"); - if (!TestUtils.tryExecuteNonQueriesOnSpecifiedDataNodeWithRetry( - baseEnv, wrapper, list, dataBaseName, BaseEnv.TABLE_SQL_DIALECT)) { - return false; - } - - return true; + return TestUtils.tryExecuteNonQueriesOnSpecifiedDataNodeWithRetry( + baseEnv, wrapper, list, dataBaseName, BaseEnv.TABLE_SQL_DIALECT); } public static boolean insertTablet( @@ -239,8 +248,8 @@ public static Set generateExpectedResults(final int start, final int end final String time = RpcUtils.formatDatetime("default", "ms", i, ZoneOffset.UTC); expectedResSet.add( String.format( - "t%d,%d,%d.0,%d,%s,%d,%d.0,%s,%s,%s,", - i, i, i, i, time, i, i, getDateStr(i), i, time)); + "t%d,t%d,t%d,t%d,%d,%d.0,%d,%s,%d,%d.0,%s,%s,%s,", + i, i, i, i, i, i, i, time, i, i, getDateStr(i), i, time)); } return expectedResSet; } @@ -287,12 +296,15 @@ public static Set generateExpectedResults(final Tablet tablet) { case FLOAT: stringBuffer.append(((float[]) tablet.values[j])[i]); stringBuffer.append(","); + break; case INT32: stringBuffer.append(((int[]) tablet.values[j])[i]); stringBuffer.append(","); + break; case INT64: stringBuffer.append(((long[]) tablet.values[j])[i]); stringBuffer.append(","); + break; } } String time = RpcUtils.formatDatetime("default", "ms", tablet.timestamps[i], ZoneOffset.UTC); @@ -330,6 +342,21 @@ public static void assertData( database); } + public static void assertData( + final String database, + final String table, + final Set expectedResults, + final BaseEnv baseEnv, + final Consumer handleFailure) { + TestUtils.assertDataEventuallyOnEnv( + baseEnv, + TableModelUtils.getQuerySql(table), + TableModelUtils.generateHeaderResults(), + expectedResults, + database, + handleFailure); + } + public static void assertData( final String database, final String table, @@ -412,14 +439,17 @@ public static Tablet generateTablet( final boolean allowNullDeviceColumn) { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); - schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); - schemaList.add(new MeasurementSchema("s2", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s1", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s2", TSDataType.STRING)); schemaList.add(new MeasurementSchema("s3", TSDataType.STRING)); - schemaList.add(new MeasurementSchema("s4", TSDataType.TIMESTAMP)); - schemaList.add(new MeasurementSchema("s5", TSDataType.INT32)); - schemaList.add(new MeasurementSchema("s6", TSDataType.DOUBLE)); - schemaList.add(new MeasurementSchema("s7", TSDataType.DATE)); - schemaList.add(new MeasurementSchema("s8", TSDataType.TEXT)); + schemaList.add(new MeasurementSchema("s4", TSDataType.INT64)); + schemaList.add(new MeasurementSchema("s5", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s6", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s7", TSDataType.TIMESTAMP)); + schemaList.add(new MeasurementSchema("s8", TSDataType.INT32)); + schemaList.add(new MeasurementSchema("s9", TSDataType.DOUBLE)); + schemaList.add(new MeasurementSchema("s10", TSDataType.DATE)); + schemaList.add(new MeasurementSchema("s11", TSDataType.TEXT)); final List columnTypes = Arrays.asList( @@ -431,6 +461,9 @@ public static Tablet generateTablet( Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT); Tablet tablet = new Tablet( @@ -443,30 +476,36 @@ public static Tablet generateTablet( Random random = new Random(); int nullDeviceIndex = allowNullDeviceColumn ? random.nextInt(4) : 4; - // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text for (long row = 0; row < end - start; row++) { - int randomNumber = allowNullValue ? random.nextInt(9) : 9; + int randomNumber = allowNullValue ? random.nextInt(12) : 12; long value = start + row; int rowIndex = tablet.getRowSize(); tablet.addTimestamp(rowIndex, value); tablet.addValue( "s0", rowIndex, new Binary(String.format("t%s", value).getBytes(StandardCharsets.UTF_8))); - tablet.addValue("s1", rowIndex, value); - tablet.addValue("s2", rowIndex, (value * 1.0f)); tablet.addValue( - "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + "s1", rowIndex, new Binary(String.format("t%s", value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue( + "s2", rowIndex, new Binary(String.format("t%s", value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue( + "s3", rowIndex, new Binary(String.format("t%s", value).getBytes(StandardCharsets.UTF_8))); tablet.addValue("s4", rowIndex, value); - tablet.addValue("s5", rowIndex, (int) value); - tablet.addValue("s6", rowIndex, value * 0.1); - tablet.addValue("s7", rowIndex, getDate((int) value)); + tablet.addValue("s5", rowIndex, (value * 1.0f)); + tablet.addValue( + "s6", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s7", rowIndex, value); + tablet.addValue("s8", rowIndex, (int) value); + tablet.addValue("s9", rowIndex, value * 0.1); + tablet.addValue("s10", rowIndex, getDate((int) value)); tablet.addValue( - "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); - if (randomNumber < 9) { + "s11", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + if (randomNumber < 11) { tablet.addValue("s" + randomNumber, rowIndex, null); } if (nullDeviceIndex < 4) { tablet.addValue("s" + nullDeviceIndex, rowIndex, null); } + tablet.setRowSize(rowIndex + 1); } return tablet; @@ -482,14 +521,17 @@ public static Tablet generateTablet( final boolean allowNullDeviceColumn) { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); - schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); - schemaList.add(new MeasurementSchema("s2", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s1", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s2", TSDataType.STRING)); schemaList.add(new MeasurementSchema("s3", TSDataType.STRING)); - schemaList.add(new MeasurementSchema("s4", TSDataType.TIMESTAMP)); - schemaList.add(new MeasurementSchema("s5", TSDataType.INT32)); - schemaList.add(new MeasurementSchema("s6", TSDataType.DOUBLE)); - schemaList.add(new MeasurementSchema("s7", TSDataType.DATE)); - schemaList.add(new MeasurementSchema("s8", TSDataType.TEXT)); + schemaList.add(new MeasurementSchema("s4", TSDataType.INT64)); + schemaList.add(new MeasurementSchema("s5", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s6", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s7", TSDataType.TIMESTAMP)); + schemaList.add(new MeasurementSchema("s8", TSDataType.INT32)); + schemaList.add(new MeasurementSchema("s9", TSDataType.DOUBLE)); + schemaList.add(new MeasurementSchema("s10", TSDataType.DATE)); + schemaList.add(new MeasurementSchema("s11", TSDataType.TEXT)); final List columnTypes = Arrays.asList( @@ -501,6 +543,9 @@ public static Tablet generateTablet( Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT); Tablet tablet = new Tablet( @@ -514,29 +559,43 @@ public static Tablet generateTablet( int nullDeviceIndex = allowNullDeviceColumn ? random.nextInt(4) : 4; for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { - // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text for (long row = start; row < end; row++) { - int randomNumber = allowNullValue ? random.nextInt(9) : 9; + int randomNumber = allowNullValue ? random.nextInt(12) : 12; int rowIndex = tablet.getRowSize(); tablet.addTimestamp(rowIndex, row); tablet.addValue( - "s0", rowIndex, new Binary(String.format("t%s", row).getBytes(StandardCharsets.UTF_8))); - tablet.addValue("s1", rowIndex, row); - tablet.addValue("s2", rowIndex, (row * 1.0f)); + "s0", + rowIndex, + new Binary(String.format("t%s", deviceIndex).getBytes(StandardCharsets.UTF_8))); tablet.addValue( - "s3", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + "s1", + rowIndex, + new Binary(String.format("t%s", deviceIndex).getBytes(StandardCharsets.UTF_8))); + tablet.addValue( + "s2", + rowIndex, + new Binary(String.format("t%s", deviceIndex).getBytes(StandardCharsets.UTF_8))); + tablet.addValue( + "s3", + rowIndex, + new Binary(String.format("t%s", deviceIndex).getBytes(StandardCharsets.UTF_8))); tablet.addValue("s4", rowIndex, row); - tablet.addValue("s5", rowIndex, (int) row); - tablet.addValue("s6", rowIndex, row * 0.1); - tablet.addValue("s7", rowIndex, getDate((int) row)); + tablet.addValue("s5", rowIndex, (row * 1.0f)); + tablet.addValue( + "s6", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s7", rowIndex, row); + tablet.addValue("s8", rowIndex, (int) row); + tablet.addValue("s9", rowIndex, row * 0.1); + tablet.addValue("s10", rowIndex, getDate((int) row)); tablet.addValue( - "s8", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); - if (randomNumber < 9) { + "s11", rowIndex, new Binary(String.valueOf(row).getBytes(StandardCharsets.UTF_8))); + if (randomNumber < 12) { tablet.addValue("s" + randomNumber, rowIndex, null); } if (nullDeviceIndex < 4) { tablet.addValue("s" + nullDeviceIndex, rowIndex, null); } + tablet.setRowSize(rowIndex + 1); } } @@ -552,14 +611,17 @@ public static Tablet generateTablet( final boolean allowNullDeviceColumn) { List schemaList = new ArrayList<>(); schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); - schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); - schemaList.add(new MeasurementSchema("s2", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s1", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s2", TSDataType.STRING)); schemaList.add(new MeasurementSchema("s3", TSDataType.STRING)); - schemaList.add(new MeasurementSchema("s4", TSDataType.TIMESTAMP)); - schemaList.add(new MeasurementSchema("s5", TSDataType.INT32)); - schemaList.add(new MeasurementSchema("s6", TSDataType.DOUBLE)); - schemaList.add(new MeasurementSchema("s7", TSDataType.DATE)); - schemaList.add(new MeasurementSchema("s8", TSDataType.TEXT)); + schemaList.add(new MeasurementSchema("s4", TSDataType.INT64)); + schemaList.add(new MeasurementSchema("s5", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s6", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s7", TSDataType.TIMESTAMP)); + schemaList.add(new MeasurementSchema("s8", TSDataType.INT32)); + schemaList.add(new MeasurementSchema("s9", TSDataType.DOUBLE)); + schemaList.add(new MeasurementSchema("s10", TSDataType.DATE)); + schemaList.add(new MeasurementSchema("s11", TSDataType.TEXT)); final List columnTypes = Arrays.asList( @@ -571,6 +633,9 @@ public static Tablet generateTablet( Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, Tablet.ColumnCategory.MEASUREMENT); Tablet tablet = new Tablet( @@ -586,30 +651,43 @@ public static Tablet generateTablet( for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text for (long row = 0; row < deviceDataSize; row++) { - int randomNumber = allowNullValue ? random.nextInt(9) : 9; + int randomNumber = allowNullValue ? random.nextInt(12) : 12; int rowIndex = tablet.getRowSize(); - int value = random.nextInt(); + long value = random.nextInt(); tablet.addTimestamp(rowIndex, row); tablet.addValue( "s0", rowIndex, - new Binary(String.format("t%s", value).getBytes(StandardCharsets.UTF_8))); - tablet.addValue("s1", rowIndex, value); - tablet.addValue("s2", rowIndex, (value * 1.0f)); + new Binary(String.format("t%s", deviceIndex).getBytes(StandardCharsets.UTF_8))); + tablet.addValue( + "s1", + rowIndex, + new Binary(String.format("t%s", deviceIndex).getBytes(StandardCharsets.UTF_8))); tablet.addValue( - "s3", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + "s2", + rowIndex, + new Binary(String.format("t%s", deviceIndex).getBytes(StandardCharsets.UTF_8))); + tablet.addValue( + "s3", + rowIndex, + new Binary(String.format("t%s", deviceIndex).getBytes(StandardCharsets.UTF_8))); tablet.addValue("s4", rowIndex, value); - tablet.addValue("s5", rowIndex, (int) value); - tablet.addValue("s6", rowIndex, value * 0.1); - tablet.addValue("s7", rowIndex, getDate((int) value)); + tablet.addValue("s5", rowIndex, (value * 1.0f)); + tablet.addValue( + "s6", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s7", rowIndex, value); + tablet.addValue("s8", rowIndex, (int) value); + tablet.addValue("s9", rowIndex, value * 0.1); + tablet.addValue("s10", rowIndex, getDate((int) value)); tablet.addValue( - "s8", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); - if (randomNumber < 9) { + "s11", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + if (randomNumber < 12) { tablet.addValue("s" + randomNumber, rowIndex, null); } if (nullDeviceIndex < 4) { tablet.addValue("s" + nullDeviceIndex, rowIndex, null); } + tablet.setRowSize(rowIndex + 1); } } From 0306ae06a8ade10e292d9cdbeb93abe8fa4506ed Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 16 Dec 2024 16:06:54 +0800 Subject: [PATCH 30/42] add test case --- .../it/tablemodel/IoTDBPipeDataSinkIT.java | 259 ++++++++++++++---- .../pipe/it/tablemodel/TableModelUtils.java | 15 +- 2 files changed, 211 insertions(+), 63 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 0b760ffd6187..5386d382f51d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -30,6 +30,7 @@ import org.apache.tsfile.write.record.Tablet; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -43,11 +44,14 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.function.BiConsumer; import java.util.function.Consumer; @RunWith(IoTDBTestRunner.class) @Category({MultiClusterIT2TableModel.class}) public class IoTDBPipeDataSinkIT extends AbstractPipeTableModelTestIT { + + @Ignore @Test public void testThriftConnectorWithRealtimeFirstDisabled() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); @@ -116,16 +120,19 @@ public void testThriftConnectorWithRealtimeFirstDisabled() throws Exception { } } + @Ignore @Test public void testSinkTabletFormat() throws Exception { testSinkFormat("tablet"); } + @Ignore @Test public void testSinkTsFileFormat() throws Exception { testSinkFormat("tsfile"); } + @Ignore @Test public void testSinkHybridFormat() throws Exception { testSinkFormat("hybrid"); @@ -189,7 +196,7 @@ private void testSinkFormat(final String format) throws Exception { return; } - TableModelUtils.assertCountData("test", "test", 100, receiverEnv); + TableModelUtils.assertCountData("test", "test", 150, receiverEnv); TestUtils.assertDataEventuallyOnEnv( receiverEnv, @@ -234,10 +241,11 @@ private void testSinkFormat(final String format) throws Exception { new HashSet<>(Arrays.asList("0,1.0,", "1,1.0,", "2,1.0,", "3,1.0,", "4,1.0,"))), handleFailure); - TableModelUtils.assertCountData("test", "test", 300, receiverEnv); + TableModelUtils.assertCountData("test", "test", 350, receiverEnv); } } + @Ignore @Test public void testWriteBackSink() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); @@ -304,6 +312,46 @@ public void testWriteBackSink() throws Exception { @Test public void testSinkTsFileFormat2() throws Exception { + doTest(this::insertTablet1); + } + + @Test + public void testSinkTsFileFormat3() throws Exception { + doTest(this::insertTablet2); + } + + @Test + public void testSinkTsFileFormat4() throws Exception { + doTest(this::insertTablet3); + } + + @Test + public void testSinkTsFileFormat5() throws Exception { + doTest(this::insertTablet4); + } + + @Test + public void testSinkTsFileFormat6() throws Exception { + doTest(this::insertTablet5); + } + + @Test + public void testSinkTsFileFormat7() throws Exception { + doTest(this::insertTablet6); + } + + @Test + public void testSinkTsFileFormat8() throws Exception { + doTest(this::insertTablet7); + } + + @Test + public void testSinkTsFileFormat9() throws Exception { + doTest(this::insertTablet8); + } + + private void doTest(BiConsumer>, Map>> consumer) + throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); final String receiverIp = receiverDataNode.getIp(); @@ -335,6 +383,7 @@ public void testSinkTsFileFormat2() throws Exception { final Map processorAttributes = new HashMap<>(); final Map connectorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.realtime.mode", "forced-log"); extractorAttributes.put("capture.table", "true"); extractorAttributes.put("capture.tree", "true"); extractorAttributes.put("extractor.database-name", "test.*"); @@ -364,7 +413,7 @@ public void testSinkTsFileFormat2() throws Exception { return; } - insertTablet(testResult, test1Result); + consumer.accept(testResult, test1Result); TestUtils.assertDataEventuallyOnEnv( receiverEnv, @@ -399,6 +448,7 @@ public void testSinkTsFileFormat2() throws Exception { tablet -> { set.addAll(TableModelUtils.generateExpectedResults(tablet)); }); + TableModelUtils.assertCountData("test0",entry.getKey(),set.size(),receiverEnv,handleFailure); TableModelUtils.assertData("test0", entry.getKey(), set, receiverEnv, handleFailure); } @@ -410,53 +460,75 @@ public void testSinkTsFileFormat2() throws Exception { tablet -> { set.addAll(TableModelUtils.generateExpectedResults(tablet)); }); + TableModelUtils.assertCountData("test1",entry.getKey(),set.size(),receiverEnv,handleFailure); TableModelUtils.assertData("test1", entry.getKey(), set, receiverEnv, handleFailure); } } - private void insertTablet( + private void insertTablet1( final Map> testResult, final Map> test1Result) { - final Random random = new Random(); int deviceIDStartIndex = 0; int deviceIDEndIndex = 100; - for (int i = 0; i < 10; i++) { - final String tableName = "test" + i; - for (int j = 0; j < 10; j++) { - final String dataBaseName = "test" + j % 2; + for (int j = 0; j < 25; j++) { + final String dataBaseName = "test" + j % 2; + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; Tablet tablet = TableModelUtils.generateTablet( - tableName, deviceIDStartIndex, deviceIDEndIndex, 0, 10, false, true); + tableName, deviceIDStartIndex, deviceIDEndIndex, 0, 10, false, false); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } Map> map = j % 2 == 0 ? testResult : test1Result; - map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); - deviceIDStartIndex += 25; - deviceIDEndIndex += 25; + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } + deviceIDStartIndex += 25; + deviceIDEndIndex += 25; } + } - deviceIDStartIndex = 0; - deviceIDEndIndex = 100; + private void insertTablet2( + final Map> testResult, final Map> test1Result) { - for (int i = 0; i < 5; i++) { - final String tableName = "test" + i; - for (int j = 0; j < 10; j++) { - final String dataBaseName = "test" + j % 2; + int deviceIDStartIndex = 0; + int deviceIDEndIndex = 30; + + for (int j = 0; j < 25; j++) { + final String dataBaseName = "test" + j % 2; + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; Tablet tablet = TableModelUtils.generateTablet( - tableName, deviceIDStartIndex, deviceIDEndIndex, 10, false, true); + tableName, deviceIDStartIndex, deviceIDEndIndex, 10, false, false); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } Map> map = j % 2 == 0 ? testResult : test1Result; - map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); - deviceIDStartIndex += 25; - deviceIDEndIndex += 25; + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } + deviceIDStartIndex += 10; + deviceIDEndIndex += 10; } + } - for (int i = 0; i < 5; i++) { - final String tableName = "test" + i; - for (int j = 0; j < 10; j++) { + private void insertTablet3( + final Map> testResult, final Map> test1Result) { + + final Random random = new Random(); + int deviceIDStartIndex = 0; + int deviceIDEndIndex = 100; + + for (int j = 0; j < 25; j++) { + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; final String dataBaseName = "test" + j % 2; deviceIDStartIndex = random.nextInt(1 << 16) - 25; deviceIDEndIndex = deviceIDStartIndex + 25; @@ -470,15 +542,28 @@ private void insertTablet( false, true); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } Map> map = j % 2 == 0 ? testResult : test1Result; - map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } } + } - for (int i = 0; i < 5; i++) { - final String tableName = "test" + i; - for (int j = 0; j < 10; j++) { - final String dataBaseName = "test" + j % 2; + private void insertTablet4( + final Map> testResult, final Map> test1Result) { + + final Random random = new Random(); + int deviceIDStartIndex = 0; + int deviceIDEndIndex = 100; + + for (int j = 0; j < 25; j++) { + final String dataBaseName = "test" + j % 2; + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; deviceIDStartIndex = random.nextInt(1 << 16) - 25; deviceIDEndIndex = deviceIDStartIndex + 25; Tablet tablet = @@ -489,65 +574,125 @@ private void insertTablet( deviceIDStartIndex, deviceIDEndIndex, false, - true); + false); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } Map> map = j % 2 == 0 ? testResult : test1Result; map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); } } + } - for (int i = 0; i < 5; i++) { - final String tableName = "test" + i; - for (int j = 0; j < 10; j++) { + private void insertTablet5( + final Map> testResult, final Map> test1Result) { + + final Random random = new Random(); + int deviceIDStartIndex = 0; + int deviceIDEndIndex = 100; + for (int j = 0; j < 25; j++) { + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; final String dataBaseName = "test" + j % 2; deviceIDStartIndex = random.nextInt(1 << 16) - 25; deviceIDEndIndex = deviceIDStartIndex + 25; Tablet tablet = TableModelUtils.generateTablet( - tableName, - deviceIDStartIndex, - deviceIDEndIndex, - deviceIDStartIndex, - deviceIDEndIndex, - false, - true); + tableName, 0, 25, deviceIDStartIndex, deviceIDEndIndex, false, true); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } Map> map = j % 2 == 0 ? testResult : test1Result; - map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } } + } - for (int i = 0; i < 5; i++) { - final String tableName = "test" + i; - for (int j = 0; j < 10; j++) { - final String dataBaseName = "test" + j % 2; + private void insertTablet6( + final Map> testResult, final Map> test1Result) { + final Random random = new Random(); + int deviceIDStartIndex = 0; + int deviceIDEndIndex = 100; + + for (int j = 0; j < 25; j++) { + final String dataBaseName = "test" + j % 2; + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; deviceIDStartIndex = random.nextInt(1 << 16) - 25; deviceIDEndIndex = deviceIDStartIndex + 25; Tablet tablet = TableModelUtils.generateTablet( tableName, deviceIDStartIndex, deviceIDEndIndex, 100, 110, false, true); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); - TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } Map> map = j % 2 == 0 ? testResult : test1Result; - map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } } + } + private void insertTablet7( + final Map> testResult, final Map> test1Result) { + + final Random random = new Random(); + int deviceIDStartIndex = 0; + int deviceIDEndIndex = 100; deviceIDStartIndex = random.nextInt(1 << 16) - 300; - deviceIDEndIndex = deviceIDStartIndex + 25; - for (int i = 0; i < 5; i++) { - final String tableName = "test" + i; - for (int j = 0; j < 10; j++) { - final String dataBaseName = "test" + j % 2; + deviceIDEndIndex = deviceIDStartIndex + 10; + for (int j = 0; j < 25; j++) { + final String dataBaseName = "test" + j % 2; + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; Tablet tablet = TableModelUtils.generateTablet( tableName, deviceIDStartIndex, deviceIDEndIndex, 100, 110, false, true); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } Map> map = j % 2 == 0 ? testResult : test1Result; - map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); - deviceIDStartIndex += 25; - deviceIDEndIndex += 25; + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); + } + deviceIDStartIndex += 25; + deviceIDEndIndex += 25; + } + } + + private void insertTablet8( + final Map> testResult, final Map> test1Result) { + final Random random = new Random(); + int deviceIDStartIndex = random.nextInt(1 << 16); + int deviceIDEndIndex = deviceIDStartIndex + 35; + for (int j = 0; j < 25; j++) { + final String dataBaseName = "test" + j % 2; + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; + Tablet tablet = + TableModelUtils.generateTablet( + tableName, 100, 125, deviceIDStartIndex, deviceIDEndIndex, false, true); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } + deviceIDStartIndex += 25; + deviceIDEndIndex += 25; } } } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index 824446345eb6..ba16943415d6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -116,7 +116,7 @@ public static boolean insertData( List list = new ArrayList<>(end - start + 1); Object[] values = new Object[12]; Random random = new Random(); - // s0 string, s1 int64, s2 float, s3 string, s4 int64, s5 float, s6 string s7 timestamp, s8 + // s0 string, s1 string, s2 string, s3 string, s4 int64, s5 float, s6 string s7 timestamp, s8 // int32, s9 double, s10 date, s11 text for (int i = start; i < end; ++i) { Arrays.fill(values, i); @@ -242,13 +242,15 @@ public static void deleteData( } } + // s0 string, s1 string, s2 string, s3 string, s4 int64, s5 float, s6 string s7 timestamp, s8 + // int32, s9 double, s10 date, s11 text public static Set generateExpectedResults(final int start, final int end) { Set expectedResSet = new HashSet<>(); for (int i = start; i < end; ++i) { final String time = RpcUtils.formatDatetime("default", "ms", i, ZoneOffset.UTC); expectedResSet.add( String.format( - "t%d,t%d,t%d,t%d,%d,%d.0,%d,%s,%d,%d.0,%s,%s,%s,", + "t%s,t%s,t%s,t%s,%s,%s.0,%s,%s,%d,%d.0,%s,%s,%s", i, i, i, i, i, i, i, time, i, i, getDateStr(i), i, time)); } return expectedResSet; @@ -317,11 +319,11 @@ public static Set generateExpectedResults(final Tablet tablet) { } public static String generateHeaderResults() { - return "s0,s3,s2,s1,s4,s5,s6,s7,s8,time,"; + return "s0,s1,s2,s3,s4,s5,s6,s7,s8,s9,s10,s11,time,"; } public static String getQuerySql(final String table) { - return "select s0,s3,s2,s1,s4,s5,s6,s7,s8,time from " + table; + return "select s0,s1,s2,s3,s4,s5,s6,s7,s8,s9,s10,s11,time from " + table; } public static String getQueryCountSql(final String table) { @@ -650,11 +652,12 @@ public static Tablet generateTablet( for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text + long value = random.nextInt(1<<16); for (long row = 0; row < deviceDataSize; row++) { int randomNumber = allowNullValue ? random.nextInt(12) : 12; int rowIndex = tablet.getRowSize(); - long value = random.nextInt(); - tablet.addTimestamp(rowIndex, row); + value += random.nextInt(100); + tablet.addTimestamp(rowIndex, value); tablet.addValue( "s0", rowIndex, From 44122687f80db89c4496060b0ed340ed6a0809db Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 16 Dec 2024 16:07:50 +0800 Subject: [PATCH 31/42] spotless --- .../iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java | 6 ++++-- .../apache/iotdb/pipe/it/tablemodel/TableModelUtils.java | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 5386d382f51d..60f1d0111408 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -448,7 +448,8 @@ private void doTest(BiConsumer>, Map { set.addAll(TableModelUtils.generateExpectedResults(tablet)); }); - TableModelUtils.assertCountData("test0",entry.getKey(),set.size(),receiverEnv,handleFailure); + TableModelUtils.assertCountData( + "test0", entry.getKey(), set.size(), receiverEnv, handleFailure); TableModelUtils.assertData("test0", entry.getKey(), set, receiverEnv, handleFailure); } @@ -460,7 +461,8 @@ private void doTest(BiConsumer>, Map { set.addAll(TableModelUtils.generateExpectedResults(tablet)); }); - TableModelUtils.assertCountData("test1",entry.getKey(),set.size(),receiverEnv,handleFailure); + TableModelUtils.assertCountData( + "test1", entry.getKey(), set.size(), receiverEnv, handleFailure); TableModelUtils.assertData("test1", entry.getKey(), set, receiverEnv, handleFailure); } } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index ba16943415d6..f69f3977cfce 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -652,7 +652,7 @@ public static Tablet generateTablet( for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text - long value = random.nextInt(1<<16); + long value = random.nextInt(1 << 16); for (long row = 0; row < deviceDataSize; row++) { int randomNumber = allowNullValue ? random.nextInt(12) : 12; int rowIndex = tablet.getRowSize(); From d4292cf70f6e8e6a9bf0667b5cf2a22ac532c835 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 16 Dec 2024 18:03:27 +0800 Subject: [PATCH 32/42] fix --- .../iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 60f1d0111408..c397a5951720 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -496,7 +496,6 @@ private void insertTablet1( private void insertTablet2( final Map> testResult, final Map> test1Result) { - int deviceIDStartIndex = 0; int deviceIDEndIndex = 30; @@ -523,7 +522,6 @@ private void insertTablet2( private void insertTablet3( final Map> testResult, final Map> test1Result) { - final Random random = new Random(); int deviceIDStartIndex = 0; int deviceIDEndIndex = 100; @@ -557,7 +555,6 @@ private void insertTablet3( private void insertTablet4( final Map> testResult, final Map> test1Result) { - final Random random = new Random(); int deviceIDStartIndex = 0; int deviceIDEndIndex = 100; @@ -584,14 +581,13 @@ private void insertTablet4( throw new RuntimeException(e); } Map> map = j % 2 == 0 ? testResult : test1Result; - map.computeIfAbsent(dataBaseName, k -> new ArrayList<>()).add(tablet); + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } } } private void insertTablet5( final Map> testResult, final Map> test1Result) { - final Random random = new Random(); int deviceIDStartIndex = 0; int deviceIDEndIndex = 100; From 5b8820ee57b5c809dcb342f0e8874078da468bba Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 16 Dec 2024 19:28:07 +0800 Subject: [PATCH 33/42] add test case --- .../it/tablemodel/IoTDBPipeDataSinkIT.java | 32 ++++++-- .../pipe/it/tablemodel/TableModelUtils.java | 76 +++++++++++++++++++ 2 files changed, 102 insertions(+), 6 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index c397a5951720..859ecbdb4e40 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -30,7 +30,6 @@ import org.apache.tsfile.write.record.Tablet; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -51,7 +50,6 @@ @Category({MultiClusterIT2TableModel.class}) public class IoTDBPipeDataSinkIT extends AbstractPipeTableModelTestIT { - @Ignore @Test public void testThriftConnectorWithRealtimeFirstDisabled() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); @@ -120,19 +118,16 @@ public void testThriftConnectorWithRealtimeFirstDisabled() throws Exception { } } - @Ignore @Test public void testSinkTabletFormat() throws Exception { testSinkFormat("tablet"); } - @Ignore @Test public void testSinkTsFileFormat() throws Exception { testSinkFormat("tsfile"); } - @Ignore @Test public void testSinkHybridFormat() throws Exception { testSinkFormat("hybrid"); @@ -245,7 +240,6 @@ private void testSinkFormat(final String format) throws Exception { } } - @Ignore @Test public void testWriteBackSink() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); @@ -350,6 +344,11 @@ public void testSinkTsFileFormat9() throws Exception { doTest(this::insertTablet8); } + @Test + public void testSinkTsFileFormat10() throws Exception { + doTest(this::insertTablet9); + } + private void doTest(BiConsumer>, Map>> consumer) throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); @@ -693,4 +692,25 @@ private void insertTablet8( deviceIDEndIndex += 25; } } + + private void insertTablet9( + final Map> testResult, final Map> test1Result) { + final Random random = new Random(); + for (int j = 0; j < 25; j++) { + final String dataBaseName = "test" + j % 2; + for (int i = 0; i < 5; i++) { + final String tableName = "test" + i; + Tablet tablet = + TableModelUtils.generateTabletDeviceIDAllIsNull(tableName, 100, 125, 25, false); + TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + Map> map = j % 2 == 0 ? testResult : test1Result; + map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); + } + } + } } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index f69f3977cfce..fcf764c448b3 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -696,4 +696,80 @@ public static Tablet generateTablet( return tablet; } + + public static Tablet generateTabletDeviceIDAllIsNull( + final String tableName, + final int deviceStartIndex, + final int deviceEndIndex, + final int deviceDataSize, + final boolean allowNullValue) { + List schemaList = new ArrayList<>(); + schemaList.add(new MeasurementSchema("s0", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s1", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s2", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s3", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s4", TSDataType.INT64)); + schemaList.add(new MeasurementSchema("s5", TSDataType.FLOAT)); + schemaList.add(new MeasurementSchema("s6", TSDataType.STRING)); + schemaList.add(new MeasurementSchema("s7", TSDataType.TIMESTAMP)); + schemaList.add(new MeasurementSchema("s8", TSDataType.INT32)); + schemaList.add(new MeasurementSchema("s9", TSDataType.DOUBLE)); + schemaList.add(new MeasurementSchema("s10", TSDataType.DATE)); + schemaList.add(new MeasurementSchema("s11", TSDataType.TEXT)); + + final List columnTypes = + Arrays.asList( + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.ID, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT, + Tablet.ColumnCategory.MEASUREMENT); + Tablet tablet = + new Tablet( + tableName, + IMeasurementSchema.getMeasurementNameList(schemaList), + IMeasurementSchema.getDataTypeList(schemaList), + columnTypes, + (deviceEndIndex - deviceStartIndex) * deviceDataSize); + tablet.initBitMaps(); + final Random random = new Random(); + + for (int deviceIndex = deviceStartIndex; deviceIndex < deviceEndIndex; deviceIndex++) { + // s2 float, s3 string, s4 timestamp, s5 int32, s6 double, s7 date, s8 text + long value = random.nextInt(1 << 16); + for (long row = 0; row < deviceDataSize; row++) { + int randomNumber = allowNullValue ? random.nextInt(12) : 12; + int rowIndex = tablet.getRowSize(); + value += random.nextInt(100); + tablet.addTimestamp(rowIndex, value); + tablet.addValue("s0", rowIndex, null); + tablet.addValue("s1", rowIndex, null); + tablet.addValue("s2", rowIndex, null); + tablet.addValue("s3", rowIndex, null); + tablet.addValue("s4", rowIndex, value); + tablet.addValue("s5", rowIndex, (value * 1.0f)); + tablet.addValue( + "s6", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + tablet.addValue("s7", rowIndex, value); + tablet.addValue("s8", rowIndex, (int) value); + tablet.addValue("s9", rowIndex, value * 0.1); + tablet.addValue("s10", rowIndex, getDate((int) value)); + tablet.addValue( + "s11", rowIndex, new Binary(String.valueOf(value).getBytes(StandardCharsets.UTF_8))); + if (randomNumber < 12) { + tablet.addValue("s" + randomNumber, rowIndex, null); + } + tablet.setRowSize(rowIndex + 1); + } + } + + return tablet; + } } From 9dd29ac823e29b5ff09fb87b48f633abd1141641 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 17 Dec 2024 12:03:34 +0800 Subject: [PATCH 34/42] update PipeTableModeTsFileBuilder --- .../util/PipeTableModeTsFileBuilder.java | 148 ++++++------------ 1 file changed, 44 insertions(+), 104 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java index fd76cee143f7..c8b90476a340 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java @@ -40,7 +40,6 @@ import java.util.LinkedHashSet; import java.util.LinkedList; import java.util.List; -import java.util.ListIterator; import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; @@ -73,7 +72,10 @@ public List> convertTabletToTSFileWithDBInfo() throws IOExcep } List> pairList = new ArrayList<>(); for (Map.Entry> entry : dataBase2TabletList.entrySet()) { - pairList.addAll(writeTableModelTabletsToTsFiles(entry.getValue(), entry.getKey())); + final LinkedHashSet>>>> linkedHashSet = + new LinkedHashSet<>(); + pairList.addAll( + writeTableModelTabletsToTsFiles(entry.getValue(), entry.getKey(), linkedHashSet)); } return pairList; } @@ -95,30 +97,42 @@ public synchronized void close() { dataBase2TabletList.clear(); } - private List> writeTableModelTabletsToTsFiles( - final List tabletList, final String dataBase) throws IOException { + private >>> + List> writeTableModelTabletsToTsFiles( + final List tabletList, + final String dataBase, + LinkedHashSet> linkedHashSet) + throws IOException { - final Map> tableName2Tablets = new HashMap<>(); + final Map> tableName2Tablets = new HashMap<>(); // Sort the tablets by dataBaseName for (final Tablet tablet : tabletList) { - tableName2Tablets.computeIfAbsent(tablet.getTableName(), k -> new ArrayList<>()).add(tablet); + tableName2Tablets + .computeIfAbsent(tablet.getTableName(), k -> new ArrayList<>()) + .add((T) new Pair<>(tablet, WriteUtils.splitTabletByDevice(tablet))); } // Replace ArrayList with LinkedList to improve performance - final LinkedHashSet>>>> table2Tablets = - new LinkedHashSet<>(); + final LinkedHashSet> table2Tablets = new LinkedHashSet<>(); + + // Sort the tablets by start time in first device + for (final List tablets : tableName2Tablets.values()) { + tablets.sort( + (o1, o2) -> { + final IDeviceID deviceID = o1.right.get(0).left; + final int result; + if ((result = deviceID.compareTo(o2.right.get(0).left)) == 0) { + return Long.compare(o1.left.timestamps[0], o2.left.timestamps[0]); + } + return result; + }); + } + // Sort the tables by table name tableName2Tablets.entrySet().stream() .sorted(Map.Entry.comparingByKey(Comparator.naturalOrder())) - .forEach( - entry -> { - LinkedList>>> list = new LinkedList<>(); - for (final Tablet tablet : entry.getValue()) { - writerPairToList(list, new Pair<>(tablet, WriteUtils.splitTabletByDevice(tablet))); - } - table2Tablets.add(list); - }); + .forEach(entry -> linkedHashSet.add(new LinkedList<>(entry.getValue()))); // Help GC tableName2Tablets.clear(); @@ -126,13 +140,13 @@ private List> writeTableModelTabletsToTsFiles( final List> sealedFiles = new ArrayList<>(); // Try making the tsfile size as large as possible - while (!table2Tablets.isEmpty()) { + while (!linkedHashSet.isEmpty()) { if (Objects.isNull(fileWriter)) { createFileWriter(); } try { - tryBestToWriteTabletsIntoOneFile(table2Tablets); + tryBestToWriteTabletsIntoOneFile(linkedHashSet); } catch (final Exception e) { LOGGER.warn( "Batch id = {}: Failed to write tablets into tsfile, because {}", @@ -186,21 +200,20 @@ private List> writeTableModelTabletsToTsFiles( return sealedFiles; } - private void tryBestToWriteTabletsIntoOneFile( - final LinkedHashSet>>>> - device2TabletsLinkedList) - throws IOException { - final Iterator>>>> iterator = - device2TabletsLinkedList.iterator(); + private >>> + void tryBestToWriteTabletsIntoOneFile( + final LinkedHashSet> device2TabletsLinkedList) throws IOException { + final Iterator> iterator = device2TabletsLinkedList.iterator(); while (iterator.hasNext()) { - final LinkedList>>> tablets = iterator.next(); + final LinkedList tablets = iterator.next(); - final List>>> tabletsToWrite = new ArrayList<>(); + final List tabletsToWrite = new ArrayList<>(); final Map deviceLastTimestampMap = new HashMap<>(); while (!tablets.isEmpty()) { - final Pair>> pair = tablets.peekFirst(); - if (timestampsAreNonOverlapping(pair, deviceLastTimestampMap)) { + final T pair = tablets.peekFirst(); + if (timestampsAreNonOverlapping( + (Pair>>) pair, deviceLastTimestampMap)) { tabletsToWrite.add(pair); tablets.pollFirst(); continue; @@ -240,9 +253,9 @@ private void tryBestToWriteTabletsIntoOneFile( * @return If false, the tablet overlaps with the previous tablet; if true, there is no time * overlap. */ - private boolean timestampsAreNonOverlapping( - final Pair>> tabletPair, - final Map deviceLastTimestampMap) { + private >>> + boolean timestampsAreNonOverlapping( + final T tabletPair, final Map deviceLastTimestampMap) { int currentTimestampIndex = 0; for (Pair deviceTimestampIndexPair : tabletPair.right) { final Long lastDeviceTimestamp = deviceLastTimestampMap.get(deviceTimestampIndexPair.left); @@ -257,77 +270,4 @@ private boolean timestampsAreNonOverlapping( return true; } - - /** - * Add the Tablet to the List and compare the IDevice minimum timestamp with each Tablet from the - * beginning. If all the IDevice minimum timestamps of the current Tablet are smaller than the - * IDevice minimum timestamps of a certain Tablet in the List, put the current Tablet in this - * position. - */ - private void writerPairToList( - final LinkedList>>> list, - final Pair>> pair) { - int lastResult = Integer.MAX_VALUE; - if (list.isEmpty()) { - list.add(pair); - return; - } - ListIterator>>> iterator = list.listIterator(); - while (iterator.hasNext()) { - final Pair>> pair2 = iterator.next(); - final int result = compareDeviceID(pair2, pair); - if (lastResult == 0 && result != 0) { - iterator.add(pair); - return; - } - lastResult = result; - } - list.add(pair); - } - - /** - * Compares the time differences of the same DeviceID in two device ID lists. If the time of the - * same DeviceID in the second device list is greater than in the first, then a positive number is - * returned; if there is no such DeviceID, then 0 is returned. - * - * @param firstDeviceList The first device ID list and its associated times - * @param secondDeviceList The second device ID list and its associated times - * @return The comparison result - */ - private int compareDeviceID( - final Pair>> firstDeviceList, - final Pair>> secondDeviceList) { - int bCount = 0; - int aIndex = 0; - int bIndex = 0; - int aLastTimeIndex = 0; - int bLastTimeIndex = 0; - final List> listA = firstDeviceList.right; - final List> listB = secondDeviceList.right; - while (aIndex < listA.size() && bIndex < listB.size()) { - int comparisonResult = listA.get(aIndex).left.compareTo(listB.get(bIndex).left); - if (comparisonResult == 0) { - long aTime = firstDeviceList.left.timestamps[aLastTimeIndex]; - long bTime = secondDeviceList.left.timestamps[bLastTimeIndex]; - if (aTime < bTime) { - bCount++; - } - aLastTimeIndex = listA.get(aIndex).right; - bLastTimeIndex = listB.get(bIndex).right; - aIndex++; - bIndex++; - continue; - } - - if (comparisonResult > 0) { - bLastTimeIndex = listB.get(bIndex).right; - bIndex++; - continue; - } - - aLastTimeIndex = listA.get(aIndex).right; - aIndex++; - } - return bCount; - } } From 972b17283eb104194bcdb264039d5acb154676f3 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 17 Dec 2024 16:49:55 +0800 Subject: [PATCH 35/42] modify test case --- .../it/tablemodel/IoTDBPipeDataSinkIT.java | 46 +++++++++---------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java index 859ecbdb4e40..bd0a94419c14 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeDataSinkIT.java @@ -470,7 +470,7 @@ private void insertTablet1( final Map> testResult, final Map> test1Result) { int deviceIDStartIndex = 0; - int deviceIDEndIndex = 100; + int deviceIDEndIndex = 10; for (int j = 0; j < 25; j++) { final String dataBaseName = "test" + j % 2; @@ -488,15 +488,15 @@ private void insertTablet1( Map> map = j % 2 == 0 ? testResult : test1Result; map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } - deviceIDStartIndex += 25; - deviceIDEndIndex += 25; + deviceIDStartIndex += 2; + deviceIDEndIndex += 2; } } private void insertTablet2( final Map> testResult, final Map> test1Result) { int deviceIDStartIndex = 0; - int deviceIDEndIndex = 30; + int deviceIDEndIndex = 10; for (int j = 0; j < 25; j++) { final String dataBaseName = "test" + j % 2; @@ -514,8 +514,8 @@ private void insertTablet2( Map> map = j % 2 == 0 ? testResult : test1Result; map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } - deviceIDStartIndex += 10; - deviceIDEndIndex += 10; + deviceIDStartIndex += 2; + deviceIDEndIndex += 2; } } @@ -529,8 +529,8 @@ private void insertTablet3( for (int i = 0; i < 5; i++) { final String tableName = "test" + i; final String dataBaseName = "test" + j % 2; - deviceIDStartIndex = random.nextInt(1 << 16) - 25; - deviceIDEndIndex = deviceIDStartIndex + 25; + deviceIDStartIndex = random.nextInt(1 << 16) - 10; + deviceIDEndIndex = deviceIDStartIndex + 10; Tablet tablet = TableModelUtils.generateTablet( tableName, @@ -562,8 +562,8 @@ private void insertTablet4( final String dataBaseName = "test" + j % 2; for (int i = 0; i < 5; i++) { final String tableName = "test" + i; - deviceIDStartIndex = random.nextInt(1 << 16) - 25; - deviceIDEndIndex = deviceIDStartIndex + 25; + deviceIDStartIndex = random.nextInt(1 << 16) - 10; + deviceIDEndIndex = deviceIDStartIndex + 10; Tablet tablet = TableModelUtils.generateTablet( tableName, @@ -594,11 +594,11 @@ private void insertTablet5( for (int i = 0; i < 5; i++) { final String tableName = "test" + i; final String dataBaseName = "test" + j % 2; - deviceIDStartIndex = random.nextInt(1 << 16) - 25; - deviceIDEndIndex = deviceIDStartIndex + 25; + deviceIDStartIndex = random.nextInt(1 << 16) - 10; + deviceIDEndIndex = deviceIDStartIndex + 10; Tablet tablet = TableModelUtils.generateTablet( - tableName, 0, 25, deviceIDStartIndex, deviceIDEndIndex, false, true); + tableName, 0, 10, deviceIDStartIndex, deviceIDEndIndex, false, true); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); try { Thread.sleep(100); @@ -621,8 +621,8 @@ private void insertTablet6( final String dataBaseName = "test" + j % 2; for (int i = 0; i < 5; i++) { final String tableName = "test" + i; - deviceIDStartIndex = random.nextInt(1 << 16) - 25; - deviceIDEndIndex = deviceIDStartIndex + 25; + deviceIDStartIndex = random.nextInt(1 << 16) - 10; + deviceIDEndIndex = deviceIDStartIndex + 10; Tablet tablet = TableModelUtils.generateTablet( tableName, deviceIDStartIndex, deviceIDEndIndex, 100, 110, false, true); @@ -644,7 +644,7 @@ private void insertTablet7( final Random random = new Random(); int deviceIDStartIndex = 0; int deviceIDEndIndex = 100; - deviceIDStartIndex = random.nextInt(1 << 16) - 300; + deviceIDStartIndex = random.nextInt(1 << 16) - 10; deviceIDEndIndex = deviceIDStartIndex + 10; for (int j = 0; j < 25; j++) { final String dataBaseName = "test" + j % 2; @@ -662,8 +662,8 @@ private void insertTablet7( Map> map = j % 2 == 0 ? testResult : test1Result; map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } - deviceIDStartIndex += 25; - deviceIDEndIndex += 25; + deviceIDStartIndex += 2; + deviceIDEndIndex += 2; } } @@ -671,14 +671,14 @@ private void insertTablet8( final Map> testResult, final Map> test1Result) { final Random random = new Random(); int deviceIDStartIndex = random.nextInt(1 << 16); - int deviceIDEndIndex = deviceIDStartIndex + 35; + int deviceIDEndIndex = deviceIDStartIndex + 10; for (int j = 0; j < 25; j++) { final String dataBaseName = "test" + j % 2; for (int i = 0; i < 5; i++) { final String tableName = "test" + i; Tablet tablet = TableModelUtils.generateTablet( - tableName, 100, 125, deviceIDStartIndex, deviceIDEndIndex, false, true); + tableName, 100, 110, deviceIDStartIndex, deviceIDEndIndex, false, true); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); try { Thread.sleep(100); @@ -688,8 +688,8 @@ private void insertTablet8( Map> map = j % 2 == 0 ? testResult : test1Result; map.computeIfAbsent(tableName, k -> new ArrayList<>()).add(tablet); } - deviceIDStartIndex += 25; - deviceIDEndIndex += 25; + deviceIDStartIndex += 2; + deviceIDEndIndex += 2; } } @@ -701,7 +701,7 @@ private void insertTablet9( for (int i = 0; i < 5; i++) { final String tableName = "test" + i; Tablet tablet = - TableModelUtils.generateTabletDeviceIDAllIsNull(tableName, 100, 125, 25, false); + TableModelUtils.generateTabletDeviceIDAllIsNull(tableName, 100, 110, 10, false); TableModelUtils.insertTablet(dataBaseName, tablet, senderEnv); try { Thread.sleep(100); From b5a924fbb5b8b374f9ce3c359b3b705f0411c6f3 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 17 Dec 2024 21:45:07 +0800 Subject: [PATCH 36/42] modify session pool max size --- .../org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index fcf764c448b3..69d596c99156 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -215,7 +215,7 @@ public static boolean insertTablet( public static boolean insertTablet( final String dataBaseName, final Tablet tablet, final BaseEnv baseEnv) { - ITableSessionPool tableSessionPool = baseEnv.getTableSessionPool(1); + ITableSessionPool tableSessionPool = baseEnv.getTableSessionPool(20); try (final ITableSession session = tableSessionPool.getSession()) { session.executeNonQueryStatement("use " + dataBaseName); session.insert(tablet); From a468b79a275d7b01dbb8d49ac6ad34a2704c83a5 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Wed, 18 Dec 2024 12:07:24 +0800 Subject: [PATCH 37/42] close session pool --- .../org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index 69d596c99156..15d783502b18 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -215,8 +215,8 @@ public static boolean insertTablet( public static boolean insertTablet( final String dataBaseName, final Tablet tablet, final BaseEnv baseEnv) { - ITableSessionPool tableSessionPool = baseEnv.getTableSessionPool(20); - try (final ITableSession session = tableSessionPool.getSession()) { + try (ITableSessionPool tableSessionPool = baseEnv.getTableSessionPool(20); + final ITableSession session = tableSessionPool.getSession()) { session.executeNonQueryStatement("use " + dataBaseName); session.insert(tablet); session.executeNonQueryStatement("flush"); From 3972607d28b78a015991293a462a46af9d433642 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Wed, 18 Dec 2024 15:36:57 +0800 Subject: [PATCH 38/42] modify TableModelUtils --- .../apache/iotdb/pipe/it/tablemodel/TableModelUtils.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index 15d783502b18..564d705a79ba 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -98,7 +98,7 @@ public static boolean insertData( for (int i = start; i < end; ++i) { list.add( String.format( - "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11 time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", + "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11, time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", tableName, i, i, i, i, i, i, i, i, i, i, getDateStr(i), i, i)); } list.add("flush"); @@ -167,7 +167,7 @@ public static boolean insertDataNotThrowError( for (int i = start; i < end; ++i) { list.add( String.format( - "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11 time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", + "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11, time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", tableName, i, i, i, i, i, i, i, i, i, i, getDateStr(i), i, i)); } return TestUtils.tryExecuteNonQueriesWithRetry( @@ -185,7 +185,7 @@ public static boolean insertData( for (int i = start; i < end; ++i) { list.add( String.format( - "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11 time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", + "insert into %s (s0, s3, s2, s1, s4, s5, s6, s7, s8, s9, s10, s11, time) values ('t%s','t%s','t%s','t%s','%s', %s.0, %s, %s, %d, %d.0, '%s', '%s', %s)", tableName, i, i, i, i, i, i, i, i, i, i, getDateStr(i), i, i)); } list.add("flush"); @@ -250,7 +250,7 @@ public static Set generateExpectedResults(final int start, final int end final String time = RpcUtils.formatDatetime("default", "ms", i, ZoneOffset.UTC); expectedResSet.add( String.format( - "t%s,t%s,t%s,t%s,%s,%s.0,%s,%s,%d,%d.0,%s,%s,%s", + "t%s,t%s,t%s,t%s,%s,%s.0,%s,%s,%d,%d.0,%s,%s,%s,", i, i, i, i, i, i, i, time, i, i, getDateStr(i), i, time)); } return expectedResSet; From f7e984d2ac7f6bc0acb9d5dc4eed68a748aa7b0b Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 3 Jan 2025 20:22:56 +0800 Subject: [PATCH 39/42] The receiver sorting rules are changed to be consistent with Session --- .../pipe/it/tablemodel/TableModelUtils.java | 8 +-- .../request/PipeTransferTabletRawReqV2.java | 2 +- .../util/PipeTableModelTabletEventSorter.java | 68 ++++++++++++++++++- 3 files changed, 71 insertions(+), 7 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index 9efb4cbe7132..2af45c5c4d88 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -462,7 +462,7 @@ public static Tablet generateTablet( Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELDT, + Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, @@ -544,7 +544,7 @@ public static Tablet generateTablet( Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELDT, + Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, @@ -634,7 +634,7 @@ public static Tablet generateTablet( Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELDT, + Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, @@ -726,7 +726,7 @@ public static Tablet generateTabletDeviceIDAllIsNull( Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELDT, + Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, Tablet.ColumnCategory.FIELD, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java index ffbad5bd9ab0..3fcbcb6c0953 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java @@ -61,7 +61,7 @@ public InsertTabletStatement constructStatement() { if (Objects.isNull(dataBaseName)) { new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); } else { - new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByTimestamp(); } try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index aa7f162e4f09..dec77a758495 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -29,8 +29,10 @@ import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; public class PipeTableModelTabletEventSorter { @@ -136,12 +138,55 @@ public void sortAndDeduplicateByDevIdTimestamp() { sortAndDeduplicateValuesAndBitMaps(); } - private void sortTimestamps(int startIndex, int endIndex) { + public void sortAndDeduplicateByTimestamp() { + if (tablet == null || tablet.getRowSize() == 0) { + return; + } + + for (int i = 1, size = tablet.getRowSize(); i < size; ++i) { + final long currentTimestamp = tablet.timestamps[i]; + final long previousTimestamp = tablet.timestamps[i - 1]; + + if (currentTimestamp < previousTimestamp) { + isUnSorted = true; + break; + } + if (currentTimestamp == previousTimestamp) { + hasDuplicates = true; + } + } + + if (!isUnSorted && hasDuplicates) { + return; + } + + index = new Integer[tablet.getRowSize()]; + for (int i = 0, size = tablet.getRowSize(); i < size; i++) { + index[i] = i; + } + + if (!isUnSorted) { + sortTimestamps(0, tablet.getRowSize()); + + // Do deduplicate anyway. + // isDeduplicated may be false positive when isSorted is false. + deduplicateTimestampsAndDeviceId(0, tablet.getRowSize()); + hasDuplicates = false; + } + + if (hasDuplicates) { + deduplicateTimestampsAndDeviceId(0, tablet.getRowSize()); + } + + sortAndDeduplicateValuesAndBitMaps(); + } + + private void sortTimestamps(final int startIndex, final int endIndex) { Arrays.sort( this.index, startIndex, endIndex, Comparator.comparingLong(i -> tablet.timestamps[i])); } - private void deduplicateTimestamps(int startIndex, int endIndex) { + private void deduplicateTimestamps(final int startIndex, final int endIndex) { long lastTime = tablet.timestamps[index[startIndex]]; index[deduplicatedSize++] = index[startIndex]; for (int i = startIndex + 1; i < endIndex; i++) { @@ -151,6 +196,25 @@ private void deduplicateTimestamps(int startIndex, int endIndex) { } } + private void deduplicateTimestampsAndDeviceId(final int startIndex, final int endIndex) { + long lastTime = tablet.timestamps[index[startIndex]]; + IDeviceID deviceID = tablet.getDeviceID(index[startIndex]); + final Set deviceIDSet = new HashSet<>(); + deviceIDSet.add(deviceID); + index[deduplicatedSize++] = index[startIndex]; + for (int i = startIndex + 1; i < endIndex; i++) { + deviceID = tablet.getDeviceID(index[i]); + if ((lastTime == (lastTime = tablet.timestamps[index[i]])) + && !deviceIDSet.contains(deviceID)) { + index[deduplicatedSize++] = index[i]; + deviceIDSet.add(deviceID); + } else { + index[deduplicatedSize++] = index[i]; + deviceIDSet.clear(); + } + } + } + private void sortAndDeduplicateValuesAndBitMaps() { int columnIndex = 0; tablet.timestamps = From 89b7d467f9d2c4efdb4bd04bb1f24c68b8dca9ec Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 6 Jan 2025 11:16:05 +0800 Subject: [PATCH 40/42] update PipeTableModelTabletEventSorter --- .../pipe/it/tablemodel/TableModelUtils.java | 78 +++++++-------- .../request/PipeTransferTabletRawReqV2.java | 2 +- .../protocol/opcua/OpcUaNameSpace.java | 4 +- .../util/PipeTableModelTabletEventSorter.java | 99 ++++++++++++------- .../connector/PipeTabletEventSorterTest.java | 55 +++++++++-- 5 files changed, 152 insertions(+), 86 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java index 2af45c5c4d88..c7da1382535d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java @@ -536,19 +536,19 @@ public static Tablet generateTablet( schemaList.add(new MeasurementSchema("s11", TSDataType.TEXT)); final List columnTypes = - Arrays.asList( - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD); + Arrays.asList( + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD); Tablet tablet = new Tablet( tableName, @@ -626,19 +626,19 @@ public static Tablet generateTablet( schemaList.add(new MeasurementSchema("s11", TSDataType.TEXT)); final List columnTypes = - Arrays.asList( - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD); + Arrays.asList( + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD); Tablet tablet = new Tablet( tableName, @@ -718,19 +718,19 @@ public static Tablet generateTabletDeviceIDAllIsNull( schemaList.add(new MeasurementSchema("s11", TSDataType.TEXT)); final List columnTypes = - Arrays.asList( - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.TAG, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD, - Tablet.ColumnCategory.FIELD); + Arrays.asList( + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD); Tablet tablet = new Tablet( tableName, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java index 3fcbcb6c0953..4caa20e7d04a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java @@ -61,7 +61,7 @@ public InsertTabletStatement constructStatement() { if (Objects.isNull(dataBaseName)) { new PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary(); } else { - new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByTimestamp(); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByTimestampIfNecessary(); } try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java index a6ec6d11b3ab..01a137dd97fc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java @@ -138,9 +138,9 @@ private void transferTabletForClientServerModel(final Tablet tablet, final boole transferTabletRowForClientServerModel( tablet.getDeviceId().split("\\."), newSchemas, timestamps, values); } else { - new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); - final List columnIndexes = new ArrayList<>(); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByTimestampIfNecessary(); + final List columnIndexes = new ArrayList<>(); for (int i = 0; i < schemas.size(); ++i) { if (tablet.getColumnTypes().get(i) == Tablet.ColumnCategory.FIELD) { columnIndexes.add(i); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java index dec77a758495..5ffb982c1797 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java @@ -138,7 +138,46 @@ public void sortAndDeduplicateByDevIdTimestamp() { sortAndDeduplicateValuesAndBitMaps(); } - public void sortAndDeduplicateByTimestamp() { + private void sortAndDeduplicateValuesAndBitMaps() { + int columnIndex = 0; + tablet.timestamps = + (long[]) + TabletSortUtil.reorderValueList( + deduplicatedSize, tablet.timestamps, TSDataType.TIMESTAMP, index); + for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) { + final IMeasurementSchema schema = tablet.getSchemas().get(i); + if (schema != null) { + tablet.values[columnIndex] = + TabletSortUtil.reorderValueList( + deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); + if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) { + tablet.bitMaps[columnIndex] = + TabletSortUtil.reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); + } + columnIndex++; + } + } + + tablet.setRowSize(deduplicatedSize); + } + + private void sortTimestamps(final int startIndex, final int endIndex) { + Arrays.sort( + this.index, startIndex, endIndex, Comparator.comparingLong(i -> tablet.timestamps[i])); + } + + private void deduplicateTimestamps(final int startIndex, final int endIndex) { + long lastTime = tablet.timestamps[index[startIndex]]; + index[deduplicatedSize++] = index[startIndex]; + for (int i = startIndex + 1; i < endIndex; i++) { + if (lastTime != (lastTime = tablet.timestamps[index[i]])) { + index[deduplicatedSize++] = index[i]; + } + } + } + + /** Sort by time only, and remove only rows with the same DeviceID and time. */ + public void sortAndDeduplicateByTimestampIfNecessary() { if (tablet == null || tablet.getRowSize() == 0) { return; } @@ -156,7 +195,7 @@ public void sortAndDeduplicateByTimestamp() { } } - if (!isUnSorted && hasDuplicates) { + if (!isUnSorted && !hasDuplicates) { return; } @@ -165,62 +204,52 @@ public void sortAndDeduplicateByTimestamp() { index[i] = i; } - if (!isUnSorted) { - sortTimestamps(0, tablet.getRowSize()); + if (isUnSorted) { + sortTimestamps(); // Do deduplicate anyway. - // isDeduplicated may be false positive when isSorted is false. - deduplicateTimestampsAndDeviceId(0, tablet.getRowSize()); + // isDeduplicated may be false positive when isUnSorted is true. + deduplicateTimestamps(); hasDuplicates = false; } if (hasDuplicates) { - deduplicateTimestampsAndDeviceId(0, tablet.getRowSize()); + deduplicateTimestamps(); } - sortAndDeduplicateValuesAndBitMaps(); - } - - private void sortTimestamps(final int startIndex, final int endIndex) { - Arrays.sort( - this.index, startIndex, endIndex, Comparator.comparingLong(i -> tablet.timestamps[i])); + sortAndDeduplicateValuesAndBitMapsIgnoreTimestamp(); } - private void deduplicateTimestamps(final int startIndex, final int endIndex) { - long lastTime = tablet.timestamps[index[startIndex]]; - index[deduplicatedSize++] = index[startIndex]; - for (int i = startIndex + 1; i < endIndex; i++) { - if (lastTime != (lastTime = tablet.timestamps[index[i]])) { - index[deduplicatedSize++] = index[i]; - } - } + private void sortTimestamps() { + Arrays.sort(this.index, Comparator.comparingLong(i -> tablet.timestamps[i])); + Arrays.sort(tablet.timestamps, 0, tablet.getRowSize()); } - private void deduplicateTimestampsAndDeviceId(final int startIndex, final int endIndex) { - long lastTime = tablet.timestamps[index[startIndex]]; - IDeviceID deviceID = tablet.getDeviceID(index[startIndex]); + private void deduplicateTimestamps() { + deduplicatedSize = 1; + long lastTime = tablet.timestamps[0]; + IDeviceID deviceID = tablet.getDeviceID(index[0]); final Set deviceIDSet = new HashSet<>(); deviceIDSet.add(deviceID); - index[deduplicatedSize++] = index[startIndex]; - for (int i = startIndex + 1; i < endIndex; i++) { + for (int i = 1, size = tablet.getRowSize(); i < size; i++) { deviceID = tablet.getDeviceID(index[i]); - if ((lastTime == (lastTime = tablet.timestamps[index[i]])) - && !deviceIDSet.contains(deviceID)) { - index[deduplicatedSize++] = index[i]; - deviceIDSet.add(deviceID); + if ((lastTime == (lastTime = tablet.timestamps[i]))) { + if (!deviceIDSet.contains(deviceID)) { + tablet.timestamps[deduplicatedSize] = lastTime; + index[deduplicatedSize++] = index[i]; + deviceIDSet.add(deviceID); + } } else { + tablet.timestamps[deduplicatedSize] = lastTime; index[deduplicatedSize++] = index[i]; deviceIDSet.clear(); + deviceIDSet.add(deviceID); } } } - private void sortAndDeduplicateValuesAndBitMaps() { + private void sortAndDeduplicateValuesAndBitMapsIgnoreTimestamp() { int columnIndex = 0; - tablet.timestamps = - (long[]) - TabletSortUtil.reorderValueList( - deduplicatedSize, tablet.timestamps, TSDataType.TIMESTAMP, index); for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) { final IMeasurementSchema schema = tablet.getSchemas().get(i); if (schema != null) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index c789e76a49c9..ca8d6ea2e6d3 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -238,6 +238,21 @@ public void testTableModelSort() { doTableModelTest(false, true); } + @Test + public void testTableModelDeduplicateAndSort1() { + doTableModelTest1(true, true); + } + + @Test + public void testTableModelDeduplicate1() { + doTableModelTest1(true, false); + } + + @Test + public void testTableModelSort1() { + doTableModelTest1(false, true); + } + public void doTableModelTest(final boolean hasDuplicates, final boolean isUnSorted) { final Tablet tablet = generateTablet("test", 10, hasDuplicates, isUnSorted); new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp(); @@ -260,6 +275,28 @@ public void doTableModelTest(final boolean hasDuplicates, final boolean isUnSort } } + public void doTableModelTest1(final boolean hasDuplicates, final boolean isUnSorted) { + final Tablet tablet = generateTablet("test", 10, hasDuplicates, isUnSorted); + new PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByTimestampIfNecessary(); + for (int i = 1; i < tablet.getRowSize(); i++) { + long time = tablet.timestamps[i]; + Assert.assertTrue(time > tablet.timestamps[i - 1]); + Assert.assertEquals( + tablet.getValue(i, 0), + new Binary(String.valueOf(i / 100).getBytes(StandardCharsets.UTF_8))); + Assert.assertEquals(tablet.getValue(i, 1), (long) i); + Assert.assertEquals(tablet.getValue(i, 2), i * 1.0f); + Assert.assertEquals( + tablet.getValue(i, 3), new Binary(String.valueOf(i).getBytes(StandardCharsets.UTF_8))); + Assert.assertEquals(tablet.getValue(i, 4), (long) i); + Assert.assertEquals(tablet.getValue(i, 5), i); + Assert.assertEquals(tablet.getValue(i, 6), i * 0.1); + Assert.assertEquals(tablet.getValue(i, 7), getDate(i)); + Assert.assertEquals( + tablet.getValue(i, 8), new Binary(String.valueOf(i).getBytes(StandardCharsets.UTF_8))); + } + } + private Tablet generateTablet( final String tableName, final int deviceIDNum, @@ -278,15 +315,15 @@ private Tablet generateTablet( final List columnTypes = Arrays.asList( - Tablet.ColumnCategory.ID, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT, - Tablet.ColumnCategory.MEASUREMENT); + Tablet.ColumnCategory.TAG, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD, + Tablet.ColumnCategory.FIELD); Tablet tablet = new Tablet( tableName, From 314dc5286a6e99ad436ec8781c99b22fdf703a4a Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Tue, 7 Jan 2025 18:27:37 +0800 Subject: [PATCH 41/42] refactor --- .../batch/PipeTabletEventTsFileBatch.java | 12 ++++++------ .../request/PipeTransferTabletRawReq.java | 2 +- .../request/PipeTransferTabletRawReqV2.java | 4 ++-- .../connector/protocol/opcua/OpcUaNameSpace.java | 4 ++-- .../PipeTransferTabletBatchEventHandler.java | 2 +- .../thrift/sync/IoTDBDataRegionSyncConnector.java | 2 +- .../{ => builder}/PipeTableModeTsFileBuilder.java | 2 +- .../PipeTreeModelTsFileBuilder.java} | 10 +++++----- .../util/{ => builder}/PipeTsFileBuilder.java | 2 +- .../util/{ => cacher}/LeaderCacheUtils.java | 2 +- .../PipeTableModelTabletEventSorter.java | 14 ++++++++------ .../PipeTabletEventSorter.java} | 4 ++-- .../PipeTreeModelTabletEventSorter.java | 7 ++++--- .../pipe/connector/PipeTabletEventSorterTest.java | 4 ++-- 14 files changed, 37 insertions(+), 34 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/{ => builder}/PipeTableModeTsFileBuilder.java (99%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/{PipeTreeModelTSFileBuilder.java => builder/PipeTreeModelTsFileBuilder.java} (97%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/{ => builder}/PipeTsFileBuilder.java (99%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/{ => cacher}/LeaderCacheUtils.java (97%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/{ => sorter}/PipeTableModelTabletEventSorter.java (94%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/{TabletSortUtil.java => sorter/PipeTabletEventSorter.java} (97%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/{ => sorter}/PipeTreeModelTabletEventSorter.java (93%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index 0ba8f98d6233..95b744e73045 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -19,11 +19,11 @@ package org.apache.iotdb.db.pipe.connector.payload.evolvable.batch; -import org.apache.iotdb.db.pipe.connector.util.PipeTableModeTsFileBuilder; -import org.apache.iotdb.db.pipe.connector.util.PipeTableModelTabletEventSorter; -import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTSFileBuilder; -import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; -import org.apache.iotdb.db.pipe.connector.util.PipeTsFileBuilder; +import org.apache.iotdb.db.pipe.connector.util.builder.PipeTableModeTsFileBuilder; +import org.apache.iotdb.db.pipe.connector.util.builder.PipeTreeModelTsFileBuilder; +import org.apache.iotdb.db.pipe.connector.util.builder.PipeTsFileBuilder; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTableModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; @@ -63,7 +63,7 @@ public PipeTabletEventTsFileBatch(final int maxDelayInMs, final long requestMaxB this.maxSizeInBytes = requestMaxBatchSizeInBytes; AtomicLong tsFileIdGenerator = new AtomicLong(0); - treeModeTsFileBuilder = new PipeTreeModelTSFileBuilder(currentBatchId, tsFileIdGenerator); + treeModeTsFileBuilder = new PipeTreeModelTsFileBuilder(currentBatchId, tsFileIdGenerator); tableModeTsFileBuilder = new PipeTableModeTsFileBuilder(currentBatchId, tsFileIdGenerator); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java index 002c6af7a504..bf27b697698c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java @@ -23,7 +23,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.IoTDBConnectorRequestVersion; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeRequestType; import org.apache.iotdb.commons.utils.PathUtils; -import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.queryengine.plan.parser.StatementGenerator; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java index 4caa20e7d04a..fd435ded99ca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReqV2.java @@ -23,8 +23,8 @@ import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.IoTDBConnectorRequestVersion; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeRequestType; import org.apache.iotdb.commons.utils.PathUtils; -import org.apache.iotdb.db.pipe.connector.util.PipeTableModelTabletEventSorter; -import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTableModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.queryengine.plan.parser.StatementGenerator; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java index 01a137dd97fc..834a14f94df8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java @@ -22,8 +22,8 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; import org.apache.iotdb.commons.utils.PathUtils; -import org.apache.iotdb.db.pipe.connector.util.PipeTableModelTabletEventSorter; -import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTableModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTreeModelTabletEventSorter; import org.apache.iotdb.db.utils.DateTimeUtils; import org.apache.iotdb.db.utils.TimestampPrecisionUtils; import org.apache.iotdb.pipe.api.event.Event; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java index 8370dd667ab3..cc03af180f9d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java @@ -26,7 +26,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.connector.payload.evolvable.batch.PipeTabletEventPlainBatch; import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector; -import org.apache.iotdb.db.pipe.connector.util.LeaderCacheUtils; +import org.apache.iotdb.db.pipe.connector.util.cacher.LeaderCacheUtils; import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index a6f02cbf92ea..93ce77974b90 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -35,7 +35,7 @@ import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFilePieceReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFilePieceWithModReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFileSealWithModReq; -import org.apache.iotdb.db.pipe.connector.util.LeaderCacheUtils; +import org.apache.iotdb.db.pipe.connector.util.cacher.LeaderCacheUtils; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java similarity index 99% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java index c8b90476a340..6199856a7caa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.connector.util; +package org.apache.iotdb.db.pipe.connector.util.builder; import org.apache.iotdb.pipe.api.exception.PipeException; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java similarity index 97% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java index 5b944ffd213b..e3ecd21d6ff5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTSFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.connector.util; +package org.apache.iotdb.db.pipe.connector.util.builder; import org.apache.commons.io.FileUtils; import org.apache.tsfile.exception.write.WriteProcessException; @@ -42,14 +42,14 @@ import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; -public class PipeTreeModelTSFileBuilder extends PipeTsFileBuilder { +public class PipeTreeModelTsFileBuilder extends PipeTsFileBuilder { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeTreeModelTSFileBuilder.class); + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTreeModelTsFileBuilder.class); private final List tabletList = new ArrayList<>(); private final List isTabletAlignedList = new ArrayList<>(); - public PipeTreeModelTSFileBuilder( + public PipeTreeModelTsFileBuilder( final AtomicLong currentBatchId, final AtomicLong tsFileIdGenerator) { super(currentBatchId, tsFileIdGenerator); } @@ -57,7 +57,7 @@ public PipeTreeModelTSFileBuilder( @Override public void bufferTableModelTablet(final String dataBase, final Tablet tablet) { throw new UnsupportedOperationException( - "PipeTreeModelTSFileBuilder does not support table model tablet to build TSFile"); + "PipeTreeModelTsFileBuilder does not support table model tablet to build TSFile"); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTsFileBuilder.java similarity index 99% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTsFileBuilder.java index 3c1082fbb409..9623811e07fb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTsFileBuilder.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.connector.util; +package org.apache.iotdb.db.pipe.connector.util.builder; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/LeaderCacheUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/cacher/LeaderCacheUtils.java similarity index 97% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/LeaderCacheUtils.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/cacher/LeaderCacheUtils.java index e31fec9b57b6..0151a83a9cff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/LeaderCacheUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/cacher/LeaderCacheUtils.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.connector.util; +package org.apache.iotdb.db.pipe.connector.util.cacher; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTableModelTabletEventSorter.java similarity index 94% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTableModelTabletEventSorter.java index 5ffb982c1797..63ac5b847a1b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTableModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTableModelTabletEventSorter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.connector.util; +package org.apache.iotdb.db.pipe.connector.util.sorter; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; @@ -142,17 +142,18 @@ private void sortAndDeduplicateValuesAndBitMaps() { int columnIndex = 0; tablet.timestamps = (long[]) - TabletSortUtil.reorderValueList( + PipeTabletEventSorter.reorderValueList( deduplicatedSize, tablet.timestamps, TSDataType.TIMESTAMP, index); for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) { final IMeasurementSchema schema = tablet.getSchemas().get(i); if (schema != null) { tablet.values[columnIndex] = - TabletSortUtil.reorderValueList( + PipeTabletEventSorter.reorderValueList( deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) { tablet.bitMaps[columnIndex] = - TabletSortUtil.reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); + PipeTabletEventSorter.reorderBitMap( + deduplicatedSize, tablet.bitMaps[columnIndex], index); } columnIndex++; } @@ -254,11 +255,12 @@ private void sortAndDeduplicateValuesAndBitMapsIgnoreTimestamp() { final IMeasurementSchema schema = tablet.getSchemas().get(i); if (schema != null) { tablet.values[columnIndex] = - TabletSortUtil.reorderValueList( + PipeTabletEventSorter.reorderValueList( deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) { tablet.bitMaps[columnIndex] = - TabletSortUtil.reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); + PipeTabletEventSorter.reorderBitMap( + deduplicatedSize, tablet.bitMaps[columnIndex], index); } columnIndex++; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/TabletSortUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTabletEventSorter.java similarity index 97% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/TabletSortUtil.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTabletEventSorter.java index 070fa31339cb..f3756b2a46c3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/TabletSortUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTabletEventSorter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.connector.util; +package org.apache.iotdb.db.pipe.connector.util.sorter; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.utils.Binary; @@ -26,7 +26,7 @@ import java.time.LocalDate; -public class TabletSortUtil { +public class PipeTabletEventSorter { public static Object reorderValueList( final int deduplicatedSize, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTreeModelTabletEventSorter.java similarity index 93% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTreeModelTabletEventSorter.java index 63a38c65eac8..42f8c2dac52c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/PipeTreeModelTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTreeModelTabletEventSorter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.connector.util; +package org.apache.iotdb.db.pipe.connector.util.sorter; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -107,11 +107,12 @@ private void sortAndDeduplicateValuesAndBitMaps() { final IMeasurementSchema schema = tablet.getSchemas().get(i); if (schema != null) { tablet.values[columnIndex] = - TabletSortUtil.reorderValueList( + PipeTabletEventSorter.reorderValueList( deduplicatedSize, tablet.values[columnIndex], schema.getType(), index); if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) { tablet.bitMaps[columnIndex] = - TabletSortUtil.reorderBitMap(deduplicatedSize, tablet.bitMaps[columnIndex], index); + PipeTabletEventSorter.reorderBitMap( + deduplicatedSize, tablet.bitMaps[columnIndex], index); } columnIndex++; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java index ca8d6ea2e6d3..e6f2d44a589f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java @@ -19,8 +19,8 @@ package org.apache.iotdb.db.pipe.connector; -import org.apache.iotdb.db.pipe.connector.util.PipeTableModelTabletEventSorter; -import org.apache.iotdb.db.pipe.connector.util.PipeTreeModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTableModelTabletEventSorter; +import org.apache.iotdb.db.pipe.connector.util.sorter.PipeTreeModelTabletEventSorter; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.utils.Binary; From bc97e11960c71597165fc697bd88d66d1eb26138 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Tue, 7 Jan 2025 18:42:34 +0800 Subject: [PATCH 42/42] refactor --- .../batch/PipeTabletEventTsFileBatch.java | 44 +++++++++---------- .../builder/PipeTableModeTsFileBuilder.java | 2 +- .../builder/PipeTreeModelTsFileBuilder.java | 2 +- .../util/builder/PipeTsFileBuilder.java | 2 +- 4 files changed, 25 insertions(+), 25 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index 95b744e73045..1162961f567e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -53,6 +53,7 @@ public class PipeTabletEventTsFileBatch extends PipeTabletEventBatch { private final AtomicLong currentBatchId = new AtomicLong(BATCH_ID_GENERATOR.incrementAndGet()); private final long maxSizeInBytes; + private final PipeTsFileBuilder treeModeTsFileBuilder; private final PipeTsFileBuilder tableModeTsFileBuilder; @@ -62,7 +63,8 @@ public PipeTabletEventTsFileBatch(final int maxDelayInMs, final long requestMaxB super(maxDelayInMs); this.maxSizeInBytes = requestMaxBatchSizeInBytes; - AtomicLong tsFileIdGenerator = new AtomicLong(0); + + final AtomicLong tsFileIdGenerator = new AtomicLong(0); treeModeTsFileBuilder = new PipeTreeModelTsFileBuilder(currentBatchId, tsFileIdGenerator); tableModeTsFileBuilder = new PipeTableModeTsFileBuilder(currentBatchId, tsFileIdGenerator); } @@ -79,22 +81,21 @@ protected boolean constructBatch(final TabletInsertionEvent event) { if (tablet.getRowSize() == 0) { continue; } - // table Model if (isTableModel) { + // table Model bufferTableModelTablet( insertNodeTabletInsertionEvent.getPipeName(), insertNodeTabletInsertionEvent.getCreationTime(), tablet, insertNodeTabletInsertionEvent.getTableModelDatabaseName()); - continue; + } else { + // tree Model + bufferTreeModelTablet( + insertNodeTabletInsertionEvent.getPipeName(), + insertNodeTabletInsertionEvent.getCreationTime(), + tablet, + insertNodeTabletInsertionEvent.isAligned(i)); } - - // tree Model - bufferTreeModelTablet( - insertNodeTabletInsertionEvent.getPipeName(), - insertNodeTabletInsertionEvent.getCreationTime(), - tablet, - insertNodeTabletInsertionEvent.isAligned(i)); } } else if (event instanceof PipeRawTabletInsertionEvent) { final PipeRawTabletInsertionEvent rawTabletInsertionEvent = @@ -103,21 +104,21 @@ protected boolean constructBatch(final TabletInsertionEvent event) { if (tablet.getRowSize() == 0) { return true; } - // table Model if (rawTabletInsertionEvent.isTableModelEvent()) { + // table Model bufferTableModelTablet( rawTabletInsertionEvent.getPipeName(), rawTabletInsertionEvent.getCreationTime(), tablet, rawTabletInsertionEvent.getTableModelDatabaseName()); - return true; + } else { + // tree Model + bufferTreeModelTablet( + rawTabletInsertionEvent.getPipeName(), + rawTabletInsertionEvent.getCreationTime(), + tablet, + rawTabletInsertionEvent.isAligned()); } - // tree Model - bufferTreeModelTablet( - rawTabletInsertionEvent.getPipeName(), - rawTabletInsertionEvent.getCreationTime(), - tablet, - rawTabletInsertionEvent.isAligned()); } else { LOGGER.warn( "Batch id = {}: Unsupported event {} type {} when constructing tsfile batch", @@ -180,13 +181,12 @@ public synchronized List> sealTsFiles() return Collections.emptyList(); } - List> list = new ArrayList<>(); + final List> list = new ArrayList<>(); if (!treeModeTsFileBuilder.isEmpty()) { - list.addAll(treeModeTsFileBuilder.convertTabletToTSFileWithDBInfo()); + list.addAll(treeModeTsFileBuilder.convertTabletToTsFileWithDBInfo()); } - if (!tableModeTsFileBuilder.isEmpty()) { - list.addAll(tableModeTsFileBuilder.convertTabletToTSFileWithDBInfo()); + list.addAll(tableModeTsFileBuilder.convertTabletToTsFileWithDBInfo()); } return list; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java index 6199856a7caa..6ac5ac911bfa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java @@ -66,7 +66,7 @@ public void bufferTreeModelTablet(Tablet tablet, Boolean isAligned) { } @Override - public List> convertTabletToTSFileWithDBInfo() throws IOException { + public List> convertTabletToTsFileWithDBInfo() throws IOException { if (dataBase2TabletList.isEmpty()) { return new ArrayList<>(0); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java index e3ecd21d6ff5..8d30dd0172f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java @@ -67,7 +67,7 @@ public void bufferTreeModelTablet(final Tablet tablet, final Boolean isAligned) } @Override - public List> convertTabletToTSFileWithDBInfo() + public List> convertTabletToTsFileWithDBInfo() throws IOException, WriteProcessException { return writeTabletsToTsFiles(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTsFileBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTsFileBuilder.java index 9623811e07fb..d320bff0d173 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTsFileBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTsFileBuilder.java @@ -108,7 +108,7 @@ private File getNextBaseDir() throws DiskSpaceInsufficientException { public abstract void bufferTreeModelTablet(Tablet tablet, Boolean isAligned); - public abstract List> convertTabletToTSFileWithDBInfo() + public abstract List> convertTabletToTsFileWithDBInfo() throws IOException, WriteProcessException; public abstract boolean isEmpty();