From b0501ae6eda7c5aa71e41a6a5584152670e705ae Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Mon, 2 Sep 2024 16:30:50 +0800 Subject: [PATCH 01/13] Pipe IT: Avoid IoTDBPipeTypeConversionIT from creating too many DataRegions and SchemaRegions (#13378) --- .../it/manual/IoTDBPipeTypeConversionIT.java | 48 +++++++++---------- 1 file changed, 22 insertions(+), 26 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java index e22a49ed279d..aa2558b480d1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java @@ -54,6 +54,7 @@ public class IoTDBPipeTypeConversionIT extends AbstractPipeDualManualIT { // Test for converting BOOLEAN to OtherType @Test public void testBooleanToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.BOOLEAN, TSDataType.INT32); executeAndVerifyTypeConversion(TSDataType.BOOLEAN, TSDataType.INT64); executeAndVerifyTypeConversion(TSDataType.BOOLEAN, TSDataType.FLOAT); @@ -68,6 +69,7 @@ public void testBooleanToOtherTypeConversion() { // Test for converting INT32 to OtherType @Test public void testInt32ToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.INT32, TSDataType.BOOLEAN); executeAndVerifyTypeConversion(TSDataType.INT32, TSDataType.INT64); executeAndVerifyTypeConversion(TSDataType.INT32, TSDataType.FLOAT); @@ -82,6 +84,7 @@ public void testInt32ToOtherTypeConversion() { // Test for converting INT64 to OtherType @Test public void testInt64ToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.INT64, TSDataType.BOOLEAN); executeAndVerifyTypeConversion(TSDataType.INT64, TSDataType.INT32); executeAndVerifyTypeConversion(TSDataType.INT64, TSDataType.FLOAT); @@ -96,6 +99,7 @@ public void testInt64ToOtherTypeConversion() { // Test for converting FLOAT to OtherType @Test public void testFloatToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.FLOAT, TSDataType.BOOLEAN); executeAndVerifyTypeConversion(TSDataType.FLOAT, TSDataType.INT32); executeAndVerifyTypeConversion(TSDataType.FLOAT, TSDataType.INT64); @@ -110,6 +114,7 @@ public void testFloatToOtherTypeConversion() { // Test for converting DOUBLE to OtherType @Test public void testDoubleToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.DOUBLE, TSDataType.BOOLEAN); executeAndVerifyTypeConversion(TSDataType.DOUBLE, TSDataType.INT32); executeAndVerifyTypeConversion(TSDataType.DOUBLE, TSDataType.INT64); @@ -124,6 +129,7 @@ public void testDoubleToOtherTypeConversion() { // Test for converting TEXT to OtherType @Test public void testTextToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.TEXT, TSDataType.BLOB); executeAndVerifyTypeConversion(TSDataType.TEXT, TSDataType.STRING); executeAndVerifyTypeConversion(TSDataType.TEXT, TSDataType.BOOLEAN); @@ -138,6 +144,7 @@ public void testTextToOtherTypeConversion() { // Test for converting TIMESTAMP to OtherType @Test public void testTimestampToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.TIMESTAMP, TSDataType.BOOLEAN); executeAndVerifyTypeConversion(TSDataType.TIMESTAMP, TSDataType.INT32); executeAndVerifyTypeConversion(TSDataType.TIMESTAMP, TSDataType.INT64); @@ -152,6 +159,7 @@ public void testTimestampToOtherTypeConversion() { // Test for converting DATE to OtherType @Test public void testDateToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.DATE, TSDataType.BOOLEAN); executeAndVerifyTypeConversion(TSDataType.DATE, TSDataType.INT32); executeAndVerifyTypeConversion(TSDataType.DATE, TSDataType.INT64); @@ -165,6 +173,7 @@ public void testDateToOtherTypeConversion() { // Test for converting BLOB to OtherType @Test public void testBlobToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.BLOB, TSDataType.TEXT); executeAndVerifyTypeConversion(TSDataType.BLOB, TSDataType.STRING); executeAndVerifyTypeConversion(TSDataType.BLOB, TSDataType.BOOLEAN); @@ -179,6 +188,7 @@ public void testBlobToOtherTypeConversion() { // Test for converting STRING to OtherType @Test public void testStringToOtherTypeConversion() { + createDataPipe(); executeAndVerifyTypeConversion(TSDataType.STRING, TSDataType.TEXT); executeAndVerifyTypeConversion(TSDataType.STRING, TSDataType.BLOB); executeAndVerifyTypeConversion(TSDataType.STRING, TSDataType.BOOLEAN); @@ -193,8 +203,8 @@ private void executeAndVerifyTypeConversion(TSDataType source, TSDataType target List pairs = prepareTypeConversionTest(source, target); TestUtils.assertDataEventuallyOnEnv( receiverEnv, - String.format("select * from root.%s2%s.**", source.name(), target.name()), - String.format("Time,root.%s2%s.test.status,", source.name(), target.name()), + String.format("select status from root.test.%s2%s", source.name(), target.name()), + String.format("Time,root.test.%s2%s.status,", source.name(), target.name()), createExpectedResultSet(pairs, source, target), 30); } @@ -206,17 +216,8 @@ private List prepareTypeConversionTest(TSDataType sourceType, TSDataType t createTimeSeries(sourceTypeName, targetTypeName, sourceTypeName, senderEnv); createTimeSeries(sourceTypeName, targetTypeName, targetTypeName, receiverEnv); - createDataPipe(sourceTypeName, targetTypeName); - List pairs = createTestDataForType(sourceTypeName); - // wait pipe start - try { - Thread.sleep(2000); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - executeDataInsertions(pairs, sourceType, targetType); return pairs; } @@ -225,25 +226,20 @@ private void createTimeSeries( String sourceTypeName, String targetTypeName, String dataType, BaseEnv env) { String timeSeriesCreationQuery = String.format( - "create timeseries root.%s2%s.test.status with datatype=%s,encoding=PLAIN", + "create timeseries root.test.%s2%s.status with datatype=%s,encoding=PLAIN", sourceTypeName, targetTypeName, dataType); TestUtils.tryExecuteNonQueriesWithRetry( env, Collections.singletonList(timeSeriesCreationQuery)); } - private void createDataPipe(String sourceTypeName, String targetTypeName) { + private void createDataPipe() { String sql = String.format( - "create pipe %s2%s" - + " with source ('source'='iotdb-source','source.path'='root.%s2%s.**','realtime.mode'='forced-log','realtime.enable'='true','history.enable'='false')" + "create pipe test" + + " with source ('source'='iotdb-source','source.path'='root.test.**','realtime.mode'='forced-log','realtime.enable'='true','history.enable'='false')" + " with processor ('processor'='do-nothing-processor')" + " with sink ('node-urls'='%s:%s','batch.enable'='false','sink.format'='tablet')", - sourceTypeName, - targetTypeName, - sourceTypeName, - targetTypeName, - receiverEnv.getIP(), - receiverEnv.getPort()); + receiverEnv.getIP(), receiverEnv.getPort()); TestUtils.tryExecuteNonQueriesWithRetry(senderEnv, Collections.singletonList(sql)); } @@ -311,7 +307,7 @@ private List createInsertStatementsForString( for (Pair pair : testData) { executes.add( String.format( - "insert into root.%s2%s.test(timestamp,status) values (%s,'%s');", + "insert into root.test.%s2%s(timestamp,status) values (%s,'%s');", sourceType, targetType, pair.left, @@ -327,7 +323,7 @@ private List createInsertStatementsForNumeric( for (Pair pair : testData) { executes.add( String.format( - "insert into root.%s2%s.test(timestamp,status) values (%s,%s);", + "insert into root.test.%s2%s(timestamp,status) values (%s,%s);", sourceType, targetType, pair.left, pair.right)); } executes.add("flush"); @@ -340,7 +336,7 @@ private List createInsertStatementsForTimestamp( for (Pair pair : testData) { executes.add( String.format( - "insert into root.%s2%s.test(timestamp,status) values (%s,%s);", + "insert into root.test.%s2%s(timestamp,status) values (%s,%s);", sourceType, targetType, pair.left, pair.right)); } executes.add("flush"); @@ -353,7 +349,7 @@ private List createInsertStatementsForLocalDate( for (Pair pair : testData) { executes.add( String.format( - "insert into root.%s2%s.test(timestamp,status) values (%s,'%s');", + "insert into root.test.%s2%s(timestamp,status) values (%s,'%s');", sourceType, targetType, pair.left, DateUtils.formatDate((Integer) pair.right))); } executes.add("flush"); @@ -367,7 +363,7 @@ private List createInsertStatementsForBlob( String value = BytesUtils.parseBlobByteArrayToString(((Binary) pair.right).getValues()); executes.add( String.format( - "insert into root.%s2%s.test(timestamp,status) values (%s,X'%s');", + "insert into root.test.%s2%s(timestamp,status) values (%s,X'%s');", sourceType, targetType, pair.left, value.substring(2))); } executes.add("flush"); From 71f765b727cbb6cb5e390a21ea263e7cc10caeb0 Mon Sep 17 00:00:00 2001 From: Zikun Ma <55695098+DanielWang2035@users.noreply.github.com> Date: Mon, 2 Sep 2024 18:28:45 +0800 Subject: [PATCH 02/13] Pipe: add timely consistency check for pipe memory control (#13354) --- .../resource/memory/PipeMemoryManager.java | 27 +++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java index 8c7bc2c8ed25..bf5db9d0bb2c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java @@ -66,7 +66,7 @@ public PipeMemoryManager() { PipeDataNodeAgent.runtime() .registerPeriodicalJob( "PipeMemoryManager#tryExpandAll()", - this::tryExpandAll, + this::tryExpandAllAndCheckConsistency, PipeConfig.getInstance().getPipeMemoryExpanderIntervalSeconds()); } @@ -290,8 +290,31 @@ private boolean tryShrink4Allocate(long sizeInBytes) { } } - public synchronized void tryExpandAll() { + public synchronized void tryExpandAllAndCheckConsistency() { allocatedBlocks.forEach(PipeMemoryBlock::expand); + + long blockSum = + allocatedBlocks.stream().mapToLong(PipeMemoryBlock::getMemoryUsageInBytes).sum(); + if (blockSum != usedMemorySizeInBytes) { + LOGGER.warn( + "tryExpandAllAndCheckConsistency: memory usage is not consistent with allocated blocks," + + " usedMemorySizeInBytes is {} but sum of all blocks is {}", + usedMemorySizeInBytes, + blockSum); + } + + long tabletBlockSum = + allocatedBlocks.stream() + .filter(PipeTabletMemoryBlock.class::isInstance) + .mapToLong(PipeMemoryBlock::getMemoryUsageInBytes) + .sum(); + if (tabletBlockSum != usedMemorySizeInBytesOfTablets) { + LOGGER.warn( + "tryExpandAllAndCheckConsistency: memory usage of tablets is not consistent with allocated blocks," + + " usedMemorySizeInBytesOfTablets is {} but sum of all tablet blocks is {}", + usedMemorySizeInBytesOfTablets, + tabletBlockSum); + } } public synchronized void release(PipeMemoryBlock block) { From 08fc8abe247d16bd89a5911081720b8a59e19517 Mon Sep 17 00:00:00 2001 From: Zikun Ma <55695098+DanielWang2035@users.noreply.github.com> Date: Mon, 2 Sep 2024 18:36:56 +0800 Subject: [PATCH 03/13] Pipe: better memory control for in-memory tablets (#13301) --- .../db/pipe/event/common/row/PipeRow.java | 15 ++ .../event/common/row/PipeRowCollector.java | 14 +- .../tablet/PipeRawTabletInsertionEvent.java | 6 +- .../TsFileInsertionDataContainer.java | 12 ++ .../TsFileInsertionQueryDataContainer.java | 3 +- ...sFileInsertionQueryDataTabletIterator.java | 46 +++++-- .../TsFileInsertionScanDataContainer.java | 43 ++++-- .../resource/memory/PipeMemoryManager.java | 59 +++++++- .../resource/memory/PipeMemoryWeightUtil.java | 128 ++++++++++++++++++ .../iotdb/commons/conf/CommonConfig.java | 9 ++ .../iotdb/commons/conf/CommonDescriptor.java | 5 + .../iotdb/commons/pipe/config/PipeConfig.java | 5 + 12 files changed, 314 insertions(+), 31 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRow.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRow.java index 8757b6613c98..6d4d25b95424 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRow.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRow.java @@ -190,6 +190,21 @@ public boolean isAligned() { return isAligned; } + public int getCurrentRowSize() { + int rowSize = 0; + rowSize += 8; // timestamp + for (int i = 0; i < valueColumnTypes.length; i++) { + if (valueColumnTypes[i] != null) { + if (valueColumnTypes[i].isBinary()) { + rowSize += getBinary(i) != null ? getBinary(i).getLength() : 0; + } else { + rowSize += valueColumnTypes[i].getDataTypeSize(); + } + } + } + return rowSize; + } + public IMeasurementSchema[] getMeasurementSchemaList() { return measurementSchemaList; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java index c1f7739534db..4646041a8a2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java @@ -19,15 +19,16 @@ package org.apache.iotdb.db.pipe.event.common.row; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.pipe.api.access.Row; import org.apache.iotdb.pipe.api.collector.RowCollector; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -66,13 +67,12 @@ public void collectRow(Row row) { final String deviceId = pipeRow.getDeviceId(); final List measurementSchemaList = new ArrayList<>(Arrays.asList(measurementSchemaArray)); - tablet = - new Tablet( - deviceId, - measurementSchemaList, - PipeConfig.getInstance().getPipeDataStructureTabletRowSize()); - isAligned = pipeRow.isAligned(); + // Calculate row count and memory size of the tablet based on the first row + Pair rowCountAndMemorySize = + PipeMemoryWeightUtil.calculateTabletRowCountAndMemory(pipeRow); + tablet = new Tablet(deviceId, measurementSchemaList, rowCountAndMemorySize.getLeft()); tablet.initBitMaps(); + isAligned = pipeRow.isAligned(); } final int rowIndex = tablet.rowSize; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java index 285121ae63ca..f06819323eca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java @@ -26,6 +26,7 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.db.pipe.resource.memory.PipeTabletMemoryBlock; import org.apache.iotdb.pipe.api.access.Row; import org.apache.iotdb.pipe.api.collector.RowCollector; @@ -110,7 +111,10 @@ public PipeRawTabletInsertionEvent( @Override public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { - allocatedMemoryBlock = PipeDataNodeResourceManager.memory().forceAllocateWithRetry(tablet); + allocatedMemoryBlock = + PipeDataNodeResourceManager.memory() + .forceAllocateForTabletWithRetry( + PipeMemoryWeightUtil.calculateTabletSizeInBytes(tablet)); return true; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java index 2e8a7ec6efa9..d41ed8b08680 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java @@ -22,6 +22,8 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.tsfile.read.TsFileSequenceReader; @@ -42,6 +44,8 @@ public abstract class TsFileInsertionDataContainer implements AutoCloseable { protected final PipeTaskMeta pipeTaskMeta; // used to report progress protected final EnrichedEvent sourceEvent; // used to report progress + protected final PipeMemoryBlock allocatedMemoryBlockForTablet; + protected TsFileSequenceReader tsFileSequenceReader; protected TsFileInsertionDataContainer( @@ -58,6 +62,10 @@ protected TsFileInsertionDataContainer( this.pipeTaskMeta = pipeTaskMeta; this.sourceEvent = sourceEvent; + + // Allocate empty memory block, will be resized later. + this.allocatedMemoryBlockForTablet = + PipeDataNodeResourceManager.memory().forceAllocateForTabletWithRetry(0); } /** @@ -74,5 +82,9 @@ public void close() { } catch (final IOException e) { LOGGER.warn("Failed to close TsFileSequenceReader", e); } + + if (allocatedMemoryBlockForTablet != null) { + allocatedMemoryBlockForTablet.close(); + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java index c62d5bc603a5..955a76148517 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java @@ -280,7 +280,8 @@ public boolean hasNext() { measurementDataTypeMap, entry.getKey(), entry.getValue(), - timeFilterExpression); + timeFilterExpression, + allocatedMemoryBlockForTablet); } catch (final Exception e) { close(); throw new PipeException("failed to create TsFileInsertionDataTabletIterator", e); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataTabletIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataTabletIterator.java index e6bcdca94b26..d644d15297a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataTabletIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataTabletIterator.java @@ -19,7 +19,9 @@ package org.apache.iotdb.db.pipe.event.common.tsfile.container.query; -import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.tsfile.common.constant.TsFileConstant; @@ -32,6 +34,7 @@ import org.apache.tsfile.read.expression.IExpression; import org.apache.tsfile.read.expression.QueryExpression; import org.apache.tsfile.read.query.dataset.QueryDataSet; +import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.MeasurementSchema; @@ -42,6 +45,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.stream.Collectors; public class TsFileInsertionQueryDataTabletIterator implements Iterator { @@ -56,12 +60,15 @@ public class TsFileInsertionQueryDataTabletIterator implements Iterator private final QueryDataSet queryDataSet; + private final PipeMemoryBlock allocatedBlockForTablet; + TsFileInsertionQueryDataTabletIterator( final TsFileReader tsFileReader, final Map measurementDataTypeMap, final IDeviceID deviceId, final List measurements, - final IExpression timeFilterExpression) + final IExpression timeFilterExpression, + final PipeMemoryBlock allocatedBlockForTablet) throws IOException { this.tsFileReader = tsFileReader; this.measurementDataTypeMap = measurementDataTypeMap; @@ -79,6 +86,8 @@ public class TsFileInsertionQueryDataTabletIterator implements Iterator this.timeFilterExpression = timeFilterExpression; this.queryDataSet = buildQueryDataSet(); + + this.allocatedBlockForTablet = Objects.requireNonNull(allocatedBlockForTablet); } private QueryDataSet buildQueryDataSet() throws IOException { @@ -118,16 +127,35 @@ private Tablet buildNextTablet() throws IOException { measurementDataTypeMap.get(deviceId + TsFileConstant.PATH_SEPARATOR + measurement); schemas.add(new MeasurementSchema(measurement, dataType)); } - final Tablet tablet = - new Tablet( - // Used for tree model - deviceId.toString(), - schemas, - PipeConfig.getInstance().getPipeDataStructureTabletRowSize()); - tablet.initBitMaps(); + Tablet tablet = null; + if (!queryDataSet.hasNext()) { + tablet = + new Tablet( + // Used for tree model + deviceId.toString(), schemas, 1); + tablet.initBitMaps(); + // Ignore the memory cost of tablet + PipeDataNodeResourceManager.memory().forceResize(allocatedBlockForTablet, 0); + return tablet; + } + + boolean isFirstRow = true; while (queryDataSet.hasNext()) { final RowRecord rowRecord = queryDataSet.next(); + if (isFirstRow) { + // Calculate row count and memory size of the tablet based on the first row + Pair rowCountAndMemorySize = + PipeMemoryWeightUtil.calculateTabletRowCountAndMemory(rowRecord); + tablet = + new Tablet( + // Used for tree model + deviceId.toString(), schemas, rowCountAndMemorySize.getLeft()); + tablet.initBitMaps(); + PipeDataNodeResourceManager.memory() + .forceResize(allocatedBlockForTablet, rowCountAndMemorySize.getRight()); + isFirstRow = false; + } final int rowIndex = tablet.rowSize; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java index 8dc4e6a07e72..3151c9512297 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java @@ -19,12 +19,13 @@ package org.apache.iotdb.db.pipe.event.common.tsfile.container.scan; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainer; +import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.pipe.api.exception.PipeException; @@ -44,6 +45,7 @@ import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.DateUtils; +import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.TsPrimitiveType; import org.apache.tsfile.write.UnSupportedDataTypeException; import org.apache.tsfile.write.record.Tablet; @@ -175,15 +177,32 @@ public Tablet next() { private Tablet getNextTablet() { try { - final Tablet tablet = - new Tablet( - currentDevice.toString(), - currentMeasurements, - PipeConfig.getInstance().getPipeDataStructureTabletRowSize()); - tablet.initBitMaps(); + Tablet tablet = null; + if (!data.hasCurrent()) { + tablet = new Tablet(currentDevice.toString(), currentMeasurements, 1); + tablet.initBitMaps(); + // Ignore the memory cost of tablet + PipeDataNodeResourceManager.memory().forceResize(allocatedMemoryBlockForTablet, 0); + return tablet; + } + + boolean isFirstRow = true; while (data.hasCurrent()) { if (isMultiPage || data.currentTime() >= startTime && data.currentTime() <= endTime) { + if (isFirstRow) { + // Calculate row count and memory size of the tablet based on the first row + Pair rowCountAndMemorySize = + PipeMemoryWeightUtil.calculateTabletRowCountAndMemory(data); + tablet = + new Tablet( + currentDevice.toString(), currentMeasurements, rowCountAndMemorySize.getLeft()); + tablet.initBitMaps(); + PipeDataNodeResourceManager.memory() + .forceResize(allocatedMemoryBlockForTablet, rowCountAndMemorySize.getRight()); + isFirstRow = false; + } + final int rowIndex = tablet.rowSize; tablet.addTimestamp(rowIndex, data.currentTime()); @@ -197,16 +216,22 @@ private Tablet getNextTablet() { data = chunkReader.nextPageData(); } - if (tablet.rowSize == tablet.getMaxRowNumber()) { + if (tablet != null && tablet.rowSize == tablet.getMaxRowNumber()) { break; } } + if (tablet == null) { + tablet = new Tablet(currentDevice.toString(), currentMeasurements, 1); + tablet.initBitMaps(); + // Ignore the memory cost of tablet + PipeDataNodeResourceManager.memory().forceResize(allocatedMemoryBlockForTablet, 0); + } + // Switch chunk reader iff current chunk is all consumed if (!data.hasCurrent()) { prepareData(); } - return tablet; } catch (final Exception e) { close(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java index bf5db9d0bb2c..4763fcc71b4f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java @@ -24,7 +24,6 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; -import org.apache.tsfile.write.record.Tablet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,7 +74,7 @@ public synchronized PipeMemoryBlock forceAllocate(long sizeInBytes) return forceAllocate(sizeInBytes, false); } - public PipeTabletMemoryBlock forceAllocateWithRetry(Tablet tablet) + public PipeTabletMemoryBlock forceAllocateForTabletWithRetry(long tabletSizeInBytes) throws PipeRuntimeOutOfMemoryCriticalException { if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { // No need to calculate the tablet size, skip it to save time @@ -107,8 +106,7 @@ public PipeTabletMemoryBlock forceAllocateWithRetry(Tablet tablet) synchronized (this) { final PipeTabletMemoryBlock block = - (PipeTabletMemoryBlock) - forceAllocate(PipeMemoryWeightUtil.calculateTabletSizeInBytes(tablet), true); + (PipeTabletMemoryBlock) forceAllocate(tabletSizeInBytes, true); usedMemorySizeInBytesOfTablets += block.getMemoryUsageInBytes(); return block; } @@ -147,6 +145,59 @@ private PipeMemoryBlock forceAllocate(long sizeInBytes, boolean isForTablet) sizeInBytes)); } + public synchronized void forceResize(PipeMemoryBlock block, long targetSize) { + if (block == null || block.isReleased()) { + LOGGER.warn("forceResize: cannot resize a null or released memory block"); + return; + } + + if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { + block.setMemoryUsageInBytes(targetSize); + return; + } + + final long oldSize = block.getMemoryUsageInBytes(); + + if (oldSize >= targetSize) { + usedMemorySizeInBytes -= oldSize - targetSize; + if (block instanceof PipeTabletMemoryBlock) { + usedMemorySizeInBytesOfTablets -= oldSize - targetSize; + } + block.setMemoryUsageInBytes(targetSize); + return; + } + + long sizeInBytes = targetSize - oldSize; + for (int i = 1; i <= MEMORY_ALLOCATE_MAX_RETRIES; i++) { + if (TOTAL_MEMORY_SIZE_IN_BYTES - usedMemorySizeInBytes >= sizeInBytes) { + usedMemorySizeInBytes += sizeInBytes; + if (block instanceof PipeTabletMemoryBlock) { + usedMemorySizeInBytesOfTablets += sizeInBytes; + } + block.setMemoryUsageInBytes(targetSize); + return; + } + + try { + tryShrink4Allocate(sizeInBytes); + this.wait(MEMORY_ALLOCATE_RETRY_INTERVAL_IN_MS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("forceResize: interrupted while waiting for available memory", e); + } + } + + throw new PipeRuntimeOutOfMemoryCriticalException( + String.format( + "forceResize: failed to allocate memory after %d retries, " + + "total memory size %d bytes, used memory size %d bytes, " + + "requested memory size %d bytes", + MEMORY_ALLOCATE_MAX_RETRIES, + TOTAL_MEMORY_SIZE_IN_BYTES, + usedMemorySizeInBytes, + sizeInBytes)); + } + /** * Allocate a {@link PipeMemoryBlock} for pipe only if memory already used is less than the * specified threshold. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java index ff9c996c6a2e..ed1eab789290 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java @@ -19,11 +19,18 @@ package org.apache.iotdb.db.pipe.resource.memory; +import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.db.pipe.event.common.row.PipeRow; import org.apache.iotdb.db.utils.MemUtils; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.common.BatchData; +import org.apache.tsfile.read.common.Field; +import org.apache.tsfile.read.common.RowRecord; import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.TsPrimitiveType; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -62,6 +69,127 @@ public static long memoryOfIDeviceID2StrList(Map> map) { return usageInBytes + 16L; // add the overhead of map } + /** + * Given a row of a tablet, calculate the row count and memory cost of the pipe tablet that will + * be constructed according to config. + * + * @return left is the row count of tablet, right is the memory cost of tablet in bytes + */ + public static Pair calculateTabletRowCountAndMemory(RowRecord row) { + int totalSizeInBytes = 0; + + // timestamp + totalSizeInBytes += 8L; + + // values + final List fields = row.getFields(); + int schemaCount = 0; + if (fields != null) { + schemaCount = fields.size(); + for (final Field field : fields) { + if (field == null) { + continue; + } + + final TSDataType tsDataType = field.getDataType(); + if (tsDataType == null) { + continue; + } + + if (tsDataType.isBinary()) { + final Binary binary = field.getBinaryV(); + totalSizeInBytes += binary == null ? 0 : binary.getLength(); + } else { + totalSizeInBytes += tsDataType.getDataTypeSize(); + } + } + } + + return calculateTabletRowCountAndMemoryBySize(totalSizeInBytes, schemaCount); + } + + /** + * Given a BatchData, calculate the row count and memory cost of the pipe tablet that will be + * constructed according to config. + * + * @return left is the row count of tablet, right is the memory cost of tablet in bytes + */ + public static Pair calculateTabletRowCountAndMemory(BatchData batchData) { + int totalSizeInBytes = 0; + int schemaCount = 0; + + // timestamp + totalSizeInBytes += 8L; + + // values + final TSDataType type = batchData.getDataType(); + if (type != null) { + if (type == TSDataType.VECTOR && batchData.getVector() != null) { + schemaCount = batchData.getVector().length; + for (int i = 0; i < schemaCount; ++i) { + final TsPrimitiveType primitiveType = batchData.getVector()[i]; + if (primitiveType == null || primitiveType.getDataType() == null) { + continue; + } + + if (primitiveType.getDataType().isBinary()) { + final Binary binary = primitiveType.getBinary(); + totalSizeInBytes += binary == null ? 0 : binary.getLength(); + } else { + totalSizeInBytes += primitiveType.getDataType().getDataTypeSize(); + } + } + } else { + schemaCount = 1; + if (type.isBinary()) { + final Binary binary = batchData.getBinary(); + totalSizeInBytes += binary == null ? 0 : binary.getLength(); + } else { + totalSizeInBytes += type.getDataTypeSize(); + } + } + } + + return calculateTabletRowCountAndMemoryBySize(totalSizeInBytes, schemaCount); + } + + /** + * Given a row of a tablet, calculate the row count and memory cost of the pipe tablet that will + * be constructed according to config. + * + * @return left is the row count of tablet, right is the memory cost of tablet in bytes + */ + public static Pair calculateTabletRowCountAndMemory(PipeRow row) { + return calculateTabletRowCountAndMemoryBySize(row.getCurrentRowSize(), row.size()); + } + + private static Pair calculateTabletRowCountAndMemoryBySize( + int rowSize, int schemaCount) { + if (rowSize <= 0) { + return new Pair<>(1, 0); + } + + // Calculate row number according to the max size of a pipe tablet. + // "-100" is the estimated size of other data structures in a pipe tablet. + // "*8" converts bytes to bits, because the bitmap size is 1 bit per schema. + int rowNumber = + 8 + * (PipeConfig.getInstance().getPipeDataStructureTabletSizeInBytes() - 100) + / (8 * rowSize + schemaCount); + rowNumber = Math.max(1, rowNumber); + + if ( // This means the row number is larger than the max row count of a pipe tablet + rowNumber > PipeConfig.getInstance().getPipeDataStructureTabletRowSize()) { + // Bound the row number, the memory cost is rowSize * rowNumber + return new Pair<>( + PipeConfig.getInstance().getPipeDataStructureTabletRowSize(), + rowSize * PipeConfig.getInstance().getPipeDataStructureTabletRowSize()); + } else { + return new Pair<>( + rowNumber, PipeConfig.getInstance().getPipeDataStructureTabletSizeInBytes()); + } + } + public static long calculateTabletSizeInBytes(Tablet tablet) { long totalSizeInBytes = 0; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index ef8f013f7a13..1a5505737cca 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -202,6 +202,7 @@ public class CommonConfig { private int pipeNonForwardingEventsProgressReportInterval = 100; private int pipeDataStructureTabletRowSize = 2048; + private int pipeDataStructureTabletSizeInBytes = 2097152; private double pipeDataStructureTabletMemoryBlockAllocationRejectThreshold = 0.4; private int pipeSubtaskExecutorBasicCheckPointIntervalByConsumedEventCount = 10_000; @@ -674,6 +675,14 @@ public void setPipeDataStructureTabletRowSize(int pipeDataStructureTabletRowSize this.pipeDataStructureTabletRowSize = pipeDataStructureTabletRowSize; } + public int getPipeDataStructureTabletSizeInBytes() { + return pipeDataStructureTabletSizeInBytes; + } + + public void setPipeDataStructureTabletSizeInBytes(int pipeDataStructureTabletSizeInBytes) { + this.pipeDataStructureTabletSizeInBytes = pipeDataStructureTabletSizeInBytes; + } + public double getPipeDataStructureTabletMemoryBlockAllocationRejectThreshold() { return pipeDataStructureTabletMemoryBlockAllocationRejectThreshold; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java index 76e0a853dc20..310400b33ce6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java @@ -275,6 +275,11 @@ private void loadPipeProps(Properties properties) { properties.getProperty( "pipe_data_structure_tablet_row_size", String.valueOf(config.getPipeDataStructureTabletRowSize())))); + config.setPipeDataStructureTabletSizeInBytes( + Integer.parseInt( + properties.getProperty( + "pipe_data_structure_tablet_size_in_bytes", + String.valueOf(config.getPipeDataStructureTabletSizeInBytes())))); config.setPipeDataStructureTabletMemoryBlockAllocationRejectThreshold( Double.parseDouble( properties.getProperty( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 449bd149e671..957065286cca 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -60,6 +60,10 @@ public int getPipeDataStructureTabletRowSize() { return COMMON_CONFIG.getPipeDataStructureTabletRowSize(); } + public int getPipeDataStructureTabletSizeInBytes() { + return COMMON_CONFIG.getPipeDataStructureTabletSizeInBytes(); + } + public double getPipeDataStructureTabletMemoryBlockAllocationRejectThreshold() { return COMMON_CONFIG.getPipeDataStructureTabletMemoryBlockAllocationRejectThreshold(); } @@ -325,6 +329,7 @@ public void printAllConfigs() { LOGGER.info("PipeHardLinkWALEnabled: {}", getPipeHardLinkWALEnabled()); LOGGER.info("PipeDataStructureTabletRowSize: {}", getPipeDataStructureTabletRowSize()); + LOGGER.info("PipeDataStructureTabletSizeInBytes: {}", getPipeDataStructureTabletSizeInBytes()); LOGGER.info( "PipeDataStructureTabletMemoryBlockAllocationRejectThreshold: {}", getPipeDataStructureTabletMemoryBlockAllocationRejectThreshold()); From f56f388fbbeaaa8a798622d03d4c2eb205910565 Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Mon, 2 Sep 2024 18:40:34 +0800 Subject: [PATCH 04/13] Pipe IT: req size automatic fragmentation integration test (#13342) --- .../env/cluster/config/MppCommonConfig.java | 10 + .../cluster/config/MppSharedCommonConfig.java | 8 + .../env/remote/config/RemoteCommonConfig.java | 6 + .../apache/iotdb/itbase/env/CommonConfig.java | 3 + .../it/autocreate/AbstractPipeDualAutoIT.java | 8 +- .../it/manual/AbstractPipeDualManualIT.java | 8 +- .../it/manual/IoTDBPipeReqAutoSliceIT.java | 482 ++++++++++++++++++ 7 files changed, 519 insertions(+), 6 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeReqAutoSliceIT.java diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java index 741d02d9bba0..1760301ac737 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java @@ -474,6 +474,16 @@ public CommonConfig setPipeMetaSyncerSyncIntervalMinutes(long pipeMetaSyncerSync return this; } + @Override + public CommonConfig setPipeConnectorRequestSliceThresholdBytes( + int pipeConnectorRequestSliceThresholdBytes) { + setProperty( + "pipe_connector_request_slice_threshold_bytes", + String.valueOf(pipeConnectorRequestSliceThresholdBytes)); + + return this; + } + // For part of the log directory public String getClusterConfigStr() { return fromConsensusFullNameToAbbr(properties.getProperty(CONFIG_NODE_CONSENSUS_PROTOCOL_CLASS)) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java index 7278fc0fb9ec..969d4bb8d41c 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java @@ -483,4 +483,12 @@ public CommonConfig setPipeMetaSyncerSyncIntervalMinutes(long pipeMetaSyncerSync cnConfig.setPipeMetaSyncerSyncIntervalMinutes(pipeMetaSyncerSyncIntervalMinutes); return this; } + + @Override + public CommonConfig setPipeConnectorRequestSliceThresholdBytes( + int pipeConnectorRequestSliceThresholdBytes) { + dnConfig.setPipeConnectorRequestSliceThresholdBytes(pipeConnectorRequestSliceThresholdBytes); + cnConfig.setPipeConnectorRequestSliceThresholdBytes(pipeConnectorRequestSliceThresholdBytes); + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java index 238a85a3a4c9..cee8aadf05f9 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java @@ -339,4 +339,10 @@ public CommonConfig setPipeMetaSyncerInitialSyncDelayMinutes( public CommonConfig setPipeMetaSyncerSyncIntervalMinutes(long pipeMetaSyncerSyncIntervalMinutes) { return this; } + + @Override + public CommonConfig setPipeConnectorRequestSliceThresholdBytes( + int pipeConnectorRequestSliceThresholdBytes) { + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java index d58c846f39e2..d806be8db1dc 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java @@ -150,4 +150,7 @@ CommonConfig setPipeHeartbeatIntervalSecondsForCollectingPipeMeta( CommonConfig setPipeMetaSyncerInitialSyncDelayMinutes(long pipeMetaSyncerInitialSyncDelayMinutes); CommonConfig setPipeMetaSyncerSyncIntervalMinutes(long pipeMetaSyncerSyncIntervalMinutes); + + CommonConfig setPipeConnectorRequestSliceThresholdBytes( + int pipeConnectorRequestSliceThresholdBytes); } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java index a0a27791a318..f2d3ec8cf6e1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java @@ -36,7 +36,12 @@ public void setUp() { MultiEnvFactory.createEnv(2); senderEnv = MultiEnvFactory.getEnv(0); receiverEnv = MultiEnvFactory.getEnv(1); + setupConfig(); + senderEnv.initClusterEnvironment(); + receiverEnv.initClusterEnvironment(); + } + protected void setupConfig() { // TODO: delete ratis configurations senderEnv .getConfig() @@ -54,9 +59,6 @@ public void setUp() { // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); - - senderEnv.initClusterEnvironment(); - receiverEnv.initClusterEnvironment(); } @After diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java index ae81f1ffb20a..cdca06025fed 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java @@ -36,7 +36,12 @@ public void setUp() { MultiEnvFactory.createEnv(2); senderEnv = MultiEnvFactory.getEnv(0); receiverEnv = MultiEnvFactory.getEnv(1); + setupConfig(); + senderEnv.initClusterEnvironment(); + receiverEnv.initClusterEnvironment(); + } + protected void setupConfig() { // TODO: delete ratis configurations senderEnv .getConfig() @@ -54,9 +59,6 @@ public void setUp() { // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); - - senderEnv.initClusterEnvironment(); - receiverEnv.initClusterEnvironment(); } @After diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeReqAutoSliceIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeReqAutoSliceIT.java new file mode 100644 index 000000000000..7ff87375905e --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeReqAutoSliceIT.java @@ -0,0 +1,482 @@ +/* + * 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.pipe.it.manual; + +import org.apache.iotdb.commons.utils.function.CheckedTriConsumer; +import org.apache.iotdb.db.it.utils.TestUtils; +import org.apache.iotdb.isession.ISession; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.MultiClusterIT2ManualCreateSchema; +import org.apache.iotdb.rpc.IoTDBConnectionException; +import org.apache.iotdb.rpc.StatementExecutionException; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.utils.Pair; +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.Ignore; +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.HashSet; +import java.util.List; +import java.util.Random; + +@RunWith(IoTDBTestRunner.class) +@Category({MultiClusterIT2ManualCreateSchema.class}) +public class IoTDBPipeReqAutoSliceIT extends AbstractPipeDualManualIT { + private static final int generateDataSize = 10; + + @Override + protected void setupConfig() { + super.setupConfig(); + senderEnv.getConfig().getCommonConfig().setPipeConnectorRequestSliceThresholdBytes(4); + receiverEnv.getConfig().getCommonConfig().setPipeConnectorRequestSliceThresholdBytes(4); + } + + @Test + public void insertTablet() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + senderSession.insertTablet(tablet); + }, + false); + } + + @Ignore + @Test + public void insertTabletReceiveByTsFile() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + senderSession.insertTablet(tablet); + }, + true); + } + + @Ignore + @Test + public void insertAlignedTablet() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + senderSession.insertAlignedTablet(tablet); + }, + false); + } + + @Ignore + @Test + public void insertAlignedTabletReceiveByTsFile() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + senderSession.insertAlignedTablet(tablet); + }, + true); + } + + @Ignore + @Test + public void insertRecordsReceiveByTsFile() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertRecordForTable(tablet); + senderSession.insertRecords( + getDeviceID(tablet), timestamps, pair.left, pair.right, values); + }, + true); + } + + @Ignore + @Test + public void insertRecord() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertRecordForTable(tablet); + for (int i = 0; i < values.size(); i++) { + senderSession.insertRecord( + tablet.getDeviceId(), + timestamps.get(i), + pair.left.get(i), + pair.right.get(i), + values.get(i).toArray()); + } + }, + false); + } + + @Ignore + @Test + public void insertRecordReceiveByTsFile() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertRecordForTable(tablet); + for (int i = 0; i < values.size(); i++) { + senderSession.insertRecord( + tablet.getDeviceId(), + timestamps.get(i), + pair.left.get(i), + pair.right.get(i), + values.get(i).toArray()); + } + }, + true); + } + + @Ignore + @Test + public void insertAlignedRecord() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertRecordForTable(tablet); + for (int i = 0; i < values.size(); i++) { + senderSession.insertAlignedRecord( + tablet.getDeviceId(), + timestamps.get(i), + pair.left.get(i), + pair.right.get(i), + values.get(i)); + } + }, + false); + } + + @Ignore + @Test + public void insertAlignedRecordReceiveByTsFile() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertRecordForTable(tablet); + for (int i = 0; i < values.size(); i++) { + senderSession.insertAlignedRecord( + tablet.getDeviceId(), + timestamps.get(i), + pair.left.get(i), + pair.right.get(i), + values.get(i)); + } + }, + true); + } + + @Ignore + @Test + public void insertRecords() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertRecordForTable(tablet); + senderSession.insertRecords( + getDeviceID(tablet), timestamps, pair.left, pair.right, values); + }, + false); + } + + @Ignore + @Test + public void insertAlignedRecords() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertRecordForTable(tablet); + senderSession.insertAlignedRecords( + getDeviceID(tablet), timestamps, pair.left, pair.right, values); + }, + false); + } + + @Ignore + @Test + public void insertAlignedRecordsReceiveByTsFile() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertRecordForTable(tablet); + senderSession.insertAlignedRecords( + getDeviceID(tablet), timestamps, pair.left, pair.right, values); + }, + true); + } + + @Ignore + @Test + public void insertStringRecordsOfOneDevice() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertStrRecordForTable(tablet); + senderSession.insertStringRecordsOfOneDevice( + tablet.getDeviceId(), timestamps, pair.left, values); + }, + false); + } + + @Ignore + @Test + public void insertStringRecordsOfOneDeviceReceiveByTsFile() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertStrRecordForTable(tablet); + senderSession.insertStringRecordsOfOneDevice( + tablet.getDeviceId(), timestamps, pair.left, values); + }, + true); + } + + @Ignore + @Test + public void insertAlignedStringRecordsOfOneDevice() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertStrRecordForTable(tablet); + senderSession.insertAlignedStringRecordsOfOneDevice( + tablet.getDeviceId(), timestamps, pair.left, values); + }, + false); + } + + @Ignore + @Test + public void insertAlignedStringRecordsOfOneDeviceReceiveByTsFile() { + prepareReqAutoSliceTest( + (ISession senderSession, ISession receiverSession, Tablet tablet) -> { + List timestamps = getTimestampList(tablet); + Pair>, List>> pair = + getMeasurementSchemasAndType(tablet); + List> values = generateTabletInsertStrRecordForTable(tablet); + senderSession.insertAlignedStringRecordsOfOneDevice( + tablet.getDeviceId(), timestamps, pair.left, values); + }, + true); + } + + private void prepareReqAutoSliceTest( + CheckedTriConsumer consumer, boolean isTsFile) { + Tablet tablet = createTablet(); + createTimeSeries(); + try (ISession senderSession = senderEnv.getSessionConnection(); + ISession receiverSession = receiverEnv.getSessionConnection()) { + if (isTsFile) { + consumer.accept(senderSession, receiverSession, tablet); + senderSession.executeNonQueryStatement("flush"); + Thread.sleep(2000); + createPipe(senderSession, true); + } else { + createPipe(senderSession, false); + Thread.sleep(2000); + consumer.accept(senderSession, receiverSession, tablet); + senderSession.executeNonQueryStatement("flush"); + } + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(); + } + verify(tablet); + } + + private void createPipe(ISession session, boolean isTsFile) + throws IoTDBConnectionException, StatementExecutionException { + session.executeNonQueryStatement( + String.format( + "create pipe test" + + " with source ('source'='iotdb-source','source.path'='root.test.**')" + + " with sink ('sink'='iotdb-thrift-sync-sink','node-urls'='%s:%s','batch.enable'='false','sink.format'='%s')", + receiverEnv.getIP(), receiverEnv.getPort(), isTsFile ? "tsfile" : "tablet")); + } + + private int[] createTestDataForInt32() { + int[] data = new int[generateDataSize]; + Random random = new Random(); + for (int i = 0; i < generateDataSize; i++) { + data[i] = random.nextInt(); + } + return data; + } + + private long[] createTestDataForInt64() { + long[] data = new long[generateDataSize]; + long time = System.currentTimeMillis(); + for (int i = 0; i < generateDataSize; i++) { + data[i] = time + i; + } + return data; + } + + private void verify(Tablet tablet) { + HashSet set = new HashSet<>(); + for (int i = 0; i < generateDataSize; i++) { + set.add( + String.format( + "%d,%d,%d,", + tablet.timestamps[i], ((int[]) tablet.values[0])[i], ((int[]) tablet.values[1])[i])); + } + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select * from root.test.** ORDER BY time ASC", + "Time,root.test.db.temperature,root.test.db.status,", + set, + 20); + } + + private void createTimeSeries() { + List timeSeriesCreation = + Arrays.asList( + "create timeseries root.test.db.status with datatype=INT32,encoding=PLAIN", + "create timeseries root.test.db.temperature with datatype=INT32,encoding=PLAIN"); + TestUtils.tryExecuteNonQueriesWithRetry(senderEnv, timeSeriesCreation); + TestUtils.tryExecuteNonQueriesWithRetry(receiverEnv, timeSeriesCreation); + } + + private Tablet createTablet() { + long[] timestamp = createTestDataForInt64(); + int[] temperature = createTestDataForInt32(); + int[] status = createTestDataForInt32(); + + Object[] objects = new Object[2]; + objects[0] = temperature; + objects[1] = status; + + List measurementSchemas = new ArrayList<>(2); + measurementSchemas.add(new MeasurementSchema("temperature", TSDataType.INT32)); + measurementSchemas.add(new MeasurementSchema("status", TSDataType.INT32)); + + BitMap[] bitMaps = new BitMap[2]; + for (int i = 0; i < bitMaps.length; i++) { + bitMaps[i] = new BitMap(generateDataSize); + } + + return new Tablet( + "root.test.db", measurementSchemas, timestamp, objects, bitMaps, generateDataSize); + } + + private List getTimestampList(Tablet tablet) { + long[] timestamps = tablet.timestamps; + List data = new ArrayList<>(timestamps.length); + for (long timestamp : timestamps) { + data.add(timestamp); + } + return data; + } + + private Pair>, List>> getMeasurementSchemasAndType( + Tablet tablet) { + List> schemaData = new ArrayList<>(tablet.rowSize); + List> typeData = new ArrayList<>(tablet.rowSize); + List measurementSchemas = new ArrayList<>(tablet.getSchemas().size()); + List types = new ArrayList<>(tablet.rowSize); + for (IMeasurementSchema measurementSchema : tablet.getSchemas()) { + measurementSchemas.add(measurementSchema.getMeasurementId()); + types.add(measurementSchema.getType()); + } + + for (int i = 0; i < tablet.rowSize; i++) { + schemaData.add(measurementSchemas); + typeData.add(types); + } + + return new Pair<>(schemaData, typeData); + } + + private List getDeviceID(Tablet tablet) { + List data = new ArrayList<>(tablet.rowSize); + for (int i = 0; i < tablet.rowSize; i++) { + data.add(tablet.getDeviceId()); + } + return data; + } + + private List> generateTabletInsertRecordForTable(final Tablet tablet) { + List> insertRecords = new ArrayList<>(tablet.rowSize); + final List schemas = tablet.getSchemas(); + final Object[] values = tablet.values; + for (int i = 0; i < tablet.rowSize; i++) { + List insertRecord = new ArrayList<>(); + for (int j = 0; j < schemas.size(); j++) { + switch (schemas.get(j).getType()) { + case INT64: + case TIMESTAMP: + insertRecord.add(((long[]) values[j])[i]); + break; + case INT32: + insertRecord.add(((int[]) values[j])[i]); + break; + } + } + insertRecords.add(insertRecord); + } + + return insertRecords; + } + + private List> generateTabletInsertStrRecordForTable(Tablet tablet) { + List> insertRecords = new ArrayList<>(tablet.rowSize); + final List schemas = tablet.getSchemas(); + final Object[] values = tablet.values; + for (int i = 0; i < tablet.rowSize; i++) { + List insertRecord = new ArrayList<>(); + for (int j = 0; j < schemas.size(); j++) { + switch (schemas.get(j).getType()) { + case INT64: + insertRecord.add(String.valueOf(((long[]) values[j])[i])); + break; + case INT32: + insertRecord.add(String.valueOf(((int[]) values[j])[i])); + break; + } + } + insertRecords.add(insertRecord); + } + + return insertRecords; + } +} From 37c58c8284054364f1e9a2740f8f3c310b8bff27 Mon Sep 17 00:00:00 2001 From: V_Galaxy Date: Mon, 2 Sep 2024 18:48:10 +0800 Subject: [PATCH 05/13] Pipe: avoid logging warn when clearing event ref after releasing the event & minor improve for javadoc (#13371) --- .../apache/iotdb/commons/pipe/event/EnrichedEvent.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java index 57a2ee6a6eba..2dc88493c216 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java @@ -103,8 +103,7 @@ protected EnrichedEvent( * * @param holderMessage the message of the invoker * @return {@code true} if the {@link EnrichedEvent#referenceCount} is increased successfully, - * {@code false} otherwise; {@link EnrichedEvent#referenceCount} will be incremented - * regardless of the circumstances + * {@code false} otherwise */ public synchronized boolean increaseReferenceCount(final String holderMessage) { boolean isSuccessful = true; @@ -156,8 +155,7 @@ public synchronized boolean increaseReferenceCount(final String holderMessage) { * * @param holderMessage the message of the invoker * @return {@code true} if the {@link EnrichedEvent#referenceCount} is decreased successfully, - * {@code false} otherwise; {@link EnrichedEvent#referenceCount} will be decremented - * regardless of the circumstances + * {@code false} otherwise */ public synchronized boolean decreaseReferenceCount( final String holderMessage, final boolean shouldReport) { @@ -222,10 +220,6 @@ public synchronized boolean decreaseReferenceCount( */ public synchronized boolean clearReferenceCount(final String holderMessage) { if (isReleased.get()) { - LOGGER.warn( - "clear reference count to event that has already been released: {}, stack trace: {}", - coreReportMessage(), - Thread.currentThread().getStackTrace()); return false; } From ed9d6c0da8bd03e259e9b0bbd692070964fd017c Mon Sep 17 00:00:00 2001 From: Emmanuel Ferdman Date: Mon, 2 Sep 2024 16:29:40 +0300 Subject: [PATCH 06/13] Update iotdb client example reference (#13386) Signed-off-by: Emmanuel Ferdman --- iotdb-client/client-py/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-client/client-py/README.md b/iotdb-client/client-py/README.md index ec0c5a65de99..d6e837dd7eda 100644 --- a/iotdb-client/client-py/README.md +++ b/iotdb-client/client-py/README.md @@ -47,7 +47,7 @@ You have to install thrift (>=0.14.1) before using the package. First, download the latest package: `pip3 install apache-iotdb` -You can get an example of using the package to read and write data at here: [Example](https://github.com/apache/iotdb/blob/master/client-py/SessionExample.py) +You can get an example of using the package to read and write data at here: [Example](https://github.com/apache/iotdb/blob/master/iotdb-client/client-py/SessionExample.py) An example of aligned timeseries: [Aligned Timeseries Session Example](https://github.com/apache/iotdb/blob/master/client-py/SessionAlignedTimeseriesExample.py) From 334c457438bc5aac6b90506f144f401a3922f848 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:09:36 +0800 Subject: [PATCH 07/13] Pipe: Fixed the bug that OPCUASink's model effect is reverted (#13377) --- .../iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java index 1885f1816771..ea3b1184f55b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java @@ -153,7 +153,7 @@ public void customize( Arrays.asList( CONNECTOR_OPC_UA_MODEL_KEY, SINK_OPC_UA_MODEL_KEY), CONNECTOR_OPC_UA_MODEL_DEFAULT_VALUE) - .equals(CONNECTOR_OPC_UA_MODEL_DEFAULT_VALUE)); + .equals(CONNECTOR_OPC_UA_MODEL_CLIENT_SERVER_VALUE)); nameSpace.startup(); newServer.startup().get(); return new Pair<>(new AtomicInteger(0), nameSpace); From 9460661d2564dba2d4ad14a3bc573fccc26640bd Mon Sep 17 00:00:00 2001 From: Jiang Tian Date: Tue, 3 Sep 2024 10:42:36 +0800 Subject: [PATCH 08/13] Fix WAL Exception catch & support new DataTypes in QueryDataSetUtils (#13359) * Fix WAL Exception catch & support new DataTypes in QueryDataSetUtils * fix review * fix review --- .../java/org/apache/iotdb/TabletExample.java | 4 + .../org/apache/iotdb/udf/UDAFExample.java | 4 + .../iotdb/db/query/udf/example/UDAFAvg.java | 4 + .../iotdb/db/query/udf/example/UDAFSum.java | 4 + .../org/apache/iotdb/tool/ImportData.java | 3 + .../jdbc/GroupedLSBWatermarkEncoder.java | 4 + .../apache/iotdb/db/conf/IoTDBDescriptor.java | 3 + ...CustomizedReadableIntermediateResults.java | 16 ++++ .../AbstractSameTypeNumericOperator.java | 18 ++++ .../aggregation/AccumulatorFactory.java | 4 + .../execution/aggregation/AvgAccumulator.java | 4 + .../aggregation/ExtremeAccumulator.java | 24 ++++++ .../aggregation/MaxMinByBaseAccumulator.java | 2 + .../aggregation/MaxValueAccumulator.java | 6 ++ .../aggregation/MinValueAccumulator.java | 6 ++ .../execution/aggregation/SumAccumulator.java | 4 + .../aggregation/VarianceAccumulator.java | 4 + .../operator/window/WindowManagerFactory.java | 10 +++ .../dag/column/unary/InColumnTransformer.java | 86 +------------------ .../unary/scalar/RoundColumnTransformer.java | 6 ++ .../RoundFunctionColumnTransformer.java | 6 ++ .../dag/input/ConstantInputReader.java | 4 + .../unary/ArithmeticNegationTransformer.java | 6 ++ .../dag/transformer/unary/InTransformer.java | 2 + .../unary/scalar/CastFunctionTransformer.java | 34 ++++++++ .../unary/scalar/DiffFunctionTransformer.java | 6 ++ .../scalar/RoundFunctionTransformer.java | 6 ++ .../dag/util/TransformUtils.java | 8 ++ .../transformation/dag/util/TypeUtils.java | 3 + .../dataregion/wal/io/WALReader.java | 12 ++- .../iotdb/db/utils/QueryDataSetUtils.java | 4 + .../dataregion/wal/io/WALFileTest.java | 40 ++++++--- .../iotdb/commons/udf/builtin/UDTFAbs.java | 20 +++++ .../commons/udf/builtin/UDTFBottomK.java | 2 + .../commons/udf/builtin/UDTFChangePoints.java | 6 ++ .../udf/builtin/UDTFCommonDerivative.java | 6 ++ .../builtin/UDTFCommonValueDifference.java | 6 ++ .../udf/builtin/UDTFContinuouslySatisfy.java | 10 +++ .../builtin/UDTFEqualSizeBucketAggSample.java | 6 ++ .../builtin/UDTFEqualSizeBucketM4Sample.java | 6 ++ .../UDTFEqualSizeBucketOutlierSample.java | 6 ++ .../UDTFEqualSizeBucketRandomSample.java | 6 ++ .../commons/udf/builtin/UDTFInRange.java | 18 ++++ .../iotdb/commons/udf/builtin/UDTFJexl.java | 15 ++++ .../iotdb/commons/udf/builtin/UDTFM4.java | 6 ++ .../iotdb/commons/udf/builtin/UDTFMath.java | 18 ++++ .../builtin/UDTFNonNegativeDerivative.java | 6 ++ .../UDTFNonNegativeValueDifference.java | 6 ++ .../iotdb/commons/udf/builtin/UDTFOnOff.java | 18 ++++ .../commons/udf/builtin/UDTFSelectK.java | 4 + .../iotdb/commons/udf/builtin/UDTFTopK.java | 2 + .../commons/udf/builtin/UDTFValueTrend.java | 6 ++ .../commons/udf/utils/MasterRepairUtil.java | 6 ++ .../iotdb/commons/utils/SerializeUtils.java | 24 ++++++ .../iotdb/library/anomaly/UDTFKSigma.java | 12 +++ .../iotdb/library/anomaly/UDTFRange.java | 6 ++ .../library/anomaly/UDTFTwoSidedFilter.java | 6 ++ .../library/dprofile/UDAFPercentile.java | 24 ++++++ .../iotdb/library/dprofile/UDAFQuantile.java | 18 ++++ .../iotdb/library/dprofile/UDAFSpread.java | 12 +++ .../iotdb/library/dprofile/UDTFDistinct.java | 16 ++++ .../iotdb/library/dprofile/UDTFSample.java | 6 ++ .../dprofile/util/ExactOrderStatistics.java | 30 +++++++ .../library/drepair/UDTFTimestampRepair.java | 6 ++ .../iotdb/library/drepair/UDTFValueFill.java | 6 ++ .../library/drepair/UDTFValueRepair.java | 7 ++ .../frequency/UDFEnvelopeAnalysis.java | 6 ++ .../org/apache/iotdb/library/util/Util.java | 15 ++++ 68 files changed, 621 insertions(+), 99 deletions(-) diff --git a/example/session/src/main/java/org/apache/iotdb/TabletExample.java b/example/session/src/main/java/org/apache/iotdb/TabletExample.java index 43e5459a22f4..39f1223efcad 100644 --- a/example/session/src/main/java/org/apache/iotdb/TabletExample.java +++ b/example/session/src/main/java/org/apache/iotdb/TabletExample.java @@ -90,9 +90,11 @@ private static Map> loadCSVData( case BOOLEAN: ret.get(measurement).add(Boolean.parseBoolean(items[idx])); break; + case DATE: case INT32: ret.get(measurement).add(Integer.parseInt(items[idx])); break; + case TIMESTAMP: case INT64: ret.get(measurement).add(Long.parseLong(items[idx])); break; @@ -102,6 +104,8 @@ private static Map> loadCSVData( case DOUBLE: ret.get(measurement).add(Double.parseDouble(items[idx])); break; + case STRING: + case BLOB: case TEXT: ret.get(measurement).add(BytesUtils.valueOf(items[idx])); break; diff --git a/example/udf/src/main/java/org/apache/iotdb/udf/UDAFExample.java b/example/udf/src/main/java/org/apache/iotdb/udf/UDAFExample.java index b2b9bec5f1ba..540f6bcef5cc 100644 --- a/example/udf/src/main/java/org/apache/iotdb/udf/UDAFExample.java +++ b/example/udf/src/main/java/org/apache/iotdb/udf/UDAFExample.java @@ -110,6 +110,10 @@ public void addInput(State state, Column[] columns, BitMap bitMap) { return; case TEXT: case BOOLEAN: + case TIMESTAMP: + case STRING: + case BLOB: + case DATE: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in aggregation AVG : %s", dataType)); diff --git a/integration-test/src/main/java/org/apache/iotdb/db/query/udf/example/UDAFAvg.java b/integration-test/src/main/java/org/apache/iotdb/db/query/udf/example/UDAFAvg.java index a17905e4b151..a8aab5d0a452 100644 --- a/integration-test/src/main/java/org/apache/iotdb/db/query/udf/example/UDAFAvg.java +++ b/integration-test/src/main/java/org/apache/iotdb/db/query/udf/example/UDAFAvg.java @@ -101,7 +101,11 @@ public void addInput(State state, Column[] columns, BitMap bitMap) { addDoubleInput(avgState, columns, bitMap); return; case TEXT: + case STRING: + case BLOB: case BOOLEAN: + case TIMESTAMP: + case DATE: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in aggregation AVG : %s", dataType)); diff --git a/integration-test/src/main/java/org/apache/iotdb/db/query/udf/example/UDAFSum.java b/integration-test/src/main/java/org/apache/iotdb/db/query/udf/example/UDAFSum.java index 087dc655d844..a829c9f2c477 100644 --- a/integration-test/src/main/java/org/apache/iotdb/db/query/udf/example/UDAFSum.java +++ b/integration-test/src/main/java/org/apache/iotdb/db/query/udf/example/UDAFSum.java @@ -101,7 +101,11 @@ public void addInput(State state, Column[] columns, BitMap bitMap) { addDoubleInput(sumState, columns, bitMap); return; case TEXT: + case STRING: + case BLOB: case BOOLEAN: + case TIMESTAMP: + case DATE: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in aggregation AVG : %s", dataType)); diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/ImportData.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/ImportData.java index a16cce67a2ed..819190d2c2c0 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/ImportData.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/ImportData.java @@ -1040,6 +1040,9 @@ private static Object typeTrans(String value, TSDataType type) { return Float.parseFloat(value); case DOUBLE: return Double.parseDouble(value); + case TIMESTAMP: + case DATE: + case BLOB: default: return null; } diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/GroupedLSBWatermarkEncoder.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/GroupedLSBWatermarkEncoder.java index 6ec6a9b5baf1..d1d830fb2c2c 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/GroupedLSBWatermarkEncoder.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/GroupedLSBWatermarkEncoder.java @@ -134,6 +134,10 @@ public RowRecord encodeRecord(RowRecord rowRecord) { double originDoubleValue = field.getDoubleV(); field.setDoubleV(encodeDouble(originDoubleValue, timestamp)); break; + case BLOB: + case STRING: + case BOOLEAN: + case TEXT: default: } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 5eaa88e3ee5d..58de8530dc4e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -2578,6 +2578,9 @@ public TSEncoding getDefaultEncodingByType(TSDataType dataType) { return conf.getDefaultFloatEncoding(); case DOUBLE: return conf.getDefaultDoubleEncoding(); + case STRING: + case BLOB: + case TEXT: default: return conf.getDefaultTextEncoding(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/CustomizedReadableIntermediateResults.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/CustomizedReadableIntermediateResults.java index f9c9c5f82880..c915bcbc1741 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/CustomizedReadableIntermediateResults.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/CustomizedReadableIntermediateResults.java @@ -66,6 +66,10 @@ public int getInt(final String key) { return (int) (float) value; case DOUBLE: return (int) (double) value; + case TEXT: + case BLOB: + case BOOLEAN: + case STRING: default: throw new UnsupportedOperationException( String.format("The type %s cannot be casted to int.", typeResultPair.getLeft())); @@ -89,6 +93,10 @@ public long getLong(final String key) { return (long) (float) value; case DOUBLE: return (long) (double) value; + case BOOLEAN: + case STRING: + case TEXT: + case BLOB: default: throw new UnsupportedOperationException( String.format("The type %s cannot be casted to long.", typeResultPair.getLeft())); @@ -112,6 +120,10 @@ public float getFloat(final String key) { return (float) value; case DOUBLE: return (float) (double) value; + case TEXT: + case BLOB: + case BOOLEAN: + case STRING: default: throw new UnsupportedOperationException( String.format("The type %s cannot be casted to float.", typeResultPair.getLeft())); @@ -135,6 +147,10 @@ public double getDouble(final String key) { return (float) value; case DOUBLE: return (double) value; + case BOOLEAN: + case STRING: + case TEXT: + case BLOB: default: throw new UnsupportedOperationException( String.format("The type %s cannot be casted to double.", typeResultPair.getLeft())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbstractSameTypeNumericOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbstractSameTypeNumericOperator.java index eab8b6425604..2e3136564768 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbstractSameTypeNumericOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbstractSameTypeNumericOperator.java @@ -128,6 +128,12 @@ public Pair getResult() { return new Pair<>(TSDataType.FLOAT, floatValue); case DOUBLE: return new Pair<>(TSDataType.DOUBLE, doubleValue); + case BLOB: + case TEXT: + case BOOLEAN: + case STRING: + case TIMESTAMP: + case DATE: default: return null; } @@ -149,6 +155,12 @@ public void serialize(final DataOutputStream outputStream) throws IOException { case DOUBLE: ReadWriteIOUtils.write(doubleValue, outputStream); break; + case TIMESTAMP: + case DATE: + case BOOLEAN: + case STRING: + case TEXT: + case BLOB: default: throw new IOException(String.format("Unsupported output datatype %s", outPutDataType)); } @@ -170,6 +182,12 @@ public void deserialize(final ByteBuffer byteBuffer) throws IOException { case DOUBLE: doubleValue = ReadWriteIOUtils.readDouble(byteBuffer); break; + case TEXT: + case BLOB: + case BOOLEAN: + case STRING: + case DATE: + case TIMESTAMP: default: throw new IOException(String.format("Unsupported output datatype %s", outPutDataType)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/AccumulatorFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/AccumulatorFactory.java index a31dc6e582d2..24a998f54a91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/AccumulatorFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/AccumulatorFactory.java @@ -159,6 +159,10 @@ private static Accumulator createModeAccumulator(TSDataType tsDataType) { return new FloatModeAccumulator(); case DOUBLE: return new DoubleModeAccumulator(); + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: default: throw new IllegalArgumentException("Unknown data type: " + tsDataType); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/AvgAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/AvgAccumulator.java index 96d963ad2b49..c6d1baa33830 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/AvgAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/AvgAccumulator.java @@ -56,7 +56,11 @@ public void addInput(Column[] columns, BitMap bitMap) { addDoubleInput(columns, bitMap); return; case TEXT: + case BLOB: + case STRING: case BOOLEAN: + case DATE: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in aggregation AVG : %s", seriesDataType)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/ExtremeAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/ExtremeAccumulator.java index 776e39112030..76a42b41c718 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/ExtremeAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/ExtremeAccumulator.java @@ -56,7 +56,11 @@ public void addInput(Column[] columns, BitMap bitMap) { addDoubleInput(columns, bitMap); return; case TEXT: + case STRING: + case BLOB: case BOOLEAN: + case DATE: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in Extreme: %s", seriesDataType)); @@ -84,7 +88,11 @@ public void addIntermediate(Column[] partialResult) { updateDoubleResult(partialResult[0].getDouble(0)); break; case TEXT: + case STRING: + case BLOB: case BOOLEAN: + case DATE: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in Extreme: %s", seriesDataType)); @@ -114,7 +122,11 @@ public void addStatistics(Statistics statistics) { updateDoubleResult((double) statistics.getMinValue()); break; case TEXT: + case STRING: + case BLOB: case BOOLEAN: + case DATE: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in Extreme: %s", seriesDataType)); @@ -141,7 +153,11 @@ public void setFinal(Column finalResult) { extremeResult.setDouble(finalResult.getDouble(0)); break; case TEXT: + case STRING: + case BLOB: case BOOLEAN: + case DATE: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in Extreme: %s", seriesDataType)); @@ -170,7 +186,11 @@ public void outputIntermediate(ColumnBuilder[] columnBuilders) { columnBuilders[0].writeDouble(extremeResult.getDouble()); break; case TEXT: + case STRING: + case BLOB: case BOOLEAN: + case DATE: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in Extreme: %s", seriesDataType)); @@ -197,7 +217,11 @@ public void outputFinal(ColumnBuilder columnBuilder) { columnBuilder.writeDouble(extremeResult.getDouble()); break; case TEXT: + case STRING: + case BLOB: case BOOLEAN: + case DATE: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in Extreme: %s", seriesDataType)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MaxMinByBaseAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MaxMinByBaseAccumulator.java index c1e9b006c053..a0e4f80120cb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MaxMinByBaseAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MaxMinByBaseAccumulator.java @@ -86,6 +86,7 @@ public void addInput(Column[] column, BitMap bitMap) { addBinaryInput(column, bitMap); return; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException(String.format(UNSUPPORTED_TYPE_MESSAGE, yDataType)); @@ -438,6 +439,7 @@ private void updateFromBytesIntermediateInput(byte[] bytes) { updateBinaryResult(time, binaryMaxVal, columnBuilder.build(), 0); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException(String.format(UNSUPPORTED_TYPE_MESSAGE, yDataType)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MaxValueAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MaxValueAccumulator.java index 75e7a2f74e71..0d58de8064f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MaxValueAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MaxValueAccumulator.java @@ -63,6 +63,7 @@ public void addInput(Column[] columns, BitMap bitMap) { addBinaryInput(columns, bitMap); return; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -96,6 +97,7 @@ public void addIntermediate(Column[] partialResult) { updateBinaryResult(partialResult[0].getBinary(0)); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -127,6 +129,7 @@ public void addStatistics(Statistics statistics) { updateBinaryResult((Binary) statistics.getMaxValue()); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -160,6 +163,7 @@ public void setFinal(Column finalResult) { maxResult.setBinary(finalResult.getBinary(0)); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -194,6 +198,7 @@ public void outputIntermediate(ColumnBuilder[] columnBuilders) { columnBuilders[0].writeBinary(maxResult.getBinary()); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -226,6 +231,7 @@ public void outputFinal(ColumnBuilder columnBuilder) { columnBuilder.writeBinary(maxResult.getBinary()); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MinValueAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MinValueAccumulator.java index c8e403bf7ec1..1d9cc59aa17d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MinValueAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/MinValueAccumulator.java @@ -63,6 +63,7 @@ public void addInput(Column[] columns, BitMap bitMap) { addBinaryInput(columns, bitMap); return; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -96,6 +97,7 @@ public void addIntermediate(Column[] partialResult) { updateBinaryResult(partialResult[0].getBinary(0)); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -127,6 +129,7 @@ public void addStatistics(Statistics statistics) { updateBinaryResult((Binary) statistics.getMinValue()); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -160,6 +163,7 @@ public void setFinal(Column finalResult) { minResult.setBinary(finalResult.getBinary(0)); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -194,6 +198,7 @@ public void outputIntermediate(ColumnBuilder[] columnBuilders) { columnBuilders[0].writeBinary(minResult.getBinary()); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -226,6 +231,7 @@ public void outputFinal(ColumnBuilder columnBuilder) { columnBuilder.writeBinary(minResult.getBinary()); break; case TEXT: + case BLOB: case BOOLEAN: default: throw new UnSupportedDataTypeException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/SumAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/SumAccumulator.java index c948ed18e3b4..37daf1a84b1e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/SumAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/SumAccumulator.java @@ -56,7 +56,11 @@ public void addInput(Column[] columns, BitMap bitMap) { addDoubleInput(columns, bitMap); return; case TEXT: + case BLOB: case BOOLEAN: + case TIMESTAMP: + case DATE: + case STRING: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in aggregation AVG : %s", seriesDataType)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/VarianceAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/VarianceAccumulator.java index 7f7dd1bfd6de..3242518c3dc2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/VarianceAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/VarianceAccumulator.java @@ -69,7 +69,11 @@ public void addInput(Column[] columns, BitMap bitMap) { addDoubleInput(columns, bitMap); return; case TEXT: + case BLOB: case BOOLEAN: + case DATE: + case STRING: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in aggregation variance : %s", seriesDataType)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/window/WindowManagerFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/window/WindowManagerFactory.java index d165d721e228..3c3fc6e616aa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/window/WindowManagerFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/window/WindowManagerFactory.java @@ -69,6 +69,10 @@ private static VariationWindowManager genEqualEventWindowManager( return new EqualBinaryWindowManager(eventWindowParameter, ascending); case BOOLEAN: return new EqualBooleanWindowManager(eventWindowParameter, ascending); + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: default: throw new UnSupportedDataTypeException( String.format( @@ -88,6 +92,12 @@ private static VariationWindowManager genVariationEventWindowManager( return new VariationFloatWindowManager(eventWindowParameter, ascending); case DOUBLE: return new VariationDoubleWindowManager(eventWindowParameter, ascending); + case TIMESTAMP: + case DATE: + case STRING: + case BOOLEAN: + case BLOB: + case TEXT: default: throw new UnSupportedDataTypeException( String.format( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/InColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/InColumnTransformer.java index 29ef3d094030..5b38a1cc9380 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/InColumnTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/InColumnTransformer.java @@ -20,12 +20,6 @@ package org.apache.iotdb.db.queryengine.transformation.dag.column.unary; import org.apache.iotdb.db.exception.sql.SemanticException; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BinaryLiteral; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DoubleLiteral; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StringLiteral; import org.apache.iotdb.db.queryengine.transformation.dag.column.ColumnTransformer; import org.apache.tsfile.block.column.Column; @@ -34,10 +28,8 @@ import org.apache.tsfile.read.common.type.Type; import org.apache.tsfile.read.common.type.TypeEnum; import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.utils.DateUtils; import java.util.HashSet; -import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -163,84 +155,8 @@ private void initTypedSet(Set values) { .map(v -> new Binary(v, TSFileConfig.STRING_CHARSET)) .collect(Collectors.toSet()); break; - default: - throw new UnsupportedOperationException("unsupported data type: " + childType); - } - } - - private void initTypedSet(List values) { - if (childType == null) { - return; - } - String errorMsg = "\"%s\" cannot be cast to [%s]"; - switch (childType) { - case INT32: - intSet = new HashSet<>(); - for (Literal value : values) { - try { - intSet.add((int) ((LongLiteral) value).getParsedValue()); - } catch (IllegalArgumentException e) { - throw new SemanticException(String.format(errorMsg, value, childType)); - } - } - break; - case DATE: - intSet = new HashSet<>(); - for (Literal value : values) { - intSet.add(DateUtils.parseDateExpressionToInt(((StringLiteral) value).getValue())); - } - break; - case INT64: - case TIMESTAMP: - longSet = new HashSet<>(); - for (Literal value : values) { - try { - longSet.add((((LongLiteral) value).getParsedValue())); - } catch (IllegalArgumentException e) { - throw new SemanticException(String.format(errorMsg, value, childType)); - } - } - break; - case FLOAT: - floatSet = new HashSet<>(); - for (Literal value : values) { - try { - floatSet.add((float) ((DoubleLiteral) value).getValue()); - } catch (IllegalArgumentException e) { - throw new SemanticException(String.format(errorMsg, value, childType)); - } - } - break; - case DOUBLE: - doubleSet = new HashSet<>(); - for (Literal value : values) { - try { - doubleSet.add(((DoubleLiteral) value).getValue()); - } catch (IllegalArgumentException e) { - throw new SemanticException(String.format(errorMsg, value, childType)); - } - } - break; - case BOOLEAN: - booleanSet = new HashSet<>(); - for (Literal value : values) { - booleanSet.add(((BooleanLiteral) value).getValue()); - } - break; - case TEXT: - case STRING: - stringSet = new HashSet<>(); - for (Literal value : values) { - stringSet.add( - new Binary(((StringLiteral) value).getValue(), TSFileConfig.STRING_CHARSET)); - } - break; case BLOB: - stringSet = new HashSet<>(); - for (Literal value : values) { - stringSet.add(new Binary(((BinaryLiteral) value).getValue())); - } - break; + case DATE: default: throw new UnsupportedOperationException("unsupported data type: " + childType); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/RoundColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/RoundColumnTransformer.java index ebd250cc86c9..1f9dfc5a0654 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/RoundColumnTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/RoundColumnTransformer.java @@ -59,6 +59,12 @@ protected void doTransform( builder.writeDouble( Math.rint(leftColumn.getDouble(i) * Math.pow(10, places)) / Math.pow(10, places)); break; + case DATE: + case TEXT: + case BOOLEAN: + case BLOB: + case STRING: + case TIMESTAMP: default: throw new UnsupportedOperationException( String.format("Unsupported source dataType: %s", sourceType)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/RoundFunctionColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/RoundFunctionColumnTransformer.java index c8167971e19d..31dcaa027f85 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/RoundFunctionColumnTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/RoundFunctionColumnTransformer.java @@ -59,6 +59,12 @@ protected void doTransform(Column column, ColumnBuilder columnBuilder) { columnBuilder.writeDouble( Math.rint(column.getDouble(i) * Math.pow(10, places)) / Math.pow(10, places)); break; + case TIMESTAMP: + case STRING: + case BOOLEAN: + case BLOB: + case TEXT: + case DATE: default: throw new UnsupportedOperationException( String.format( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/input/ConstantInputReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/input/ConstantInputReader.java index ffe16112f62d..32125c110ef5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/input/ConstantInputReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/input/ConstantInputReader.java @@ -91,6 +91,10 @@ public ConstantInputReader(ConstantOperand expression) throws QueryProcessExcept Column booleanColumn = new BooleanColumn(1, Optional.empty(), booleanArray); cachedColumns[0] = new RunLengthEncodedColumn(booleanColumn, count); break; + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: default: throw new QueryProcessException("Unsupported type: " + expression.getDataType()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/ArithmeticNegationTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/ArithmeticNegationTransformer.java index 5ecd69142d62..fcbb30b7ca13 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/ArithmeticNegationTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/ArithmeticNegationTransformer.java @@ -55,6 +55,12 @@ protected void transform(Column[] columns, ColumnBuilder builder) case DOUBLE: transformDouble(columns, builder); return; + case DATE: + case TEXT: + case TIMESTAMP: + case BLOB: + case BOOLEAN: + case STRING: default: throw new QueryProcessException("Unsupported data type: " + layerReaderDataType); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/InTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/InTransformer.java index 7630112726de..5bdfa8f96a7f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/InTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/InTransformer.java @@ -87,6 +87,7 @@ private void initTypedSet(Set values) { case STRING: stringSet = values; break; + case BLOB: default: throw new UnsupportedOperationException("unsupported data type: " + layerReaderDataType); } @@ -122,6 +123,7 @@ protected void transform(Column[] columns, ColumnBuilder builder) case STRING: transformBinary(columns, builder); return; + case BLOB: default: throw new QueryProcessException("unsupported data type: " + layerReaderDataType); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/CastFunctionTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/CastFunctionTransformer.java index 9c6ba87da66a..11cf952e0b43 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/CastFunctionTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/CastFunctionTransformer.java @@ -67,6 +67,10 @@ protected Column[] transform(Column[] columns) throws QueryProcessException, IOE return castBooleans(columns); case TEXT: return castBinaries(columns); + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: default: throw new UnsupportedOperationException( String.format("Unsupported source dataType: %s", layerReaderDataType)); @@ -133,6 +137,11 @@ private Column[] castInts(Column[] columns) { } } break; + case STRING: + case BLOB: + case TIMESTAMP: + case DATE: + case INT32: default: throw new UnsupportedOperationException( String.format("Unsupported target dataType: %s", layerReaderDataType)); @@ -203,6 +212,11 @@ private Column[] castLongs(Column[] columns) { } } break; + case BLOB: + case STRING: + case DATE: + case TIMESTAMP: + case INT64: default: throw new UnsupportedOperationException( String.format("Unsupported target dataType: %s", layerReaderDataType)); @@ -273,6 +287,11 @@ private Column[] castFloats(Column[] columns) { } } break; + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: + case FLOAT: default: throw new UnsupportedOperationException( String.format("Unsupported target dataType: %s", layerReaderDataType)); @@ -343,6 +362,11 @@ private Column[] castDoubles(Column[] columns) { } } break; + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: + case DOUBLE: default: throw new UnsupportedOperationException( String.format("Unsupported target dataType: %s", layerReaderDataType)); @@ -413,6 +437,11 @@ private Column[] castBooleans(Column[] columns) { } } break; + case STRING: + case BLOB: + case DATE: + case TIMESTAMP: + case BOOLEAN: default: throw new UnsupportedOperationException( String.format("Unsupported target dataType: %s", layerReaderDataType)); @@ -488,6 +517,11 @@ private Column[] castBinaries(Column[] columns) { } } break; + case TIMESTAMP: + case DATE: + case STRING: + case BLOB: + case TEXT: default: throw new UnsupportedOperationException( String.format("Unsupported target dataType: %s", layerReaderDataType)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/DiffFunctionTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/DiffFunctionTransformer.java index d7ab2df5c4a4..ee7ba10e60e4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/DiffFunctionTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/DiffFunctionTransformer.java @@ -61,6 +61,12 @@ public void transform(Column[] columns, ColumnBuilder builder) throws QueryProce case DOUBLE: transformDouble(columns, builder); return; + case BLOB: + case TEXT: + case DATE: + case STRING: + case TIMESTAMP: + case BOOLEAN: default: throw new QueryProcessException("Unsupported data type: " + layerReaderDataType); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/RoundFunctionTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/RoundFunctionTransformer.java index 24061fb07649..a4ddff58bab4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/RoundFunctionTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/transformer/unary/scalar/RoundFunctionTransformer.java @@ -61,6 +61,12 @@ protected void transform(Column[] columns, ColumnBuilder builder) case DOUBLE: transformDouble(columns, builder); return; + case TIMESTAMP: + case BOOLEAN: + case DATE: + case STRING: + case TEXT: + case BLOB: default: throw new UnsupportedOperationException( String.format("Unsupported source dataType: %s", layerReaderDataType)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/util/TransformUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/util/TransformUtils.java index 187386916f5c..c05d7adfecc2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/util/TransformUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/util/TransformUtils.java @@ -76,6 +76,10 @@ public static Column transformConstantOperandToColumn(ConstantOperand constantOp return new BinaryColumn(1, Optional.empty(), new Binary[] {(Binary) value}); case BOOLEAN: return new BooleanColumn(1, Optional.empty(), new boolean[] {(boolean) value}); + case STRING: + case BLOB: + case DATE: + case TIMESTAMP: default: throw new UnSupportedDataTypeException( "Unsupported type: " + constantOperand.getDataType()); @@ -151,6 +155,10 @@ public static boolean splitWindowForStateWindow( valueRecorder.recordString(str); } break; + case TIMESTAMP: + case DATE: + case BLOB: + case STRING: default: throw new UnsupportedOperationException( "The data type of the state window strategy is not valid."); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/util/TypeUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/util/TypeUtils.java index 1bdee2344cad..0eedb86a31a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/util/TypeUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/util/TypeUtils.java @@ -72,6 +72,9 @@ public static double castValueToDouble(Column column, TSDataType type, int index return column.getDouble(index); case BOOLEAN: return column.getBoolean(index) ? 1 : 0; + case BLOB: + case STRING: + case TEXT: default: throw new QueryProcessException("Unsupported data type: " + type); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALReader.java index ce0d9689a850..261e95eb52f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALReader.java @@ -67,23 +67,27 @@ public boolean hasNext() { return true; } // read WALEntries from log stream + if (fileCorrupted) { + return false; + } try { - if (fileCorrupted) { - return false; - } nextEntry = WALEntry.deserialize(logStream); if (nextEntry.getType() == WALEntryType.WAL_FILE_INFO_END_MARKER) { nextEntry = null; return false; } } catch (EOFException e) { + fileCorrupted = true; return false; - } catch (Exception e) { + } catch (IOException e) { fileCorrupted = true; // log only when file should be complete if (!fileMayCorrupt) { logger.warn("Fail to read WALEntry from wal file {}, skip broken WALEntries.", logFile, e); } + } catch (Exception e) { + fileCorrupted = true; + logger.warn("Fail to read WALEntry from wal file {}, skip broken WALEntries.", logFile, e); } return nextEntry != null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java index ab9ea47b2261..dec3864353cc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java @@ -772,9 +772,11 @@ public static Object[] readTabletValuesFromStream( case BOOLEAN: parseBooleanColumn(size, stream, values, i); break; + case DATE: case INT32: parseInt32Column(size, stream, values, i); break; + case TIMESTAMP: case INT64: parseInt64Column(size, stream, values, i); break; @@ -785,6 +787,8 @@ public static Object[] readTabletValuesFromStream( parseDoubleColumn(size, stream, values, i); break; case TEXT: + case BLOB: + case STRING: parseTextColumn(size, stream, values, i); break; default: diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALFileTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALFileTest.java index 8a13bce60aa8..5f05e021802b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALFileTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALFileTest.java @@ -181,23 +181,27 @@ public static InsertRowNode getInsertRowNode(String devicePath) throws IllegalPa TSDataType.INT64, TSDataType.INT32, TSDataType.BOOLEAN, - TSDataType.TEXT + TSDataType.TEXT, + TSDataType.STRING, + TSDataType.BLOB }; - Object[] columns = new Object[6]; + Object[] columns = new Object[8]; columns[0] = 1.0; columns[1] = 2.0f; columns[2] = 10000L; columns[3] = 100; columns[4] = false; columns[5] = new Binary("hh" + 0, TSFileConfig.STRING_CHARSET); + columns[6] = new Binary("jj" + 0, TSFileConfig.STRING_CHARSET); + columns[7] = new Binary("kk" + 0, TSFileConfig.STRING_CHARSET); InsertRowNode insertRowNode = new InsertRowNode( new PlanNodeId(""), new PartialPath(devicePath), false, - new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}, + new String[] {"s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8"}, dataTypes, time, columns, @@ -211,6 +215,8 @@ public static InsertRowNode getInsertRowNode(String devicePath) throws IllegalPa new MeasurementSchema("s4", dataTypes[3]), new MeasurementSchema("s5", dataTypes[4]), new MeasurementSchema("s6", dataTypes[5]), + new MeasurementSchema("s7", dataTypes[6]), + new MeasurementSchema("s8", dataTypes[7]), }; insertRowNode.setMeasurementSchemas(schemas); return insertRowNode; @@ -226,23 +232,27 @@ public static InsertRowsNode getInsertRowsNode(String devicePath) throws Illegal TSDataType.INT64, TSDataType.INT32, TSDataType.BOOLEAN, - TSDataType.TEXT + TSDataType.TEXT, + TSDataType.STRING, + TSDataType.BLOB }; - Object[] columns = new Object[6]; + Object[] columns = new Object[8]; columns[0] = 1.0; columns[1] = 2.0f; columns[2] = 10000L; columns[3] = 100; columns[4] = false; columns[5] = new Binary("hh" + 0, TSFileConfig.STRING_CHARSET); + columns[6] = new Binary("jj" + 0, TSFileConfig.STRING_CHARSET); + columns[7] = new Binary("kk" + 0, TSFileConfig.STRING_CHARSET); InsertRowNode insertRowNode = new InsertRowNode( new PlanNodeId(""), new PartialPath(devicePath), false, - new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}, + new String[] {"s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8"}, dataTypes, time, columns, @@ -256,6 +266,8 @@ public static InsertRowsNode getInsertRowsNode(String devicePath) throws Illegal new MeasurementSchema("s4", dataTypes[3]), new MeasurementSchema("s5", dataTypes[4]), new MeasurementSchema("s6", dataTypes[5]), + new MeasurementSchema("s7", dataTypes[6]), + new MeasurementSchema("s8", dataTypes[7]), }; insertRowNode.setMeasurementSchemas(schemas); insertRowsNode.addOneInsertRowNode(insertRowNode, 0); @@ -266,7 +278,7 @@ public static InsertRowsNode getInsertRowsNode(String devicePath) throws Illegal new PlanNodeId(""), new PartialPath(devicePath), false, - new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}, + new String[] {"s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8"}, dataTypes, time, columns, @@ -289,16 +301,20 @@ public static InsertTabletNode getInsertTabletNode(String devicePath) TSDataType.INT64, TSDataType.INT32, TSDataType.BOOLEAN, - TSDataType.TEXT + TSDataType.TEXT, + TSDataType.STRING, + TSDataType.BLOB }; - Object[] columns = new Object[6]; + Object[] columns = new Object[8]; columns[0] = new double[4]; columns[1] = new float[4]; columns[2] = new long[4]; columns[3] = new int[4]; columns[4] = new boolean[4]; columns[5] = new Binary[4]; + columns[6] = new Binary[4]; + columns[7] = new Binary[4]; for (int r = 0; r < 4; r++) { ((double[]) columns[0])[r] = 1.0 + r; @@ -307,6 +323,8 @@ public static InsertTabletNode getInsertTabletNode(String devicePath) ((int[]) columns[3])[r] = 100 + r; ((boolean[]) columns[4])[r] = (r % 2 == 0); ((Binary[]) columns[5])[r] = new Binary("hh" + r, TSFileConfig.STRING_CHARSET); + ((Binary[]) columns[6])[r] = new Binary("jj" + r, TSFileConfig.STRING_CHARSET); + ((Binary[]) columns[7])[r] = new Binary("kk" + r, TSFileConfig.STRING_CHARSET); } BitMap[] bitMaps = new BitMap[dataTypes.length]; @@ -324,13 +342,15 @@ public static InsertTabletNode getInsertTabletNode(String devicePath) new MeasurementSchema("s4", dataTypes[3]), new MeasurementSchema("s5", dataTypes[4]), new MeasurementSchema("s6", dataTypes[5]), + new MeasurementSchema("s7", dataTypes[6]), + new MeasurementSchema("s8", dataTypes[7]), }; return new InsertTabletNode( new PlanNodeId(""), new PartialPath(devicePath), false, - new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}, + new String[] {"s1", "s2", "s3", "s4", "s5", "s6", "s7", "s8"}, dataTypes, schemas, times, diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFAbs.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFAbs.java index 24dc7021badc..be0d16807e21 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFAbs.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFAbs.java @@ -63,6 +63,14 @@ public void transform(Row row, PointCollector collector) case DOUBLE: collector.putDouble(time, Math.abs(row.getDouble(0))); break; + case BLOB: + case STRING: + case TIMESTAMP: + case TEXT: + case DATE: + case BOOLEAN: + case VECTOR: + case UNKNOWN: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -89,6 +97,12 @@ public Object transform(Row row) throws IOException { return Math.abs(row.getFloat(0)); case DOUBLE: return Math.abs(row.getDouble(0)); + case DATE: + case BOOLEAN: + case TEXT: + case TIMESTAMP: + case STRING: + case BLOB: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -116,6 +130,12 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception case DOUBLE: transformDouble(columns, builder); return; + case BLOB: + case STRING: + case TEXT: + case TIMESTAMP: + case BOOLEAN: + case DATE: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFBottomK.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFBottomK.java index 2e9cb0c0de2c..bb12f7ff4a58 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFBottomK.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFBottomK.java @@ -72,6 +72,8 @@ protected void constructPQ() throws UDFInputSeriesDataTypeNotValidException { return cs1.length() - cs2.length(); }); break; + case BLOB: + case BOOLEAN: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFChangePoints.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFChangePoints.java index c90a90fdd68f..dd19304384b8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFChangePoints.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFChangePoints.java @@ -134,6 +134,12 @@ public void transform(Row row, PointCollector collector) throws Exception { collector.putString(row.getTime(), cacheString); } } + case STRING: + case BLOB: + case DATE: + case TIMESTAMP: + default: + break; } } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFCommonDerivative.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFCommonDerivative.java index 2070ba75f395..00b3fd475501 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFCommonDerivative.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFCommonDerivative.java @@ -55,6 +55,12 @@ protected void doTransform(Row row, PointCollector collector) collector.putDouble(currentTime, (currentDouble - previousDouble) / timeDelta); previousDouble = currentDouble; break; + case DATE: + case BOOLEAN: + case TIMESTAMP: + case TEXT: + case STRING: + case BLOB: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFCommonValueDifference.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFCommonValueDifference.java index fa29e1d2e126..75305f731651 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFCommonValueDifference.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFCommonValueDifference.java @@ -54,6 +54,12 @@ protected void doTransform(Row row, PointCollector collector) collector.putDouble(time, currentDouble - previousDouble); previousDouble = currentDouble; break; + case STRING: + case BLOB: + case TIMESTAMP: + case TEXT: + case BOOLEAN: + case DATE: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFContinuouslySatisfy.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFContinuouslySatisfy.java index de8a097d1b7b..b76b0276e7cf 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFContinuouslySatisfy.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFContinuouslySatisfy.java @@ -100,6 +100,11 @@ public void transform(Row row, PointCollector collector) case BOOLEAN: needAddNewRecord = transformBoolean(row.getTime(), row.getBoolean(0)); break; + case TEXT: + case STRING: + case BLOB: + case TIMESTAMP: + case DATE: default: // This will not happen throw new UDFInputSeriesDataTypeNotValidException( @@ -221,6 +226,11 @@ public void terminate(PointCollector collector) } } break; + case TIMESTAMP: + case DATE: + case STRING: + case BLOB: + case TEXT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketAggSample.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketAggSample.java index 096ef8fdcdee..d01673276137 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketAggSample.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketAggSample.java @@ -481,6 +481,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) case DOUBLE: aggregator.aggregateDouble(rowWindow, collector); break; + case BLOB: + case TEXT: + case DATE: + case STRING: + case TIMESTAMP: + case BOOLEAN: default: // This will not happen throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketM4Sample.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketM4Sample.java index 564bb57a8763..cf80380eea2a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketM4Sample.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketM4Sample.java @@ -58,6 +58,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) case DOUBLE: transformDouble(rowWindow, collector); break; + case TIMESTAMP: + case BOOLEAN: + case DATE: + case STRING: + case TEXT: + case BLOB: default: // This will not happen throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketOutlierSample.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketOutlierSample.java index 2658d75837fc..f3b3aacba378 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketOutlierSample.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketOutlierSample.java @@ -656,6 +656,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) case DOUBLE: outlierSampler.outlierSampleDouble(rowWindow, collector); break; + case TEXT: + case BLOB: + case DATE: + case STRING: + case BOOLEAN: + case TIMESTAMP: default: // This will not happen throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketRandomSample.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketRandomSample.java index 52a679059829..b9c733a2df0d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketRandomSample.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFEqualSizeBucketRandomSample.java @@ -61,6 +61,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) case DOUBLE: collector.putDouble(row.getTime(), row.getDouble(0)); break; + case BOOLEAN: + case TIMESTAMP: + case DATE: + case STRING: + case BLOB: + case TEXT: default: // This will not happen throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFInRange.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFInRange.java index 137d9d228171..f24b9d6ae7b8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFInRange.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFInRange.java @@ -85,6 +85,12 @@ public void transform(Row row, PointCollector collector) case DOUBLE: collector.putBoolean(time, row.getDouble(0) >= lower && upper >= row.getDouble(0)); break; + case BLOB: + case TEXT: + case DATE: + case STRING: + case TIMESTAMP: + case BOOLEAN: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -111,6 +117,12 @@ public Object transform(Row row) throws Exception { return row.getFloat(0) >= lower && upper >= row.getFloat(0); case DOUBLE: return row.getDouble(0) >= lower && upper >= row.getDouble(0); + case TIMESTAMP: + case BOOLEAN: + case DATE: + case STRING: + case TEXT: + case BLOB: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -138,6 +150,12 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception case DOUBLE: transformDouble(columns, builder); return; + case BLOB: + case TEXT: + case DATE: + case STRING: + case BOOLEAN: + case TIMESTAMP: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFJexl.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFJexl.java index 24b5d419c91e..d554cfe25720 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFJexl.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFJexl.java @@ -94,6 +94,10 @@ public void beforeStart(UDFParameters parameters, UDTFConfigurations configurati case BOOLEAN: evaluator = new EvaluatorBooleanInput(); break; + case STRING: + case TIMESTAMP: + case DATE: + case BLOB: default: throw new UDFInputSeriesDataTypeNotValidException( 0, @@ -162,6 +166,13 @@ public void transform(Row row, PointCollector collector) case BOOLEAN: evaluator.evaluateBoolean(row, collector); break; + case TIMESTAMP: + case DATE: + case STRING: + case BLOB: + case INT64: + case INT32: + case FLOAT: default: // This will not happen. throw new UDFOutputSeriesDataTypeNotValidException(0, "[Number, String, Boolean]"); @@ -333,6 +344,10 @@ public void getValues(Row row) throws IOException, UDFInputSeriesDataTypeNotVali case BOOLEAN: values[i] = row.getBoolean(i); break; + case STRING: + case BLOB: + case DATE: + case TIMESTAMP: default: throw new UDFInputSeriesDataTypeNotValidException( i, diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFM4.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFM4.java index 0b8ba6c8ac46..824740b8c9b0 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFM4.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFM4.java @@ -137,6 +137,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) case DOUBLE: transformDouble(rowWindow, collector); break; + case BLOB: + case DATE: + case STRING: + case TIMESTAMP: + case BOOLEAN: + case TEXT: default: // This will not happen throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFMath.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFMath.java index 779cc278d492..de4f8edcec5d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFMath.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFMath.java @@ -85,6 +85,12 @@ public void transform(Row row, PointCollector collector) case DOUBLE: collector.putDouble(time, transformer.transform(row.getDouble(0))); break; + case BOOLEAN: + case TEXT: + case STRING: + case TIMESTAMP: + case DATE: + case BLOB: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -111,6 +117,12 @@ public Object transform(Row row) throws IOException { return transformer.transform(row.getFloat(0)); case DOUBLE: return transformer.transform(row.getDouble(0)); + case DATE: + case BLOB: + case STRING: + case TIMESTAMP: + case TEXT: + case BOOLEAN: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -138,6 +150,12 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception case DOUBLE: transformDouble(columns, builder); return; + case TEXT: + case BOOLEAN: + case STRING: + case TIMESTAMP: + case BLOB: + case DATE: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFNonNegativeDerivative.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFNonNegativeDerivative.java index 176714b78e38..6812d2d2d8ee 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFNonNegativeDerivative.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFNonNegativeDerivative.java @@ -55,6 +55,12 @@ protected void doTransform(Row row, PointCollector collector) collector.putDouble(currentTime, Math.abs(currentDouble - previousDouble) / timeDelta); previousDouble = currentDouble; break; + case DATE: + case BLOB: + case STRING: + case TIMESTAMP: + case BOOLEAN: + case TEXT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFNonNegativeValueDifference.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFNonNegativeValueDifference.java index 87643bf8c617..1183e5397fef 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFNonNegativeValueDifference.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFNonNegativeValueDifference.java @@ -54,6 +54,12 @@ protected void doTransform(Row row, PointCollector collector) collector.putDouble(time, Math.abs(currentDouble - previousDouble)); previousDouble = currentDouble; break; + case BOOLEAN: + case TEXT: + case STRING: + case TIMESTAMP: + case BLOB: + case DATE: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFOnOff.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFOnOff.java index e626af34b6ea..66b5cd1add8e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFOnOff.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFOnOff.java @@ -79,6 +79,12 @@ public void transform(Row row, PointCollector collector) case DOUBLE: collector.putBoolean(time, (row.getDouble(0) >= threshold)); break; + case DATE: + case BLOB: + case STRING: + case TIMESTAMP: + case TEXT: + case BOOLEAN: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -105,6 +111,12 @@ public Object transform(Row row) throws IOException { return row.getFloat(0) >= threshold; case DOUBLE: return row.getDouble(0) >= threshold; + case TEXT: + case BOOLEAN: + case STRING: + case TIMESTAMP: + case BLOB: + case DATE: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -132,6 +144,12 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception case DOUBLE: transformDouble(columns, builder); return; + case BLOB: + case DATE: + case STRING: + case TIMESTAMP: + case BOOLEAN: + case TEXT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFSelectK.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFSelectK.java index ad84e1dc70c1..49faa398f1a7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFSelectK.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFSelectK.java @@ -106,6 +106,8 @@ public void transform(Row row, PointCollector collector) case STRING: transformString(row.getTime(), row.getString(0)); break; + case BLOB: + case BOOLEAN: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -177,6 +179,8 @@ public void terminate(PointCollector collector) collector.putString(pair.left, pair.right); } break; + case BLOB: + case BOOLEAN: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFTopK.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFTopK.java index e015f0de945e..1838b85022b3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFTopK.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFTopK.java @@ -51,6 +51,8 @@ protected void constructPQ() throws UDFInputSeriesDataTypeNotValidException { case STRING: stringPQ = new PriorityQueue<>(k, Comparator.comparing(o -> o.right)); break; + case BOOLEAN: + case BLOB: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFValueTrend.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFValueTrend.java index bc7d52fb7044..c99002193294 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFValueTrend.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFValueTrend.java @@ -65,6 +65,12 @@ protected void updatePreviousValue(Row row) case DOUBLE: previousDouble = row.getDouble(0); break; + case TEXT: + case BOOLEAN: + case TIMESTAMP: + case STRING: + case DATE: + case BLOB: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/utils/MasterRepairUtil.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/utils/MasterRepairUtil.java index f8367822cad7..4fe3faf89810 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/utils/MasterRepairUtil.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/utils/MasterRepairUtil.java @@ -104,6 +104,12 @@ public static double getValueAsDouble(Row row, int index) throws Exception { case DOUBLE: ans = row.getDouble(index); break; + case DATE: + case BLOB: + case STRING: + case TIMESTAMP: + case BOOLEAN: + case TEXT: default: throw new Exception("The value of the input time series is not numeric.\n"); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/SerializeUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/SerializeUtils.java index 853915bd31f8..8e5893edd05c 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/SerializeUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/SerializeUtils.java @@ -141,16 +141,20 @@ public static BatchData deserializeBatchData(ByteBuffer buffer) { TSDataType dataType = TSDataType.values()[buffer.get()]; BatchData batchData = BatchDataType.deserialize(buffer.get(), dataType); switch (dataType) { + case DATE: case INT32: for (int i = 0; i < length; i++) { batchData.putInt(buffer.getLong(), buffer.getInt()); } break; + case TIMESTAMP: case INT64: for (int i = 0; i < length; i++) { batchData.putLong(buffer.getLong(), buffer.getLong()); } break; + case BLOB: + case STRING: case TEXT: for (int i = 0; i < length; i++) { long time = buffer.getLong(); @@ -193,15 +197,19 @@ public static BatchData deserializeBatchData(ByteBuffer buffer) { case FLOAT: values[j] = new TsPrimitiveType.TsFloat(buffer.getFloat()); break; + case BLOB: + case STRING: case TEXT: int len = buffer.getInt(); byte[] bytes = new byte[len]; buffer.get(bytes); values[j] = new TsPrimitiveType.TsBinary(new Binary(bytes)); break; + case TIMESTAMP: case INT64: values[j] = new TsPrimitiveType.TsLong(buffer.getLong()); break; + case DATE: case INT32: values[j] = new TsPrimitiveType.TsInt(buffer.getInt()); break; @@ -286,15 +294,19 @@ public static void serializeTVPairs( dataOutputStream.write(dataType.ordinal()); dataOutputStream.writeInt(timeValuePairs.size()); switch (timeValuePairs.get(0).getValue().getDataType()) { + case BLOB: + case STRING: case TEXT: serializeTextTVPairs(timeValuePairs, dataOutputStream); break; case BOOLEAN: serializeBooleanTVPairs(timeValuePairs, dataOutputStream); break; + case TIMESTAMP: case INT64: serializeLongTVPairs(timeValuePairs, dataOutputStream); break; + case DATE: case INT32: serializeIntTVPairs(timeValuePairs, dataOutputStream); break; @@ -321,6 +333,8 @@ public static void serializeTVPair( try { dataOutputStream.write(dataType.ordinal()); switch (dataType) { + case STRING: + case BLOB: case TEXT: dataOutputStream.writeLong(timeValuePair.getTimestamp()); if (timeValuePair.getTimestamp() != Long.MIN_VALUE) { @@ -334,12 +348,14 @@ public static void serializeTVPair( dataOutputStream.writeBoolean(timeValuePair.getValue().getBoolean()); } break; + case TIMESTAMP: case INT64: dataOutputStream.writeLong(timeValuePair.getTimestamp()); if (timeValuePair.getTimestamp() != Long.MIN_VALUE) { dataOutputStream.writeLong(timeValuePair.getValue().getLong()); } break; + case DATE: case INT32: dataOutputStream.writeLong(timeValuePair.getTimestamp()); if (timeValuePair.getTimestamp() != Long.MIN_VALUE) { @@ -458,15 +474,19 @@ public static List deserializeTVPairs(ByteBuffer buffer) { case FLOAT: deserializeFloatTVPairs(buffer, ret, size, dataType); break; + case DATE: case INT32: deserializeIntTVPairs(buffer, ret, size, dataType); break; + case TIMESTAMP: case INT64: deserializeLongTVPairs(buffer, ret, size, dataType); break; case BOOLEAN: deserializeBooleanTVPairs(buffer, ret, size, dataType); break; + case BLOB: + case STRING: case TEXT: deserializeTextTVPairs(buffer, ret, size, dataType); break; @@ -490,12 +510,16 @@ public static TimeValuePair deserializeTVPair(ByteBuffer buffer) { return new TimeValuePair(time, TsPrimitiveType.getByType(dataType, buffer.getDouble())); case FLOAT: return new TimeValuePair(time, TsPrimitiveType.getByType(dataType, buffer.getFloat())); + case DATE: case INT32: return new TimeValuePair(time, TsPrimitiveType.getByType(dataType, buffer.getInt())); + case TIMESTAMP: case INT64: return new TimeValuePair(time, TsPrimitiveType.getByType(dataType, buffer.getLong())); case BOOLEAN: return new TimeValuePair(time, TsPrimitiveType.getByType(dataType, buffer.get() == 1)); + case BLOB: + case STRING: case TEXT: int bytesLen = buffer.getInt(); byte[] bytes = new byte[bytesLen]; diff --git a/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFKSigma.java b/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFKSigma.java index beab9bb32272..1d63d040d87f 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFKSigma.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFKSigma.java @@ -91,6 +91,12 @@ public void transform(Row row, PointCollector collector) throws Exception { case FLOAT: v.push(row.getFloat(0)); break; + case TIMESTAMP: + case DATE: + case TEXT: + case STRING: + case BLOB: + case BOOLEAN: default: break; } @@ -118,6 +124,12 @@ public void transform(Row row, PointCollector collector) throws Exception { case FLOAT: v.push(row.getFloat(0)); break; + case BLOB: + case BOOLEAN: + case STRING: + case TEXT: + case DATE: + case TIMESTAMP: default: break; } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFRange.java b/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFRange.java index e559c82e9390..41ca813331cc 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFRange.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFRange.java @@ -86,6 +86,12 @@ public void transform(Row row, PointCollector collector) throws Exception { collector.putDouble(timestamp, doubleValue); } break; + case DATE: + case TIMESTAMP: + case TEXT: + case STRING: + case BOOLEAN: + case BLOB: default: throw new UDFException("No such kind of data type."); } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFTwoSidedFilter.java b/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFTwoSidedFilter.java index 018f8c95a42e..1191c2543680 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFTwoSidedFilter.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/anomaly/UDTFTwoSidedFilter.java @@ -81,6 +81,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) throws Exce collector.putLong(time[i], Math.round(repaired[i])); } break; + case BOOLEAN: + case BLOB: + case STRING: + case TEXT: + case TIMESTAMP: + case DATE: default: throw new UDFException("No such kind of data type."); } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFPercentile.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFPercentile.java index 8f45200777f2..d3baa6a7bf0d 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFPercentile.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFPercentile.java @@ -89,6 +89,12 @@ public void beforeStart(UDFParameters parameters, UDTFConfigurations configurati case DOUBLE: doubleDic = new HashMap<>(); break; + case TIMESTAMP: + case DATE: + case TEXT: + case STRING: + case BLOB: + case BOOLEAN: default: break; } @@ -111,6 +117,12 @@ public void transform(Row row, PointCollector collector) throws Exception { case DOUBLE: doubleDic.put(row.getDouble(0), row.getTime()); break; + case BLOB: + case BOOLEAN: + case STRING: + case TEXT: + case DATE: + case TIMESTAMP: default: break; } @@ -145,6 +157,12 @@ public void terminate(PointCollector collector) throws Exception { time = doubleDic.getOrDefault(dres, 0L); collector.putDouble(time, dres); break; + case DATE: + case TIMESTAMP: + case TEXT: + case STRING: + case BOOLEAN: + case BLOB: default: break; } @@ -163,6 +181,12 @@ public void terminate(PointCollector collector) throws Exception { case DOUBLE: collector.putDouble(0, res); break; + case BOOLEAN: + case BLOB: + case STRING: + case TEXT: + case TIMESTAMP: + case DATE: default: break; } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFQuantile.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFQuantile.java index 67523a5ff60d..047beafe1fc6 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFQuantile.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFQuantile.java @@ -86,6 +86,12 @@ public void terminate(PointCollector collector) throws Exception { case DOUBLE: collector.putDouble(0, res); break; + case TIMESTAMP: + case DATE: + case TEXT: + case STRING: + case BLOB: + case BOOLEAN: default: break; } @@ -104,6 +110,12 @@ private long dataToLong(Object data) { case DOUBLE: result = Double.doubleToLongBits((double) data); return (double) data >= 0d ? result : result ^ Long.MAX_VALUE; + case BLOB: + case BOOLEAN: + case STRING: + case TEXT: + case DATE: + case TIMESTAMP: default: return (long) data; } @@ -119,6 +131,12 @@ private double longToResult(long result) { return Double.longBitsToDouble(result); case INT64: case INT32: + case DATE: + case TEXT: + case STRING: + case BOOLEAN: + case BLOB: + case TIMESTAMP: default: return (result); } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFSpread.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFSpread.java index 7c615662355a..bc7d9ad7c846 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFSpread.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFSpread.java @@ -76,6 +76,12 @@ public void transform(Row row, PointCollector pc) throws Exception { case DOUBLE: transformDouble(row); break; + case BLOB: + case TIMESTAMP: + case BOOLEAN: + case STRING: + case TEXT: + case DATE: default: break; } @@ -96,6 +102,12 @@ public void terminate(PointCollector pc) throws Exception { case DOUBLE: pc.putDouble(0, doubleMax - doubleMin); break; + case TEXT: + case DATE: + case STRING: + case BOOLEAN: + case TIMESTAMP: + case BLOB: default: throw new NoNumberException(); } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDTFDistinct.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDTFDistinct.java index 778fd60db8eb..1698164a944f 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDTFDistinct.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDTFDistinct.java @@ -86,6 +86,10 @@ public void beforeStart(UDFParameters parameters, UDTFConfigurations configurati case BOOLEAN: booleanSet = new BooleanHashSet(); break; + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: default: break; } @@ -112,6 +116,10 @@ public void transform(Row row, PointCollector pc) throws Exception { case BOOLEAN: booleanSet.add(row.getBoolean(0)); break; + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: default: break; } @@ -162,6 +170,10 @@ public void terminate(PointCollector pc) throws Exception { i++; } break; + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: default: break; } @@ -188,6 +200,10 @@ public void beforeDestroy() { case BOOLEAN: booleanSet.clear(); break; + case BLOB: + case STRING: + case TIMESTAMP: + case DATE: default: break; } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDTFSample.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDTFSample.java index bf014a63de99..545d4096f51b 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDTFSample.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDTFSample.java @@ -151,6 +151,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) throws Exce case DOUBLE: collector.putDouble(p.getLeft(), p.getRight()); break; + case TIMESTAMP: + case DATE: + case BLOB: + case BOOLEAN: + case STRING: + case TEXT: default: throw new NoNumberException(); } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/ExactOrderStatistics.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/ExactOrderStatistics.java index 5eb0b4701ee3..e1f0baa7c060 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/ExactOrderStatistics.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/ExactOrderStatistics.java @@ -55,6 +55,12 @@ public ExactOrderStatistics(Type type) throws UDFInputSeriesDataTypeNotValidExce case DOUBLE: doubleArrayList = new DoubleArrayList(); break; + case STRING: + case TEXT: + case BOOLEAN: + case BLOB: + case DATE: + case TIMESTAMP: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -82,6 +88,12 @@ public void insert(Row row) throws UDFInputSeriesDataTypeNotValidException, IOEx doubleArrayList.add(vd); } break; + case DATE: + case TIMESTAMP: + case BLOB: + case BOOLEAN: + case TEXT: + case STRING: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -99,6 +111,12 @@ public double getMedian() throws UDFInputSeriesDataTypeNotValidException { return getMedian(floatArrayList); case DOUBLE: return getMedian(doubleArrayList); + case TEXT: + case STRING: + case BOOLEAN: + case BLOB: + case TIMESTAMP: + case DATE: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -181,6 +199,12 @@ public double getMad() throws UDFInputSeriesDataTypeNotValidException { return getMad(floatArrayList); case DOUBLE: return getMad(doubleArrayList); + case TIMESTAMP: + case DATE: + case BLOB: + case BOOLEAN: + case STRING: + case TEXT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -255,6 +279,12 @@ public String getPercentile(double phi) throws UDFInputSeriesDataTypeNotValidExc return Float.toString(getPercentile(floatArrayList, phi)); case DOUBLE: return Double.toString(getPercentile(doubleArrayList, phi)); + case STRING: + case TEXT: + case BOOLEAN: + case BLOB: + case DATE: + case TIMESTAMP: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFTimestampRepair.java b/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFTimestampRepair.java index 78c462e8ea5f..9554cff6d18c 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFTimestampRepair.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFTimestampRepair.java @@ -95,6 +95,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) throws Exce collector.putLong(timestamp[i], (long) value[i]); } break; + case DATE: + case TIMESTAMP: + case BLOB: + case BOOLEAN: + case TEXT: + case STRING: default: throw new UDFException(""); } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFValueFill.java b/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFValueFill.java index ab539730acf4..5727badc3fd6 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFValueFill.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFValueFill.java @@ -98,6 +98,12 @@ public void transform(RowWindow rowWindow, PointCollector collector) throws Exce collector.putLong(time[i], Math.round(repaired[i])); } break; + case TEXT: + case STRING: + case BOOLEAN: + case BLOB: + case TIMESTAMP: + case DATE: default: throw new UDFException(""); } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFValueRepair.java b/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFValueRepair.java index bc73a04e4b19..17f589d00ded 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFValueRepair.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/drepair/UDTFValueRepair.java @@ -115,6 +115,13 @@ public void transform(RowWindow rowWindow, PointCollector collector) throws Exce collector.putLong(time[i], Math.round(repaired[i])); } break; + case TIMESTAMP: + case DATE: + case BLOB: + case BOOLEAN: + case STRING: + case TEXT: + default: throw new Exception(); } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/frequency/UDFEnvelopeAnalysis.java b/library-udf/src/main/java/org/apache/iotdb/library/frequency/UDFEnvelopeAnalysis.java index 8180f747d765..af06b4fb9de4 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/frequency/UDFEnvelopeAnalysis.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/frequency/UDFEnvelopeAnalysis.java @@ -226,6 +226,12 @@ public double getValueAsDouble(Row row, int index) throws IOException { case DOUBLE: ans = row.getDouble(index); break; + case TEXT: + case STRING: + case BOOLEAN: + case BLOB: + case DATE: + case TIMESTAMP: default: throw new UDFOutputSeriesDataTypeNotValidException( index, "Fail to get data type in row " + row.getTime()); diff --git a/library-udf/src/main/java/org/apache/iotdb/library/util/Util.java b/library-udf/src/main/java/org/apache/iotdb/library/util/Util.java index d7060de7692f..5c65c4314ce9 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/util/Util.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/util/Util.java @@ -60,6 +60,12 @@ public static double getValueAsDouble(Row row, int index) throws IOException, No case DOUBLE: ans = row.getDouble(index); break; + case DATE: + case BLOB: + case BOOLEAN: + case STRING: + case TEXT: + case TIMESTAMP: default: throw new NoNumberException(); } @@ -107,6 +113,10 @@ public static Object getValueAsObject(Row row) throws IOException { case TEXT: ans = row.getString(0); break; + case BLOB: + case STRING: + case DATE: + case TIMESTAMP: default: break; } @@ -138,6 +148,11 @@ public static void putValue(PointCollector pc, Type type, long t, Object o) thro case BOOLEAN: pc.putBoolean(t, (Boolean) o); break; + case DATE: + case TIMESTAMP: + case TEXT: + case STRING: + case BLOB: default: break; } From 0d9fc3a8801a714d7df879a06df185eb005fe57f Mon Sep 17 00:00:00 2001 From: Beyyes Date: Tue, 3 Sep 2024 11:45:11 +0800 Subject: [PATCH 09/13] Add query metrics for table model --- .../metric/QueryPlanCostMetricSet.java | 149 +++++++++++++++--- .../db/queryengine/plan/Coordinator.java | 14 +- .../plan/analyze/AnalyzeVisitor.java | 11 +- .../db/queryengine/plan/analyze/Analyzer.java | 4 +- .../plan/analyze/TemplatedAnalyze.java | 3 +- .../plan/execution/QueryExecution.java | 9 +- ...ableModelStatementMemorySourceVisitor.java | 3 +- .../plan/planner/LogicalPlanner.java | 7 +- .../plan/relational/analyzer/Analyzer.java | 9 ++ ...lPlanner.java => TableLogicalPlanner.java} | 19 ++- ...delPlanner.java => TableModelPlanner.java} | 6 +- .../distribute/TableDistributedPlanner.java | 12 +- .../PushPredicateIntoTableScan.java | 13 ++ .../relational/analyzer/AnalyzerTest.java | 79 +++++----- .../analyzer/LimitOffsetPushDownTest.java | 18 +-- .../plan/relational/analyzer/SortTest.java | 32 ++-- .../relational/analyzer/SubQueryTest.java | 14 +- .../plan/relational/planner/PlanTester.java | 8 +- 18 files changed, 287 insertions(+), 123 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/{LogicalPlanner.java => TableLogicalPlanner.java} (94%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/{RelationalModelPlanner.java => TableModelPlanner.java} (97%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/metric/QueryPlanCostMetricSet.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/metric/QueryPlanCostMetricSet.java index da775515688b..0b0721fe44b0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/metric/QueryPlanCostMetricSet.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/metric/QueryPlanCostMetricSet.java @@ -32,10 +32,14 @@ public class QueryPlanCostMetricSet implements IMetricSet { private static final QueryPlanCostMetricSet INSTANCE = new QueryPlanCostMetricSet(); + + public static final String TABLE_TYPE = "table"; + public static final String TREE_TYPE = "tree"; + public static final String ANALYZER = "analyzer"; public static final String LOGICAL_PLANNER = "logical_planner"; + public static final String LOGICAL_PLAN_OPTIMIZE = "logical_plan_optimize"; public static final String DISTRIBUTION_PLANNER = "distribution_planner"; - public static final String PARTITION_FETCHER = "partition_fetcher"; public static final String SCHEMA_FETCHER = "schema_fetcher"; @@ -43,28 +47,58 @@ private QueryPlanCostMetricSet() { // empty constructor } - private Timer analyzerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; - private Timer logicalPlannerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; - private Timer distributionPlannerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; - private Timer partitionFetcherTimer = DoNothingMetricManager.DO_NOTHING_TIMER; - private Timer schemaFetcherTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer treeAnalyzerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer treeLogicalPlannerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer treeDistributionPlannerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer treePartitionFetcherTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer treeSchemaFetcherTimer = DoNothingMetricManager.DO_NOTHING_TIMER; - public void recordPlanCost(String stage, long costTimeInNanos) { + private Timer tableAnalyzerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer tableLogicalPlannerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer tableLogicalPlanOptimizerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer tableDistributionPlannerTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer tablePartitionFetcherTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + private Timer tableSchemaFetcherTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + + public void recordPlanCost(String type, String stage, long costTimeInNanos) { switch (stage) { case ANALYZER: - analyzerTimer.updateNanos(costTimeInNanos); + if (TREE_TYPE.equals(type)) { + treeAnalyzerTimer.updateNanos(costTimeInNanos); + } else { + tableAnalyzerTimer.updateNanos(costTimeInNanos); + } break; case LOGICAL_PLANNER: - logicalPlannerTimer.updateNanos(costTimeInNanos); + if (TREE_TYPE.equals(type)) { + treeLogicalPlannerTimer.updateNanos(costTimeInNanos); + } else { + tableLogicalPlannerTimer.updateNanos(costTimeInNanos); + } + break; + case LOGICAL_PLAN_OPTIMIZE: + tableLogicalPlanOptimizerTimer.updateNanos(costTimeInNanos); break; case DISTRIBUTION_PLANNER: - distributionPlannerTimer.updateNanos(costTimeInNanos); + if (TREE_TYPE.equals(type)) { + treeDistributionPlannerTimer.updateNanos(costTimeInNanos); + } else { + tableDistributionPlannerTimer.updateNanos(costTimeInNanos); + } break; case PARTITION_FETCHER: - partitionFetcherTimer.updateNanos(costTimeInNanos); + if (TREE_TYPE.equals(type)) { + treePartitionFetcherTimer.updateNanos(costTimeInNanos); + } else { + tablePartitionFetcherTimer.updateNanos(costTimeInNanos); + } break; case SCHEMA_FETCHER: - schemaFetcherTimer.updateNanos(costTimeInNanos); + if (TREE_TYPE.equals(type)) { + treeSchemaFetcherTimer.updateNanos(costTimeInNanos); + } else { + tableSchemaFetcherTimer.updateNanos(costTimeInNanos); + } break; default: throw new UnsupportedOperationException("Unsupported stage: " + stage); @@ -73,34 +107,93 @@ public void recordPlanCost(String stage, long costTimeInNanos) { @Override public void bindTo(AbstractMetricService metricService) { - analyzerTimer = + treeAnalyzerTimer = metricService.getOrCreateTimer( Metric.QUERY_PLAN_COST.toString(), MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TREE_TYPE, Tag.STAGE.toString(), ANALYZER); - logicalPlannerTimer = + treeLogicalPlannerTimer = metricService.getOrCreateTimer( Metric.QUERY_PLAN_COST.toString(), MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TREE_TYPE, Tag.STAGE.toString(), LOGICAL_PLANNER); - distributionPlannerTimer = + treeDistributionPlannerTimer = + metricService.getOrCreateTimer( + Metric.QUERY_PLAN_COST.toString(), + MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TREE_TYPE, + Tag.STAGE.toString(), + DISTRIBUTION_PLANNER); + treePartitionFetcherTimer = + metricService.getOrCreateTimer( + Metric.QUERY_PLAN_COST.toString(), + MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TREE_TYPE, + Tag.STAGE.toString(), + PARTITION_FETCHER); + treeSchemaFetcherTimer = + metricService.getOrCreateTimer( + Metric.QUERY_PLAN_COST.toString(), + MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TREE_TYPE, + Tag.STAGE.toString(), + SCHEMA_FETCHER); + + tableAnalyzerTimer = + metricService.getOrCreateTimer( + Metric.QUERY_PLAN_COST.toString(), + MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TABLE_TYPE, + Tag.STAGE.toString(), + ANALYZER); + tableLogicalPlannerTimer = + metricService.getOrCreateTimer( + Metric.QUERY_PLAN_COST.toString(), + MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TABLE_TYPE, + Tag.STAGE.toString(), + LOGICAL_PLANNER); + tableLogicalPlanOptimizerTimer = + metricService.getOrCreateTimer( + Metric.QUERY_PLAN_COST.toString(), + MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TABLE_TYPE, + Tag.STAGE.toString(), + LOGICAL_PLAN_OPTIMIZE); + tableDistributionPlannerTimer = metricService.getOrCreateTimer( Metric.QUERY_PLAN_COST.toString(), MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TABLE_TYPE, Tag.STAGE.toString(), DISTRIBUTION_PLANNER); - partitionFetcherTimer = + tablePartitionFetcherTimer = metricService.getOrCreateTimer( Metric.QUERY_PLAN_COST.toString(), MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TABLE_TYPE, Tag.STAGE.toString(), PARTITION_FETCHER); - schemaFetcherTimer = + tableSchemaFetcherTimer = metricService.getOrCreateTimer( Metric.QUERY_PLAN_COST.toString(), MetricLevel.IMPORTANT, + Tag.TYPE.toString(), + TABLE_TYPE, Tag.STAGE.toString(), SCHEMA_FETCHER); } @@ -108,14 +201,24 @@ public void bindTo(AbstractMetricService metricService) { @Override public void unbindFrom(AbstractMetricService metricService) { Arrays.asList( - ANALYZER, LOGICAL_PLANNER, DISTRIBUTION_PLANNER, PARTITION_FETCHER, SCHEMA_FETCHER) + ANALYZER, + LOGICAL_PLANNER, + LOGICAL_PLAN_OPTIMIZE, + DISTRIBUTION_PLANNER, + PARTITION_FETCHER, + SCHEMA_FETCHER) .forEach( stage -> - metricService.remove( - MetricType.TIMER, - Metric.QUERY_PLAN_COST.toString(), - Tag.STAGE.toString(), - stage)); + Arrays.asList(TREE_TYPE, TABLE_TYPE) + .forEach( + type -> + metricService.remove( + MetricType.TIMER, + Metric.QUERY_PLAN_COST.toString(), + Tag.TYPE.toString(), + type, + Tag.STAGE.toString(), + stage))); } public static QueryPlanCostMetricSet getInstance() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index fff9085efa67..a86dfe1dacfd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -46,7 +46,7 @@ import org.apache.iotdb.db.queryengine.plan.execution.config.TreeConfigTaskVisitor; import org.apache.iotdb.db.queryengine.plan.planner.TreeModelPlanner; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; -import org.apache.iotdb.db.queryengine.plan.relational.planner.RelationalModelPlanner; +import org.apache.iotdb.db.queryengine.plan.relational.planner.TableModelPlanner; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AddColumn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CreateDB; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CreateTable; @@ -291,8 +291,8 @@ private IQueryExecution createQueryExecutionForTableModel( queryContext.setTableQuery(true); queryContext.setTimeOut(timeOut); queryContext.setStartTime(startTime); - RelationalModelPlanner relationalModelPlanner = - new RelationalModelPlanner( + TableModelPlanner tableModelPlanner = + new TableModelPlanner( statement.toRelationalStatement(queryContext), sqlParser, metadata, @@ -301,7 +301,7 @@ private IQueryExecution createQueryExecutionForTableModel( scheduledExecutor, SYNC_INTERNAL_SERVICE_CLIENT_MANAGER, ASYNC_INTERNAL_SERVICE_CLIENT_MANAGER); - return new QueryExecution(relationalModelPlanner, queryContext, executor); + return new QueryExecution(tableModelPlanner, queryContext, executor); } private IQueryExecution createQueryExecutionForTableModel( @@ -340,8 +340,8 @@ private IQueryExecution createQueryExecutionForTableModel( if (statement instanceof WrappedInsertStatement) { ((WrappedInsertStatement) statement).setContext(queryContext); } - RelationalModelPlanner relationalModelPlanner = - new RelationalModelPlanner( + TableModelPlanner tableModelPlanner = + new TableModelPlanner( statement, sqlParser, metadata, @@ -350,7 +350,7 @@ private IQueryExecution createQueryExecutionForTableModel( scheduledExecutor, SYNC_INTERNAL_SERVICE_CLIENT_MANAGER, ASYNC_INTERNAL_SERVICE_CLIENT_MANAGER); - return new QueryExecution(relationalModelPlanner, queryContext, executor); + return new QueryExecution(tableModelPlanner, queryContext, executor); } public IQueryExecution getQueryExecution(Long queryId) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index 3c55f9588cf5..f0a91e5e766f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -200,6 +200,7 @@ import static org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant.ENDTIME; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.PARTITION_FETCHER; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.SCHEMA_FETCHER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TREE_TYPE; import static org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeUtils.removeLogicalView; import static org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeUtils.validateSchema; import static org.apache.iotdb.db.queryengine.plan.analyze.ExpressionAnalyzer.bindSchemaForExpression; @@ -530,7 +531,8 @@ private ISchemaTree analyzeSchema( logger.debug("[EndFetchSchema]"); long schemaFetchCost = System.nanoTime() - startTime; context.setFetchSchemaCost(schemaFetchCost); - QueryPlanCostMetricSet.getInstance().recordPlanCost(SCHEMA_FETCHER, schemaFetchCost); + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TREE_TYPE, SCHEMA_FETCHER, schemaFetchCost); } analysis.setSchemaTree(schemaTree); @@ -2215,7 +2217,8 @@ private DataPartition fetchDataPartitionByDevices( } } finally { long partitionFetchCost = System.nanoTime() - startTime; - QueryPlanCostMetricSet.getInstance().recordPlanCost(PARTITION_FETCHER, partitionFetchCost); + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TREE_TYPE, PARTITION_FETCHER, partitionFetchCost); context.setFetchPartitionCost(partitionFetchCost); } } @@ -2378,7 +2381,7 @@ private void analyzeInto( long startTime = System.nanoTime(); ISchemaTree targetSchemaTree = schemaFetcher.fetchSchema(targetPathTree, true, context); QueryPlanCostMetricSet.getInstance() - .recordPlanCost(SCHEMA_FETCHER, System.nanoTime() - startTime); + .recordPlanCost(TREE_TYPE, SCHEMA_FETCHER, System.nanoTime() - startTime); deviceViewIntoPathDescriptor.bindType(targetSchemaTree); analysis.setDeviceViewIntoPathDescriptor(deviceViewIntoPathDescriptor); @@ -2456,7 +2459,7 @@ private void analyzeInto( ISchemaTree targetSchemaTree = schemaFetcher.fetchSchema(targetPathTree, true, context); updateSchemaTreeByViews(analysis, targetSchemaTree, context); QueryPlanCostMetricSet.getInstance() - .recordPlanCost(SCHEMA_FETCHER, System.nanoTime() - startTime); + .recordPlanCost(TREE_TYPE, SCHEMA_FETCHER, System.nanoTime() - startTime); intoPathDescriptor.bindType(targetSchemaTree); analysis.setIntoPathDescriptor(intoPathDescriptor); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analyzer.java index eca06360c9c5..238620b6f48d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analyzer.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.Statement; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.ANALYZER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TREE_TYPE; /** Analyze the statement and generate Analysis. */ public class Analyzer { @@ -51,7 +52,8 @@ public Analysis analyze(Statement statement) { } if (statement.isQuery()) { - QueryPlanCostMetricSet.getInstance().recordPlanCost(ANALYZER, System.nanoTime() - startTime); + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TREE_TYPE, ANALYZER, System.nanoTime() - startTime); } return analysis; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/TemplatedAnalyze.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/TemplatedAnalyze.java index d3bf070d2bec..c1f0971b2931 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/TemplatedAnalyze.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/TemplatedAnalyze.java @@ -58,6 +58,7 @@ import java.util.stream.Collectors; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.PARTITION_FETCHER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TREE_TYPE; import static org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeVisitor.CONFIG; import static org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeVisitor.DEVICE_EXPRESSION; import static org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeVisitor.END_TIME_EXPRESSION; @@ -418,7 +419,7 @@ private static DataPartition fetchDataPartitionByDevices( } } finally { QueryPlanCostMetricSet.getInstance() - .recordPlanCost(PARTITION_FETCHER, System.nanoTime() - startTime); + .recordPlanCost(TREE_TYPE, PARTITION_FETCHER, System.nanoTime() - startTime); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java index 108b117298c9..ff2509f1d693 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java @@ -72,6 +72,7 @@ import static org.apache.iotdb.db.queryengine.common.DataNodeEndPoints.isSameNode; import static org.apache.iotdb.db.queryengine.metric.QueryExecutionMetricSet.WAIT_FOR_RESULT; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.DISTRIBUTION_PLANNER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TREE_TYPE; /** * QueryExecution stores all the status of a query which is being prepared or running inside the MPP @@ -153,7 +154,7 @@ interface ISourceHandleSupplier { @Override public void start() { - final long startTime = System.nanoTime(); + long startTime = System.nanoTime(); if (skipExecute()) { LOGGER.debug("[SkipExecute]"); if (analysis.isFailed()) { @@ -184,8 +185,7 @@ public void start() { doDistributedPlan(); // update timeout after finishing plan stage - context.setTimeOut( - context.getTimeOut() - (System.currentTimeMillis() - context.getStartTime())); + context.setTimeOut(context.getTimeOut() - (System.nanoTime() - context.getStartTime())); stateMachine.transitionToPlanned(); if (context.getQueryType() == QueryType.READ) { @@ -302,7 +302,8 @@ public void doDistributedPlan() { if (analysis.isQuery()) { long distributionPlanCost = System.nanoTime() - startTime; context.setDistributionPlanCost(distributionPlanCost); - QUERY_PLAN_COST_METRIC_SET.recordPlanCost(DISTRIBUTION_PLANNER, distributionPlanCost); + QUERY_PLAN_COST_METRIC_SET.recordPlanCost( + TREE_TYPE, DISTRIBUTION_PLANNER, distributionPlanCost); } // if is this Statement is ShowQueryStatement, set its instances to the highest priority, so diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java index ce8065659fa2..83c0cdb5af1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.TableLogicalPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.distribute.TableDistributedPlanGenerator; import org.apache.iotdb.db.queryengine.plan.relational.planner.distribute.TableDistributedPlanner; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AstVisitor; @@ -65,7 +66,7 @@ public StatementMemorySource visitExplain( new ColumnHeader(IoTDBConstant.COLUMN_DISTRIBUTION_PLAN, TSDataType.TEXT)), true); final LogicalQueryPlan logicalPlan = - new org.apache.iotdb.db.queryengine.plan.relational.planner.LogicalPlanner( + new TableLogicalPlanner( context.getQueryContext(), LocalExecutionPlanner.getInstance().metadata, context.getQueryContext().getSession(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanner.java index 65f28cd4a9e5..1a05c5fc6e86 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanner.java @@ -31,6 +31,7 @@ import java.util.List; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.LOGICAL_PLANNER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TREE_TYPE; /** Generate a logical plan for the statement. */ public class LogicalPlanner { @@ -52,13 +53,15 @@ public LogicalQueryPlan plan(Analysis analysis) { if (analysis.isQuery()) { long planFinishTime = System.nanoTime(); - QueryPlanCostMetricSet.getInstance() - .recordPlanCost(LOGICAL_PLANNER, System.nanoTime() - planFinishTime); context.setLogicalPlanCost(planFinishTime - startTime); + for (PlanOptimizer optimizer : optimizers) { rootNode = optimizer.optimize(rootNode, analysis, context); } context.setLogicalOptimizationCost(System.nanoTime() - planFinishTime); + + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TREE_TYPE, LOGICAL_PLANNER, System.nanoTime() - planFinishTime); } return new LogicalQueryPlan(context, rootNode); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java index f5861cf45fee..683650cb87f4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java @@ -23,8 +23,10 @@ import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.SessionInfo; import org.apache.iotdb.db.queryengine.execution.warnings.WarningCollector; +import org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Parameter; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WrappedInsertStatement; @@ -32,6 +34,8 @@ import java.util.Map; import static java.util.Objects.requireNonNull; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.ANALYZER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TABLE_TYPE; public class Analyzer { @@ -76,11 +80,16 @@ public Analysis analyze(Statement statement) { analysis.setDatabaseName(session.getDatabaseName().get()); } + long startTime = System.nanoTime(); StatementAnalyzer analyzer = statementAnalyzerFactory.createStatementAnalyzer( analysis, context, session, warningCollector, CorrelationSupport.ALLOWED); analyzer.analyze(statement); + if (statement instanceof Query) { + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TABLE_TYPE, ANALYZER, System.nanoTime() - startTime); + } // TODO access control // check column access permissions for each table diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/LogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java similarity index 94% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/LogicalPlanner.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index 8ba4286183b0..5de5aa292362 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/LogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.queryengine.common.header.ColumnHeader; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.execution.warnings.WarningCollector; +import org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; @@ -64,11 +65,14 @@ import java.util.Optional; import static java.util.Objects.requireNonNull; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.LOGICAL_PLANNER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.LOGICAL_PLAN_OPTIMIZE; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TABLE_TYPE; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowDevice.getDeviceColumnHeaderList; import static org.apache.iotdb.db.queryengine.plan.relational.type.InternalTypeManager.getTSDataType; -public class LogicalPlanner { - private static final Logger LOG = LoggerFactory.getLogger(LogicalPlanner.class); +public class TableLogicalPlanner { + private static final Logger LOG = LoggerFactory.getLogger(TableLogicalPlanner.class); private final MPPQueryContext queryContext; private final SessionInfo sessionInfo; private final SymbolAllocator symbolAllocator = new SymbolAllocator(); @@ -76,7 +80,7 @@ public class LogicalPlanner { private final Metadata metadata; private final WarningCollector warningCollector; - public LogicalPlanner( + public TableLogicalPlanner( MPPQueryContext queryContext, Metadata metadata, SessionInfo sessionInfo, @@ -91,7 +95,7 @@ public LogicalPlanner( } @TestOnly - public LogicalPlanner( + public TableLogicalPlanner( MPPQueryContext queryContext, Metadata metadata, SessionInfo sessionInfo, @@ -105,9 +109,14 @@ public LogicalPlanner( } public LogicalQueryPlan plan(Analysis analysis) { + long startTime = System.nanoTime(); PlanNode planNode = planStatement(analysis, analysis.getStatement()); if (analysis.getStatement() instanceof Query) { + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TABLE_TYPE, LOGICAL_PLANNER, System.nanoTime() - startTime); + startTime = System.nanoTime(); + for (PlanOptimizer optimizer : planOptimizers) { planNode = optimizer.optimize( @@ -123,6 +132,8 @@ public LogicalQueryPlan plan(Analysis analysis) { warningCollector, PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector())); } + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TABLE_TYPE, LOGICAL_PLAN_OPTIMIZE, System.nanoTime() - startTime); } return new LogicalQueryPlan(queryContext, planNode); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationalModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java similarity index 97% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationalModelPlanner.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java index 2a2f40e94ca0..2df9f797158b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationalModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java @@ -47,7 +47,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; -public class RelationalModelPlanner implements IPlanner { +public class TableModelPlanner implements IPlanner { private final Statement statement; @@ -69,7 +69,7 @@ public class RelationalModelPlanner implements IPlanner { private final IClientManager asyncInternalServiceClientManager; - public RelationalModelPlanner( + public TableModelPlanner( Statement statement, SqlParser sqlParser, Metadata metadata, @@ -107,7 +107,7 @@ public IAnalysis analyze(MPPQueryContext context) { @Override public LogicalQueryPlan doLogicalPlan(IAnalysis analysis, MPPQueryContext context) { - return new LogicalPlanner(context, metadata, context.getSession(), warningCollector) + return new TableLogicalPlanner(context, metadata, context.getSession(), warningCollector) .plan((Analysis) analysis); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java index 363bcde7a442..069d4024d658 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java @@ -16,6 +16,7 @@ import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelLocation; +import org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.analyze.QueryType; import org.apache.iotdb.db.queryengine.plan.planner.distribution.WriteFragmentParallelPlanner; import org.apache.iotdb.db.queryengine.plan.planner.plan.DistributedQueryPlan; @@ -43,6 +44,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.iotdb.db.queryengine.execution.warnings.WarningCollector.NOOP; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.DISTRIBUTION_PLANNER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TABLE_TYPE; public class TableDistributedPlanner { @@ -62,6 +65,7 @@ public TableDistributedPlanner( } public DistributedQueryPlan plan() { + long startTime = System.nanoTime(); TableDistributedPlanGenerator.PlanContext planContext = new TableDistributedPlanGenerator.PlanContext(); PlanNode outputNodeWithExchange = generateDistributedPlanWithOptimize(planContext); @@ -71,9 +75,15 @@ public DistributedQueryPlan plan() { .getRespDatasetHeader() .setTableColumnToTsBlockIndexMap((OutputNode) outputNodeWithExchange); } + adjustUpStream(outputNodeWithExchange, planContext); + DistributedQueryPlan resultDistributedPlan = generateDistributedPlan(outputNodeWithExchange); - return generateDistributedPlan(outputNodeWithExchange); + if (analysis.getStatement() instanceof Query) { + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TABLE_TYPE, DISTRIBUTION_PLANNER, System.nanoTime() - startTime); + } + return resultDistributedPlan; } public PlanNode generateDistributedPlanWithOptimize( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 6e738a1c4aa4..3e525cc32d8e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -20,6 +20,7 @@ import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.MultiChildProcessNode; @@ -56,6 +57,9 @@ import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.ATTRIBUTE; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.MEASUREMENT; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TIME; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.PARTITION_FETCHER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.SCHEMA_FETCHER; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.TABLE_TYPE; import static org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeVisitor.getTimePartitionSlotList; import static org.apache.iotdb.db.queryengine.plan.relational.planner.ir.GlobalTimePredicateExtractVisitor.extractGlobalTimeFilter; @@ -274,10 +278,14 @@ private void tableMetadataIndexScan(TableScanNode node, List metadat node.getIdAndAttributeIndexMap().put(columnName, attributeIndex++); } } + + long startTime = System.nanoTime(); List deviceEntries = metadata.indexScan( node.getQualifiedObjectName(), metadataExpressions, attributeColumns, queryContext); node.setDeviceEntries(deviceEntries); + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TABLE_TYPE, SCHEMA_FETCHER, System.nanoTime() - startTime); if (deviceEntries.isEmpty()) { analysis.setFinishQueryAfterAnalyze(); @@ -289,6 +297,8 @@ private void tableMetadataIndexScan(TableScanNode node, List metadat .orElse(null); node.setTimeFilter(timeFilter); String treeModelDatabase = "root." + node.getQualifiedObjectName().getDatabaseName(); + + startTime = System.nanoTime(); DataPartition dataPartition = fetchDataPartitionByDevices(treeModelDatabase, deviceEntries, timeFilter); @@ -303,6 +313,9 @@ private void tableMetadataIndexScan(TableScanNode node, List metadat } else { analysis.upsertDataPartition(dataPartition); } + + QueryPlanCostMetricSet.getInstance() + .recordPlanCost(TABLE_TYPE, PARTITION_FETCHER, System.nanoTime() - startTime); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java index 250208e6fd5a..9024de680b39 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java @@ -51,8 +51,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.OperatorNotFoundException; import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; -import org.apache.iotdb.db.queryengine.plan.relational.planner.LogicalPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.planner.TableLogicalPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.distribute.TableDistributedPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CollectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; @@ -123,7 +123,7 @@ public class AnalyzerTest { String sql; Analysis actualAnalysis; MPPQueryContext context; - LogicalPlanner logicalPlanner; + TableLogicalPlanner logicalPlanner; LogicalQueryPlan logicalQueryPlan; PlanNode rootNode; TableDistributedPlanner distributionPlanner; @@ -182,7 +182,7 @@ public void singleTableNoFilterTest() { assertNotNull(actualAnalysis); assertEquals(1, actualAnalysis.getTables().size()); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -221,7 +221,7 @@ public void singleTableWithFilterTest1() { actualAnalysis = analyzeSQL(sql, metadata, context); assertNotNull(actualAnalysis); assertEquals(1, actualAnalysis.getTables().size()); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -272,7 +272,7 @@ public void singleTableWithFilterTest2() { actualAnalysis = analyzeSQL(sql, metadata, context); assertNotNull(actualAnalysis); assertEquals(1, actualAnalysis.getTables().size()); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -331,7 +331,7 @@ public void singleTableWithFilterTest3() { actualAnalysis = analyzeSQL(sql, metadata, context); assertNotNull(actualAnalysis); assertEquals(1, actualAnalysis.getTables().size()); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -361,7 +361,7 @@ public void singleTableWithFilterTest4() { actualAnalysis = analyzeSQL(sql, metadata, context); assertNotNull(actualAnalysis); assertEquals(1, actualAnalysis.getTables().size()); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -390,7 +390,7 @@ public void singleTableWithFilterTest5() { actualAnalysis = analyzeSQL(sql, metadata, context); assertNotNull(actualAnalysis); assertEquals(1, actualAnalysis.getTables().size()); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -431,7 +431,7 @@ public void singleTableWithFilterTest6() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); assertEquals(1, actualAnalysis.getTables().size()); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -479,7 +479,7 @@ public void singleTableWithFilterTest6() { actualAnalysis = analyzeSQL(sql, metadata, context); assertEquals(1, actualAnalysis.getTables().size()); context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -512,7 +512,7 @@ public void singleTableProjectTest() { actualAnalysis = analyzeSQL(sql, metadata, context); assertNotNull(actualAnalysis); assertEquals(1, actualAnalysis.getTables().size()); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -527,7 +527,7 @@ public void singleTableProjectTest() { assertNotNull(actualAnalysis); assertEquals(1, actualAnalysis.getTables().size()); context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -549,7 +549,7 @@ public void singleTableProjectTest() { actualAnalysis = analyzeSQL(sql, metadata, context); context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode.getChildren().get(0) instanceof ProjectNode); @@ -570,7 +570,7 @@ public void singleTableProjectTest() { actualAnalysis = analyzeSQL(sql, metadata, context); context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -587,7 +587,7 @@ public void expressionTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode.getChildren().get(0) instanceof FilterNode); @@ -606,7 +606,7 @@ public void expressionTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); @@ -623,7 +623,7 @@ public void expressionTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode.getChildren().get(0) instanceof ProjectNode); @@ -640,7 +640,7 @@ public void expressionTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); @@ -649,7 +649,7 @@ public void expressionTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); @@ -664,7 +664,7 @@ public void expressionTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); @@ -681,7 +681,7 @@ public void functionTest() { sql = "SELECT CAST(s2 AS DOUBLE) FROM table1 WHERE CAST(s1 AS DOUBLE) > 1.0"; context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); @@ -690,7 +690,7 @@ public void functionTest() { "SELECT SUBSTRING(tag1, 2), SUBSTRING(tag2, s1) FROM table1 WHERE SUBSTRING(tag2, 1) = 'A'"; context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); @@ -698,7 +698,7 @@ public void functionTest() { sql = "SELECT ROUND(s1, 1) FROM table1 WHERE ROUND(s2, 2) > 1.0"; context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); @@ -706,7 +706,7 @@ public void functionTest() { sql = "SELECT REPLACE(tag1, 'A', 'B') FROM table1 WHERE REPLACE(attr1, 'C', 'D') = 'D'"; context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); } @@ -717,7 +717,7 @@ public void diffTest() { sql = "SELECT DIFF(s1) FROM table1 WHERE DIFF(s2) > 0"; context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode.getChildren().get(0) instanceof ProjectNode); @@ -760,7 +760,7 @@ public void diffTest() { sql = "SELECT s1 FROM table1 WHERE DIFF(s2) > 0 and time > 5 and tag1 = 'A' and s1 = 1"; context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); - logicalPlanner = new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + logicalPlanner = new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); logicalQueryPlan = logicalPlanner.plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode.getChildren().get(0) instanceof ProjectNode); @@ -779,7 +779,7 @@ public void predicatePushDownTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -805,7 +805,8 @@ public void limitOffsetTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(actualAnalysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector) + .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode.getChildren().get(0) instanceof OffsetNode); OffsetNode offsetNode = (OffsetNode) rootNode.getChildren().get(0); @@ -819,7 +820,8 @@ public void limitOffsetTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(actualAnalysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector) + .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode.getChildren().get(0) instanceof ProjectNode); assertTrue(getChildrenNode(rootNode, 2) instanceof OffsetNode); @@ -835,7 +837,8 @@ public void predicateCannotNormalizedTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(actualAnalysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector) + .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); assertTrue(getChildrenNode(rootNode, 1) instanceof TableScanNode); @@ -852,7 +855,8 @@ public void limitEliminationTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(actualAnalysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector) + .plan(actualAnalysis); // logical plan: `OutputNode - ProjectNode - LimitNode - TableScanNode` rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -885,7 +889,8 @@ public void limitEliminationTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(actualAnalysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector) + .plan(actualAnalysis); // logical plan: `OutputNode - ProjectNode - LimitNode - TableScanNode` rootNode = logicalQueryPlan.getRootNode(); assertTrue(rootNode instanceof OutputNode); @@ -908,7 +913,8 @@ public void limitEliminationTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(actualAnalysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector) + .plan(actualAnalysis); // logical plan: `OutputNode - ProjectNode - LimitNode - TableScanNode` rootNode = logicalQueryPlan.getRootNode(); // distributed plan: `IdentitySink - OutputNode - ProjectNode - LimitNode - TableScanNode` @@ -928,7 +934,8 @@ public void duplicateProjectionsTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(actualAnalysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector) + .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); distributionPlanner = new TableDistributedPlanner(actualAnalysis, logicalQueryPlan, context); distributedQueryPlan = distributionPlanner.plan(); @@ -1039,7 +1046,7 @@ public void analyzeInsertTablet() { assertEquals(3, partitionSlotMapMap.size()); logicalQueryPlan = - new LogicalPlanner(context, mockMetadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, mockMetadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); RelationalInsertTabletNode insertTabletNode = @@ -1085,7 +1092,7 @@ public void analyzeInsertRow() { assertEquals(1, partitionSlotMapMap.size()); logicalQueryPlan = - new LogicalPlanner(context, mockMetadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, mockMetadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); RelationalInsertRowNode insertNode = (RelationalInsertRowNode) logicalQueryPlan.getRootNode(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/LimitOffsetPushDownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/LimitOffsetPushDownTest.java index 2e6f3bca6dfd..6e2bb2601f12 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/LimitOffsetPushDownTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/LimitOffsetPushDownTest.java @@ -25,7 +25,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ExchangeNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; -import org.apache.iotdb.db.queryengine.plan.relational.planner.LogicalPlanner; +import org.apache.iotdb.db.queryengine.plan.relational.planner.TableLogicalPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.distribute.TableDistributedPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CollectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LimitNode; @@ -59,7 +59,7 @@ public class LimitOffsetPushDownTest { String sql; Analysis actualAnalysis; MPPQueryContext context; - LogicalPlanner logicalPlanner; + TableLogicalPlanner logicalPlanner; LogicalQueryPlan logicalQueryPlan; PlanNode rootNode; TableDistributedPlanner distributionPlanner; @@ -73,7 +73,7 @@ public void noOrderByTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output - Project - Offset - Limit - TableScan` @@ -113,7 +113,7 @@ public void orderByAllIDsTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output - Offset - Limit - Project - StreamSort - Project - TableScan` @@ -147,7 +147,7 @@ public void orderByAllIDsTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output - Limit - StreamSort - TableScan` @@ -186,7 +186,7 @@ public void orderBySomeTagsTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); rootNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output - Offset - Limit - Project - StreamSort - Project - TableScan` @@ -227,7 +227,7 @@ public void orderByTimeTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); // LogicalPlan: `Output - Offset - Project - TopK - Project - TableScan` rootNode = logicalQueryPlan.getRootNode(); @@ -267,7 +267,7 @@ public void orderByOthersTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); // LogicalPlan: `Output - Offset - Project - TopK - Project - TableScan` rootNode = logicalQueryPlan.getRootNode(); @@ -304,7 +304,7 @@ public void limitDiffProjectTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); actualAnalysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP) .plan(actualAnalysis); // LogicalPlan: `Output - Project - Limit - TableScan` rootNode = logicalQueryPlan.getRootNode(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SortTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SortTest.java index a07678ccaaa2..045de6debb93 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SortTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SortTest.java @@ -31,7 +31,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ExchangeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.sink.IdentitySinkNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; -import org.apache.iotdb.db.queryengine.plan.relational.planner.LogicalPlanner; +import org.apache.iotdb.db.queryengine.plan.relational.planner.TableLogicalPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.distribute.TableDistributedPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LimitNode; @@ -100,7 +100,7 @@ public void someIDColumnTimeOthersSortTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output-Offset-Limit-Project-StreamSort-Project-Filter-TableScan` @@ -176,7 +176,7 @@ public void someIDColumnTimeOthersSortTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output-Offset-Limit-StreamSort-TableScan` assertTrue(getChildrenNode(logicalPlanNode, 3) instanceof StreamSortNode); @@ -202,7 +202,7 @@ public void allIDColumnTimeSortWithLimitTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output-Offset-Limit-Project-StreamSort-Project-Filter-TableScan` @@ -280,7 +280,7 @@ public void allIDColumnTimeSortNoLimitTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output-Project-StreamSort-Project-Filter-TableScan` @@ -348,7 +348,7 @@ public void someIDColumnOthersTimeSortTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output-Offset-Limit-Project-StreamSort-Project-Filter-TableScan` @@ -428,7 +428,7 @@ public void allIDColumnOthersTimeSortTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output-Offset-Limit-Project-StreamSort-Project-Filter-TableScan` @@ -506,7 +506,7 @@ public void orderByTimeTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); assertTopKNoFilter(originalDeviceEntries1, originalDeviceEntries2, DESC, 15, 0, true); @@ -518,7 +518,7 @@ public void orderByTimeTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); assertTopKWithFilter(originalDeviceEntries1, originalDeviceEntries2, DESC, 0, 0, false); @@ -529,7 +529,7 @@ public void orderByTimeTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); assertTopKWithFilter(originalDeviceEntries1, originalDeviceEntries2, DESC, 0, 0, false); @@ -540,7 +540,7 @@ public void orderByTimeTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); assertTopKWithFilter(originalDeviceEntries1, originalDeviceEntries2, DESC, 0, 0, false); @@ -555,7 +555,7 @@ public void orderByOthersTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); assertTopKNoFilter(originalDeviceEntries1, originalDeviceEntries2, ASC, 0, 0, false); @@ -566,7 +566,7 @@ public void orderByOthersTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); assertTopKWithFilter(originalDeviceEntries1, originalDeviceEntries2, ASC, 0, 0, false); @@ -577,7 +577,7 @@ public void orderByOthersTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); assertTopKWithFilter(originalDeviceEntries1, originalDeviceEntries2, ASC, 0, 0, false); @@ -588,7 +588,7 @@ public void orderByOthersTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); assertTopKWithFilter(originalDeviceEntries1, originalDeviceEntries2, ASC, 0, 0, false); } @@ -600,7 +600,7 @@ public void projectSortTest() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); distributionPlanner = new TableDistributedPlanner(analysis, logicalQueryPlan, context); distributedQueryPlan = distributionPlanner.plan(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SubQueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SubQueryTest.java index 6db97aac6e4e..66fd695ae83d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SubQueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SubQueryTest.java @@ -26,7 +26,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ExchangeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.sink.IdentitySinkNode; -import org.apache.iotdb.db.queryengine.plan.relational.planner.LogicalPlanner; +import org.apache.iotdb.db.queryengine.plan.relational.planner.TableLogicalPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.distribute.TableDistributedPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LimitNode; @@ -94,7 +94,7 @@ public void subQueryTest1() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output - Offset - Limit - StreamSort - Project - TableScan` @@ -179,7 +179,7 @@ public void subQueryTest2() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output - Offset - TopK - Project - Limit - Project - StreamSort - Project - @@ -275,7 +275,7 @@ public void subQueryTest3() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output - Offset - ProjectNode - TopK - Project - Limit - Project - StreamSort - @@ -390,7 +390,7 @@ public void subQueryTest4() { context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); analysis = analyzeSQL(sql, metadata, context); logicalQueryPlan = - new LogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); + new TableLogicalPlanner(context, metadata, sessionInfo, warningCollector).plan(analysis); logicalPlanNode = logicalQueryPlan.getRootNode(); // LogicalPlan: `Output - Offset - ProjectNode - TopK - Project - Filter - Limit - Project - @@ -487,7 +487,7 @@ public void subQueryTest5() { sql = "SELECT * FROM (SELECT * FROM table1 WHERE s1>1) WHERE s2>2"; analysis = analyzeSQL(sql, TEST_MATADATA, QUERY_CONTEXT); logicalPlanNode = - new LogicalPlanner(QUERY_CONTEXT, TEST_MATADATA, SESSION_INFO, DEFAULT_WARNING) + new TableLogicalPlanner(QUERY_CONTEXT, TEST_MATADATA, SESSION_INFO, DEFAULT_WARNING) .plan(analysis) .getRootNode(); assertNodeMatches(logicalPlanNode, OutputNode.class, TableScanNode.class); @@ -502,7 +502,7 @@ public void subQueryTest6() { sql = "SELECT * FROM (SELECT * FROM table1 limit 10) limit 5"; analysis = analyzeSQL(sql, TEST_MATADATA, QUERY_CONTEXT); logicalPlanNode = - new LogicalPlanner(QUERY_CONTEXT, TEST_MATADATA, SESSION_INFO, DEFAULT_WARNING) + new TableLogicalPlanner(QUERY_CONTEXT, TEST_MATADATA, SESSION_INFO, DEFAULT_WARNING) .plan(analysis) .getRootNode(); assertNodeMatches(logicalPlanNode, OutputNode.class, LimitNode.class, TableScanNode.class); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java index 254fd5f0912f..dd3fcae48c6f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java @@ -84,8 +84,8 @@ public LogicalQueryPlan createPlan( Analysis analysis = analyze(sql, metadata); this.analysis = analysis; - LogicalPlanner logicalPlanner = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); + TableLogicalPlanner logicalPlanner = + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP); plan = logicalPlanner.plan(analysis); @@ -102,8 +102,8 @@ public LogicalQueryPlan createPlan( Analysis analysis = analyze(sql, metadata); - LogicalPlanner logicalPlanner = - new LogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP, optimizers); + TableLogicalPlanner logicalPlanner = + new TableLogicalPlanner(context, metadata, sessionInfo, WarningCollector.NOOP, optimizers); return logicalPlanner.plan(analysis); } From 64cf3844c027b1f8c44ec84d28ef554dbe834e78 Mon Sep 17 00:00:00 2001 From: Haonan Date: Tue, 3 Sep 2024 15:39:29 +0800 Subject: [PATCH 10/13] Change FileTimeIndexCache to region level (#13353) * Check FileTimeIndexCache to region level * add log * fix ut * fix it error * fix error log --- .../iotdb/db/storageengine/StorageEngine.java | 4 + .../storageengine/dataregion/DataRegion.java | 73 +++++------ .../dataregion/tsfile/TsFileManager.java | 9 +- .../dataregion/tsfile/TsFileResource.java | 12 +- .../timeindex/FileTimeIndexCacheRecorder.java | 118 ++++++++---------- .../FileTimeIndexCacheReader.java | 10 +- .../FileTimeIndexCacheWriter.java | 4 + 7 files changed, 112 insertions(+), 118 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java index 208fac15061d..11b2a1c6c632 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java @@ -355,6 +355,7 @@ private void timedFlushUnseqMemTable() { private void asyncRecoverTsFileResource() { List> futures = new LinkedList<>(); + long startRecoverTime = System.currentTimeMillis(); for (DataRegion dataRegion : dataRegionMap.values()) { if (dataRegion != null) { List> asyncTsFileResourceRecoverTasks = @@ -378,6 +379,9 @@ private void asyncRecoverTsFileResource() { checkResults(futures, "async recover tsfile resource meets error."); recoverRepairData(); isReadyForNonReadWriteFunctions.set(true); + LOGGER.info( + "TsFile Resource recover cost: {}s.", + (System.currentTimeMillis() - startRecoverTime) / 1000); }, ThreadName.STORAGE_ENGINE_RECOVER_TRIGGER.getName()); recoverEndTrigger.start(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 3953fd8d1a63..09b30bf1bb6d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -546,12 +546,24 @@ private void recover() throws DataRegionException { latestPartitionId, ((TreeMap>) partitionTmpUnseqTsFiles).lastKey()); } + File logFile = SystemFileFactory.INSTANCE.getFile(dataRegionSysDir, "FileTimeIndexCache_0"); + Map fileTimeIndexMap = new HashMap<>(); + if (logFile.exists()) { + try { + FileTimeIndexCacheReader logReader = + new FileTimeIndexCacheReader(logFile, dataRegionId); + logReader.read(fileTimeIndexMap); + } catch (Exception e) { + throw new RuntimeException(e); + } + } for (Entry> partitionFiles : partitionTmpSeqTsFiles.entrySet()) { Callable asyncRecoverTask = recoverFilesInPartition( partitionFiles.getKey(), dataRegionRecoveryContext, partitionFiles.getValue(), + fileTimeIndexMap, true); if (asyncRecoverTask != null) { asyncTsFileResourceRecoverTaskList.add(asyncRecoverTask); @@ -564,6 +576,7 @@ private void recover() throws DataRegionException { partitionFiles.getKey(), dataRegionRecoveryContext, partitionFiles.getValue(), + fileTimeIndexMap, false); if (asyncRecoverTask != null) { asyncTsFileResourceRecoverTaskList.add(asyncRecoverTask); @@ -870,45 +883,29 @@ private Callable recoverFilesInPartition( long partitionId, DataRegionRecoveryContext context, List resourceList, + Map fileTimeIndexMap, boolean isSeq) { - - File partitionSysDir = - SystemFileFactory.INSTANCE.getFile(dataRegionSysDir, String.valueOf(partitionId)); - File logFile = SystemFileFactory.INSTANCE.getFile(partitionSysDir, "FileTimeIndexCache_0"); - if (logFile.exists()) { - Map fileTimeIndexMap; - try { - FileTimeIndexCacheReader logReader = - new FileTimeIndexCacheReader(logFile, dataRegionId, partitionId); - fileTimeIndexMap = logReader.read(); - } catch (Exception e) { - throw new RuntimeException(e); - } - List resourceListForAsyncRecover = new ArrayList<>(); - List resourceListForSyncRecover = new ArrayList<>(); - Callable asyncRecoverTask = null; - for (TsFileResource tsFileResource : resourceList) { - if (fileTimeIndexMap.containsKey(tsFileResource.getTsFileID())) { - tsFileResource.setTimeIndex(fileTimeIndexMap.get(tsFileResource.getTsFileID())); - tsFileResource.setStatus(TsFileResourceStatus.NORMAL); - tsFileManager.add(tsFileResource, isSeq); - resourceListForAsyncRecover.add(tsFileResource); - } else { - resourceListForSyncRecover.add(tsFileResource); - } - } - if (!resourceListForAsyncRecover.isEmpty()) { - asyncRecoverTask = - asyncRecoverFilesInPartition(partitionId, context, resourceListForAsyncRecover, isSeq); - } - if (!resourceListForSyncRecover.isEmpty()) { - syncRecoverFilesInPartition(partitionId, context, resourceListForSyncRecover, isSeq); + List resourceListForAsyncRecover = new ArrayList<>(); + List resourceListForSyncRecover = new ArrayList<>(); + Callable asyncRecoverTask = null; + for (TsFileResource tsFileResource : resourceList) { + if (fileTimeIndexMap.containsKey(tsFileResource.getTsFileID())) { + tsFileResource.setTimeIndex(fileTimeIndexMap.get(tsFileResource.getTsFileID())); + tsFileResource.setStatus(TsFileResourceStatus.NORMAL); + tsFileManager.add(tsFileResource, isSeq); + resourceListForAsyncRecover.add(tsFileResource); + } else { + resourceListForSyncRecover.add(tsFileResource); } - return asyncRecoverTask; - } else { - syncRecoverFilesInPartition(partitionId, context, resourceList, isSeq); - return null; } + if (!resourceListForAsyncRecover.isEmpty()) { + asyncRecoverTask = + asyncRecoverFilesInPartition(partitionId, context, resourceListForAsyncRecover, isSeq); + } + if (!resourceListForSyncRecover.isEmpty()) { + syncRecoverFilesInPartition(partitionId, context, resourceListForSyncRecover, isSeq); + } + return asyncRecoverTask; } private Callable asyncRecoverFilesInPartition( @@ -3919,9 +3916,7 @@ private void renameAndHandleError(String originFileName, String newFileName) { } public void compactFileTimeIndexCache() { - for (long timePartition : partitionMaxFileVersions.keySet()) { - tsFileManager.compactFileTimeIndexCache(timePartition); - } + tsFileManager.compactFileTimeIndexCache(); } @TestOnly diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java index 06409cf3cf86..50aee584f9c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java @@ -392,16 +392,17 @@ public boolean isLatestTimePartition(long timePartitionId) { && unsequenceFiles.higherKey(timePartitionId) == null); } - public void compactFileTimeIndexCache(long timePartition) { + public void compactFileTimeIndexCache() { + int currentResourceSize = size(true) + size(false); readLock(); try { FileTimeIndexCacheRecorder.getInstance() .compactFileTimeIndexIfNeeded( storageGroupName, Integer.parseInt(dataRegionId), - timePartition, - sequenceFiles.get(timePartition), - unsequenceFiles.get(timePartition)); + currentResourceSize, + sequenceFiles, + unsequenceFiles); } finally { readUnlock(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 97dcada99849..0ee97fd2a636 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -296,12 +296,18 @@ public void deserialize() throws IOException { } public static int getFileTimeIndexSerializedSize() { - // 5 * 8 Byte means 5 long numbers of tsFileID.timestamp, tsFileID.fileVersion - // tsFileID.compactionVersion, timeIndex.getMinStartTime(), timeIndex.getMaxStartTime() - return 5 * Long.BYTES; + // 6 * 8 Byte means 6 long numbers of + // tsFileID.timePartitionId, + // tsFileID.timestamp, + // tsFileID.fileVersion, + // tsFileID.compactionVersion, + // timeIndex.getMinStartTime(), + // timeIndex.getMaxStartTime() + return 6 * Long.BYTES; } public void serializeFileTimeIndexToByteBuffer(ByteBuffer buffer) { + buffer.putLong(tsFileID.timePartitionId); buffer.putLong(tsFileID.timestamp); buffer.putLong(tsFileID.fileVersion); buffer.putLong(tsFileID.compactionVersion); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndexCacheRecorder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndexCacheRecorder.java index b0c805bba6bc..d6eb597922e1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndexCacheRecorder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndexCacheRecorder.java @@ -57,8 +57,7 @@ public class FileTimeIndexCacheRecorder { private final BlockingQueue taskQueue = new LinkedBlockingQueue<>(); - private final Map> writerMap = - new ConcurrentHashMap<>(); + private final Map writerMap = new ConcurrentHashMap<>(); private FileTimeIndexCacheRecorder() { recordFileIndexThread = @@ -86,11 +85,10 @@ public void logFileTimeIndex(TsFileResource... tsFileResources) { TsFileResource firstResource = tsFileResources[0]; TsFileID tsFileID = firstResource.getTsFileID(); int dataRegionId = tsFileID.regionId; - long partitionId = tsFileID.timePartitionId; File dataRegionSysDir = StorageEngine.getDataRegionSystemDir( firstResource.getDatabaseName(), firstResource.getDataRegionId()); - FileTimeIndexCacheWriter writer = getWriter(dataRegionId, partitionId, dataRegionSysDir); + FileTimeIndexCacheWriter writer = getWriter(dataRegionId, dataRegionSysDir); boolean result = taskQueue.offer( () -> { @@ -116,18 +114,14 @@ public void logFileTimeIndex(TsFileResource... tsFileResources) { public void compactFileTimeIndexIfNeeded( String dataBaseName, int dataRegionId, - long partitionId, - TsFileResourceList sequenceFiles, - TsFileResourceList unsequenceFiles) { + int currentResourceCount, + Map sequenceFiles, + Map unsequenceFiles) { FileTimeIndexCacheWriter writer = getWriter( dataRegionId, - partitionId, StorageEngine.getDataRegionSystemDir(dataBaseName, String.valueOf(dataRegionId))); - int currentResourceCount = - (sequenceFiles == null ? 0 : sequenceFiles.size()) - + (unsequenceFiles == null ? 0 : unsequenceFiles.size()); if (writer.getLogFile().length() > currentResourceCount * getFileTimeIndexSerializedSize() * 100L) { @@ -136,25 +130,29 @@ public void compactFileTimeIndexIfNeeded( () -> { try { writer.clearFile(); - if (sequenceFiles != null && !sequenceFiles.isEmpty()) { - ByteBuffer buffer = - ByteBuffer.allocate( - getFileTimeIndexSerializedSize() * sequenceFiles.size()); - for (TsFileResource tsFileResource : sequenceFiles) { - tsFileResource.serializeFileTimeIndexToByteBuffer(buffer); + for (TsFileResourceList sequenceList : sequenceFiles.values()) { + if (sequenceList != null && !sequenceList.isEmpty()) { + ByteBuffer buffer = + ByteBuffer.allocate( + getFileTimeIndexSerializedSize() * sequenceList.size()); + for (TsFileResource tsFileResource : sequenceList) { + tsFileResource.serializeFileTimeIndexToByteBuffer(buffer); + } + buffer.flip(); + writer.write(buffer); } - buffer.flip(); - writer.write(buffer); } - if (unsequenceFiles != null && !unsequenceFiles.isEmpty()) { - ByteBuffer buffer = - ByteBuffer.allocate( - getFileTimeIndexSerializedSize() * unsequenceFiles.size()); - for (TsFileResource tsFileResource : unsequenceFiles) { - tsFileResource.serializeFileTimeIndexToByteBuffer(buffer); + for (TsFileResourceList unsequenceList : unsequenceFiles.values()) { + if (unsequenceList != null && !unsequenceList.isEmpty()) { + ByteBuffer buffer = + ByteBuffer.allocate( + getFileTimeIndexSerializedSize() * unsequenceList.size()); + for (TsFileResource tsFileResource : unsequenceList) { + tsFileResource.serializeFileTimeIndexToByteBuffer(buffer); + } + buffer.flip(); + writer.write(buffer); } - buffer.flip(); - writer.write(buffer); } } catch (IOException e) { LOGGER.warn("Meet error when compact FileTimeIndexCache: {}", e.getMessage()); @@ -166,51 +164,41 @@ public void compactFileTimeIndexIfNeeded( } } - private FileTimeIndexCacheWriter getWriter( - int dataRegionId, long partitionId, File dataRegionSysDir) { - return writerMap - .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) - .computeIfAbsent( - partitionId, - k -> { - File partitionDir = - SystemFileFactory.INSTANCE.getFile(dataRegionSysDir, String.valueOf(partitionId)); - File logFile = SystemFileFactory.INSTANCE.getFile(partitionDir, FILE_NAME); - try { - if (!partitionDir.exists() && !partitionDir.mkdirs()) { - LOGGER.debug( - "Partition directory has existed,filePath:{}", - partitionDir.getAbsolutePath()); - } - if (!logFile.createNewFile()) { - LOGGER.debug( - "Partition log file has existed,filePath:{}", logFile.getAbsolutePath()); - } - return new FileTimeIndexCacheWriter(logFile, true); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); + private FileTimeIndexCacheWriter getWriter(int dataRegionId, File dataRegionSysDir) { + return writerMap.computeIfAbsent( + dataRegionId, + k -> { + File logFile = SystemFileFactory.INSTANCE.getFile(dataRegionSysDir, FILE_NAME); + try { + if (!dataRegionSysDir.exists() && !dataRegionSysDir.mkdirs()) { + LOGGER.debug( + "DataRegionSysDir has existed,filePath:{}", dataRegionSysDir.getAbsolutePath()); + } + if (!logFile.createNewFile()) { + LOGGER.debug("FileTimeIndex file has existed,filePath:{}", logFile.getAbsolutePath()); + } + return new FileTimeIndexCacheWriter(logFile, true); + } catch (IOException e) { + LOGGER.error( + "FileTimeIndex log file create filed,filePath:{}", logFile.getAbsolutePath(), e); + throw new RuntimeException(e); + } + }); } public void close() throws IOException { - for (Map partitionWriterMap : writerMap.values()) { - for (FileTimeIndexCacheWriter writer : partitionWriterMap.values()) { - writer.close(); - } + for (FileTimeIndexCacheWriter writer : writerMap.values()) { + writer.close(); } } public void removeFileTimeIndexCache(int dataRegionId) { - Map partitionWriterMap = writerMap.get(dataRegionId); - if (partitionWriterMap != null) { - for (FileTimeIndexCacheWriter writer : partitionWriterMap.values()) { - try { - writer.close(); - deleteDirectoryAndEmptyParent(writer.getLogFile()); - } catch (IOException e) { - LOGGER.warn("Meet error when close FileTimeIndexCache: {}", e.getMessage()); - } + for (FileTimeIndexCacheWriter writer : writerMap.values()) { + try { + writer.close(); + deleteDirectoryAndEmptyParent(writer.getLogFile()); + } catch (IOException e) { + LOGGER.warn("Meet error when close FileTimeIndexCache: {}", e.getMessage()); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/fileTimeIndexCache/FileTimeIndexCacheReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/fileTimeIndexCache/FileTimeIndexCacheReader.java index 35682bd80203..d07210736e6b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/fileTimeIndexCache/FileTimeIndexCacheReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/fileTimeIndexCache/FileTimeIndexCacheReader.java @@ -32,7 +32,6 @@ import java.nio.channels.FileChannel; import java.nio.file.Files; import java.nio.file.StandardOpenOption; -import java.util.HashMap; import java.util.Map; import static org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource.getFileTimeIndexSerializedSize; @@ -44,21 +43,19 @@ public class FileTimeIndexCacheReader { private final File logFile; private final long fileLength; private final int dataRegionId; - private final long partitionId; - public FileTimeIndexCacheReader(File logFile, String dataRegionId, long partitionId) { + public FileTimeIndexCacheReader(File logFile, String dataRegionId) { this.logFile = logFile; this.fileLength = logFile.length(); this.dataRegionId = Integer.parseInt(dataRegionId); - this.partitionId = partitionId; } - public Map read() throws IOException { - Map fileTimeIndexMap = new HashMap<>(); + public void read(Map fileTimeIndexMap) throws IOException { long readLength = 0L; try (DataInputStream logStream = new DataInputStream(new BufferedInputStream(Files.newInputStream(logFile.toPath())))) { while (readLength < fileLength) { + long partitionId = logStream.readLong(); long timestamp = logStream.readLong(); long fileVersion = logStream.readLong(); long compactionVersion = logStream.readLong(); @@ -79,6 +76,5 @@ public Map read() throws IOException { channel.truncate(readLength); } } - return fileTimeIndexMap; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/fileTimeIndexCache/FileTimeIndexCacheWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/fileTimeIndexCache/FileTimeIndexCacheWriter.java index dce4f8b11039..3d88575ba683 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/fileTimeIndexCache/FileTimeIndexCacheWriter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/fileTimeIndexCache/FileTimeIndexCacheWriter.java @@ -53,6 +53,10 @@ public FileTimeIndexCacheWriter(File logFile, boolean forceEachWrite) public void write(ByteBuffer logBuffer) throws IOException { try { + if (!this.logFile.exists()) { + // For UT env, logFile may not be created + return; + } channel.write(logBuffer); if (this.forceEachWrite) { channel.force(true); From 91f5c6ca9214b1b446879f2139738dd6ef340793 Mon Sep 17 00:00:00 2001 From: Li Yu Heng Date: Tue, 3 Sep 2024 16:48:47 +0800 Subject: [PATCH 11/13] Fix IoTDBSetConfigurationIT.setClusterNameTest (wait DataNode's restarting for more time, and slow down requesting frequency) #13373 --- .../java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java index f6e9f98726b5..9039f94dc18a 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSetConfigurationIT.java @@ -101,7 +101,8 @@ public void testSetClusterName() throws Exception { EnvFactory.getEnv().getDataNodeWrapper(0).start(); // set cluster name on datanode Awaitility.await() - .atMost(10, TimeUnit.SECONDS) + .atMost(30, TimeUnit.SECONDS) + .pollDelay(1, TimeUnit.SECONDS) .until( () -> { try (Connection connection = EnvFactory.getEnv().getConnection(); From aaa72b1161a79055b9489b3855124c0bb0653bec Mon Sep 17 00:00:00 2001 From: Haonan Date: Tue, 3 Sep 2024 21:08:51 +0800 Subject: [PATCH 12/13] Fix cpp client IT on macos (#13392) --- .github/workflows/multi-language-client.yml | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/.github/workflows/multi-language-client.yml b/.github/workflows/multi-language-client.yml index ac3f3a63cdb9..c3e1ff7ba2a5 100644 --- a/.github/workflows/multi-language-client.yml +++ b/.github/workflows/multi-language-client.yml @@ -56,11 +56,16 @@ jobs: sudo apt-get update sudo apt-get install libboost-all-dev - name: Install CPP Dependencies (Mac) + # remove some xcode to release disk space if: runner.os == 'macOS' shell: bash run: | brew install boost - brew install bison + sudo rm -rf /Applications/Xcode_14.3.1.app + sudo rm -rf /Applications/Xcode_15.0.1.app + sudo rm -rf /Applications/Xcode_15.1.app + sudo rm -rf /Applications/Xcode_15.2.app + sudo rm -rf /Applications/Xcode_15.3.app - name: Install CPP Dependencies (Windows) if: runner.os == 'Windows' run: | @@ -76,7 +81,7 @@ jobs: uses: actions/cache@v4 with: path: ~/.m2 - key: client-${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} + key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} restore-keys: ${{ runner.os }}-m2- - name: Build IoTDB server shell: bash @@ -99,7 +104,7 @@ jobs: uses: actions/upload-artifact@v4 with: name: cpp-IT-${{ runner.os }} - path: iotdb-client/client-cpp/target/build/test/Testing + path: distribution/target/apache-iotdb-*-all-bin/apache-iotdb-*-all-bin/logs retention-days: 1 go: From 7f6211e80c90fb1b8ec26aecf18c417ed23a7771 Mon Sep 17 00:00:00 2001 From: Jiang Tian Date: Wed, 4 Sep 2024 09:17:04 +0800 Subject: [PATCH 13/13] Remove unnecessary type cast log (#13391) --- .../queryengine/plan/statement/crud/InsertRowStatement.java | 6 ------ .../plan/statement/crud/InsertTabletStatement.java | 6 ------ 2 files changed, 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java index f7b4f5f32a81..45a381293a37 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java @@ -191,12 +191,6 @@ public Object getFirstValueOfIndex(int index) { @Override protected boolean checkAndCastDataType(int columnIndex, TSDataType dataType) { if (CommonUtils.checkCanCastType(dataTypes[columnIndex], dataType)) { - LOGGER.warn( - "Inserting to {}.{} : Cast from {} to {}", - devicePath, - measurements[columnIndex], - dataTypes[columnIndex], - dataType); values[columnIndex] = CommonUtils.castValue(dataTypes[columnIndex], dataType, values[columnIndex]); dataTypes[columnIndex] = dataType; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java index cb844ef3175c..6029937d04cf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java @@ -194,12 +194,6 @@ public List getSchemaValidationList() { @Override protected boolean checkAndCastDataType(int columnIndex, TSDataType dataType) { if (CommonUtils.checkCanCastType(dataTypes[columnIndex], dataType)) { - LOGGER.warn( - "Inserting to {}.{} : Cast from {} to {}", - devicePath, - measurements[columnIndex], - dataTypes[columnIndex], - dataType); columns[columnIndex] = CommonUtils.castArray(dataTypes[columnIndex], dataType, columns[columnIndex]); dataTypes[columnIndex] = dataType;