From c82ab13c93e88ab4069eeff22db6fabd6c4b9a51 Mon Sep 17 00:00:00 2001 From: Jackie Tien Date: Fri, 11 Jul 2025 14:11:18 +0800 Subject: [PATCH 001/180] Add Object Type --- .../java/org/apache/iotdb/ObjectExample.java | 148 ++++++++ .../org/apache/iotdb/ObjectReadExample.java | 114 ++++++ .../iotdb/TableModelSessionPoolExample.java | 22 +- .../org/apache/iotdb/pipe/api/type/Type.java | 5 +- .../org/apache/iotdb/udf/api/type/Type.java | 10 +- .../org/apache/iotdb/cli/AbstractCli.java | 1 + .../apache/iotdb/isession/SessionDataSet.java | 1 + .../jdbc/IoTDBAbstractDatabaseMetadata.java | 1 + .../apache/iotdb/rpc/IoTDBJDBCDataSet.java | 6 + .../org/apache/iotdb/rpc/IoTDBRpcDataSet.java | 6 + .../org/apache/iotdb/rpc/TSStatusCode.java | 3 + .../org/apache/iotdb/session/Session.java | 1 + .../payload/SubscriptionSessionDataSet.java | 1 + .../iotdb/session/util/SessionUtils.java | 3 + iotdb-core/datanode/pom.xml | 5 + .../iotdb/db/conf/DataNodeMemoryConfig.java | 5 +- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 9 + .../apache/iotdb/db/conf/IoTDBDescriptor.java | 12 + .../dataregion/DataExecutionVisitor.java | 13 + .../aggregate/AggregateProcessor.java | 3 + .../memory/InsertNodeMemoryEstimator.java | 1 + .../protocol/opcda/OpcDaServerHandle.java | 2 + .../util/sorter/PipeTabletEventSorter.java | 1 + .../thrift/impl/ClientRPCServiceImpl.java | 2 +- .../aggregation/AccumulatorFactory.java | 1 + .../execution/aggregation/AvgAccumulator.java | 1 + .../aggregation/ExtremeAccumulator.java | 6 + .../aggregation/FirstValueAccumulator.java | 6 + .../aggregation/LastValueAccumulator.java | 6 + .../aggregation/MaxMinByBaseAccumulator.java | 6 + .../aggregation/MaxValueAccumulator.java | 6 + .../aggregation/MinValueAccumulator.java | 6 + .../execution/aggregation/SumAccumulator.java | 1 + .../aggregation/VarianceAccumulator.java | 1 + .../executor/RegionWriteExecutor.java | 14 + .../execution/memory/LocalMemoryManager.java | 1 - .../execution/operator/AggregationUtil.java | 1 + .../operator/process/TopKOperator.java | 2 + .../operator/process/TransformOperator.java | 1 + .../join/merge/MergeSortComparator.java | 1 + .../window/function/value/LagFunction.java | 1 + .../window/function/value/LeadFunction.java | 1 + .../process/window/utils/RowComparator.java | 3 + .../operator/source/SeriesScanUtil.java | 1 + .../LastQueryAggTableScanOperator.java | 1 + .../aggregation/AccumulatorFactory.java | 2 + .../ApproxCountDistinctAccumulator.java | 1 + .../aggregation/AvgAccumulator.java | 2 + .../aggregation/ExtremeAccumulator.java | 5 + .../aggregation/FirstAccumulator.java | 4 + .../aggregation/FirstByAccumulator.java | 4 + .../aggregation/LastAccumulator.java | 4 + .../aggregation/LastByAccumulator.java | 4 + .../aggregation/SumAccumulator.java | 2 + .../TableMaxMinByBaseAccumulator.java | 5 + .../aggregation/TableVarianceAccumulator.java | 2 + .../source/relational/aggregation/Utils.java | 2 + ...GroupedApproxCountDistinctAccumulator.java | 1 + .../grouped/GroupedAvgAccumulator.java | 1 + .../grouped/GroupedExtremeAccumulator.java | 8 + .../grouped/GroupedFirstAccumulator.java | 8 + .../grouped/GroupedFirstByAccumulator.java | 9 + .../grouped/GroupedLastAccumulator.java | 8 + .../grouped/GroupedLastByAccumulator.java | 9 + .../grouped/GroupedMaxAccumulator.java | 8 + .../GroupedMaxMinByBaseAccumulator.java | 15 + .../grouped/GroupedMinAccumulator.java | 8 + .../grouped/GroupedModeAccumulator.java | 4 + .../grouped/GroupedSumAccumulator.java | 1 + .../grouped/GroupedVarianceAccumulator.java | 1 + .../operator/window/WindowManagerFactory.java | 2 + .../relational/ColumnTransformerBuilder.java | 34 +- .../plan/planner/TableOperatorGenerator.java | 23 +- .../plan/planner/plan/node/PlanNodeType.java | 8 + .../plan/planner/plan/node/PlanVisitor.java | 5 + .../plan/node/write/InsertTabletNode.java | 29 +- .../planner/plan/node/write/ObjectNode.java | 346 ++++++++++++++++++ .../write/RelationalInsertTabletNode.java | 74 ++++ .../metadata/TableMetadataImpl.java | 19 + .../plan/relational/sql/ast/DeleteDevice.java | 3 +- .../relational/type/InternalTypeManager.java | 6 + .../plan/relational/utils/TypeUtil.java | 13 +- .../scalar/ReadObjectColumnTransformer.java | 129 +++++++ .../dag/input/ConstantInputReader.java | 1 + .../unary/ArithmeticNegationTransformer.java | 1 + .../dag/transformer/unary/InTransformer.java | 2 + .../unary/scalar/CastFunctionTransformer.java | 7 + .../unary/scalar/DiffFunctionTransformer.java | 1 + .../scalar/RoundFunctionTransformer.java | 1 + .../dag/util/TransformUtils.java | 2 + .../transformation/dag/util/TypeUtils.java | 1 + .../row/SerializableRowList.java | 2 + .../datastructure/tv/SerializableTVList.java | 1 + .../impl/SchemaRegionMemoryImpl.java | 6 +- .../iotdb/db/storageengine/StorageEngine.java | 38 ++ .../storageengine/dataregion/DataRegion.java | 63 +++- .../impl/FastCompactionPerformer.java | 3 +- .../task/CrossSpaceCompactionTask.java | 6 + .../task/InnerSpaceCompactionTask.java | 6 + .../execute/task/SettleCompactionTask.java | 8 +- .../execute/utils/CompactionUtils.java | 168 +++++++++ .../utils/MultiTsFileDeviceIterator.java | 11 +- .../FastAlignedSeriesCompactionExecutor.java | 25 +- ...astNonAlignedSeriesCompactionExecutor.java | 6 +- .../fast/element/ChunkMetadataElement.java | 10 +- ...dChunkAlignedSeriesCompactionExecutor.java | 1 + .../SingleSeriesCompactionExecutor.java | 1 + .../writer/AbstractCompactionWriter.java | 1 + .../memtable/AlignedReadOnlyMemChunk.java | 2 + .../memtable/AlignedWritableMemChunk.java | 1 + .../dataregion/memtable/WritableMemChunk.java | 1 + .../reader/chunk/MemAlignedPageReader.java | 1 + .../read/reader/chunk/MemPageReader.java | 2 + .../tsfile/generator/TsFileNameGenerator.java | 15 + .../dataregion/wal/buffer/WALBuffer.java | 3 + .../dataregion/wal/buffer/WALEntry.java | 6 + .../dataregion/wal/buffer/WALEntryType.java | 4 +- .../dataregion/wal/buffer/WALInfoEntry.java | 4 + .../dataregion/wal/node/IWALNode.java | 3 + .../dataregion/wal/node/WALFakeNode.java | 6 + .../dataregion/wal/node/WALNode.java | 52 ++- .../file/UnsealedTsFileRecoverPerformer.java | 2 + .../BatchedAlignedValueChunkData.java | 2 + .../rescon/disk/TierManager.java | 51 +++ .../rescon/memory/PrimitiveArrayManager.java | 2 + .../db/tools/TsFileSplitByPartitionTool.java | 1 + .../apache/iotdb/db/utils/CommonUtils.java | 1 + .../db/utils/EncodingInferenceUtils.java | 1 + .../org/apache/iotdb/db/utils/MemUtils.java | 4 +- .../iotdb/db/utils/ObjectTypeUtils.java | 85 +++++ .../apache/iotdb/db/utils/ObjectWriter.java | 82 +++++ .../iotdb/db/utils/QueryDataSetUtils.java | 4 + .../iotdb/db/utils/TimeValuePairUtils.java | 2 + .../iotdb/db/utils/TypeInferenceUtils.java | 1 + .../db/utils/datastructure/AlignedTVList.java | 12 + .../MergeSortMultiAlignedTVListIterator.java | 1 + .../MergeSortMultiTVListIterator.java | 1 + .../MultiAlignedTVListIterator.java | 1 + .../datastructure/MultiTVListIterator.java | 1 + .../iotdb/db/utils/datastructure/TVList.java | 5 + .../db/utils/windowing/window/WindowImpl.java | 1 + .../CompactionDeleteObjectFileTest.java | 122 ++++++ .../PrimitiveArrayManagerTest.java | 1 + .../conf/iotdb-system.properties.template | 5 + .../iotdb/commons/conf/IoTDBConstant.java | 1 + .../commons/exception/ObjectFileNotExist.java | 31 ++ .../iotdb/commons/udf/builtin/UDTFAbs.java | 3 + .../commons/udf/builtin/UDTFBottomK.java | 1 + .../udf/builtin/UDTFCommonDerivative.java | 1 + .../builtin/UDTFCommonValueDifference.java | 1 + .../iotdb/commons/udf/builtin/UDTFConst.java | 5 + .../udf/builtin/UDTFContinuouslySatisfy.java | 2 + .../builtin/UDTFEqualSizeBucketAggSample.java | 1 + .../builtin/UDTFEqualSizeBucketM4Sample.java | 1 + .../UDTFEqualSizeBucketOutlierSample.java | 1 + .../UDTFEqualSizeBucketRandomSample.java | 1 + .../commons/udf/builtin/UDTFInRange.java | 3 + .../iotdb/commons/udf/builtin/UDTFJexl.java | 3 + .../iotdb/commons/udf/builtin/UDTFM4.java | 1 + .../iotdb/commons/udf/builtin/UDTFMath.java | 3 + .../builtin/UDTFNonNegativeDerivative.java | 1 + .../UDTFNonNegativeValueDifference.java | 1 + .../iotdb/commons/udf/builtin/UDTFOnOff.java | 1 + .../commons/udf/builtin/UDTFSelectK.java | 2 + .../iotdb/commons/udf/builtin/UDTFTopK.java | 1 + .../commons/udf/builtin/UDTFValueTrend.java | 1 + .../TableBuiltinScalarFunction.java | 1 + .../udf/utils/UDFDataTypeTransformer.java | 7 + .../iotdb/commons/utils/SerializeUtils.java | 6 + .../src/main/thrift/client.thrift | 2 +- pom.xml | 2 +- 171 files changed, 2153 insertions(+), 42 deletions(-) create mode 100644 example/session/src/main/java/org/apache/iotdb/ObjectExample.java create mode 100644 example/session/src/main/java/org/apache/iotdb/ObjectReadExample.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ObjectNode.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/ReadObjectColumnTransformer.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ObjectTypeUtils.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ObjectWriter.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionDeleteObjectFileTest.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/ObjectFileNotExist.java diff --git a/example/session/src/main/java/org/apache/iotdb/ObjectExample.java b/example/session/src/main/java/org/apache/iotdb/ObjectExample.java new file mode 100644 index 0000000000000..3339d3aae5a4c --- /dev/null +++ b/example/session/src/main/java/org/apache/iotdb/ObjectExample.java @@ -0,0 +1,148 @@ +/* + * 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; + +import org.apache.iotdb.isession.ITableSession; +import org.apache.iotdb.rpc.IoTDBConnectionException; +import org.apache.iotdb.rpc.StatementExecutionException; +import org.apache.iotdb.session.TableSessionBuilder; + +import org.apache.tsfile.enums.ColumnCategory; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.write.record.Tablet; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class ObjectExample { + private static final String LOCAL_URL = "127.0.0.1:6667"; + + public static void main(String[] args) { + + // don't specify database in constructor + try (ITableSession session = + new TableSessionBuilder() + .nodeUrls(Collections.singletonList(LOCAL_URL)) + .username("root") + .password("root") + .build()) { + session.executeNonQueryStatement("CREATE DATABASE IF NOT EXISTS test1"); + session.executeNonQueryStatement("use test1"); + + // insert table data by tablet + List columnNameList = + Arrays.asList("region_id", "plant_id", "device_id", "temperature", "file"); + List dataTypeList = + Arrays.asList( + TSDataType.STRING, + TSDataType.STRING, + TSDataType.STRING, + TSDataType.FLOAT, + TSDataType.OBJECT); + List columnTypeList = + new ArrayList<>( + Arrays.asList( + ColumnCategory.TAG, + ColumnCategory.TAG, + ColumnCategory.TAG, + ColumnCategory.FIELD, + ColumnCategory.FIELD)); + Tablet tablet = new Tablet("tsfile_table", columnNameList, dataTypeList, columnTypeList, 1); + int rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, 1); + tablet.addValue(rowIndex, 0, "1"); + tablet.addValue(rowIndex, 1, "5"); + tablet.addValue(rowIndex, 2, "3"); + tablet.addValue(rowIndex, 3, 37.6F); + tablet.addValue( + rowIndex, + 4, + true, + 0, + Files.readAllBytes( + Paths.get( + "/Users/jackietien/Downloads/2_1746622362350_fa24aa15233f4e76bcda789a5771f43f"))); + session.insert(tablet); + tablet.reset(); + + tablet = new Tablet("tsfile_table", columnNameList, dataTypeList, columnTypeList, 1); + rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, 2); + tablet.addValue(rowIndex, 0, "1"); + tablet.addValue(rowIndex, 1, "5"); + tablet.addValue(rowIndex, 2, "3"); + tablet.addValue(rowIndex, 3, 37.6F); + tablet.addValue( + rowIndex, + 4, + true, + 0, + Files.readAllBytes( + Paths.get( + "/Users/jackietien/Downloads/2_1746622367063_8fb5ac8e21724140874195b60b878664"))); + session.insert(tablet); + tablet.reset(); + + tablet = new Tablet("tiff_table", columnNameList, dataTypeList, columnTypeList, 1); + rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, 1); + tablet.addValue(rowIndex, 0, "1"); + tablet.addValue(rowIndex, 1, "5"); + tablet.addValue(rowIndex, 2, "3"); + tablet.addValue(rowIndex, 3, 37.6F); + tablet.addValue( + rowIndex, + 4, + true, + 0, + Files.readAllBytes(Paths.get("/Users/jackietien/Downloads/1751891240130.tiff"))); + session.insert(tablet); + tablet.reset(); + + tablet = new Tablet("tiff_table", columnNameList, dataTypeList, columnTypeList, 1); + rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, 2); + tablet.addValue(rowIndex, 0, "1"); + tablet.addValue(rowIndex, 1, "5"); + tablet.addValue(rowIndex, 2, "4"); + tablet.addValue(rowIndex, 3, 37.6F); + tablet.addValue( + rowIndex, + 4, + true, + 0, + Files.readAllBytes(Paths.get("/Users/jackietien/Downloads/1751891242743.tiff"))); + session.insert(tablet); + tablet.reset(); + + } catch (IoTDBConnectionException e) { + e.printStackTrace(); + } catch (StatementExecutionException e) { + e.printStackTrace(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/example/session/src/main/java/org/apache/iotdb/ObjectReadExample.java b/example/session/src/main/java/org/apache/iotdb/ObjectReadExample.java new file mode 100644 index 0000000000000..344939175db78 --- /dev/null +++ b/example/session/src/main/java/org/apache/iotdb/ObjectReadExample.java @@ -0,0 +1,114 @@ +/* + * 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; + +import org.apache.iotdb.isession.ITableSession; +import org.apache.iotdb.isession.SessionDataSet; +import org.apache.iotdb.rpc.IoTDBConnectionException; +import org.apache.iotdb.rpc.StatementExecutionException; +import org.apache.iotdb.session.TableSessionBuilder; + +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.tsfile.utils.Binary; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; + +public class ObjectReadExample { + private static final String LOCAL_URL = "127.0.0.1:6667"; + + public static void main(String[] args) { + + // don't specify database in constructor + try (ITableSession session = + new TableSessionBuilder() + .nodeUrls(Collections.singletonList(LOCAL_URL)) + .username("root") + .password("root") + .database("test1") + .thriftMaxFrameSize(256 * 1024 * 1024) + .build()) { + try (SessionDataSet dataSet = + session.executeQueryStatement( + "select READ_OBJECT(file) from tsfile_table where time = 1")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + Binary binary = iterator.getBlob(1); + System.out.println(DigestUtils.md5Hex(binary.getValues())); + } + } + + try (SessionDataSet dataSet = + session.executeQueryStatement( + "select READ_OBJECT(file) from tsfile_table where time = 2")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + Binary binary = iterator.getBlob(1); + System.out.println(DigestUtils.md5Hex(binary.getValues())); + } + } + + try (SessionDataSet dataSet = + session.executeQueryStatement("select READ_OBJECT(file) from tsfile_table")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + Binary binary = iterator.getBlob(1); + System.out.println(DigestUtils.md5Hex(binary.getValues())); + } + } + + try (SessionDataSet dataSet = + session.executeQueryStatement( + "select geo_penetrate(file, '0,3,7501,7504') from tsfile_table")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + Binary binary = iterator.getBlob(1); + ByteBuffer byteBuffer = ByteBuffer.wrap(binary.getValues()); + float[] res = new float[byteBuffer.limit() / Float.BYTES]; + for (int i = 0; i < res.length; i++) { + res[i] = byteBuffer.getFloat(); + } + System.out.println(Arrays.toString(res)); + } + } + + try (SessionDataSet dataSet = + session.executeQueryStatement( + "select geo_penetrate(file, '0,3,7501,7504', 'UNCOMPRESSED_TIFF') from tiff_table")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + Binary binary = iterator.getBlob(1); + ByteBuffer byteBuffer = ByteBuffer.wrap(binary.getValues()); + float[] res = new float[byteBuffer.limit() / Float.BYTES]; + for (int i = 0; i < res.length; i++) { + res[i] = byteBuffer.getFloat(); + } + System.out.println(Arrays.toString(res)); + } + } + + } catch (IoTDBConnectionException e) { + e.printStackTrace(); + } catch (StatementExecutionException e) { + e.printStackTrace(); + } + } +} diff --git a/example/session/src/main/java/org/apache/iotdb/TableModelSessionPoolExample.java b/example/session/src/main/java/org/apache/iotdb/TableModelSessionPoolExample.java index 0a335f02c9699..19b245653ba1a 100644 --- a/example/session/src/main/java/org/apache/iotdb/TableModelSessionPoolExample.java +++ b/example/session/src/main/java/org/apache/iotdb/TableModelSessionPoolExample.java @@ -113,7 +113,7 @@ public static void main(String[] args) { int rowIndex = tablet.getRowSize(); tablet.addTimestamp(rowIndex, timestamp); tablet.addValue("region_id", rowIndex, "1"); - tablet.addValue("plant_id", rowIndex, "5"); + tablet.addValue("plant_id", rowIndex, null); tablet.addValue("device_id", rowIndex, "3"); tablet.addValue("model", rowIndex, "A"); tablet.addValue("temperature", rowIndex, 37.6F); @@ -128,6 +128,26 @@ public static void main(String[] args) { tablet.reset(); } + // query device leader + List isSetTag = Arrays.asList(true, true, false, true); + String correctURL = + session.getDeviceLeaderURL("test2", Arrays.asList("test1", "1", "3"), isSetTag, 66); + System.out.println("Correct device leader URL: " + correctURL); + String errorDbURL = + session.getDeviceLeaderURL("test3", Arrays.asList("test1", "1", "3"), isSetTag, 66); + System.out.println("Error dbName device leader URL: " + errorDbURL); + String errorDeviceURL = + session.getDeviceLeaderURL("test2", Arrays.asList("test1", "3", "1"), isSetTag, 66); + System.out.println("Error deviceId device leader URL: " + errorDeviceURL); + List falseTagList = Arrays.asList(false, true, true, true); + String errorTagURL = + session.getDeviceLeaderURL("test2", Arrays.asList("test1", "1", "3"), falseTagList, 66); + System.out.println("Error tag device leader URL: " + errorTagURL); + String errorTimeURL = + session.getDeviceLeaderURL( + "test2", Arrays.asList("test1", "1", "3"), isSetTag, 6666666666666666L); + System.out.println("Error time device leader URL: " + errorTimeURL); + // query table data try (SessionDataSet dataSet = session.executeQueryStatement( diff --git a/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/type/Type.java b/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/type/Type.java index 9093133631a0e..59819ee418e6f 100644 --- a/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/type/Type.java +++ b/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/type/Type.java @@ -49,7 +49,10 @@ public enum Type { BLOB((byte) 10), /* STRING */ - STRING((byte) 11); + STRING((byte) 11), + + /* OBJECT */ + OBJECT((byte) 12); private final byte dataType; diff --git a/iotdb-api/udf-api/src/main/java/org/apache/iotdb/udf/api/type/Type.java b/iotdb-api/udf-api/src/main/java/org/apache/iotdb/udf/api/type/Type.java index a5c2852ac1b95..c4c38c285dce9 100644 --- a/iotdb-api/udf-api/src/main/java/org/apache/iotdb/udf/api/type/Type.java +++ b/iotdb-api/udf-api/src/main/java/org/apache/iotdb/udf/api/type/Type.java @@ -56,7 +56,11 @@ public enum Type { BLOB((byte) 10), /* STRING */ - STRING((byte) 11); + STRING((byte) 11), + + /* OBJECT */ + OBJECT((byte) 12); + private final byte dataType; Type(byte type) { @@ -92,6 +96,7 @@ public boolean checkObjectType(Object o) { case DATE: return o instanceof LocalDate; case BLOB: + case OBJECT: return o instanceof Binary; case STRING: case TEXT: @@ -102,7 +107,8 @@ public boolean checkObjectType(Object o) { } public static List allTypes() { - return Arrays.asList(BOOLEAN, INT32, INT64, FLOAT, DOUBLE, TEXT, TIMESTAMP, DATE, BLOB, STRING); + return Arrays.asList( + BOOLEAN, INT32, INT64, FLOAT, DOUBLE, TEXT, TIMESTAMP, DATE, BLOB, STRING, OBJECT); } public static List numericTypes() { diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/cli/AbstractCli.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/cli/AbstractCli.java index 1b890b680c416..118b30b44049d 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/cli/AbstractCli.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/cli/AbstractCli.java @@ -757,6 +757,7 @@ private static String getStringByColumnIndex( case DOUBLE: case TEXT: case STRING: + case OBJECT: return resultSet.getString(columnIndex); case BLOB: byte[] v = resultSet.getBytes(columnIndex); diff --git a/iotdb-client/isession/src/main/java/org/apache/iotdb/isession/SessionDataSet.java b/iotdb-client/isession/src/main/java/org/apache/iotdb/isession/SessionDataSet.java index fa67708fed0c7..bb113680d8e5b 100644 --- a/iotdb-client/isession/src/main/java/org/apache/iotdb/isession/SessionDataSet.java +++ b/iotdb-client/isession/src/main/java/org/apache/iotdb/isession/SessionDataSet.java @@ -189,6 +189,7 @@ private RowRecord constructRowRecordFromValueArray() throws StatementExecutionEx case TEXT: case BLOB: case STRING: + case OBJECT: field.setBinaryV(ioTDBRpcDataSet.getBinary(columnName)); break; default: diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBAbstractDatabaseMetadata.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBAbstractDatabaseMetadata.java index 8baf1f9b6f218..f14e2ee6b3270 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBAbstractDatabaseMetadata.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBAbstractDatabaseMetadata.java @@ -396,6 +396,7 @@ public static ByteBuffer convertTsBlock( case TEXT: case STRING: case BLOB: + case OBJECT: tsBlockBuilder .getColumnBuilder(j) .writeBinary( diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBJDBCDataSet.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBJDBCDataSet.java index 5bba6d0ea718d..43064ecdaa9fb 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBJDBCDataSet.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBJDBCDataSet.java @@ -184,6 +184,7 @@ public IoTDBJDBCDataSet( case TEXT: case BLOB: case STRING: + case OBJECT: values[i] = null; break; default: @@ -307,6 +308,7 @@ public IoTDBJDBCDataSet( case TEXT: case BLOB: case STRING: + case OBJECT: values[i] = null; break; default: @@ -418,6 +420,7 @@ public void constructOneRow() { case TEXT: case BLOB: case STRING: + case OBJECT: int length = valueBuffer.getInt(); values[i] = ReadWriteIOUtils.readBytes(valueBuffer, length); break; @@ -596,6 +599,8 @@ public String getString(int index, TSDataType tsDataType, byte[][] values) { case TEXT: case STRING: return new String(values[index], StandardCharsets.UTF_8); + case OBJECT: + return BytesUtils.parseObjectByteArrayToString(values[index]); case BLOB: return BytesUtils.parseBlobByteArrayToString(values[index]); case DATE: @@ -634,6 +639,7 @@ public Object getObject(int index, TSDataType tsDataType, byte[][] values) { case TEXT: case STRING: return new String(values[index], StandardCharsets.UTF_8); + case OBJECT: case BLOB: return new Binary(values[index]); case TIMESTAMP: diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBRpcDataSet.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBRpcDataSet.java index a40197fac7988..940b3460d927b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBRpcDataSet.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBRpcDataSet.java @@ -494,6 +494,9 @@ private Object getObjectByTsBlockIndex(int tsBlockColumnIndex) .getColumn(tsBlockColumnIndex) .getBinary(tsBlockIndex) .getStringValue(TSFileConfig.STRING_CHARSET); + case OBJECT: + return BytesUtils.parseObjectByteArrayToString( + curTsBlock.getColumn(tsBlockColumnIndex).getBinary(tsBlockIndex).getValues()); case BLOB: return BytesUtils.parseBlobByteArrayToString( curTsBlock.getColumn(tsBlockColumnIndex).getBinary(tsBlockIndex).getValues()); @@ -554,6 +557,9 @@ private String getString(int index, TSDataType tsDataType) { .getColumn(index) .getBinary(tsBlockIndex) .getStringValue(TSFileConfig.STRING_CHARSET); + case OBJECT: + return BytesUtils.parseObjectByteArrayToString( + curTsBlock.getColumn(index).getBinary(tsBlockIndex).getValues()); case BLOB: return BytesUtils.parseBlobByteArrayToString( curTsBlock.getColumn(index).getBinary(tsBlockIndex).getValues()); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index fc738783483e7..c9ab41e90908e 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -146,6 +146,9 @@ public enum TSStatusCode { PLAN_FAILED_NETWORK_PARTITION(721), CANNOT_FETCH_FI_STATE(722), + // OBJECT + OBJECT_NOT_EXISTS(740), + // Arithmetic NUMERIC_VALUE_OUT_OF_RANGE(750), DIVISION_BY_ZERO(751), diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java index 8844f36c247d2..ab28bd10ad9a3 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java @@ -3745,6 +3745,7 @@ private Object sortList(Object valueList, TSDataType dataType, Integer[] index) case TEXT: case BLOB: case STRING: + case OBJECT: Binary[] binaryValues = (Binary[]) valueList; Binary[] sortedBinaryValues = new Binary[binaryValues.length]; for (int i = 0; i < index.length; i++) { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java index a3f7f8e3f26d1..a3f01b3e11e12 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java @@ -242,6 +242,7 @@ private static Field generateFieldFromTabletValue( case TEXT: case STRING: case BLOB: + case OBJECT: final Binary binaryValue = new Binary((((Binary[]) value)[index]).getValues()); field.setBinaryV(binaryValue); break; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/util/SessionUtils.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/util/SessionUtils.java index dd30496090c03..1b5ad290678a0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/util/SessionUtils.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/util/SessionUtils.java @@ -135,6 +135,7 @@ private static int calOccupationOfOneColumn( case TEXT: case BLOB: case STRING: + case OBJECT: valueOccupation += rowSize * 4; Binary[] binaries = (Binary[]) values[columnIndex]; for (int rowIndex = 0; rowIndex < rowSize; rowIndex++) { @@ -185,6 +186,7 @@ public static int calculateLength(List types, List break; case TEXT: case STRING: + case OBJECT: res += Integer.BYTES; if (values.get(i) instanceof Binary) { res += ((Binary) values.get(i)).getValues().length; @@ -336,6 +338,7 @@ private static void getValueBufferOfDataType( case TEXT: case STRING: case BLOB: + case OBJECT: Binary[] binaryValues = (Binary[]) tablet.getValues()[i]; for (int index = 0; index < tablet.getRowSize(); index++) { if (!tablet.isNull(index, i) && binaryValues[index] != null) { diff --git a/iotdb-core/datanode/pom.xml b/iotdb-core/datanode/pom.xml index 626d8265eed2e..e8409508a035d 100644 --- a/iotdb-core/datanode/pom.xml +++ b/iotdb-core/datanode/pom.xml @@ -371,6 +371,11 @@ 1.3.0 test + + org.gdal + gdal + 3.11.0 + diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/DataNodeMemoryConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/DataNodeMemoryConfig.java index efb6758066f10..551d8a9031d63 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/DataNodeMemoryConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/DataNodeMemoryConfig.java @@ -60,8 +60,7 @@ public class DataNodeMemoryConfig { private int queryThreadCount = Runtime.getRuntime().availableProcessors(); /** Max bytes of each FragmentInstance for DataExchange */ - private long maxBytesPerFragmentInstance = - Runtime.getRuntime().maxMemory() * 3 / 10 * 200 / 1001 / queryThreadCount; + private long maxBytesPerFragmentInstance = Runtime.getRuntime().maxMemory() * 3 / 10 * 200 / 1001; /** The memory manager of on heap */ private MemoryManager onHeapMemoryManager; @@ -483,7 +482,7 @@ private void initQueryEngineMemoryAllocate( operatorsMemorySize += partForOperators; } // set max bytes per fragment instance - setMaxBytesPerFragmentInstance(dataExchangeMemorySize / getQueryThreadCount()); + setMaxBytesPerFragmentInstance(dataExchangeMemorySize); bloomFilterCacheMemoryManager = queryEngineMemoryManager.getOrCreateMemoryManager( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index c86ccfdd2c367..bb62d241cc995 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -1190,6 +1190,7 @@ public class IoTDBConfig { private ConcurrentHashMap tsFileDBToEncryptMap = new ConcurrentHashMap<>( Collections.singletonMap("root.__audit", new EncryptParameter("UNENCRYPTED", null))); + private long maxObjectSizeInByte = 4 * 1024 * 1024 * 1024L; IoTDBConfig() {} @@ -4264,4 +4265,12 @@ public void setPasswordLockTimeMinutes(int passwordLockTimeMinutes) { public ConcurrentHashMap getTSFileDBToEncryptMap() { return tsFileDBToEncryptMap; } + + public long getMaxObjectSizeInByte() { + return maxObjectSizeInByte; + } + + public void setMaxObjectSizeInByte(long maxObjectSizeInByte) { + this.maxObjectSizeInByte = maxObjectSizeInByte; + } } 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 c9a55f59d59c4..7e903948c1944 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 @@ -899,10 +899,17 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException } conf.setExtPipeDir(properties.getProperty("ext_pipe_dir", conf.getExtPipeDir()).trim()); + conf.setPipeTaskThreadCount( Integer.parseInt( properties.getProperty( "pipe_task_thread_count", Integer.toString(conf.getPipeTaskThreadCount()).trim()))); + + conf.setMaxObjectSizeInByte( + Long.parseLong( + properties.getProperty( + "max_object_file_size_in_byte", String.valueOf(conf.getMaxObjectSizeInByte())))); + // At the same time, set TSFileConfig List fsTypes = new ArrayList<>(); fsTypes.add(FSType.LOCAL); @@ -2163,6 +2170,10 @@ public synchronized void loadHotModifiedProps(TrimProperties properties) "include_null_value_in_write_throughput_metric", ConfigurationFileUtils.getConfigurationDefaultValue( "include_null_value_in_write_throughput_metric")))); + conf.setMaxObjectSizeInByte( + Long.parseLong( + properties.getProperty( + "max_object_file_size_in_byte", String.valueOf(conf.getMaxObjectSizeInByte())))); } catch (Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); @@ -2704,6 +2715,7 @@ public TSEncoding getDefaultEncodingByType(TSDataType dataType) { return conf.getDefaultDoubleEncoding(); case STRING: case BLOB: + case OBJECT: case TEXT: default: return conf.getDefaultTextEncoding(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java index cdce05b46c0b7..643f62830c293 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java @@ -39,6 +39,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsOfOneDeviceNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertRowsNode; @@ -292,4 +293,16 @@ public TSStatus visitPipeEnrichedDeleteDataNode( node.getDeleteDataNode().markAsGeneratedByPipe(); return node.getDeleteDataNode().accept(this, context); } + + @Override + public TSStatus visitWriteObjectFile(ObjectNode node, DataRegion dataRegion) { + try { + dataRegion.writeObject(node); + dataRegion.insertSeparatorToWAL(); + return StatusUtils.OK; + } catch (final Exception e) { + LOGGER.error("Error in executing plan node: {}", node, e); + return new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AggregateProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AggregateProcessor.java index d16c404e4226b..10a72a0d9c5dc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AggregateProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AggregateProcessor.java @@ -498,6 +498,7 @@ private Map> processRow( timestamp, row.getString(index), outputMinReportIntervalMilliseconds); break; case BLOB: + case OBJECT: result = state.updateWindows( timestamp, row.getBinary(index), outputMinReportIntervalMilliseconds); @@ -685,6 +686,7 @@ public void collectWindowOutputs( break; case TEXT: case BLOB: + case OBJECT: case STRING: valueColumns[columnIndex] = new Binary[distinctOutputs.size()]; break; @@ -734,6 +736,7 @@ public void collectWindowOutputs( TSFileConfig.STRING_CHARSET); break; case BLOB: + case OBJECT: ((Binary[]) valueColumns[columnIndex])[rowIndex] = (Binary) aggregatedResults.get(columnNameStringList[columnIndex]).getRight(); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java index 476ec83b7877d..60076c6139e34 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java @@ -630,6 +630,7 @@ public static long sizeOfColumns( case STRING: case TEXT: case BLOB: + case OBJECT: { size += getBinarySize((Binary[]) columns[i]); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcda/OpcDaServerHandle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcda/OpcDaServerHandle.java index 8ca78a2da5540..1ba0a8841d2f7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcda/OpcDaServerHandle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcda/OpcDaServerHandle.java @@ -318,6 +318,7 @@ private short convertTsDataType2VariantType(final TSDataType dataType) { // Note that "Variant" does not support "VT_BLOB" data, and not all the DA server // support this, thus we use "VT_BSTR" to substitute case BLOB: + case OBJECT: return Variant.VT_BSTR; default: throw new UnSupportedDataTypeException("UnSupported dataType " + dataType); @@ -354,6 +355,7 @@ private Variant.VARIANT getTabletObjectValue4Opc( case TEXT: case STRING: case BLOB: + case OBJECT: bstr = OleAuto.INSTANCE.SysAllocString(((Binary[]) column)[rowIndex].toString()); value.setValue(Variant.VT_BSTR, bstr); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/util/sorter/PipeTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/util/sorter/PipeTabletEventSorter.java index c9857c9eabaca..6569c6be8953c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/util/sorter/PipeTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/util/sorter/PipeTabletEventSorter.java @@ -136,6 +136,7 @@ protected Object reorderValueListAndBitMap( return deDuplicatedDoubleValues; case TEXT: case BLOB: + case OBJECT: case STRING: final Binary[] binaryValues = (Binary[]) valueList; final Binary[] deDuplicatedBinaryValues = new Binary[binaryValues.length]; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index 5655758f1694e..ea7646efb2b24 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -861,7 +861,7 @@ private List executeGroupByQueryInternal( IMeasurementSchema measurementSchema = new MeasurementSchema(measurement, dataType); AbstractSeriesAggregationScanOperator operator; boolean canUseStatistics = - !TSDataType.BLOB.equals(dataType) + (!TSDataType.BLOB.equals(dataType) && !TSDataType.OBJECT.equals(dataType)) || (!TAggregationType.LAST_VALUE.equals(aggregationType) && !TAggregationType.FIRST_VALUE.equals(aggregationType)); IFullPath path; 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 c61d769bcb3a4..fb7938309af09 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 @@ -163,6 +163,7 @@ private static Accumulator createModeAccumulator(TSDataType tsDataType) { return new FloatModeAccumulator(); case DOUBLE: return new DoubleModeAccumulator(); + case OBJECT: 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 c6d1baa33830a..6549a7a6748e9 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 @@ -57,6 +57,7 @@ public void addInput(Column[] columns, BitMap bitMap) { return; case TEXT: case BLOB: + case OBJECT: case STRING: case BOOLEAN: case DATE: 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 76a42b41c7180..ff54a48ce5e79 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 @@ -58,6 +58,7 @@ public void addInput(Column[] columns, BitMap bitMap) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -90,6 +91,7 @@ public void addIntermediate(Column[] partialResult) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -124,6 +126,7 @@ public void addStatistics(Statistics statistics) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -155,6 +158,7 @@ public void setFinal(Column finalResult) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -188,6 +192,7 @@ public void outputIntermediate(ColumnBuilder[] columnBuilders) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -219,6 +224,7 @@ public void outputFinal(ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/FirstValueAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/FirstValueAccumulator.java index 3bc2e0f9c8961..640008f0a5fda 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/FirstValueAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/FirstValueAccumulator.java @@ -63,6 +63,7 @@ public void addInput(Column[] columns, BitMap bitMap) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(columns, bitMap); return; case BOOLEAN: @@ -98,6 +99,7 @@ public void addIntermediate(Column[] partialResult) { break; case TEXT: case BLOB: + case OBJECT: case STRING: updateBinaryFirstValue(partialResult[0].getBinary(0), partialResult[1].getLong(0)); break; @@ -132,6 +134,7 @@ public void addStatistics(Statistics statistics) { break; case TEXT: case BLOB: + case OBJECT: case STRING: updateBinaryFirstValue((Binary) statistics.getFirstValue(), statistics.getStartTime()); break; @@ -167,6 +170,7 @@ public void setFinal(Column finalResult) { break; case TEXT: case BLOB: + case OBJECT: case STRING: firstValue.setBinary(finalResult.getBinary(0)); break; @@ -206,6 +210,7 @@ public void outputIntermediate(ColumnBuilder[] columnBuilders) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilders[0].writeBinary(firstValue.getBinary()); break; @@ -242,6 +247,7 @@ public void outputFinal(ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilder.writeBinary(firstValue.getBinary()); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/LastValueAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/LastValueAccumulator.java index 603f3427860ad..36c6ed5f88df7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/LastValueAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/LastValueAccumulator.java @@ -63,6 +63,7 @@ public void addInput(Column[] columns, BitMap bitMap) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(columns, bitMap); return; case BOOLEAN: @@ -99,6 +100,7 @@ public void addIntermediate(Column[] partialResult) { case TEXT: case BLOB: case STRING: + case OBJECT: updateBinaryLastValue(partialResult[0].getBinary(0), partialResult[1].getLong(0)); break; case BOOLEAN: @@ -133,6 +135,7 @@ public void addStatistics(Statistics statistics) { case TEXT: case BLOB: case STRING: + case OBJECT: updateBinaryLastValue((Binary) statistics.getLastValue(), statistics.getEndTime()); break; case BOOLEAN: @@ -168,6 +171,7 @@ public void setFinal(Column finalResult) { case TEXT: case BLOB: case STRING: + case OBJECT: lastValue.setBinary(finalResult.getBinary(0)); break; case BOOLEAN: @@ -207,6 +211,7 @@ public void outputIntermediate(ColumnBuilder[] columnBuilders) { case TEXT: case BLOB: case STRING: + case OBJECT: columnBuilders[0].writeBinary(lastValue.getBinary()); break; case BOOLEAN: @@ -243,6 +248,7 @@ public void outputFinal(ColumnBuilder columnBuilder) { case TEXT: case BLOB: case STRING: + case OBJECT: columnBuilder.writeBinary(lastValue.getBinary()); break; case BOOLEAN: 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 a0e4f80120cb0..ebbd1aa6dc149 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 @@ -88,6 +88,7 @@ public void addInput(Column[] column, BitMap bitMap) { case TEXT: case BLOB: case BOOLEAN: + case OBJECT: default: throw new UnSupportedDataTypeException(String.format(UNSUPPORTED_TYPE_MESSAGE, yDataType)); } @@ -302,6 +303,7 @@ private void writeX(ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: columnBuilder.writeBinary(xResult.getBinary()); break; case BOOLEAN: @@ -335,6 +337,7 @@ private void updateX(Column xColumn, int xIndex) { case TEXT: case STRING: case BLOB: + case OBJECT: xResult.setBinary(xColumn.getBinary(xIndex)); break; case BOOLEAN: @@ -385,6 +388,7 @@ private void writeIntermediateToStream( case TEXT: case STRING: case BLOB: + case OBJECT: String content = value.getBinary().toString(); dataOutputStream.writeInt(content.length()); dataOutputStream.writeBytes(content); @@ -441,6 +445,7 @@ private void updateFromBytesIntermediateInput(byte[] bytes) { case TEXT: case BLOB: case BOOLEAN: + case OBJECT: default: throw new UnSupportedDataTypeException(String.format(UNSUPPORTED_TYPE_MESSAGE, yDataType)); } @@ -471,6 +476,7 @@ private void readXFromBytesIntermediateInput( case TEXT: case STRING: case BLOB: + case OBJECT: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; columnBuilder.writeBinary(new Binary(BytesUtils.subBytes(bytes, offset, length))); 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 0d58de8064f26..d02678bd6225a 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 @@ -64,6 +64,7 @@ public void addInput(Column[] columns, BitMap bitMap) { return; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -98,6 +99,7 @@ public void addIntermediate(Column[] partialResult) { break; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -130,6 +132,7 @@ public void addStatistics(Statistics statistics) { break; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -164,6 +167,7 @@ public void setFinal(Column finalResult) { break; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -199,6 +203,7 @@ public void outputIntermediate(ColumnBuilder[] columnBuilders) { break; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -232,6 +237,7 @@ public void outputFinal(ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: 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 1d9cc59aa17de..fd55ea0f1bc74 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 @@ -64,6 +64,7 @@ public void addInput(Column[] columns, BitMap bitMap) { return; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -98,6 +99,7 @@ public void addIntermediate(Column[] partialResult) { break; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -130,6 +132,7 @@ public void addStatistics(Statistics statistics) { break; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -164,6 +167,7 @@ public void setFinal(Column finalResult) { break; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -199,6 +203,7 @@ public void outputIntermediate(ColumnBuilder[] columnBuilders) { break; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -232,6 +237,7 @@ public void outputFinal(ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: 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 37daf1a84b1e0..066d8cd4fc5d3 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 @@ -57,6 +57,7 @@ public void addInput(Column[] columns, BitMap bitMap) { return; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: case TIMESTAMP: case DATE: 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 3242518c3dc2d..8d27f1497e06d 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 @@ -70,6 +70,7 @@ public void addInput(Column[] columns, BitMap bitMap) { return; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case STRING: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java index f4aca5e157e1a..63af0423d1a9f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java @@ -66,6 +66,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsOfOneDeviceNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertTabletNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode; @@ -370,6 +371,19 @@ public RegionExecutionResult visitDeleteData( } } + @Override + public RegionExecutionResult visitWriteObjectFile( + final ObjectNode node, final WritePlanNodeExecutionContext context) { + // data deletion don't need to block data insertion, but there are some creation operation + // require write lock on data region. + context.getRegionWriteValidationRWLock().writeLock().lock(); + try { + return super.visitWriteObjectFile(node, context); + } finally { + context.getRegionWriteValidationRWLock().writeLock().unlock(); + } + } + @Override public RegionExecutionResult visitDeleteTimeseries( final DeleteTimeSeriesNode node, final WritePlanNodeExecutionContext context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/memory/LocalMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/memory/LocalMemoryManager.java index 03766ab52762d..b6ce8b521100a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/memory/LocalMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/memory/LocalMemoryManager.java @@ -30,7 +30,6 @@ public class LocalMemoryManager { private final MemoryPool queryPool; public LocalMemoryManager() { - // TODO @spricoder: why this pool is only used for query data exchange queryPool = new MemoryPool( "read", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationUtil.java index 7c671e7b27498..2df0be4c3394b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationUtil.java @@ -262,6 +262,7 @@ public static long getOutputColumnSizePerLine(TSDataType tsDataType) { return BooleanColumn.SIZE_IN_BYTES_PER_POSITION; case TEXT: case BLOB: + case OBJECT: case STRING: return StatisticsManager.getInstance().getMaxBinarySizeInBytes(); default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TopKOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TopKOperator.java index 4668da6413ed9..bf021e5b4f409 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TopKOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TopKOperator.java @@ -306,6 +306,7 @@ private void initResultTsBlock() { case TEXT: case STRING: case BLOB: + case OBJECT: columns[i] = new BinaryColumn( positionCount, @@ -382,6 +383,7 @@ private long getMemoryUsageOfOneMergeSortKey() { case TEXT: case STRING: case BLOB: + case OBJECT: memory += 16; break; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TransformOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TransformOperator.java index 6dd24e3795a9c..28a80462ea771 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TransformOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TransformOperator.java @@ -344,6 +344,7 @@ protected YieldableState collectDataPoint(ColumnBuilder writer, long currentTime break; case TEXT: case BLOB: + case OBJECT: case STRING: writer.writeBinary(valueColumn.getBinary(currentIndex)); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/MergeSortComparator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/MergeSortComparator.java index 205808fd12076..468e340200af9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/MergeSortComparator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/MergeSortComparator.java @@ -113,6 +113,7 @@ public static Comparator getComparator(TSDataType dataType, int index, break; case TEXT: case BLOB: + case OBJECT: case STRING: comparator = Comparator.comparing( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunction.java index 90b519d52cd97..91e39753344d9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunction.java @@ -105,6 +105,7 @@ private void writeDefaultValue( case TEXT: case STRING: case BLOB: + case OBJECT: builder.writeBinary(partition.getBinary(defaultValChannel, index)); return; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunction.java index e21ff13a861bb..c51eac1563596 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunction.java @@ -106,6 +106,7 @@ private void writeDefaultValue( case TEXT: case STRING: case BLOB: + case OBJECT: builder.writeBinary(partition.getBinary(defaultValChannel, index)); return; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/RowComparator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/RowComparator.java index ac6f225880931..73ff0fe60e64c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/RowComparator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/RowComparator.java @@ -99,6 +99,7 @@ private boolean equal(Column column, TSDataType dataType, int offset1, int offse case STRING: case TEXT: case BLOB: + case OBJECT: Binary bin1 = column.getBinary(offset1); Binary bin2 = column.getBinary(offset2); if (!bin1.equals(bin2)) { @@ -178,6 +179,7 @@ private boolean equal(ColumnList column, TSDataType dataType, int offset1, int o case TEXT: case STRING: case BLOB: + case OBJECT: Binary bin1 = column.getBinary(offset1); Binary bin2 = column.getBinary(offset2); if (!bin1.equals(bin2)) { @@ -242,6 +244,7 @@ public boolean equal(List columns1, int offset1, List columns2, case TEXT: case STRING: case BLOB: + case OBJECT: Binary bin1 = column1.getBinary(offset1); Binary bin2 = column2.getBinary(offset2); if (!bin1.equals(bin2)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java index 078408c8ff97e..b49b9b199d1a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java @@ -1063,6 +1063,7 @@ private void addTimeValuePairToResult(TimeValuePair timeValuePair, TsBlockBuilde break; case TEXT: case BLOB: + case OBJECT: case STRING: builder.getColumnBuilder(0).writeBinary(timeValuePair.getValue().getBinary()); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/LastQueryAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/LastQueryAggTableScanOperator.java index fffd43043d307..999842cece369 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/LastQueryAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/LastQueryAggTableScanOperator.java @@ -726,6 +726,7 @@ private TsPrimitiveType cloneTsPrimitiveType(TsPrimitiveType originalValue) { return new TsPrimitiveType.TsDouble(originalValue.getDouble()); case TEXT: case BLOB: + case OBJECT: case STRING: return new TsPrimitiveType.TsBinary(originalValue.getBinary()); case VECTOR: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AccumulatorFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AccumulatorFactory.java index d7195cc6dfb52..45617de31d1c2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AccumulatorFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AccumulatorFactory.java @@ -391,6 +391,7 @@ public static GroupedAccumulator getGroupedApproxMostFrequentAccumulator(TSDataT return new BinaryGroupedApproxMostFrequentAccumulator(); case BLOB: return new BlobGroupedApproxMostFrequentAccumulator(); + case OBJECT: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in APPROX_COUNT_DISTINCT Aggregation: %s", type)); @@ -416,6 +417,7 @@ public static TableAccumulator getApproxMostFrequentAccumulator(TSDataType type) return new BinaryApproxMostFrequentAccumulator(); case BLOB: return new BlobApproxMostFrequentAccumulator(); + case OBJECT: default: throw new UnSupportedDataTypeException( String.format("Unsupported data type in APPROX_COUNT_DISTINCT Aggregation: %s", type)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/ApproxCountDistinctAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/ApproxCountDistinctAccumulator.java index b78aa1b0177d5..ef31759ee677b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/ApproxCountDistinctAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/ApproxCountDistinctAccumulator.java @@ -78,6 +78,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(arguments[0], mask, hll); return; case BOOLEAN: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AvgAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AvgAccumulator.java index 464820e22f4dc..7bb03faadeb7d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AvgAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AvgAccumulator.java @@ -73,6 +73,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { return; case TEXT: case BLOB: + case OBJECT: case STRING: case BOOLEAN: case DATE: @@ -101,6 +102,7 @@ public void removeInput(Column[] arguments) { return; case TEXT: case BLOB: + case OBJECT: case STRING: case BOOLEAN: case DATE: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/ExtremeAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/ExtremeAccumulator.java index db50a3cb7d704..cc9d3e3354cb0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/ExtremeAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/ExtremeAccumulator.java @@ -68,6 +68,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -100,6 +101,7 @@ public void addIntermediate(Column argument) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -136,6 +138,7 @@ public void addStatistics(Statistics[] statistics) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -168,6 +171,7 @@ public void evaluateIntermediate(ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -200,6 +204,7 @@ public void evaluateFinal(ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/FirstAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/FirstAccumulator.java index 3e4111fd4540b..2ca9b7785ccd1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/FirstAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/FirstAccumulator.java @@ -82,6 +82,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(arguments[0], arguments[1], mask); return; case BOOLEAN: @@ -132,6 +133,7 @@ public void addIntermediate(Column argument) { case TEXT: case BLOB: case STRING: + case OBJECT: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; Binary binaryVal = new Binary(BytesUtils.subBytes(bytes, offset, length)); @@ -184,6 +186,7 @@ public void evaluateFinal(ColumnBuilder columnBuilder) { case TEXT: case BLOB: case STRING: + case OBJECT: columnBuilder.writeBinary(firstValue.getBinary()); break; case BOOLEAN: @@ -225,6 +228,7 @@ public void addStatistics(Statistics[] statistics) { case TEXT: case BLOB: case STRING: + case OBJECT: updateBinaryFirstValue( (Binary) statistics[0].getFirstValue(), statistics[0].getStartTime()); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/FirstByAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/FirstByAccumulator.java index e53bfd0bfd5bf..2ca6c201f9471 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/FirstByAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/FirstByAccumulator.java @@ -104,6 +104,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(arguments[0], arguments[1], arguments[2], mask); return; case BOOLEAN: @@ -164,6 +165,7 @@ public void addIntermediate(Column argument) { break; case TEXT: case BLOB: + case OBJECT: case STRING: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; @@ -219,6 +221,7 @@ public void evaluateFinal(ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilder.writeBinary(xResult.getBinary()); break; @@ -278,6 +281,7 @@ public void addStatistics(Statistics[] statistics) { break; case TEXT: case BLOB: + case OBJECT: case STRING: xResult.setBinary((Binary) statistics[0].getFirstValue()); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/LastAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/LastAccumulator.java index d805500b118be..40f8e2ef97abd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/LastAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/LastAccumulator.java @@ -92,6 +92,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(arguments[0], arguments[1], mask); return; case BOOLEAN: @@ -141,6 +142,7 @@ public void addIntermediate(Column argument) { break; case TEXT: case BLOB: + case OBJECT: case STRING: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; @@ -193,6 +195,7 @@ public void evaluateFinal(ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilder.writeBinary(lastValue.getBinary()); break; @@ -234,6 +237,7 @@ public void addStatistics(Statistics[] statistics) { break; case TEXT: case BLOB: + case OBJECT: case STRING: updateBinaryLastValue((Binary) statistics[0].getLastValue(), statistics[0].getEndTime()); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/LastByAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/LastByAccumulator.java index 7002427315732..2ee09380b83ab 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/LastByAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/LastByAccumulator.java @@ -112,6 +112,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(arguments[0], arguments[1], arguments[2], mask); return; case BOOLEAN: @@ -173,6 +174,7 @@ public void addIntermediate(Column argument) { case TEXT: case BLOB: case STRING: + case OBJECT: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; Binary binaryVal = new Binary(BytesUtils.subBytes(bytes, offset, length)); @@ -228,6 +230,7 @@ public void evaluateFinal(ColumnBuilder columnBuilder) { case TEXT: case BLOB: case STRING: + case OBJECT: columnBuilder.writeBinary(xResult.getBinary()); break; case BOOLEAN: @@ -287,6 +290,7 @@ public void addStatistics(Statistics[] statistics) { case TEXT: case BLOB: case STRING: + case OBJECT: xResult.setBinary((Binary) statistics[0].getLastValue()); break; case BOOLEAN: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/SumAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/SumAccumulator.java index aaab07aa90b82..6f42e1a9995f3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/SumAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/SumAccumulator.java @@ -67,6 +67,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { return; case TEXT: case BLOB: + case OBJECT: case STRING: case BOOLEAN: case DATE: @@ -95,6 +96,7 @@ public void removeInput(Column[] arguments) { return; case TEXT: case BLOB: + case OBJECT: case STRING: case BOOLEAN: case DATE: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/TableMaxMinByBaseAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/TableMaxMinByBaseAccumulator.java index c33fd5f703a98..c6002304b1070 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/TableMaxMinByBaseAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/TableMaxMinByBaseAccumulator.java @@ -85,6 +85,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { case STRING: case TEXT: case BLOB: + case OBJECT: addBinaryInput(arguments, mask); return; case BOOLEAN: @@ -350,6 +351,7 @@ private void writeX(ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: columnBuilder.writeBinary(xResult.getBinary()); break; case BOOLEAN: @@ -383,6 +385,7 @@ private void updateX(Column xColumn, int xIndex) { case TEXT: case STRING: case BLOB: + case OBJECT: xResult.setBinary(xColumn.getBinary(xIndex)); break; case BOOLEAN: @@ -451,6 +454,7 @@ private void updateFromBytesIntermediateInput(byte[] bytes) { case STRING: case TEXT: case BLOB: + case OBJECT: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; Binary binaryMaxVal = new Binary(BytesUtils.subBytes(bytes, offset, length)); @@ -494,6 +498,7 @@ private void readXFromBytesIntermediateInput( case TEXT: case STRING: case BLOB: + case OBJECT: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; columnBuilder.writeBinary(new Binary(BytesUtils.subBytes(bytes, offset, length))); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/TableVarianceAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/TableVarianceAccumulator.java index 355acba8aa6f7..92ea803f2d0ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/TableVarianceAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/TableVarianceAccumulator.java @@ -79,6 +79,7 @@ public void addInput(Column[] arguments, AggregationMask mask) { return; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case STRING: @@ -106,6 +107,7 @@ public void removeInput(Column[] arguments) { return; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case STRING: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/Utils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/Utils.java index 37f2f5d912dc5..99d5fef12089f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/Utils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/Utils.java @@ -54,6 +54,7 @@ public static void serializeValue( case TEXT: case STRING: case BLOB: + case OBJECT: BytesUtils.intToBytes(value.getBinary().getValues().length, valueBytes, offset); offset += 4; System.arraycopy( @@ -116,6 +117,7 @@ public static int calcTypeSize(TSDataType dataType, TsPrimitiveType value) { return 8; case TEXT: case BLOB: + case OBJECT: case STRING: return 4 + value.getBinary().getValues().length; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedApproxCountDistinctAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedApproxCountDistinctAccumulator.java index 2573350a31ddc..9fac50c2ad24c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedApproxCountDistinctAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedApproxCountDistinctAccumulator.java @@ -78,6 +78,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(groupIds, arguments[0], mask, hlls, maxStandardError); return; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedAvgAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedAvgAccumulator.java index 2947c258e79ed..4e177c352b7de 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedAvgAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedAvgAccumulator.java @@ -77,6 +77,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { return; case TEXT: case BLOB: + case OBJECT: case STRING: case BOOLEAN: case DATE: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedExtremeAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedExtremeAccumulator.java index 8d46ab274f4c3..f53ea59314231 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedExtremeAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedExtremeAccumulator.java @@ -62,6 +62,7 @@ public GroupedExtremeAccumulator(TSDataType seriesDataType) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -92,6 +93,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: break; case BOOLEAN: break; @@ -124,6 +126,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: default: throw new UnSupportedDataTypeException( @@ -150,6 +153,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -183,6 +187,7 @@ public void addIntermediate(int[] groupIds, Column argument) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -214,6 +219,7 @@ public void evaluateIntermediate(int groupId, ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -245,6 +251,7 @@ public void evaluateFinal(int groupId, ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: @@ -277,6 +284,7 @@ public void reset() { case TEXT: case STRING: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case TIMESTAMP: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedFirstAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedFirstAccumulator.java index a1884162c4948..b8fe8bbe1b224 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedFirstAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedFirstAccumulator.java @@ -78,6 +78,7 @@ public GroupedFirstAccumulator(TSDataType seriesDataType) { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues = new BinaryBigArray(); return; case BOOLEAN: @@ -110,6 +111,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: valuesSize += binaryValues.sizeOf(); break; case BOOLEAN: @@ -144,6 +146,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues.ensureCapacity(groupCount); return; case BOOLEAN: @@ -176,6 +179,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(groupIds, arguments[0], arguments[1], mask); return; case BOOLEAN: @@ -225,6 +229,7 @@ public void addIntermediate(int[] groupIds, Column argument) { break; case TEXT: case BLOB: + case OBJECT: case STRING: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; @@ -276,6 +281,7 @@ public void evaluateFinal(int groupId, ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilder.writeBinary(binaryValues.get(groupId)); break; @@ -313,6 +319,7 @@ public void reset() { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues.reset(); return; case BOOLEAN: @@ -356,6 +363,7 @@ private byte[] serializeTimeWithValue(int groupId) { return bytes; case TEXT: case BLOB: + case OBJECT: case STRING: byte[] values = binaryValues.get(groupId).getValues(); length += Integer.BYTES + values.length; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedFirstByAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedFirstByAccumulator.java index 33ee0f9087395..d56ba59ef946b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedFirstByAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedFirstByAccumulator.java @@ -87,6 +87,7 @@ public GroupedFirstByAccumulator(TSDataType xDataType, TSDataType yDataType) { break; case TEXT: case BLOB: + case OBJECT: case STRING: xBinaryValues = new BinaryBigArray(); break; @@ -120,6 +121,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: valuesSize += xBinaryValues.sizeOf(); break; case BOOLEAN: @@ -156,6 +158,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: xBinaryValues.ensureCapacity(groupCount); return; case BOOLEAN: @@ -192,6 +195,7 @@ public void reset() { break; case TEXT: case BLOB: + case OBJECT: case STRING: xBinaryValues.reset(); break; @@ -227,6 +231,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(groupIds, arguments[0], arguments[1], arguments[2], mask); return; case BOOLEAN: @@ -288,6 +293,7 @@ public void addIntermediate(int[] groupIds, Column argument) { break; case TEXT: case BLOB: + case OBJECT: case STRING: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; @@ -343,6 +349,7 @@ private byte[] serializeTimeWithValue(int groupId) { return bytes; case TEXT: case BLOB: + case OBJECT: case STRING: byte[] values = xBinaryValues.get(groupId).getValues(); intToBytes(values.length, bytes, Long.BYTES + 1); @@ -373,6 +380,7 @@ private int calculateValueLength(int groupId) { return Double.BYTES; case TEXT: case BLOB: + case OBJECT: case STRING: return Integer.BYTES + xBinaryValues.get(groupId).getValues().length; case BOOLEAN: @@ -407,6 +415,7 @@ public void evaluateFinal(int groupId, ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilder.writeBinary(xBinaryValues.get(groupId)); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedLastAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedLastAccumulator.java index 75cc674ef967f..b21a683ab0a1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedLastAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedLastAccumulator.java @@ -78,6 +78,7 @@ public GroupedLastAccumulator(TSDataType seriesDataType) { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues = new BinaryBigArray(); return; case BOOLEAN: @@ -110,6 +111,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: valuesSize += binaryValues.sizeOf(); break; case BOOLEAN: @@ -144,6 +146,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues.ensureCapacity(groupCount); return; case BOOLEAN: @@ -176,6 +179,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(groupIds, arguments[0], arguments[1], mask); return; case BOOLEAN: @@ -225,6 +229,7 @@ public void addIntermediate(int[] groupIds, Column argument) { break; case TEXT: case BLOB: + case OBJECT: case STRING: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; @@ -276,6 +281,7 @@ public void evaluateFinal(int groupId, ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilder.writeBinary(binaryValues.get(groupId)); break; @@ -313,6 +319,7 @@ public void reset() { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues.reset(); return; case BOOLEAN: @@ -356,6 +363,7 @@ private byte[] serializeTimeWithValue(int groupId) { return bytes; case TEXT: case BLOB: + case OBJECT: case STRING: byte[] values = binaryValues.get(groupId).getValues(); length += Integer.BYTES + values.length; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedLastByAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedLastByAccumulator.java index 386d4c2f34f03..db48d221f18b2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedLastByAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedLastByAccumulator.java @@ -87,6 +87,7 @@ public GroupedLastByAccumulator(TSDataType xDataType, TSDataType yDataType) { break; case TEXT: case BLOB: + case OBJECT: case STRING: xBinaryValues = new BinaryBigArray(); break; @@ -120,6 +121,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: valuesSize += xBinaryValues.sizeOf(); break; case BOOLEAN: @@ -156,6 +158,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: xBinaryValues.ensureCapacity(groupCount); return; case BOOLEAN: @@ -192,6 +195,7 @@ public void reset() { break; case TEXT: case BLOB: + case OBJECT: case STRING: xBinaryValues.reset(); break; @@ -227,6 +231,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(groupIds, arguments[0], arguments[1], arguments[2], mask); return; case BOOLEAN: @@ -288,6 +293,7 @@ public void addIntermediate(int[] groupIds, Column argument) { break; case TEXT: case BLOB: + case OBJECT: case STRING: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; @@ -343,6 +349,7 @@ private byte[] serializeTimeWithValue(int groupId) { return bytes; case TEXT: case BLOB: + case OBJECT: case STRING: byte[] values = xBinaryValues.get(groupId).getValues(); intToBytes(values.length, bytes, Long.BYTES + 1); @@ -373,6 +380,7 @@ private int calculateValueLength(int groupId) { return Double.BYTES; case TEXT: case BLOB: + case OBJECT: case STRING: return Integer.BYTES + xBinaryValues.get(groupId).getValues().length; case BOOLEAN: @@ -407,6 +415,7 @@ public void evaluateFinal(int groupId, ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilder.writeBinary(xBinaryValues.get(groupId)); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMaxAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMaxAccumulator.java index 3b538f0244bae..ed47c96d25137 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMaxAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMaxAccumulator.java @@ -69,6 +69,7 @@ public GroupedMaxAccumulator(TSDataType seriesDataType) { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues = new BinaryBigArray(); return; case BOOLEAN: @@ -101,6 +102,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: valuesSize += binaryValues.sizeOf(); break; case BOOLEAN: @@ -135,6 +137,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues.ensureCapacity(groupCount); return; case BOOLEAN: @@ -167,6 +170,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(groupIds, arguments[0], mask); return; case BOOLEAN: @@ -204,6 +208,7 @@ public void addIntermediate(int[] groupIds, Column argument) { case TEXT: case BLOB: case STRING: + case OBJECT: updateBinaryValue(groupIds[i], argument.getBinary(i)); break; case BOOLEAN: @@ -240,6 +245,7 @@ public void evaluateIntermediate(int groupId, ColumnBuilder columnBuilder) { case TEXT: case BLOB: case STRING: + case OBJECT: columnBuilder.writeBinary(binaryValues.get(groupId)); break; case BOOLEAN: @@ -275,6 +281,7 @@ public void evaluateFinal(int groupId, ColumnBuilder columnBuilder) { case TEXT: case BLOB: case STRING: + case OBJECT: columnBuilder.writeBinary(binaryValues.get(groupId)); break; case BOOLEAN: @@ -311,6 +318,7 @@ public void reset() { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues.reset(); return; case BOOLEAN: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMaxMinByBaseAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMaxMinByBaseAccumulator.java index 7cfe93629eaea..b9b25f6271552 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMaxMinByBaseAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMaxMinByBaseAccumulator.java @@ -96,6 +96,7 @@ protected GroupedMaxMinByBaseAccumulator(TSDataType xDataType, TSDataType yDataT break; case TEXT: case BLOB: + case OBJECT: case STRING: xBinaryValues = new BinaryBigArray(); break; @@ -124,6 +125,7 @@ protected GroupedMaxMinByBaseAccumulator(TSDataType xDataType, TSDataType yDataT break; case TEXT: case BLOB: + case OBJECT: case STRING: yBinaryValues = new BinaryBigArray(); break; @@ -157,6 +159,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: valuesSize += xBinaryValues.sizeOf(); break; case BOOLEAN: @@ -185,6 +188,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: valuesSize += yBinaryValues.sizeOf(); break; case BOOLEAN: @@ -220,6 +224,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: xBinaryValues.ensureCapacity(groupCount); break; case BOOLEAN: @@ -247,6 +252,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: yBinaryValues.ensureCapacity(groupCount); break; case BOOLEAN: @@ -282,6 +288,7 @@ public void reset() { break; case TEXT: case BLOB: + case OBJECT: case STRING: xBinaryValues.reset(); break; @@ -310,6 +317,7 @@ public void reset() { break; case TEXT: case BLOB: + case OBJECT: case STRING: yBinaryValues.reset(); break; @@ -341,6 +349,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { return; case TEXT: case BLOB: + case OBJECT: case STRING: addBinaryInput(groupIds, arguments, mask); return; @@ -596,6 +605,7 @@ private void writeX(int groupId, ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: columnBuilder.writeBinary(xBinaryValues.get(groupId)); break; case BOOLEAN: @@ -630,6 +640,7 @@ private void updateX(int groupId, Column xColumn, int xIndex) { case TEXT: case STRING: case BLOB: + case OBJECT: xBinaryValues.set(groupId, xColumn.getBinary(xIndex)); break; case BOOLEAN: @@ -692,6 +703,7 @@ private void writeIntermediate( case TEXT: case STRING: case BLOB: + case OBJECT: byte[] values = isX ? xBinaryValues.get(groupId).getValues() : yBinaryValues.get(groupId).getValues(); intToBytes(values.length, bytes, offset); @@ -725,6 +737,7 @@ private int calculateValueLength(int groupId, TSDataType dataType, boolean isX) return Double.BYTES; case TEXT: case BLOB: + case OBJECT: case STRING: return Integer.BYTES + (isX @@ -774,6 +787,7 @@ private void updateFromBytesIntermediateInput(int groupId, byte[] bytes) { case STRING: case TEXT: case BLOB: + case OBJECT: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; Binary binaryMaxVal = new Binary(BytesUtils.subBytes(bytes, offset, length)); @@ -818,6 +832,7 @@ private void readXFromBytesIntermediateInput( case TEXT: case STRING: case BLOB: + case OBJECT: int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; columnBuilder.writeBinary(new Binary(BytesUtils.subBytes(bytes, offset, length))); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMinAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMinAccumulator.java index 1c7206928ff79..8fa9b4d8fa018 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMinAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedMinAccumulator.java @@ -69,6 +69,7 @@ public GroupedMinAccumulator(TSDataType seriesDataType) { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues = new BinaryBigArray(); return; case BOOLEAN: @@ -101,6 +102,7 @@ public long getEstimatedSize() { case TEXT: case STRING: case BLOB: + case OBJECT: valuesSize += binaryValues.sizeOf(); break; case BOOLEAN: @@ -135,6 +137,7 @@ public void setGroupCount(long groupCount) { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues.ensureCapacity(groupCount); return; case BOOLEAN: @@ -167,6 +170,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(groupIds, arguments[0], mask); return; case BOOLEAN: @@ -203,6 +207,7 @@ public void addIntermediate(int[] groupIds, Column argument) { break; case TEXT: case BLOB: + case OBJECT: case STRING: updateBinaryValue(groupIds[i], argument.getBinary(i)); break; @@ -240,6 +245,7 @@ public void evaluateIntermediate(int groupId, ColumnBuilder columnBuilder) { case STRING: case TEXT: case BLOB: + case OBJECT: columnBuilder.writeBinary(binaryValues.get(groupId)); break; case BOOLEAN: @@ -274,6 +280,7 @@ public void evaluateFinal(int groupId, ColumnBuilder columnBuilder) { break; case TEXT: case BLOB: + case OBJECT: case STRING: columnBuilder.writeBinary(binaryValues.get(groupId)); break; @@ -311,6 +318,7 @@ public void reset() { case TEXT: case STRING: case BLOB: + case OBJECT: binaryValues.reset(); return; case BOOLEAN: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedModeAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedModeAccumulator.java index 697a7d3b44a4a..a5bf82c5f409b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedModeAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedModeAccumulator.java @@ -95,6 +95,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { case TEXT: case STRING: case BLOB: + case OBJECT: addBinaryInput(groupIds, arguments[0], mask); break; default: @@ -166,6 +167,7 @@ public void evaluateFinal(int groupId, ColumnBuilder columnBuilder) { case TEXT: case STRING: case BLOB: + case OBJECT: columnBuilder.writeBinary(maxEntry.getKey().getBinary()); break; default: @@ -270,6 +272,7 @@ private byte[] serializeCountMap(int groupId) { case TEXT: case STRING: case BLOB: + case OBJECT: bytes = new byte [offset @@ -366,6 +369,7 @@ private void deserializeAndMergeCountMap(int groupId, byte[] bytes) { case TEXT: case STRING: case BLOB: + case OBJECT: for (int i = 0; i < size; i++) { int length = BytesUtils.bytesToInt(bytes, offset); offset += Integer.BYTES; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedSumAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedSumAccumulator.java index 4734e7c36718f..0c0325ebd0b30 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedSumAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedSumAccumulator.java @@ -70,6 +70,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { return; case TEXT: case BLOB: + case OBJECT: case STRING: case BOOLEAN: case DATE: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedVarianceAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedVarianceAccumulator.java index ea8dead07bdd6..cb6123b636f33 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedVarianceAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/grouped/GroupedVarianceAccumulator.java @@ -83,6 +83,7 @@ public void addInput(int[] groupIds, Column[] arguments, AggregationMask mask) { return; case TEXT: case BLOB: + case OBJECT: case BOOLEAN: case DATE: case STRING: 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 3c3fc6e616aaa..ae4975dff9ad9 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 @@ -70,6 +70,7 @@ private static VariationWindowManager genEqualEventWindowManager( case BOOLEAN: return new EqualBooleanWindowManager(eventWindowParameter, ascending); case BLOB: + case OBJECT: case STRING: case TIMESTAMP: case DATE: @@ -97,6 +98,7 @@ private static VariationWindowManager genVariationEventWindowManager( case STRING: case BOOLEAN: case BLOB: + case OBJECT: case TEXT: default: throw new UnSupportedDataTypeException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/relational/ColumnTransformerBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/relational/ColumnTransformerBuilder.java index 6361b9edf0bcb..0378c80f042be 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/relational/ColumnTransformerBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/relational/ColumnTransformerBuilder.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.queryengine.common.SessionInfo; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.plan.analyze.TypeProvider; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.InputLocation; import org.apache.iotdb.db.queryengine.plan.relational.function.arithmetic.AdditionResolver; @@ -165,6 +166,7 @@ import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar.RTrim2ColumnTransformer; import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar.RTrimColumnTransformer; import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar.RadiansColumnTransformer; +import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar.ReadObjectColumnTransformer; import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar.RegexpLike2ColumnTransformer; import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar.RegexpLikeColumnTransformer; import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar.Replace2ColumnTransformer; @@ -211,6 +213,8 @@ import org.apache.tsfile.read.common.type.TypeEnum; import org.apache.tsfile.utils.Binary; +import javax.annotation.Nullable; + import java.time.ZoneId; import java.util.ArrayList; import java.util.Arrays; @@ -239,6 +243,7 @@ import static org.apache.tsfile.read.common.type.FloatType.FLOAT; import static org.apache.tsfile.read.common.type.IntType.INT32; import static org.apache.tsfile.read.common.type.LongType.INT64; +import static org.apache.tsfile.read.common.type.ObjectType.OBJECT; import static org.apache.tsfile.read.common.type.StringType.STRING; public class ColumnTransformerBuilder @@ -1446,6 +1451,29 @@ private ColumnTransformer getFunctionColumnTransformer( this.process(children.get(0), context), this.process(children.get(1), context), this.process(children.get(2), context)); + } else if (TableBuiltinScalarFunction.READ_OBJECT + .getFunctionName() + .equalsIgnoreCase(functionName)) { + ColumnTransformer first = this.process(children.get(0), context); + if (children.size() == 1) { + return new ReadObjectColumnTransformer(OBJECT, first, context.fragmentInstanceContext); + } else if (children.size() == 2) { + return new ReadObjectColumnTransformer( + OBJECT, + ((LongLiteral) children.get(1)).getParsedValue(), + first, + context.fragmentInstanceContext); + } else { + long offset = ((LongLiteral) children.get(1)).getParsedValue(); + long length = ((LongLiteral) children.get(2)).getParsedValue(); + checkArgument(offset >= 0 && length >= 0); + return new ReadObjectColumnTransformer( + OBJECT, + ((LongLiteral) children.get(1)).getParsedValue(), + ((LongLiteral) children.get(2)).getParsedValue(), + first, + context.fragmentInstanceContext); + } } else { // user defined function if (TableUDFUtils.isScalarFunction(functionName)) { @@ -1910,6 +1938,8 @@ public static class Context { private final Metadata metadata; + private final Optional fragmentInstanceContext; + public Context( SessionInfo sessionInfo, List leafList, @@ -1920,7 +1950,8 @@ public Context( List inputDataTypes, int originSize, TypeProvider typeProvider, - Metadata metadata) { + Metadata metadata, + @Nullable FragmentInstanceContext fragmentInstanceContext) { this.sessionInfo = sessionInfo; this.leafList = leafList; this.inputLocations = inputLocations; @@ -1931,6 +1962,7 @@ public Context( this.originSize = originSize; this.typeProvider = typeProvider; this.metadata = metadata; + this.fragmentInstanceContext = Optional.ofNullable(fragmentInstanceContext); } public Type getType(SymbolReference symbolReference) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index c5ffb2752ee4a..d7d4a951cb860 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -41,6 +41,7 @@ import org.apache.iotdb.db.queryengine.execution.exchange.sink.ISinkHandle; import org.apache.iotdb.db.queryengine.execution.exchange.sink.ShuffleSinkHandle; import org.apache.iotdb.db.queryengine.execution.exchange.source.ISourceHandle; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceManager; import org.apache.iotdb.db.queryengine.execution.operator.EmptyDataOperator; import org.apache.iotdb.db.queryengine.execution.operator.ExplainAnalyzeOperator; @@ -1338,13 +1339,15 @@ private Operator constructFilterAndProjectOperator( ColumnTransformerBuilder visitor = new ColumnTransformerBuilder(); + FragmentInstanceContext fragmentInstanceContext = + context.getDriverContext().getFragmentInstanceContext(); ColumnTransformer filterOutputTransformer = predicate .map( p -> { ColumnTransformerBuilder.Context filterColumnTransformerContext = new ColumnTransformerBuilder.Context( - context.getDriverContext().getFragmentInstanceContext().getSessionInfo(), + fragmentInstanceContext.getSessionInfo(), filterLeafColumnTransformerList, inputLocations, filterExpressionColumnTransformerMap, @@ -1353,7 +1356,8 @@ private Operator constructFilterAndProjectOperator( ImmutableList.of(), 0, context.getTypeProvider(), - metadata); + metadata, + fragmentInstanceContext); return visitor.process(p, filterColumnTransformerContext); }) @@ -1371,7 +1375,7 @@ private Operator constructFilterAndProjectOperator( ColumnTransformerBuilder.Context projectColumnTransformerContext = new ColumnTransformerBuilder.Context( - context.getDriverContext().getFragmentInstanceContext().getSessionInfo(), + fragmentInstanceContext.getSessionInfo(), projectLeafColumnTransformerList, inputLocations, projectExpressionColumnTransformerMap, @@ -1380,7 +1384,8 @@ private Operator constructFilterAndProjectOperator( filterOutputDataTypes, inputLocations.size(), context.getTypeProvider(), - metadata); + metadata, + fragmentInstanceContext); for (Expression expression : projectExpressions) { projectOutputTransformerList.add( @@ -2453,6 +2458,8 @@ public Operator visitTableDeviceQueryCount( // In "count" we have to reuse filter operator per "next" final List filterLeafColumnTransformerList = new ArrayList<>(); + FragmentInstanceContext fragmentInstanceContext = + context.getDriverContext().getFragmentInstanceContext(); return new SchemaCountOperator<>( node.getPlanNodeId(), context @@ -2474,10 +2481,7 @@ public Operator visitTableDeviceQueryCount( .process( node.getTagFuzzyPredicate(), new ColumnTransformerBuilder.Context( - context - .getDriverContext() - .getFragmentInstanceContext() - .getSessionInfo(), + fragmentInstanceContext.getSessionInfo(), filterLeafColumnTransformerList, makeLayout(Collections.singletonList(node)), new HashMap<>(), @@ -2486,7 +2490,8 @@ public Operator visitTableDeviceQueryCount( ImmutableList.of(), 0, context.getTypeProvider(), - metadata)), + metadata, + fragmentInstanceContext)), columnSchemaList, database, table) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java index a834f7e076d2d..61b0b1d4a568a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java @@ -111,6 +111,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsOfOneDeviceNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertRowsNode; @@ -315,6 +316,7 @@ public enum PlanNodeType { RELATIONAL_INSERT_ROW((short) 2001), RELATIONAL_INSERT_ROWS((short) 2002), RELATIONAL_DELETE_DATA((short) 2003), + OBJECT_FILE_NODE((short) 2004), ; public static final int BYTES = Short.BYTES; @@ -358,6 +360,8 @@ public static PlanNode deserializeFromWAL(DataInputStream stream) throws IOExcep return RelationalInsertRowsNode.deserializeFromWAL(stream); case 2003: return RelationalDeleteDataNode.deserializeFromWAL(stream); + case 2004: + return ObjectNode.deserializeFromWAL(stream); default: throw new IllegalArgumentException("Invalid node type: " + nodeType); } @@ -384,6 +388,8 @@ public static PlanNode deserializeFromWAL(ByteBuffer buffer) { return RelationalInsertRowsNode.deserializeFromWAL(buffer); case 2003: return RelationalDeleteDataNode.deserializeFromWAL(buffer); + case 2004: + return ObjectNode.deserialize(buffer); default: throw new IllegalArgumentException("Invalid node type: " + nodeType); } @@ -705,6 +711,8 @@ public static PlanNode deserialize(ByteBuffer buffer, short nodeType) { return RelationalInsertRowsNode.deserialize(buffer); case 2003: return RelationalDeleteDataNode.deserialize(buffer); + case 2004: + return ObjectNode.deserialize(buffer); default: throw new IllegalArgumentException("Invalid node type: " + nodeType); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java index b884ac01935b7..629b06f547855 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java @@ -115,6 +115,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsOfOneDeviceNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertRowsNode; @@ -626,6 +627,10 @@ public R visitDeleteData(RelationalDeleteDataNode node, C context) { return visitPlan(node, context); } + public R visitWriteObjectFile(ObjectNode node, C context) { + return visitPlan(node, context); + } + ///////////////////////////////////////////////////////////////////////////////////////////////// // Pipe Related Node ///////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java index edbc262971a6b..52386b7e077d1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java @@ -94,10 +94,26 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue { // proper positions. protected List range; + private Boolean shouldCheckTTL; + public InsertTabletNode(PlanNodeId id) { super(id); } + public boolean shouldCheckTTL() { + if (shouldCheckTTL != null) { + return shouldCheckTTL; + } + shouldCheckTTL = true; + for (MeasurementSchema measurementSchema : measurementSchemas) { + if (measurementSchema.getType() == TSDataType.OBJECT) { + shouldCheckTTL = false; + break; + } + } + return shouldCheckTTL; + } + @Override public InsertNode mergeInsertNode(List insertNodes) { List index = new ArrayList<>(); @@ -219,7 +235,6 @@ public List splitByPartition(IAnalysis analysis) { final Map deviceIDSplitInfoMap = collectSplitRanges(); final Map> splitMap = splitByReplicaSet(deviceIDSplitInfoMap, analysis); - return doSplit(splitMap); } @@ -289,7 +304,7 @@ protected Map> splitByReplicaSet( return splitMap; } - private List doSplit(Map> splitMap) { + protected List doSplit(Map> splitMap) { List result = new ArrayList<>(); if (splitMap.size() == 1) { @@ -326,7 +341,7 @@ protected InsertTabletNode getEmptySplit(int count) { subTimes.length); } - private WritePlanNode generateOneSplit(Map.Entry> entry) { + protected WritePlanNode generateOneSplit(Map.Entry> entry) { List locs; // generate a new times and values locs = entry.getValue(); @@ -387,6 +402,7 @@ protected Object[] initTabletValues(int columnSize, int rowSize, TSDataType[] da case TEXT: case BLOB: case STRING: + case OBJECT: values[i] = new Binary[rowSize]; break; case FLOAT: @@ -640,6 +656,7 @@ private void serializeColumn(TSDataType dataType, Object column, ByteBuffer buff case TEXT: case BLOB: case STRING: + case OBJECT: Binary[] binaryValues = (Binary[]) column; for (int j = 0; j < rowCount; j++) { if (binaryValues[j] != null && binaryValues[j].getValues() != null) { @@ -692,6 +709,7 @@ private void serializeColumn(TSDataType dataType, Object column, DataOutputStrea case STRING: case TEXT: case BLOB: + case OBJECT: Binary[] binaryValues = (Binary[]) column; for (int j = 0; j < rowCount; j++) { if (binaryValues[j] != null && binaryValues[j].getValues() != null) { @@ -833,6 +851,7 @@ private int getColumnSize(TSDataType dataType, Object column, int start, int end case TEXT: case BLOB: case STRING: + case OBJECT: Binary[] binaryValues = (Binary[]) column; for (int j = start; j < end; j++) { size += ReadWriteIOUtils.sizeToWrite(binaryValues[j]); @@ -964,6 +983,7 @@ private void serializeColumn( case STRING: case TEXT: case BLOB: + case OBJECT: Binary[] binaryValues = (Binary[]) column; for (int j = start; j < end; j++) { if (binaryValues[j] != null && binaryValues[j].getValues() != null) { @@ -1127,6 +1147,7 @@ private boolean equals(Object[] columns) { case TEXT: case BLOB: case STRING: + case OBJECT: if (!Arrays.equals((Binary[]) this.columns[i], (Binary[]) columns[i])) { return false; } @@ -1199,6 +1220,8 @@ public TimeValuePair composeLastTimeValuePair( Binary[] binaryValues = (Binary[]) columns[measurementIndex]; value = new TsPrimitiveType.TsBinary(binaryValues[lastIdx]); break; + case OBJECT: + return null; default: throw new UnSupportedDataTypeException( String.format(DATATYPE_UNSUPPORTED, dataTypes[measurementIndex])); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ObjectNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ObjectNode.java new file mode 100644 index 0000000000000..ebe839bbc0c6d --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ObjectNode.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.planner.plan.node.write; + +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.exception.ObjectFileNotExist; +import org.apache.iotdb.commons.exception.runtime.SerializationRunTimeException; +import org.apache.iotdb.db.queryengine.plan.analyze.IAnalysis; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; +import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; +import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; +import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryType; +import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; +import org.apache.iotdb.db.storageengine.rescon.disk.TierManager; + +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Optional; + +public class ObjectNode extends SearchNode implements WALEntryValue { + + private static final Logger LOGGER = LoggerFactory.getLogger(ObjectNode.class); + + private final boolean isEOF; + + private final long offset; + + private byte[] content; + + private String filePath; + + private final int contentLength; + + private TRegionReplicaSet dataRegionReplicaSet; + + private boolean isGeneratedByRemoteConsensusLeader; + + private Long time = null; + + private String table = null; + + public ObjectNode(boolean isEOF, long offset, byte[] content, String filePath) { + super(new PlanNodeId("")); + this.isEOF = isEOF; + this.offset = offset; + this.filePath = filePath; + this.content = content; + this.contentLength = content.length; + } + + public ObjectNode(boolean isEOF, long offset, int contentLength, String filePath) { + super(new PlanNodeId("")); + this.isEOF = isEOF; + this.offset = offset; + this.filePath = filePath; + this.contentLength = contentLength; + } + + public long getTimestamp() { + calculateTimeAndTableName(); + return time; + } + + public String getTable() { + calculateTimeAndTableName(); + return table; + } + + private void calculateTimeAndTableName() { + if (time != null && table != null) { + return; + } + File file = new File(filePath); + String fileName = new File(filePath).getName(); + String timeStr = fileName.substring(0, fileName.length() - ".bin".length()); + time = Long.parseLong(timeStr); + table = file.getParentFile().getParentFile().getParentFile().getParentFile().getName(); + } + + public boolean isEOF() { + return isEOF; + } + + public byte[] getContent() { + return content; + } + + public long getOffset() { + return offset; + } + + public void setFilePath(String filePath) { + this.filePath = filePath; + } + + public String getFilePath() { + return filePath; + } + + @Override + public void serializeToWAL(IWALByteBufferView buffer) { + buffer.putShort(getType().getNodeType()); + buffer.putLong(searchIndex); + buffer.put((byte) (isEOF ? 1 : 0)); + buffer.putLong(offset); + WALWriteUtils.write(filePath, buffer); + buffer.putInt(content.length); + } + + @Override + public int serializedSize() { + return Short.BYTES + + Long.BYTES + + Byte.BYTES + + Long.BYTES + + Integer.BYTES + + ReadWriteIOUtils.sizeToWrite(filePath); + } + + public static ObjectNode deserializeFromWAL(DataInputStream stream) throws IOException { + long searchIndex = stream.readLong(); + boolean isEOF = stream.readByte() == 1; + long offset = stream.readLong(); + String filePath = ReadWriteIOUtils.readString(stream); + int contentLength = stream.readInt(); + ObjectNode objectNode = new ObjectNode(isEOF, offset, contentLength, filePath); + objectNode.setSearchIndex(searchIndex); + return objectNode; + } + + public static ObjectNode deserializeFromWAL(ByteBuffer buffer) { + long searchIndex = buffer.getLong(); + boolean isEOF = buffer.get() == 1; + long offset = buffer.getLong(); + String filePath = ReadWriteIOUtils.readString(buffer); + Optional objectFile = TierManager.getInstance().getAbsoluteObjectFilePath(filePath); + int contentLength = buffer.getInt(); + byte[] contents = new byte[contentLength]; + if (objectFile.isPresent()) { + try (RandomAccessFile raf = new RandomAccessFile(objectFile.get(), "r")) { + raf.seek(offset); + raf.read(contents); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else { + throw new ObjectFileNotExist(filePath); + } + + ObjectNode objectNode = new ObjectNode(isEOF, offset, contents, filePath); + objectNode.setSearchIndex(searchIndex); + return objectNode; + } + + public static ObjectNode deserialize(ByteBuffer byteBuffer) { + boolean isEoF = ReadWriteIOUtils.readBool(byteBuffer); + long offset = ReadWriteIOUtils.readLong(byteBuffer); + String filePath = ReadWriteIOUtils.readString(byteBuffer); + int contentLength = ReadWriteIOUtils.readInt(byteBuffer); + byte[] content = ReadWriteIOUtils.readBytes(byteBuffer, contentLength); + return new ObjectNode(isEoF, offset, content, filePath); + } + + @Override + public SearchNode merge(List searchNodes) { + if (searchNodes.size() == 1) { + return searchNodes.get(0); + } + throw new UnsupportedOperationException("Merge is not supported"); + } + + @Override + public ProgressIndex getProgressIndex() { + return null; + } + + @Override + public void setProgressIndex(ProgressIndex progressIndex) {} + + @Override + public List splitByPartition(IAnalysis analysis) { + return null; + } + + @Override + public TRegionReplicaSet getRegionReplicaSet() { + return dataRegionReplicaSet; + } + + public void setDataRegionReplicaSet(TRegionReplicaSet dataRegionReplicaSet) { + this.dataRegionReplicaSet = dataRegionReplicaSet; + } + + @Override + public List getChildren() { + return null; + } + + @Override + public void addChild(PlanNode child) {} + + @Override + public PlanNode clone() { + return null; + } + + @Override + public int allowedChildCount() { + return NO_CHILD_ALLOWED; + } + + @Override + public List getOutputColumnNames() { + return null; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + getType().serialize(byteBuffer); + ReadWriteIOUtils.write(isEOF, byteBuffer); + ReadWriteIOUtils.write(offset, byteBuffer); + ReadWriteIOUtils.write(filePath, byteBuffer); + ReadWriteIOUtils.write(contentLength, byteBuffer); + byteBuffer.put(content); + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + getType().serialize(stream); + ReadWriteIOUtils.write(isEOF, stream); + ReadWriteIOUtils.write(offset, stream); + ReadWriteIOUtils.write(filePath, stream); + ReadWriteIOUtils.write(contentLength, stream); + stream.write(content); + } + + public ByteBuffer serialize() { + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream stream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(WALEntryType.OBJECT_FILE_NODE.getCode(), stream); + ReadWriteIOUtils.write((long) TsFileProcessor.MEMTABLE_NOT_EXIST, stream); + ReadWriteIOUtils.write(getType().getNodeType(), stream); + byte[] contents = new byte[contentLength]; + boolean readSuccess = false; + for (int i = 0; i < 2; i++) { + Optional objectFile = TierManager.getInstance().getAbsoluteObjectFilePath(filePath); + if (objectFile.isPresent()) { + try { + readContentFromFile(objectFile.get(), contents); + readSuccess = true; + } catch (IOException e) { + LOGGER.error("Error when read object file {}.", objectFile.get(), e); + } + if (readSuccess) { + break; + } + } + Optional objectTmpFile = + TierManager.getInstance().getAbsoluteObjectFilePath(filePath + ".tmp"); + if (objectTmpFile.isPresent()) { + try { + readContentFromFile(objectTmpFile.get(), contents); + readSuccess = true; + } catch (IOException e) { + LOGGER.error("Error when read tmp object file {}.", objectTmpFile.get(), e); + } + if (readSuccess) { + break; + } + } + } + ReadWriteIOUtils.write(readSuccess && isEOF, stream); + ReadWriteIOUtils.write(offset, stream); + ReadWriteIOUtils.write(filePath, stream); + ReadWriteIOUtils.write(contentLength, stream); + stream.write(contents); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } catch (IOException e) { + throw new SerializationRunTimeException(e); + } + } + + private void readContentFromFile(File file, byte[] contents) throws IOException { + try (RandomAccessFile raf = new RandomAccessFile(file, "r")) { + raf.seek(offset); + raf.read(contents); + } + } + + @Override + public PlanNodeType getType() { + return PlanNodeType.OBJECT_FILE_NODE; + } + + @Override + public long getMemorySize() { + return contentLength; + } + + @Override + public void markAsGeneratedByRemoteConsensusLeader() { + isGeneratedByRemoteConsensusLeader = true; + } + + public boolean isGeneratedByRemoteConsensusLeader() { + return isGeneratedByRemoteConsensusLeader; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitWriteObjectFile(this, context); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java index 6e5e8eeb59821..63ca8d385ff6b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java @@ -30,14 +30,18 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.cache.TableDeviceSchemaCache; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.generator.TsFileNameGenerator; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.IDeviceID.Factory; import org.apache.tsfile.read.TimeValuePair; +import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.utils.BytesUtils; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.write.schema.MeasurementSchema; @@ -46,10 +50,12 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; public class RelationalInsertTabletNode extends InsertTabletNode { @@ -58,6 +64,8 @@ public class RelationalInsertTabletNode extends InsertTabletNode { private boolean singleDevice; + private Object[] convertedColumns; + public RelationalInsertTabletNode( PlanNodeId id, PartialPath devicePath, @@ -108,6 +116,16 @@ public void setSingleDevice() { this.singleDevice = true; } + public List getObjectColumns() { + List objectColumns = new ArrayList<>(); + for (int i = 0; i < columns.length; i++) { + if (dataTypes[i] == TSDataType.OBJECT) { + objectColumns.add((Binary[]) columns[i]); + } + } + return objectColumns; + } + @Override public IDeviceID getDeviceID(int rowIdx) { if (singleDevice) { @@ -372,4 +390,60 @@ public void updateLastCache(final String databaseName) { startOffset = endOffset; } } + + @Override + protected List doSplit(Map> splitMap) { + List result = new ArrayList<>(); + + if (splitMap.size() == 1) { + final Entry> entry = splitMap.entrySet().iterator().next(); + if (entry.getValue().size() == 2) { + // Avoid using system arraycopy when there is no need to split + setRange(entry.getValue()); + setDataRegionReplicaSet(entry.getKey()); + for (int i = 0; i < columns.length; i++) { + if (dataTypes[i] == TSDataType.OBJECT) { + for (int j = 0; j < times.length; j++) { + byte[] binary = ((Binary[]) columns[i])[j].getValues(); + ByteBuffer buffer = ByteBuffer.wrap(binary); + boolean isEoF = buffer.get() == 1; + long offset = buffer.getLong(); + byte[] content = ReadWriteIOUtils.readBytes(buffer, buffer.remaining()); + String relativePath = + TsFileNameGenerator.generateObjectFilePath( + dataRegionReplicaSet.regionId.getId(), times[j], getDeviceID(j)); + ObjectNode objectNode = new ObjectNode(isEoF, offset, content, relativePath); + objectNode.setDataRegionReplicaSet(entry.getKey()); + result.add(objectNode); + if (isEoF) { + byte[] filePathBytes = relativePath.getBytes(StandardCharsets.UTF_8); + byte[] valueBytes = new byte[filePathBytes.length + Long.BYTES]; + System.arraycopy( + BytesUtils.longToBytes(offset + content.length), 0, valueBytes, 0, Long.BYTES); + System.arraycopy(filePathBytes, 0, valueBytes, Long.BYTES, filePathBytes.length); + ((Binary[]) columns[i])[j] = new Binary(valueBytes); + } else { + ((Binary[]) columns[i])[j] = null; + if (bitMaps == null) { + bitMaps = new BitMap[columns.length]; + } + if (bitMaps[i] == null) { + bitMaps[i] = new BitMap(rowCount); + } + bitMaps[i].mark(j); + } + } + } + } + result.add(this); + return result; + } + } + + for (Map.Entry> entry : splitMap.entrySet()) { + // TODO: add ObjectNode for split + result.add(generateOneSplit(entry)); + } + return result; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java index f83e527113856..e4742b873b191 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java @@ -65,6 +65,7 @@ import org.apache.iotdb.udf.api.relational.TableFunction; import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.common.type.ObjectType; import org.apache.tsfile.read.common.type.StringType; import org.apache.tsfile.read.common.type.Type; import org.apache.tsfile.read.common.type.TypeFactory; @@ -1076,6 +1077,20 @@ && isIntegerNumber(argumentTypes.get(2)))) { functionName)); } return BLOB; + } else if (TableBuiltinScalarFunction.READ_OBJECT + .getFunctionName() + .equalsIgnoreCase(functionName)) { + if (argumentTypes.isEmpty() + || argumentTypes.size() > 3 + || !isObjectType(argumentTypes.get(0)) + || (argumentTypes.size() >= 2 && !isIntegerNumber(argumentTypes.get(1))) + || (argumentTypes.size() >= 3 && !isIntegerNumber(argumentTypes.get(2)))) { + throw new SemanticException( + "Scalar function " + + functionName.toLowerCase(Locale.ENGLISH) + + " must have at 1~3 arguments, and first argument must be OBJECT type, other arguments must be int32 or int64 type"); + } + return BLOB; } // builtin aggregation function @@ -1501,6 +1516,10 @@ public static boolean isCharType(Type type) { return TEXT.equals(type) || StringType.STRING.equals(type); } + public static boolean isObjectType(Type type) { + return ObjectType.OBJECT.equals(type); + } + public static boolean isBlobType(Type type) { return BLOB.equals(type); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/DeleteDevice.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/DeleteDevice.java index 9b1487b0a48a6..ed5bc0d3719ee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/DeleteDevice.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/DeleteDevice.java @@ -226,7 +226,8 @@ public static DeviceBlackListConstructor constructDevicePredicateUpdater( ImmutableList.of(), 0, mockTypeProvider, - metadata)) + metadata, + null)) : null; return new DeviceBlackListConstructor( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/type/InternalTypeManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/type/InternalTypeManager.java index 6f2aaef76ab14..5719a4555ff12 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/type/InternalTypeManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/type/InternalTypeManager.java @@ -35,6 +35,7 @@ import static org.apache.tsfile.read.common.type.FloatType.FLOAT; import static org.apache.tsfile.read.common.type.IntType.INT32; import static org.apache.tsfile.read.common.type.LongType.INT64; +import static org.apache.tsfile.read.common.type.ObjectType.OBJECT; import static org.apache.tsfile.read.common.type.StringType.STRING; import static org.apache.tsfile.read.common.type.TimestampType.TIMESTAMP; import static org.apache.tsfile.read.common.type.UnknownType.UNKNOWN; @@ -52,6 +53,7 @@ public InternalTypeManager() { types.put(new TypeSignature(TypeEnum.TEXT.name().toLowerCase(Locale.ENGLISH)), TEXT); types.put(new TypeSignature(TypeEnum.STRING.name().toLowerCase(Locale.ENGLISH)), STRING); types.put(new TypeSignature(TypeEnum.BLOB.name().toLowerCase(Locale.ENGLISH)), BLOB); + types.put(new TypeSignature(TypeEnum.OBJECT.name().toLowerCase(Locale.ENGLISH)), OBJECT); types.put(new TypeSignature(TypeEnum.DATE.name().toLowerCase(Locale.ENGLISH)), DATE); types.put(new TypeSignature(TypeEnum.TIMESTAMP.name().toLowerCase(Locale.ENGLISH)), TIMESTAMP); types.put(new TypeSignature(TypeEnum.UNKNOWN.name().toLowerCase(Locale.ENGLISH)), UNKNOWN); @@ -105,6 +107,8 @@ public static TSDataType getTSDataType(Type type) { return TSDataType.BLOB; case STRING: return TSDataType.STRING; + case OBJECT: + return TSDataType.OBJECT; default: throw new IllegalArgumentException(); } @@ -132,6 +136,8 @@ public static Type fromTSDataType(TSDataType dataType) { return TIMESTAMP; case BLOB: return BLOB; + case OBJECT: + return OBJECT; case STRING: return STRING; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/utils/TypeUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/utils/TypeUtil.java index 0cc31381072f3..fe951d1e4ce10 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/utils/TypeUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/utils/TypeUtil.java @@ -28,6 +28,7 @@ import org.apache.tsfile.read.common.type.BlobType; import org.apache.tsfile.read.common.type.DoubleType; import org.apache.tsfile.read.common.type.FloatType; +import org.apache.tsfile.read.common.type.ObjectType; import org.apache.tsfile.read.common.type.RowType; import org.apache.tsfile.read.common.type.StringType; import org.apache.tsfile.read.common.type.TimestampType; @@ -116,6 +117,8 @@ public static Type getType(TypeEnum typeEnum, List subTypes) { return TimestampType.TIMESTAMP; case DATE: return DATE; + case OBJECT: + return ObjectType.OBJECT; case ROW: return RowType.anonymous(subTypes); default: @@ -123,7 +126,6 @@ public static Type getType(TypeEnum typeEnum, List subTypes) { } } - // TODO move these methods into each Type to avoid branch miss public static boolean isFlatVariableWidth(Type type) { switch (type.getTypeEnum()) { case BOOLEAN: @@ -137,6 +139,7 @@ public static boolean isFlatVariableWidth(Type type) { case TEXT: case STRING: case BLOB: + case OBJECT: return true; default: throw new UnsupportedOperationException(); @@ -160,6 +163,7 @@ public static int getFlatFixedSize(Type type) { case TEXT: case STRING: case BLOB: + case OBJECT: return 16; default: throw new UnsupportedOperationException(); @@ -179,6 +183,7 @@ public static int getFlatVariableWidthSize(Type type, Column column, int positio case TEXT: case STRING: case BLOB: + case OBJECT: return column.isNull(position) ? 0 : column.getBinary(position).getLength(); default: throw new UnsupportedOperationException(); @@ -198,6 +203,7 @@ public static int getFlatVariableWidthSize(Type type, Column column, int[] posit case TEXT: case STRING: case BLOB: + case OBJECT: int result = 0; for (int i = 0; i < position.length; i++) { if (!column.isNull(i)) { @@ -237,6 +243,7 @@ public static void readFlat( case TEXT: case STRING: case BLOB: + case OBJECT: int length = bytesToInt(fixedChunk, fixedOffset); byte[] result = new byte[length]; if (length <= 12) { @@ -282,6 +289,7 @@ public static void writeFlat( case TEXT: case STRING: case BLOB: + case OBJECT: byte[] value = column.getBinary(position).getValues(); intToBytes(value.length, fixedChunk, fixedOffset); if (value.length <= 12) { @@ -320,6 +328,7 @@ public static boolean notDistinctFrom( case TEXT: case STRING: case BLOB: + case OBJECT: int leftLength = bytesToInt(fixedChunk, fixedOffset); byte[] leftValue = new byte[leftLength]; byte[] rightValue = column.getBinary(position).getValues(); @@ -414,6 +423,7 @@ public static long hash(Type type, byte[] fixedChunk, int fixedOffset, byte[] va case TEXT: case STRING: case BLOB: + case OBJECT: int length = bytesToInt(fixedChunk, fixedOffset); byte[] values = new byte[length]; @@ -490,6 +500,7 @@ public static long hash(Type type, Column column, int position) { case TEXT: case STRING: case BLOB: + case OBJECT: return XxHash64.hash(column.getBinary(position).getValues()); default: throw new UnsupportedOperationException(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/ReadObjectColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/ReadObjectColumnTransformer.java new file mode 100644 index 0000000000000..84a094e39fe69 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/ReadObjectColumnTransformer.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar; + +import org.apache.iotdb.commons.exception.IoTDBRuntimeException; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.transformation.dag.column.ColumnTransformer; +import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.UnaryColumnTransformer; +import org.apache.iotdb.db.utils.ObjectTypeUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.block.column.ColumnBuilder; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.type.Type; +import org.apache.tsfile.utils.Binary; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.Optional; + +public class ReadObjectColumnTransformer extends UnaryColumnTransformer { + + private final Optional fragmentInstanceContext; + private long offset = 0; + private long length = -1; + + public ReadObjectColumnTransformer( + Type type, + ColumnTransformer childColumnTransformer, + Optional fragmentInstanceContext) { + super(type, childColumnTransformer); + this.fragmentInstanceContext = fragmentInstanceContext; + } + + public ReadObjectColumnTransformer( + Type type, + long offset, + ColumnTransformer childColumnTransformer, + Optional fragmentInstanceContext) { + super(type, childColumnTransformer); + this.offset = offset; + this.fragmentInstanceContext = fragmentInstanceContext; + } + + public ReadObjectColumnTransformer( + Type type, + long offset, + long length, + ColumnTransformer childColumnTransformer, + Optional fragmentInstanceContext) { + super(type, childColumnTransformer); + this.offset = offset; + this.length = length; + this.fragmentInstanceContext = fragmentInstanceContext; + } + + @Override + protected void doTransform(Column column, ColumnBuilder columnBuilder) { + for (int i = 0, n = column.getPositionCount(); i < n; i++) { + if (!column.isNull(i)) { + transform(column, columnBuilder, i); + } else { + columnBuilder.appendNull(); + } + } + } + + @Override + protected void doTransform(Column column, ColumnBuilder columnBuilder, boolean[] selection) { + for (int i = 0, n = column.getPositionCount(); i < n; i++) { + if (selection[i] && !column.isNull(i)) { + transform(column, columnBuilder, i); + } else { + columnBuilder.appendNull(); + } + } + } + + private void transform(Column column, ColumnBuilder columnBuilder, int i) { + // BinaryColumn.getDataType() returns TSDataType.TEXT + if (TSDataType.TEXT == column.getDataType()) { + Binary binary = column.getBinary(i); + columnBuilder.writeBinary(readObject(binary)); + } + } + + private Binary readObject(Binary binary) { + File file = ObjectTypeUtils.getObjectPathFromBinary(binary); + long fileSize = file.length(); + if (offset >= fileSize) { + throw new UnsupportedOperationException("offset is greater than object size"); + } + long actualReadSize = Math.min(length < 0 ? fileSize : length, fileSize - offset); + if (actualReadSize > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Read object size is too large (size > 2G)"); + } + fragmentInstanceContext.ifPresent( + context -> context.getMemoryReservationContext().reserveMemoryCumulatively(actualReadSize)); + byte[] bytes = new byte[(int) actualReadSize]; + ByteBuffer buffer = ByteBuffer.wrap(bytes); + try (FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.READ)) { + fileChannel.read(buffer); + } catch (IOException e) { + throw new IoTDBRuntimeException(e, TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + return new Binary(bytes); + } +} 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 32125c110ef57..14643a7b6ac50 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 @@ -92,6 +92,7 @@ public ConstantInputReader(ConstantOperand expression) throws QueryProcessExcept cachedColumns[0] = new RunLengthEncodedColumn(booleanColumn, count); break; case BLOB: + case OBJECT: case STRING: case TIMESTAMP: case DATE: 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 fcbb30b7ca139..59b3dbbc8b636 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 @@ -59,6 +59,7 @@ protected void transform(Column[] columns, ColumnBuilder builder) case TEXT: case TIMESTAMP: case BLOB: + case OBJECT: case BOOLEAN: case STRING: default: 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 5bdfa8f96a7f7..09cc6e76ad380 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 @@ -88,6 +88,7 @@ private void initTypedSet(Set values) { stringSet = values; break; case BLOB: + case OBJECT: default: throw new UnsupportedOperationException("unsupported data type: " + layerReaderDataType); } @@ -124,6 +125,7 @@ protected void transform(Column[] columns, ColumnBuilder builder) transformBinary(columns, builder); return; case BLOB: + case OBJECT: 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 11cf952e0b435..195acb6de8527 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 @@ -68,6 +68,7 @@ protected Column[] transform(Column[] columns) throws QueryProcessException, IOE case TEXT: return castBinaries(columns); case BLOB: + case OBJECT: case STRING: case TIMESTAMP: case DATE: @@ -139,6 +140,7 @@ private Column[] castInts(Column[] columns) { break; case STRING: case BLOB: + case OBJECT: case TIMESTAMP: case DATE: case INT32: @@ -213,6 +215,7 @@ private Column[] castLongs(Column[] columns) { } break; case BLOB: + case OBJECT: case STRING: case DATE: case TIMESTAMP: @@ -288,6 +291,7 @@ private Column[] castFloats(Column[] columns) { } break; case BLOB: + case OBJECT: case STRING: case TIMESTAMP: case DATE: @@ -363,6 +367,7 @@ private Column[] castDoubles(Column[] columns) { } break; case BLOB: + case OBJECT: case STRING: case TIMESTAMP: case DATE: @@ -439,6 +444,7 @@ private Column[] castBooleans(Column[] columns) { break; case STRING: case BLOB: + case OBJECT: case DATE: case TIMESTAMP: case BOOLEAN: @@ -521,6 +527,7 @@ private Column[] castBinaries(Column[] columns) { case DATE: case STRING: case BLOB: + case OBJECT: case TEXT: default: throw new UnsupportedOperationException( 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 ee7ba10e60e49..03081f9206f1f 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 @@ -62,6 +62,7 @@ public void transform(Column[] columns, ColumnBuilder builder) throws QueryProce transformDouble(columns, builder); return; case BLOB: + case OBJECT: case TEXT: case DATE: case STRING: 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 a4ddff58bab46..47e4261807730 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 @@ -67,6 +67,7 @@ protected void transform(Column[] columns, ColumnBuilder builder) case STRING: case TEXT: case BLOB: + case OBJECT: 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 c05d7adfecc24..015904399c201 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 @@ -78,6 +78,7 @@ public static Column transformConstantOperandToColumn(ConstantOperand constantOp return new BooleanColumn(1, Optional.empty(), new boolean[] {(boolean) value}); case STRING: case BLOB: + case OBJECT: case DATE: case TIMESTAMP: default: @@ -158,6 +159,7 @@ public static boolean splitWindowForStateWindow( case TIMESTAMP: case DATE: case BLOB: + case OBJECT: case STRING: default: throw new UnsupportedOperationException( 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 0eedb86a31a9e..e8c679e2bb55f 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 @@ -50,6 +50,7 @@ public static ColumnBuilder initColumnBuilder(TSDataType type, int count) { case TEXT: case BLOB: case STRING: + case OBJECT: return new BinaryColumnBuilder(null, count); default: throw new UnSupportedDataTypeException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/datastructure/row/SerializableRowList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/datastructure/row/SerializableRowList.java index 6ec6537de362f..3bacd1ac5e79a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/datastructure/row/SerializableRowList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/datastructure/row/SerializableRowList.java @@ -107,6 +107,7 @@ protected static int calculateCapacity( case TEXT: case BLOB: case STRING: + case OBJECT: rowLength += MIN_OBJECT_HEADER_SIZE + MIN_ARRAY_HEADER_SIZE + byteArrayLengthForMemoryControl; break; @@ -214,6 +215,7 @@ private Object[] getRowSkipPrefixNulls(int index) { case TEXT: case BLOB: case STRING: + case OBJECT: row[i] = block[i].getBinary(offset); break; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/datastructure/tv/SerializableTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/datastructure/tv/SerializableTVList.java index 5f76cd46c3dc7..57ac89f073a2f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/datastructure/tv/SerializableTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/datastructure/tv/SerializableTVList.java @@ -92,6 +92,7 @@ protected static int calculateCapacity(TSDataType dataType, float memoryLimitInM case TEXT: case STRING: case BLOB: + case OBJECT: rowLength += MIN_OBJECT_HEADER_SIZE + MIN_ARRAY_HEADER_SIZE diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java index fbb407d333880..12873ab563cf2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java @@ -1522,7 +1522,8 @@ private DeviceAttributeUpdater constructDevicePredicateUpdater( ImmutableList.of(), 0, mockTypeProvider, - metadata)) + metadata, + null)) : null; final List filterOutputDataTypes = @@ -1549,7 +1550,8 @@ private DeviceAttributeUpdater constructDevicePredicateUpdater( filterOutputDataTypes, inputLocations.size(), mockTypeProvider, - metadata); + metadata, + null); final List attributeNames = updateNode.getAssignments().stream() 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 97e3f4f961c16..bb1aecf05e38d 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 @@ -76,6 +76,7 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.recover.WALRecoverManager; import org.apache.iotdb.db.storageengine.load.LoadTsFileManager; import org.apache.iotdb.db.storageengine.load.limiter.LoadTsFileRateLimiter; +import org.apache.iotdb.db.storageengine.rescon.disk.TierManager; import org.apache.iotdb.db.storageengine.rescon.memory.SystemInfo; import org.apache.iotdb.db.utils.ThreadUtils; import org.apache.iotdb.rpc.RpcUtils; @@ -83,6 +84,8 @@ import org.apache.tsfile.exception.write.PageException; import org.apache.tsfile.external.commons.io.FileUtils; +import org.apache.tsfile.fileSystem.FSFactoryProducer; +import org.apache.tsfile.fileSystem.fsFactory.FSFactory; import org.apache.tsfile.utils.FilePathUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,6 +93,8 @@ import java.io.File; import java.io.IOException; import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -106,6 +111,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -119,6 +125,8 @@ public class StorageEngine implements IService { private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); private static final WritingMetrics WRITING_METRICS = WritingMetrics.getInstance(); + private final FSFactory fsFactory = FSFactoryProducer.getFSFactory(); + /** * a folder (system/databases/ by default) that persist system info. Each database will have a * subfolder under the systemDir. @@ -156,6 +164,8 @@ public class StorageEngine implements IService { private final LoadTsFileManager loadTsFileManager = new LoadTsFileManager(); + public final AtomicLong objectFileId = new AtomicLong(0); + private StorageEngine() {} public static StorageEngine getInstance() { @@ -226,6 +236,8 @@ private void asyncRecoverDataRegion() throws StartupException { } private void asyncRecover(List> futures) { + checkObjectFiles(); + Map> localDataRegionInfo = getLocalDataRegionInfo(); localDataRegionInfo.values().forEach(list -> recoverDataRegionNum += list.size()); readyDataRegionNum = new AtomicInteger(0); @@ -1063,6 +1075,32 @@ public static File getDataRegionSystemDir(String dataBaseName, String dataRegion systemDir + File.separator + dataBaseName, dataRegionId); } + private void checkObjectFiles() { + List folders = TierManager.getInstance().getAllObjectFileFolders(); + for (String baseDir : folders) { + File fileFolder = fsFactory.getFile(baseDir); + try (Stream paths = Files.walk(fileFolder.toPath())) { + paths + .filter(Files::isRegularFile) + .filter( + path -> { + String name = path.getFileName().toString(); + return name.endsWith(".bin.back"); + }) + .forEach( + path -> { + try { + Files.delete(path); + } catch (IOException e) { + LOGGER.error("Failed to delete: {} -> {}", path, e.getMessage()); + } + }); + } catch (IOException e) { + LOGGER.error("Failed to check Object Files: {}", e.getMessage()); + } + } + } + public Runnable executeCompactFileTimeIndexCache() { return () -> { if (!isReadyForNonReadWriteFunctions()) { 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 ed9040f9f982e..6b3a05df7e5a1 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 @@ -81,6 +81,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsOfOneDeviceNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; @@ -155,6 +156,7 @@ import org.apache.iotdb.db.utils.DateTimeUtils; import org.apache.iotdb.db.utils.EncryptDBUtils; import org.apache.iotdb.db.utils.ModificationUtils; +import org.apache.iotdb.db.utils.ObjectWriter; import org.apache.iotdb.metrics.utils.MetricLevel; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -181,6 +183,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -1319,7 +1322,10 @@ private boolean executeInsertTablet( InsertTabletNode insertTabletNode, TSStatus[] results, long[] infoForMetrics) throws OutOfTTLException { boolean noFailure; - int loc = insertTabletNode.checkTTL(results, getTTL(insertTabletNode)); + int loc = + insertTabletNode.shouldCheckTTL() + ? insertTabletNode.checkTTL(results, getTTL(insertTabletNode)) + : 0; noFailure = loc == 0; List> deviceEndOffsetPairs = insertTabletNode.splitByDevice(loc, insertTabletNode.getRowCount()); @@ -1863,6 +1869,7 @@ public void syncDeleteDataFiles() throws TsFileProcessorException { } }); deleteAllSGFolders(TierManager.getInstance().getAllFilesFolders()); + deleteAllObjectFiles(TierManager.getInstance().getAllObjectFileFolders()); this.workSequenceTsFileProcessors.clear(); this.workUnsequenceTsFileProcessors.clear(); this.tsFileManager.clear(); @@ -1900,6 +1907,23 @@ private void deleteAllSGFolders(List folder) { } } + private void deleteAllObjectFiles(List folders) { + for (String objectFolder : folders) { + File dataRegionObjectFolder = fsFactory.getFile(objectFolder, dataRegionId); + if (FSUtils.getFSType(dataRegionObjectFolder) != FSType.LOCAL) { + try { + fsFactory.deleteDirectory(dataRegionObjectFolder.getPath()); + } catch (IOException e) { + logger.error("Fail to delete data region object folder {}", dataRegionObjectFolder); + } + } else { + if (dataRegionObjectFolder.exists()) { + org.apache.iotdb.commons.utils.FileUtils.deleteFileOrDirectory(dataRegionObjectFolder); + } + } + } + } + public void timedFlushSeqMemTable() { int count = 0; writeLock("timedFlushSeqMemTable"); @@ -3333,6 +3357,43 @@ public int compact() { } } + public void writeObject(ObjectNode objectNode) throws Exception { + writeLock("writeObject"); + try { + String relativeTmpPathString = objectNode.getFilePath() + ".tmp"; + String objectFileDir = TierManager.getInstance().getNextFolderForObjectFile(); + File objectTmpFile = + FSFactoryProducer.getFSFactory().getFile(objectFileDir, relativeTmpPathString); + try (ObjectWriter writer = new ObjectWriter(objectTmpFile)) { + writer.write( + objectNode.isGeneratedByRemoteConsensusLeader(), + objectNode.getOffset(), + objectNode.getContent()); + } + if (objectNode.isEOF()) { + File objectFile = + FSFactoryProducer.getFSFactory().getFile(objectFileDir, objectNode.getFilePath()); + if (objectFile.exists()) { + String relativeBackPathString = objectNode.getFilePath() + ".back"; + File objectBackFile = + FSFactoryProducer.getFSFactory().getFile(objectFileDir, relativeBackPathString); + Files.move( + objectFile.toPath(), objectBackFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + Files.move( + objectTmpFile.toPath(), objectFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + Files.delete(objectBackFile.toPath()); + } else { + Files.move( + objectTmpFile.toPath(), objectFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + } + } + getWALNode() + .ifPresent(walNode -> walNode.log(TsFileProcessor.MEMTABLE_NOT_EXIST, objectNode)); + } finally { + writeUnlock(); + } + } + /** * Load a new tsfile to unsequence dir. * diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java index 91184aaec82cb..3a5695a737634 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java @@ -181,8 +181,7 @@ public void perform() throws Exception { sortedSourceFiles.addAll(unseqFiles); boolean isTreeModel = !isAligned || device.getTableName().startsWith("root."); long ttl = deviceIterator.getTTLForCurrentDevice(); - sortedSourceFiles.removeIf( - x -> x.definitelyNotContains(device) || !x.isDeviceAlive(device, ttl)); + sortedSourceFiles.removeIf(x -> x.definitelyNotContains(device)); // checked above //noinspection OptionalGetWithoutIsPresent sortedSourceFiles.sort(Comparator.comparingLong(x -> x.getStartTime(device).get())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CrossSpaceCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CrossSpaceCompactionTask.java index 41d859924f073..290466ffeb6b1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CrossSpaceCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CrossSpaceCompactionTask.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.exception.CompactionRecoverException; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICrossCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadPointCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.subtask.FastCompactionTaskSummary; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionUtils; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.log.CompactionLogAnalyzer; @@ -192,6 +193,11 @@ public boolean doCompaction() { performer.setTargetFiles(targetTsfileResourceList); performer.setSummary(summary); performer.perform(); + if (performer instanceof ReadPointCompactionPerformer) { + for (TsFileResource resource : getAllSourceTsFiles()) { + CompactionUtils.removeDeletedObjectFiles(resource); + } + } CompactionUtils.updateProgressIndexAndMark( targetTsfileResourceList, selectedSequenceFiles, selectedUnsequenceFiles); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java index 3cd5ff532af23..9c7fcc5424fdd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java @@ -30,6 +30,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.IInnerCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadPointCompactionPerformer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.subtask.FastCompactionTaskSummary; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionUtils; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.log.CompactionLogAnalyzer; @@ -389,6 +390,11 @@ protected void compact(SimpleCompactionLogger compactionLogger) throws Exception performer.setTargetFiles(filesView.targetFilesInPerformer); performer.setSummary(summary); performer.perform(); + if (performer instanceof ReadPointCompactionPerformer) { + for (TsFileResource resource : filesView.sourceFilesInCompactionPerformer) { + CompactionUtils.removeDeletedObjectFiles(resource); + } + } prepareTargetFiles(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/SettleCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/SettleCompactionTask.java index e00703e5e7521..79d6a41025bcf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/SettleCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/SettleCompactionTask.java @@ -18,10 +18,12 @@ */ package org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task; +import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.db.service.metrics.FileMetrics; import org.apache.iotdb.db.storageengine.dataregion.compaction.constant.CompactionTaskType; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.exception.CompactionRecoverException; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.ICompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionUtils; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.log.CompactionLogAnalyzer; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.log.CompactionLogger; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.log.SimpleCompactionLogger; @@ -204,7 +206,7 @@ protected boolean doCompaction() { return isSuccess; } - public boolean settleWithFullyDirtyFiles() { + public boolean settleWithFullyDirtyFiles() throws IllegalPathException, IOException { if (fullyDirtyFiles.isEmpty()) { return true; } @@ -213,6 +215,8 @@ public boolean settleWithFullyDirtyFiles() { if (recoverMemoryStatus) { tsFileManager.remove(resource, resource.isSeq()); } + CompactionUtils.removeDeletedObjectFiles(resource); + boolean res = deleteTsFileOnDisk(resource); if (res) { fullyDeletedSuccessNum++; @@ -288,7 +292,7 @@ public void recover() { } } - public void recoverFullyDirtyFiles() { + public void recoverFullyDirtyFiles() throws IllegalPathException, IOException { if (!settleWithFullyDirtyFiles()) { throw new CompactionRecoverException("Failed to delete fully_dirty source file."); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java index 4f58e8fc1323e..1668355db311e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java @@ -31,6 +31,8 @@ import org.apache.iotdb.db.service.metrics.CompactionMetrics; import org.apache.iotdb.db.service.metrics.FileMetrics; import org.apache.iotdb.db.storageengine.dataregion.compaction.constant.CompactionTaskType; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.fast.reader.CompactionAlignedChunkReader; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.fast.reader.CompactionChunkReader; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionTaskManager; import org.apache.iotdb.db.storageengine.dataregion.modification.DeletionPredicate; import org.apache.iotdb.db.storageengine.dataregion.modification.IDPredicate.FullExactMatch; @@ -42,21 +44,33 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.ArrayDeviceTimeIndex; import org.apache.iotdb.db.utils.ModificationUtils; +import org.apache.iotdb.db.utils.ObjectTypeUtils; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory; import org.apache.iotdb.metrics.utils.MetricLevel; import org.apache.iotdb.metrics.utils.SystemMetric; import org.apache.tsfile.common.constant.TsFileConstant; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.header.PageHeader; +import org.apache.tsfile.file.metadata.AbstractAlignedChunkMetadata; import org.apache.tsfile.file.metadata.ChunkMetadata; +import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.fileSystem.FSFactoryProducer; +import org.apache.tsfile.read.TimeValuePair; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.read.common.Chunk; import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.read.reader.IPointReader; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.writer.TsFileIOWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -497,4 +511,158 @@ public static ModEntry convertTtlToDeletion(IDeviceID deviceID, long timeLowerBo new TimeRange(Long.MIN_VALUE, timeLowerBound)); } } + + public static void removeDeletedObjectFiles(TsFileResource resource) + throws IOException, IllegalPathException { + try (MultiTsFileDeviceIterator deviceIterator = + new MultiTsFileDeviceIterator(Collections.singletonList(resource))) { + while (deviceIterator.hasNextDevice()) { + deviceIterator.nextDevice(); + deviceIterator.getReaderAndChunkMetadataForCurrentAlignedSeries(); + } + } + } + + public static void removeDeletedObjectFiles( + TsFileSequenceReader reader, + List alignedChunkMetadataList, + List timeMods, + List> valueMods) + throws IOException { + if (alignedChunkMetadataList.isEmpty()) { + return; + } + List objectColumnIndexList = new ArrayList<>(); + List> objectDeletionIntervalList = new ArrayList<>(); + boolean objectColumnHasDeletion = false; + + TSDataType[] dataTypes = new TSDataType[valueMods.size()]; + for (AbstractAlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadataList) { + boolean hasNull = false; + for (int i = 0; i < alignedChunkMetadata.getValueChunkMetadataList().size(); i++) { + if (dataTypes[i] != null) { + continue; + } + IChunkMetadata chunkMetadata = alignedChunkMetadata.getValueChunkMetadataList().get(i); + if (chunkMetadata == null) { + hasNull = true; + continue; + } + dataTypes[i] = chunkMetadata.getDataType(); + if (dataTypes[i] == TSDataType.OBJECT) { + objectColumnIndexList.add(i); + List deletionInterval = ModificationUtils.sortAndMerge(valueMods.get(i)); + objectColumnHasDeletion |= (!deletionInterval.isEmpty() || !timeMods.isEmpty()); + objectDeletionIntervalList.add(deletionInterval); + } + } + if (!hasNull) { + break; + } + } + if (!objectColumnHasDeletion) { + return; + } + int[] deletionCursors = new int[objectColumnIndexList.size() + 1]; + List timeDeletionIntervalList = ModificationUtils.sortAndMerge(timeMods); + for (AbstractAlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadataList) { + CompactionUtils.removeDeletedObjectFiles( + reader, + alignedChunkMetadata, + objectColumnIndexList, + timeDeletionIntervalList, + objectDeletionIntervalList, + deletionCursors); + } + } + + private static void removeDeletedObjectFiles( + TsFileSequenceReader reader, + AbstractAlignedChunkMetadata alignedChunkMetadata, + List objectColumnIndexList, + List timeDeletions, + List> objectDeletions, + int[] deletionCursors) + throws IOException { + Chunk timeChunk = + reader.readMemChunk((ChunkMetadata) alignedChunkMetadata.getTimeChunkMetadata()); + CompactionChunkReader compactionChunkReader = new CompactionChunkReader(timeChunk); + List> timePages = + compactionChunkReader.readPageDataWithoutUncompressing(); + + List valueChunks = new ArrayList<>(); + List>> valuePages = new ArrayList<>(); + + for (int i = 0; i < objectColumnIndexList.size(); i++) { + int idxInAlignedChunkMetadata = objectColumnIndexList.get(i); + if (timeDeletions.isEmpty() && objectDeletions.get(i).isEmpty()) { + continue; + } + ChunkMetadata valueChunkMetadata = + (ChunkMetadata) + alignedChunkMetadata.getValueChunkMetadataList().get(idxInAlignedChunkMetadata); + if (valueChunkMetadata == null) { + continue; + } + Chunk chunk = reader.readMemChunk(valueChunkMetadata); + valueChunks.add(chunk); + valuePages.add( + chunk == null + ? null + : new CompactionChunkReader(chunk).readPageDataWithoutUncompressing()); + } + + CompactionAlignedChunkReader alignedChunkReader = + new CompactionAlignedChunkReader(timeChunk, valueChunks, true); + for (int i = 0; i < timePages.size(); i++) { + Pair timePage = timePages.get(i); + List valuePageHeaders = new ArrayList<>(valuePages.size()); + List compressedValuePages = new ArrayList<>(valuePages.size()); + for (int j = 0; j < valuePages.size(); j++) { + Pair valuePage = valuePages.get(j).get(i); + valuePageHeaders.add(valuePage.getLeft()); + compressedValuePages.add(valuePage.getRight()); + } + IPointReader pagePointReader = + alignedChunkReader.getPagePointReader( + timePage.getLeft(), valuePageHeaders, timePage.getRight(), compressedValuePages); + + while (pagePointReader.hasNextTimeValuePair()) { + TimeValuePair timeValuePair = pagePointReader.nextTimeValuePair(); + removeDeletedObjectFiles(timeValuePair, deletionCursors, timeDeletions, objectDeletions); + } + } + } + + private static void removeDeletedObjectFiles( + TimeValuePair timeValuePair, + int[] cursors, + List timeDeletions, + List> objectDeletions) { + long timestamp = timeValuePair.getTimestamp(); + boolean timeDeleted = isDeleted(timestamp, timeDeletions, cursors, 0); + for (int i = 0; i < timeValuePair.getValues().length; i++) { + Binary value = (Binary) timeValuePair.getValues()[i]; + if (value == null) { + continue; + } + if (timeDeleted || isDeleted(timestamp, objectDeletions.get(i), cursors, i + 1)) { + ObjectTypeUtils.deleteObjectPathFromBinary(value); + } + } + } + + private static boolean isDeleted( + long timestamp, List deleteIntervalList, int[] deleteCursors, int idx) { + while (deleteIntervalList != null && deleteCursors[idx] < deleteIntervalList.size()) { + if (deleteIntervalList.get(deleteCursors[idx]).getTimeRange().contains(timestamp)) { + return true; + } else if (deleteIntervalList.get(deleteCursors[idx]).getTimeRange().getMax() < timestamp) { + deleteCursors[idx]++; + } else { + return false; + } + } + return false; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/MultiTsFileDeviceIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/MultiTsFileDeviceIterator.java index 1889182a2db4a..75900689ca2b0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/MultiTsFileDeviceIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/MultiTsFileDeviceIterator.java @@ -443,7 +443,7 @@ public Map getAllSchemasOfCurrentDevice() throws IOEx */ private void applyModificationForAlignedChunkMetadataList( TsFileResource tsFileResource, List alignedChunkMetadataList) - throws IllegalPathException { + throws IllegalPathException, IOException { if (alignedChunkMetadataList.isEmpty()) { // all the value chunks is empty chunk return; @@ -482,6 +482,15 @@ private void applyModificationForAlignedChunkMetadataList( modificationList.isEmpty() ? Collections.emptyList() : modificationList); } + if (ttlDeletion != null) { + List emptyList = Collections.emptyList(); + CompactionUtils.removeDeletedObjectFiles( + readerMap.get(tsFileResource), + alignedChunkMetadataList, + Collections.singletonList(ttlDeletion), + modificationForValueColumns.stream().map(v -> emptyList).collect(Collectors.toList())); + } + ModificationUtils.modifyAlignedChunkMetaData( alignedChunkMetadataList, modificationForTimeColumn, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastAlignedSeriesCompactionExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastAlignedSeriesCompactionExecutor.java index 4e71c03d25f4b..166417a97eaf0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastAlignedSeriesCompactionExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastAlignedSeriesCompactionExecutor.java @@ -22,8 +22,11 @@ import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.AlignedPath; import org.apache.iotdb.commons.path.PatternTreeMap; +import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.db.exception.WriteProcessException; +import org.apache.iotdb.db.queryengine.plan.analyze.cache.schema.DataNodeTTLCache; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.subtask.FastCompactionTaskSummary; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionUtils; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.ModifiedStatus; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.batch.utils.AlignedSeriesBatchCompactionUtils; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.fast.element.AlignedPageElement; @@ -59,9 +62,11 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class FastAlignedSeriesCompactionExecutor extends SeriesCompactionExecutor { @@ -175,7 +180,8 @@ void deserializeFileIntoChunkMetadataQueue(List fileElements) new ChunkMetadataElement( alignedChunkMetadataList.get(i), i == alignedChunkMetadataList.size() - 1, - fileElement)); + fileElement, + measurementSchemas)); } } } @@ -264,6 +270,23 @@ protected List getAlignedChunkMetadataList(TsFileR } }); + long ttlForTable = + deviceId.isTableModel() + ? DataNodeTTLCache.getInstance() + .getTTLForTable(resource.getDatabaseName(), deviceId.getTableName()) + : Long.MAX_VALUE; + if (ttlForTable != Long.MAX_VALUE) { + ModEntry ttlDeletion = + CompactionUtils.convertTtlToDeletion( + deviceId, CommonDateTimeUtils.currentTime() - ttlForTable); + List emptyList = Collections.emptyList(); + CompactionUtils.removeDeletedObjectFiles( + readerCacheMap.get(resource), + alignedChunkMetadataList, + Collections.singletonList(ttlDeletion), + valueModifications.stream().map(v -> emptyList).collect(Collectors.toList())); + } + // modify aligned chunk metadatas ModificationUtils.modifyAlignedChunkMetaData( alignedChunkMetadataList, timeModifications, valueModifications, ignoreAllNullRows); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastNonAlignedSeriesCompactionExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastNonAlignedSeriesCompactionExecutor.java index b07407c7db6f0..0a77bf16ea7a7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastNonAlignedSeriesCompactionExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/FastNonAlignedSeriesCompactionExecutor.java @@ -52,6 +52,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -160,7 +161,10 @@ void deserializeFileIntoChunkMetadataQueue(List fileElements) // add into queue chunkMetadataQueue.add( new ChunkMetadataElement( - chunkMetadata, i == iChunkMetadataList.size() - 1, fileElement)); + chunkMetadata, + i == iChunkMetadataList.size() - 1, + fileElement, + Collections.emptyList())); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/element/ChunkMetadataElement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/element/ChunkMetadataElement.java index 6828841f1bda1..2d29e3944d858 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/element/ChunkMetadataElement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/fast/element/ChunkMetadataElement.java @@ -23,6 +23,7 @@ import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.read.common.Chunk; +import org.apache.tsfile.write.schema.IMeasurementSchema; import java.util.List; @@ -42,12 +43,19 @@ public class ChunkMetadataElement { public boolean needForceDecodingPage; + // for aligned series + public List measurementSchemasOfValueChunks; + public ChunkMetadataElement( - IChunkMetadata chunkMetadata, boolean isLastChunk, FileElement fileElement) { + IChunkMetadata chunkMetadata, + boolean isLastChunk, + FileElement fileElement, + List measurementSchemasOfValueChunks) { this.chunkMetadata = chunkMetadata; this.startTime = chunkMetadata.getStartTime(); this.isLastChunk = isLastChunk; this.fileElement = fileElement; + this.measurementSchemasOfValueChunks = measurementSchemasOfValueChunks; } public void clearChunks() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/ReadChunkAlignedSeriesCompactionExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/ReadChunkAlignedSeriesCompactionExecutor.java index 928c0cdbc6b20..c2e0959e65234 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/ReadChunkAlignedSeriesCompactionExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/ReadChunkAlignedSeriesCompactionExecutor.java @@ -565,6 +565,7 @@ private long estimateMemorySizeAsPageWriter(PageLoader pageLoader) { case TEXT: case STRING: case BLOB: + case OBJECT: size = pageLoader.getHeader().getUncompressedSize(); break; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/SingleSeriesCompactionExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/SingleSeriesCompactionExecutor.java index b09adb17bba8b..cff44c7618a05 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/SingleSeriesCompactionExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/executor/readchunk/SingleSeriesCompactionExecutor.java @@ -322,6 +322,7 @@ private void writeTimeAndValueToChunkWriter(TimeValuePair timeValuePair) { case TEXT: case BLOB: case STRING: + case OBJECT: chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getBinary()); break; case FLOAT: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCompactionWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCompactionWriter.java index 3458f9870d466..f3cd5185b58af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCompactionWriter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCompactionWriter.java @@ -149,6 +149,7 @@ protected void writeDataPoint(long timestamp, TsPrimitiveType value, IChunkWrite case TEXT: case STRING: case BLOB: + case OBJECT: chunkWriterImpl.write(timestamp, value.getBinary()); break; case DOUBLE: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index e69a5b8b8c398..25c4a876ab7d9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -220,6 +220,7 @@ public void initChunkMetaFromTvLists(Filter globalTimeFilter) { case TEXT: case BLOB: case STRING: + case OBJECT: for (int i = 0; i < tsBlock.getPositionCount(); i++) { if (tsBlock.getColumn(column).isNull(i)) { continue; @@ -412,6 +413,7 @@ private void writeValidValuesIntoTsBlock(TsBlockBuilder builder) throws IOExcept case TEXT: case BLOB: case STRING: + case OBJECT: valueBuilder.writeBinary(values[columnIndex].getBinary()); break; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index f81238bef71ec..5f6af57746fe7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -614,6 +614,7 @@ private void handleEncoding( case TEXT: case STRING: case BLOB: + case OBJECT: alignedChunkWriter.writeByColumn( time, isNull ? null : list.getBinaryByValueIndex(originRowIndex, columnIndex), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index ebf23154d1502..c4a871a29cc13 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -166,6 +166,7 @@ public void writeNonAlignedTablet( case TEXT: case BLOB: case STRING: + case OBJECT: Binary[] binaryValues = (Binary[]) valueList; putBinaries(times, binaryValues, bitMap, start, end); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java index 7ee43a93aea8d..dbe948c01b860 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java @@ -293,6 +293,7 @@ private void updatePageStatisticsFromTsBlock( break; case TEXT: case BLOB: + case OBJECT: case STRING: for (int i = 0; i < tsBlock.getPositionCount(); i++) { valueStatistics[column].update( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java index a868024007222..e7427ef87d8dc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java @@ -105,6 +105,7 @@ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { case TEXT: case STRING: case BLOB: + case OBJECT: batchData.putBinary( tsBlock.getTimeColumn().getLong(i), tsBlock.getColumn(0).getBinary(i)); break; @@ -269,6 +270,7 @@ private void updatePageStatisticsFromTsBlock(Statistics statistics) { case TEXT: case BLOB: case STRING: + case OBJECT: for (int i = 0; i < tsBlock.getPositionCount(); i++) { statistics.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getBinary(i)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/generator/TsFileNameGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/generator/TsFileNameGenerator.java index 16be82188e9ca..b3f1327483a9e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/generator/TsFileNameGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/generator/TsFileNameGenerator.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.storageengine.rescon.disk.TierManager; import org.apache.tsfile.common.constant.TsFileConstant; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.fileSystem.FSFactoryProducer; import org.apache.tsfile.fileSystem.fsFactory.FSFactory; import org.slf4j.Logger; @@ -175,6 +176,20 @@ public static TsFileName getTsFileName(String fileName) throws IOException { } } + public static String generateObjectFilePath(int regionId, long time, IDeviceID iDeviceID) { + String objectFileName = time + ".bin"; + Object[] segments = iDeviceID.getSegments(); + StringBuilder relativePathString = + new StringBuilder(String.valueOf(regionId)).append(File.separator); + for (Object segment : segments) { + relativePathString + .append(segment == null ? "null" : segment.toString().toLowerCase()) + .append(File.separator); + } + relativePathString.append(objectFileName); + return relativePathString.toString(); + } + @TestOnly public static TsFileResource increaseCrossCompactionCnt(TsFileResource tsFileResource) throws IOException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java index 8cc080016ca4c..a7d79f92b5753 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.service.metrics.WritingMetrics; import org.apache.iotdb.db.storageengine.dataregion.wal.checkpoint.Checkpoint; @@ -332,6 +333,8 @@ private void handleInfoEntry(WALEntry walEntry) { searchIndex = ((DeleteDataNode) walEntry.getValue()).getSearchIndex(); } else if (walEntry.getType() == WALEntryType.RELATIONAL_DELETE_DATA_NODE) { searchIndex = ((RelationalDeleteDataNode) walEntry.getValue()).getSearchIndex(); + } else if (walEntry.getType() == WALEntryType.OBJECT_FILE_NODE) { + searchIndex = ((ObjectNode) walEntry.getValue()).getSearchIndex(); } else { searchIndex = ((InsertNode) walEntry.getValue()).getSearchIndex(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntry.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntry.java index f7b85089febec..4c0ff8809d260 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntry.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntry.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.storageengine.dataregion.memtable.AbstractMemTable; import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; @@ -77,6 +78,8 @@ protected WALEntry(long memTableId, WALEntryValue value, boolean wait) { this.type = WALEntryType.CONTINUOUS_SAME_SEARCH_INDEX_SEPARATOR_NODE; } else if (value instanceof RelationalDeleteDataNode) { this.type = WALEntryType.RELATIONAL_DELETE_DATA_NODE; + } else if (value instanceof ObjectNode) { + this.type = WALEntryType.OBJECT_FILE_NODE; } else { throw new RuntimeException("Unknown WALEntry type"); } @@ -134,6 +137,9 @@ public static WALEntry deserialize(DataInputStream stream) throws IOException { case CONTINUOUS_SAME_SEARCH_INDEX_SEPARATOR_NODE: value = (ContinuousSameSearchIndexSeparatorNode) PlanNodeType.deserializeFromWAL(stream); break; + case OBJECT_FILE_NODE: + value = (ObjectNode) PlanNodeType.deserializeFromWAL(stream); + break; default: throw new RuntimeException("Unknown WALEntry type " + type); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntryType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntryType.java index 1c3ddcbf702ad..829affd1b4e18 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntryType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntryType.java @@ -45,6 +45,7 @@ public enum WALEntryType { /** {@link org.apache.iotdb.db.storageengine.dataregion.memtable.AbstractMemTable} */ MEMORY_TABLE_SNAPSHOT((byte) 10), RELATIONAL_DELETE_DATA_NODE((byte) 11), + OBJECT_FILE_NODE((byte) 12), // endregion // region signal entry type // signal wal buffer has been closed @@ -71,7 +72,8 @@ public boolean needSearch() { || this == INSERT_ROW_NODE || this == INSERT_ROWS_NODE || this == DELETE_DATA_NODE - || this == RELATIONAL_DELETE_DATA_NODE; + || this == RELATIONAL_DELETE_DATA_NODE + || this == OBJECT_FILE_NODE; } public static WALEntryType valueOf(byte code) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALInfoEntry.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALInfoEntry.java index 6da50edba4f8f..791ef63a7525d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALInfoEntry.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALInfoEntry.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALMode; @@ -89,6 +90,7 @@ public void serialize(IWALByteBufferView buffer) { case RELATIONAL_DELETE_DATA_NODE: case MEMORY_TABLE_SNAPSHOT: case CONTINUOUS_SAME_SEARCH_INDEX_SEPARATOR_NODE: + case OBJECT_FILE_NODE: value.serializeToWAL(buffer); break; case MEMORY_TABLE_CHECKPOINT: @@ -166,6 +168,8 @@ public long getMemorySize() { case CONTINUOUS_SAME_SEARCH_INDEX_SEPARATOR_NODE: case MEMORY_TABLE_CHECKPOINT: return RamUsageEstimator.sizeOfObject(value); + case OBJECT_FILE_NODE: + return ((ObjectNode) value).serializedSize(); default: throw new RuntimeException("Unsupported wal entry type " + type); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/IWALNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/IWALNode.java index 0d0e0a527f0f7..a8fbbee0dc4f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/IWALNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/IWALNode.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.storageengine.dataregion.flush.FlushListener; import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; @@ -54,6 +55,8 @@ public interface IWALNode extends FlushListener, AutoCloseable, ConsensusReqRead /** Log BatchDoneNode */ WALFlushListener log(long memTableId, ContinuousSameSearchIndexSeparatorNode separatorNode); + WALFlushListener log(long memTableId, ObjectNode objectNode); + /** Callback when memTable created. */ void onMemTableCreated(IMemTable memTable, String targetTsFile); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALFakeNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALFakeNode.java index 48bf4c1cfb990..e35d5e79fc019 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALFakeNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALFakeNode.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.WALException; @@ -82,6 +83,11 @@ public WALFlushListener log( return getResult(); } + @Override + public WALFlushListener log(long memTableId, ObjectNode objectNode) { + return getResult(); + } + private WALFlushListener getResult() { switch (status) { case SUCCESS: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java index 8803aca56c447..07dd4d78f6605 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java @@ -33,6 +33,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ObjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalDeleteDataNode; import org.apache.iotdb.db.service.metrics.WritingMetrics; import org.apache.iotdb.db.storageengine.StorageEngine; @@ -62,6 +63,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -190,6 +193,12 @@ public WALFlushListener log( return log(walEntry); } + @Override + public WALFlushListener log(long memTableId, ObjectNode objectNode) { + WALEntry walEntry = new WALInfoEntry(memTableId, objectNode); + return log(walEntry); + } + private WALFlushListener log(WALEntry walEntry) { buffer.write(walEntry); @@ -646,6 +655,8 @@ public boolean hasNext() { AtomicBoolean notFirstFile = new AtomicBoolean(false); AtomicBoolean hasCollectedSufficientData = new AtomicBoolean(false); + long memorySize = 0; + // try to collect current tmpNodes to insertNodes, return true if successfully collect an // insert node Runnable tryToCollectInsertNodeAndBumpIndex = @@ -684,7 +695,23 @@ public boolean hasNext() { } else if (currentWalEntryIndex < nextSearchIndex) { // WAL entry is outdated, do nothing, continue to see next WAL entry } else if (currentWalEntryIndex == nextSearchIndex) { - tmpNodes.get().add(new IoTConsensusRequest(buffer)); + if (type == WALEntryType.OBJECT_FILE_NODE) { + WALEntry walEntry = + WALEntry.deserialize( + new DataInputStream(new ByteArrayInputStream(buffer.array()))); + // only be called by leader read from wal + // wal only has relativePath, offset, eof, length + // need to add WALEntryType + memtableId + relativePath, offset, eof, length + + // content + // need to add IoTConsensusRequest instead of ObjectNode + tmpNodes + .get() + .add(new IoTConsensusRequest(((ObjectNode) walEntry.getValue()).serialize())); + memorySize += ((ObjectNode) walEntry.getValue()).getMemorySize(); + } else { + tmpNodes.get().add(new IoTConsensusRequest(buffer)); + memorySize += buffer.remaining(); + } } else { // currentWalEntryIndex > targetIndex // WAL entry of targetIndex has been fully collected, put them into insertNodes @@ -696,12 +723,31 @@ public boolean hasNext() { currentWalEntryIndex); nextSearchIndex = currentWalEntryIndex; } - tmpNodes.get().add(new IoTConsensusRequest(buffer)); + if (type == WALEntryType.OBJECT_FILE_NODE) { + WALEntry walEntry = + WALEntry.deserialize( + new DataInputStream(new ByteArrayInputStream(buffer.array()))); + // only be called by leader read from wal + // wal only has relativePath, offset, eof, length + // need to add WALEntryType + memtableId + relativePath, offset, eof, length + + // content + // need to add IoTConsensusRequest instead of ObjectNode + tmpNodes + .get() + .add(new IoTConsensusRequest(((ObjectNode) walEntry.getValue()).serialize())); + memorySize += ((ObjectNode) walEntry.getValue()).getMemorySize(); + } else { + tmpNodes.get().add(new IoTConsensusRequest(buffer)); + memorySize += buffer.remaining(); + } } } else { tryToCollectInsertNodeAndBumpIndex.run(); } - if (hasCollectedSufficientData.get()) { + if (memorySize > config.getWalBufferSize()) { + tryToCollectInsertNodeAndBumpIndex.run(); + } + if (memorySize > config.getWalBufferSize() || hasCollectedSufficientData.get()) { break COLLECT_FILE_LOOP; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/UnsealedTsFileRecoverPerformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/UnsealedTsFileRecoverPerformer.java index f90cb0353bb8f..75224d8867f37 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/UnsealedTsFileRecoverPerformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/UnsealedTsFileRecoverPerformer.java @@ -223,6 +223,8 @@ public void redoLog(WALEntry walEntry) { case CONTINUOUS_SAME_SEARCH_INDEX_SEPARATOR_NODE: // The CONTINUOUS_SAME_SEARCH_INDEX_SEPARATOR_NODE doesn't need redo break; + case OBJECT_FILE_NODE: + break; default: throw new RuntimeException("Unsupported type " + walEntry.getType()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/BatchedAlignedValueChunkData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/BatchedAlignedValueChunkData.java index be31a03d76ae3..6e8d3850b8e24 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/BatchedAlignedValueChunkData.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/BatchedAlignedValueChunkData.java @@ -109,6 +109,7 @@ public void writeDecodeValuePage(long[] times, TsPrimitiveType[] values, TSDataT break; case TEXT: case BLOB: + case OBJECT: case STRING: dataSize += ReadWriteIOUtils.write(values[i].getBinary(), stream); break; @@ -199,6 +200,7 @@ private void buildValueChunkWriter( break; case TEXT: case BLOB: + case OBJECT: case STRING: final Binary binaryValue = isNull ? DEFAULT_BINARY : ReadWriteIOUtils.readBinary(stream); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/disk/TierManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/disk/TierManager.java index d929f7656077c..61bb6ffafba58 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/disk/TierManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/disk/TierManager.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.storageengine.rescon.disk; import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.exception.ObjectFileNotExist; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; @@ -45,6 +46,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -65,12 +67,16 @@ public class TierManager { */ private final List unSeqTiers = new ArrayList<>(); + private final List objectTiers = new ArrayList<>(); + /** seq file folder's rawFsPath path -> tier level */ private final Map seqDir2TierLevel = new HashMap<>(); /** unSeq file folder's rawFsPath path -> tier level */ private final Map unSeqDir2TierLevel = new HashMap<>(); + private List objectDirs; + /** total space of each tier, Long.MAX_VALUE when one tier contains remote storage */ private long[] tierDiskTotalSpace; @@ -151,6 +157,22 @@ public synchronized void initFolders() { for (String dir : unSeqDirs) { unSeqDir2TierLevel.put(dir, tierLevel); } + + objectDirs = + Arrays.stream(tierDirs[tierLevel]) + .filter(Objects::nonNull) + .map( + v -> + FSFactoryProducer.getFSFactory() + .getFile(v, IoTDBConstant.OBJECT_FOLDER_NAME) + .getPath()) + .collect(Collectors.toList()); + + try { + objectTiers.add(new FolderManager(objectDirs, directoryStrategyType)); + } catch (DiskSpaceInsufficientException e) { + logger.error("All disks of tier {} are full.", tierLevel, e); + } } tierDiskTotalSpace = getTierDiskSpace(DiskSpaceType.TOTAL); @@ -160,6 +182,7 @@ public synchronized void resetFolders() { long startTime = System.currentTimeMillis(); seqTiers.clear(); unSeqTiers.clear(); + objectTiers.clear(); seqDir2TierLevel.clear(); unSeqDir2TierLevel.clear(); @@ -190,6 +213,10 @@ public String getNextFolderForTsFile(int tierLevel, boolean sequence) : unSeqTiers.get(tierLevel).getNextFolder(); } + public String getNextFolderForObjectFile() throws DiskSpaceInsufficientException { + return objectTiers.get(0).getNextFolder(); + } + public FolderManager getFolderManager(int tierLevel, boolean sequence) { return sequence ? seqTiers.get(tierLevel) : unSeqTiers.get(tierLevel); } @@ -222,6 +249,20 @@ public List getAllLocalUnSequenceFileFolders() { .collect(Collectors.toList()); } + public List getAllObjectFileFolders() { + return objectDirs; + } + + public Optional getAbsoluteObjectFilePath(String filePath) { + for (String objectDir : objectDirs) { + File objectFile = FSFactoryProducer.getFSFactory().getFile(objectDir, filePath); + if (objectFile.exists()) { + return Optional.of(objectFile); + } + } + return Optional.empty(); + } + public int getTiersNum() { return seqTiers.size(); } @@ -294,6 +335,16 @@ private long[] getTierDiskSpace(DiskSpaceType type) { return tierDiskSpace; } + public File getObjectFile(String relativePath) { + for (String folder : objectDirs) { + File file = new File(folder, relativePath); + if (file.exists()) { + return file; + } + } + throw new ObjectFileNotExist(relativePath); + } + private enum DiskSpaceType { TOTAL, USABLE, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/memory/PrimitiveArrayManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/memory/PrimitiveArrayManager.java index e8a2a9bb83741..dbf3b989800eb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/memory/PrimitiveArrayManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/memory/PrimitiveArrayManager.java @@ -249,6 +249,7 @@ private static Object createPrimitiveArray(TSDataType dataType) { case TEXT: case STRING: case BLOB: + case OBJECT: dataArray = new Binary[ARRAY_SIZE]; break; default: @@ -342,6 +343,7 @@ public static Object createDataListsByType(TSDataType dataType, int size) { case TEXT: case STRING: case BLOB: + case OBJECT: Binary[][] binaries = new Binary[arrayNumber][]; for (int i = 0; i < arrayNumber; i++) { binaries[i] = new Binary[ARRAY_SIZE]; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java index f50e472c414f3..38686bee04774 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java @@ -430,6 +430,7 @@ protected void rewritePageIntoFiles( case TEXT: case BLOB: case STRING: + case OBJECT: chunkWriter.write(time, (Binary) value); break; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java index b9dbcfe86853f..1fa1ec44fc189 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java @@ -380,6 +380,7 @@ public static Object createValueColumnOfDataType( case TEXT: case STRING: case BLOB: + case OBJECT: valueColumn = new Binary[rowNum]; break; case DATE: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/EncodingInferenceUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/EncodingInferenceUtils.java index 402921338a494..d9d4cecc4e423 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/EncodingInferenceUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/EncodingInferenceUtils.java @@ -51,6 +51,7 @@ public static TSEncoding getDefaultEncoding(TSDataType dataType) { case TEXT: case BLOB: case STRING: + case OBJECT: return conf.getDefaultTextEncoding(); default: throw new UnSupportedDataTypeException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/MemUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/MemUtils.java index 843bcdb8937c4..1de1422b07bef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/MemUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/MemUtils.java @@ -108,7 +108,9 @@ public static long getBinaryColumnSize(Binary[] column, int start, int end, TSSt if (results == null || results[i] == null || results[i].code == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - memSize += RamUsageEstimator.sizeOf(column[i].getValues()); + if (column[i] != null) { + memSize += RamUsageEstimator.sizeOf(column[i].getValues()); + } } } return memSize; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ObjectTypeUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ObjectTypeUtils.java new file mode 100644 index 0000000000000..fc1bb40b93fb7 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ObjectTypeUtils.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.utils; + +import org.apache.iotdb.commons.exception.ObjectFileNotExist; +import org.apache.iotdb.db.storageengine.rescon.disk.TierManager; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.utils.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Optional; + +public class ObjectTypeUtils { + + private static final Logger logger = LoggerFactory.getLogger(ObjectTypeUtils.class); + private static final TierManager TIER_MANAGER = TierManager.getInstance(); + + private ObjectTypeUtils() {} + + public static File getObjectPathFromBinary(Binary binary) { + byte[] bytes = binary.getValues(); + String relativeObjectFilePath = + new String(bytes, 8, bytes.length - 8, TSFileConfig.STRING_CHARSET); + Optional file = TIER_MANAGER.getAbsoluteObjectFilePath(relativeObjectFilePath); + if (!file.isPresent()) { + throw new ObjectFileNotExist(relativeObjectFilePath); + } + return file.get(); + } + + public static Optional getNullableObjectPathFromBinary(Binary binary) { + byte[] bytes = binary.getValues(); + String relativeObjectFilePath = + new String(bytes, 8, bytes.length - 8, TSFileConfig.STRING_CHARSET); + return TIER_MANAGER.getAbsoluteObjectFilePath(relativeObjectFilePath); + } + + public static void deleteObjectPathFromBinary(Binary binary) { + Optional file = ObjectTypeUtils.getNullableObjectPathFromBinary(binary); + if (!file.isPresent()) { + return; + } + File parentFile = file.get().getParentFile(); + File tmpFile = new File(parentFile, file.get().getName().replace(".bin", ".tmp")); + File bakFile = new File(parentFile, file.get().getName().replace(".bin", ".bak")); + logger.info("Remove object file {}", file.get().getAbsolutePath()); + for (int i = 0; i < 2; i++) { + try { + if (Files.deleteIfExists(file.get().toPath())) { + return; + } + if (Files.deleteIfExists(tmpFile.toPath())) { + return; + } + if (Files.deleteIfExists(bakFile.toPath())) { + return; + } + } catch (IOException e) { + logger.error("Failed to remove object file {}", file.get().getAbsolutePath(), e); + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ObjectWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ObjectWriter.java new file mode 100644 index 0000000000000..e007027bd6d81 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ObjectWriter.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.utils; + +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; + +import org.apache.tsfile.external.commons.io.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.file.Files; + +public class ObjectWriter implements AutoCloseable { + + private static final Logger LOGGER = LoggerFactory.getLogger(ObjectWriter.class); + + private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + + private final FileOutputStream fos; + + private final File file; + + public ObjectWriter(File filePath) throws FileNotFoundException { + try { + FileUtils.forceMkdir(filePath.getParentFile()); + } catch (final IOException e) { + throw new FileNotFoundException("Error occurred during creating directory " + filePath); + } + if (!Files.exists(filePath.toPath())) { + try { + Files.createFile(filePath.toPath()); + } catch (IOException e) { + throw new FileNotFoundException(e.getMessage()); + } + } + file = filePath; + fos = new FileOutputStream(filePath, true); + } + + public void write(boolean isGeneratedByConsensus, long offset, byte[] content) + throws IOException { + if (file.length() != offset) { + if (isGeneratedByConsensus || offset == 0) { + fos.getChannel().truncate(offset); + } else { + throw new IOException( + "The file length " + file.length() + " is not equal to the offset " + offset); + } + } + if (file.length() + content.length > config.getMaxObjectSizeInByte()) { + throw new IOException("The file length is larger than max_object_file_size_in_bytes"); + } + fos.write(content); + } + + @Override + public void close() throws Exception { + fos.close(); + } +} 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 cd4ca79ac4387..fed80815f692a 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 @@ -236,6 +236,7 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) case TEXT: case BLOB: case STRING: + case OBJECT: for (int i = 0; i < currentCount; i++) { rowCount++; if (column.isNull(i)) { @@ -379,6 +380,7 @@ private static void serializeTsBlock( case TEXT: case BLOB: case STRING: + case OBJECT: doWithTextColumn( rowCount, column, @@ -753,6 +755,7 @@ public static Object[] readTabletValuesFromBuffer( case TEXT: case BLOB: case STRING: + case OBJECT: Binary[] binaryValues = new Binary[size]; for (int index = 0; index < size; index++) { int binarySize = buffer.getInt(); @@ -795,6 +798,7 @@ public static Object[] readTabletValuesFromStream( case TEXT: case BLOB: case STRING: + case OBJECT: parseTextColumn(size, stream, values, i); break; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java index e11b586e7e427..a66e7d7b856bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java @@ -62,6 +62,7 @@ public static void setTimeValuePair(TimeValuePair from, TimeValuePair to) { case TEXT: case BLOB: case STRING: + case OBJECT: to.getValue().setBinary(from.getValue().getBinary()); break; case BOOLEAN: @@ -89,6 +90,7 @@ public static TimeValuePair getEmptyTimeValuePair(TSDataType dataType) { case TEXT: case BLOB: case STRING: + case OBJECT: return new TimeValuePair(0, new TsBinary(new Binary("", TSFileConfig.STRING_CHARSET))); default: throw new UnsupportedOperationException("Unrecognized datatype: " + dataType); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java index cca6cd86a6ed2..aa2855f7b0592 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java @@ -325,6 +325,7 @@ public static boolean canAutoCast(TSDataType fromType, TSDataType toType) { case DATE: case TIMESTAMP: case BLOB: + case OBJECT: case STRING: return false; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 7cc8490434830..2a91c6f09ceca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -204,6 +204,7 @@ public synchronized void putAlignedValue(long timestamp, Object[] value) { case TEXT: case BLOB: case STRING: + case OBJECT: ((Binary[]) columnValues.get(arrayIndex))[elementIndex] = columnValue != null ? (Binary) columnValue : Binary.EMPTY_VALUE; memoryBinaryChunkSize[i] += @@ -303,6 +304,7 @@ private TsPrimitiveType getAlignedValueByValueIndex( case TEXT: case BLOB: case STRING: + case OBJECT: Binary valueT = ((Binary[]) columnValues.get(arrayIndex))[elementIndex]; vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.TEXT, valueT); break; @@ -364,6 +366,7 @@ public void extendColumn(TSDataType dataType) { case TEXT: case STRING: case BLOB: + case OBJECT: columnValue.add(getPrimitiveArraysByType(TSDataType.TEXT)); break; case FLOAT: @@ -643,6 +646,7 @@ protected Object cloneValue(TSDataType type, Object value) { case TEXT: case BLOB: case STRING: + case OBJECT: Binary[] valueT = (Binary[]) value; Binary[] cloneT = new Binary[valueT.length]; System.arraycopy(valueT, 0, cloneT, 0, valueT.length); @@ -873,6 +877,7 @@ private void arrayCopy(Object[] value, int idx, int arrayIndex, int elementIndex case TEXT: case BLOB: case STRING: + case OBJECT: Binary[] arrayT = ((Binary[]) columnValues.get(arrayIndex)); System.arraycopy(value[i], idx, arrayT, elementIndex, remaining); @@ -1174,6 +1179,7 @@ public TsBlock buildTsBlock( case TEXT: case BLOB: case STRING: + case OBJECT: valueBuilder.writeBinary(getBinaryByValueIndex(originRowIndex, columnIndex)); break; default: @@ -1251,6 +1257,7 @@ public int serializedSize() { case TEXT: case BLOB: case STRING: + case OBJECT: for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { size += ReadWriteIOUtils.sizeToWrite(getBinaryByValueIndex(rowIdx, columnIndex)); } @@ -1311,6 +1318,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { case TEXT: case BLOB: case STRING: + case OBJECT: Binary valueT = ((Binary[]) columnValues.get(arrayIndex))[elementIndex]; // In some scenario, the Binary in AlignedTVList will be null if this field is empty in // current row. We need to handle this scenario to get rid of NPE. See the similar issue @@ -1390,6 +1398,7 @@ public static AlignedTVList deserialize(DataInputStream stream) throws IOExcepti case TEXT: case BLOB: case STRING: + case OBJECT: Binary[] binaryValues = new Binary[rowCount]; for (int rowIndex = 0; rowIndex < rowCount; ++rowIndex) { binaryValues[rowIndex] = ReadWriteIOUtils.readBinary(stream); @@ -1851,6 +1860,7 @@ public TsPrimitiveType getPrimitiveTypeObject(int rowIndex, int columnIndex) { case TEXT: case BLOB: case STRING: + case OBJECT: return TsPrimitiveType.getByType( TSDataType.TEXT, getBinaryByValueIndex(valueIndex, validColumnIndex)); default: @@ -2086,6 +2096,7 @@ private void writeToColumn( case TEXT: case BLOB: case STRING: + case OBJECT: valueBuilder.writeBinary(getBinaryByValueIndex(originRowIndex, validColumnIndex)); break; default: @@ -2259,6 +2270,7 @@ public void encodeBatch(IChunkWriter chunkWriter, BatchEncodeInfo encodeInfo, lo case TEXT: case BLOB: case STRING: + case OBJECT: valueChunkWriter.write( time, isNull ? null : getBinaryByValueIndex(originRowIndex, validColumnIndex), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiAlignedTVListIterator.java index b115713b6eb97..a10018159818b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiAlignedTVListIterator.java @@ -272,6 +272,7 @@ public void encodeBatch(IChunkWriter chunkWriter, BatchEncodeInfo encodeInfo, lo break; case TEXT: case BLOB: + case OBJECT: case STRING: valueChunkWriter.write( currentTime, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiTVListIterator.java index 9f6b6c1b4f636..8e6dd012265e8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiTVListIterator.java @@ -165,6 +165,7 @@ public void encodeBatch(IChunkWriter chunkWriter, BatchEncodeInfo encodeInfo, lo break; case TEXT: case BLOB: + case OBJECT: case STRING: Binary value = currIterator.getTVList().getBinary(row); chunkWriterImpl.write(time, value); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiAlignedTVListIterator.java index 42de51ac4b72a..3a522f5e71462 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiAlignedTVListIterator.java @@ -254,6 +254,7 @@ public TsBlock nextBatch() { case TEXT: case BLOB: case STRING: + case OBJECT: valueBuilder.writeBinary( alignedTVList.getBinaryByValueIndex(valueIndex, validColumnIndex)); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiTVListIterator.java index b735cc7927bef..4b3943518399e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiTVListIterator.java @@ -219,6 +219,7 @@ public TsBlock nextBatch() { case TEXT: case BLOB: case STRING: + case OBJECT: while (hasNextTimeValuePair() && builder.getPositionCount() < maxNumberOfPointsInPage) { TVList.TVListIterator iterator = tvListIterators.get(iteratorIndex); Binary binary = iterator.getTVList().getBinary(iterator.getScanOrderIndex(rowIndex)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 39fd680f11a81..e20575be36278 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -114,6 +114,7 @@ public static TVList newList(TSDataType dataType) { case TEXT: case BLOB: case STRING: + case OBJECT: return BinaryTVList.newList(); case FLOAT: return FloatTVList.newList(); @@ -676,6 +677,7 @@ public static TVList deserialize(DataInputStream stream) throws IOException { switch (dataType) { case TEXT: case BLOB: + case OBJECT: case STRING: return BinaryTVList.deserialize(stream); case FLOAT: @@ -702,6 +704,7 @@ public static TVList deserializeWithoutBitMap(DataInputStream stream) throws IOE case TEXT: case BLOB: case STRING: + case OBJECT: return BinaryTVList.deserializeWithoutBitMap(stream); case FLOAT: return FloatTVList.deserializeWithoutBitMap(stream); @@ -1126,6 +1129,7 @@ && isTimeSatisfied(time)) { case TEXT: case BLOB: case STRING: + case OBJECT: while (index < rows && builder.getPositionCount() < maxNumberOfPointsInPage && paginationController.hasCurLimit()) { @@ -1226,6 +1230,7 @@ public void encodeBatch(IChunkWriter chunkWriter, BatchEncodeInfo encodeInfo, lo case TEXT: case BLOB: case STRING: + case OBJECT: Binary value = getBinary(index); chunkWriterImpl.write(time, value); encodeInfo.dataSizeInChunk += 8L + getBinarySize(value); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/windowing/window/WindowImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/windowing/window/WindowImpl.java index 0bc72b3205aa8..a87fb480d9d2a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/windowing/window/WindowImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/windowing/window/WindowImpl.java @@ -87,6 +87,7 @@ private void init(EvictableBatchList list, int begin) { case TEXT: case BLOB: case STRING: + case OBJECT: binaryValues = new Binary[size]; for (int i = 0; i < size; ++i) { binaryValues[i] = list.getBinaryByIndex(begin + i); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionDeleteObjectFileTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionDeleteObjectFileTest.java new file mode 100644 index 0000000000000..a4b35c01da6d8 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionDeleteObjectFileTest.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.compaction; + +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.schema.table.TsTable; +import org.apache.iotdb.commons.schema.table.column.FieldColumnSchema; +import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; +import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.SettleCompactionTask; +import org.apache.iotdb.db.storageengine.dataregion.modification.DeletionPredicate; +import org.apache.iotdb.db.storageengine.dataregion.modification.IDPredicate; +import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; +import org.apache.iotdb.db.storageengine.dataregion.modification.TableDeletionEntry; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.metadata.StringArrayDeviceID; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.read.common.TimeRange; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class CompactionDeleteObjectFileTest extends AbstractCompactionTest { + @Before + public void setUp() + throws IOException, WriteProcessException, MetadataException, InterruptedException { + super.setUp(); + } + + @After + public void tearDown() throws IOException, StorageEngineException { + super.tearDown(); + } + + @Test + public void test1() throws IOException { + createTable("tsfile_table", 100); + File dir = new File("/Users/shuww/Downloads/0708/1_副本"); + List resources = new ArrayList<>(); + for (File file : dir.listFiles()) { + if (!file.getName().endsWith(".tsfile")) { + continue; + } + TsFileResource resource = new TsFileResource(file); + + try (ModificationFile modificationFile = resource.getExclusiveModFile()) { + modificationFile.write( + new TableDeletionEntry( + new DeletionPredicate( + "tsfile_table", + new IDPredicate.FullExactMatch( + new StringArrayDeviceID(new String[] {"tsfile_table", "1", "5", "3"})), + Arrays.asList("file")), + new TimeRange(-1, 0))); + modificationFile.write( + new TableDeletionEntry( + new DeletionPredicate( + "tsfile_table", + new IDPredicate.FullExactMatch( + new StringArrayDeviceID(new String[] {"tsfile_table", "1", "5", "3"})), + Arrays.asList("file")), + new TimeRange(2, 2))); + } + resource.deserialize(); + resources.add(resource); + } + + // InnerSpaceCompactionTask task = + // new InnerSpaceCompactionTask( + // 0, tsFileManager, resources, true, new ReadChunkCompactionPerformer(), 0); + SettleCompactionTask task = + new SettleCompactionTask( + 0, + tsFileManager, + resources, + Collections.emptyList(), + true, + new FastCompactionPerformer(false), + 0); + task.start(); + } + + public void createTable(String tableName, long ttl) { + TsTable tsTable = new TsTable(tableName); + tsTable.addColumnSchema(new TagColumnSchema("id_column", TSDataType.STRING)); + tsTable.addColumnSchema( + new FieldColumnSchema("s1", TSDataType.STRING, TSEncoding.PLAIN, CompressionType.LZ4)); + tsTable.addProp(TsTable.TTL_PROPERTY, ttl + ""); + DataNodeTableCache.getInstance().preUpdateTable("Downloads", tsTable, null); + DataNodeTableCache.getInstance().commitUpdateTable("Downloads", tableName, null); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/PrimitiveArrayManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/PrimitiveArrayManagerTest.java index 624ff6eb01243..a2877a8c3836e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/PrimitiveArrayManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/PrimitiveArrayManagerTest.java @@ -98,6 +98,7 @@ public void testUpdateLimits() { break; case TEXT: case BLOB: + case OBJECT: case STRING: Assert.assertTrue(o instanceof Binary[]); Assert.assertEquals(ARRAY_SIZE, ((Binary[]) o).length); diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 6c0d71addf322..e4d6b56895ac0 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -1296,6 +1296,11 @@ enable_tsfile_validation=false # Unit: ms tier_ttl_in_ms=-1 +# The maximum size limit for a single object file. +# effectiveMode: hot_reload +# Datatype: long +max_object_file_size_in_byte=4294967296 + #################### ### Compaction Configurations #################### diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java index 4a54d774c9447..8b4352cb95abd 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java @@ -248,6 +248,7 @@ private IoTDBConstant() {} public static final String DATA_FOLDER_NAME = "data"; public static final String SEQUENCE_FOLDER_NAME = "sequence"; public static final String UNSEQUENCE_FOLDER_NAME = "unsequence"; + public static final String OBJECT_FOLDER_NAME = "object"; public static final String FILE_NAME_SEPARATOR = "-"; public static final String CONSENSUS_FOLDER_NAME = "consensus"; public static final String DATA_REGION_FOLDER_NAME = "data_region"; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/ObjectFileNotExist.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/ObjectFileNotExist.java new file mode 100644 index 0000000000000..05add08b218f7 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/ObjectFileNotExist.java @@ -0,0 +1,31 @@ +/* + * 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.commons.exception; + +import static org.apache.iotdb.rpc.TSStatusCode.OBJECT_NOT_EXISTS; + +public class ObjectFileNotExist extends IoTDBRuntimeException { + + private static final String ERROR_MSG = "Object file %s does not exist"; + + public ObjectFileNotExist(String relativeObjectPath) { + super(String.format(ERROR_MSG, relativeObjectPath), OBJECT_NOT_EXISTS.getStatusCode()); + } +} 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 be0d16807e216..c3fe93420660c 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 @@ -64,6 +64,7 @@ public void transform(Row row, PointCollector collector) collector.putDouble(time, Math.abs(row.getDouble(0))); break; case BLOB: + case OBJECT: case STRING: case TIMESTAMP: case TEXT: @@ -103,6 +104,7 @@ public Object transform(Row row) throws IOException { case TIMESTAMP: case STRING: case BLOB: + case OBJECT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -131,6 +133,7 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception transformDouble(columns, builder); return; case BLOB: + case OBJECT: case STRING: case TEXT: case TIMESTAMP: 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 bb12f7ff4a58b..045e49b064b73 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 @@ -73,6 +73,7 @@ protected void constructPQ() throws UDFInputSeriesDataTypeNotValidException { }); break; case BLOB: + case OBJECT: case BOOLEAN: default: // This will not happen. 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 00b3fd4755016..cb5cbcc0486b8 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 @@ -61,6 +61,7 @@ protected void doTransform(Row row, PointCollector collector) case TEXT: case STRING: case BLOB: + case OBJECT: 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 75305f7316513..7f8d81f6b8299 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 @@ -56,6 +56,7 @@ protected void doTransform(Row row, PointCollector collector) break; case STRING: case BLOB: + case OBJECT: case TIMESTAMP: case TEXT: case BOOLEAN: diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFConst.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFConst.java index 209802c26cb90..f3436859f2a41 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFConst.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/UDTFConst.java @@ -57,6 +57,7 @@ public class UDTFConst implements UDTF { VALID_TYPES.add(TSDataType.TEXT.name()); VALID_TYPES.add(TSDataType.STRING.name()); VALID_TYPES.add(TSDataType.BLOB.name()); + VALID_TYPES.add(TSDataType.OBJECT.name()); } private TSDataType dataType; @@ -107,6 +108,7 @@ public void beforeStart(UDFParameters parameters, UDTFConfigurations configurati binaryValue = BytesUtils.valueOf(parameters.getString("value")); break; case BLOB: + case OBJECT: binaryValue = new Binary(BlobUtils.parseBlobString(parameters.getString("value"))); break; default: @@ -141,6 +143,7 @@ public void transform(Row row, PointCollector collector) throws Exception { case TEXT: case STRING: case BLOB: + case OBJECT: collector.putBinary(row.getTime(), UDFBinaryTransformer.transformToUDFBinary(binaryValue)); break; default: @@ -166,6 +169,7 @@ public Object transform(Row row) throws IOException { case TEXT: case STRING: case BLOB: + case OBJECT: return UDFBinaryTransformer.transformToUDFBinary(binaryValue); default: throw new UnsupportedOperationException(); @@ -257,6 +261,7 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception case TEXT: case STRING: case BLOB: + case OBJECT: for (int i = 0; i < count; i++) { boolean hasWritten = false; for (int j = 0; j < columns.length - 1; j++) { 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 b76b0276e7cfd..de6232ac6db34 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 @@ -103,6 +103,7 @@ public void transform(Row row, PointCollector collector) case TEXT: case STRING: case BLOB: + case OBJECT: case TIMESTAMP: case DATE: default: @@ -230,6 +231,7 @@ public void terminate(PointCollector collector) case DATE: case STRING: case BLOB: + case OBJECT: case TEXT: default: // This will not happen. 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 d016732761373..21506698605fe 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 @@ -482,6 +482,7 @@ public void transform(RowWindow rowWindow, PointCollector collector) aggregator.aggregateDouble(rowWindow, collector); break; case BLOB: + case OBJECT: case TEXT: case DATE: case STRING: 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 cf80380eea2a2..fea5cb694fa73 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 @@ -64,6 +64,7 @@ public void transform(RowWindow rowWindow, PointCollector collector) case STRING: case TEXT: case BLOB: + case OBJECT: 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 f3b3aacba378c..85490ac534e47 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 @@ -658,6 +658,7 @@ public void transform(RowWindow rowWindow, PointCollector collector) break; case TEXT: case BLOB: + case OBJECT: case DATE: case STRING: case BOOLEAN: 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 f77b7ab8a7298..566e02e7b58f6 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 @@ -66,6 +66,7 @@ public void transform(RowWindow rowWindow, PointCollector collector) case DATE: case STRING: case BLOB: + case OBJECT: case TEXT: default: // This will not happen 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 f24b9d6ae7b8b..4c17a27069733 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 @@ -86,6 +86,7 @@ public void transform(Row row, PointCollector collector) collector.putBoolean(time, row.getDouble(0) >= lower && upper >= row.getDouble(0)); break; case BLOB: + case OBJECT: case TEXT: case DATE: case STRING: @@ -123,6 +124,7 @@ public Object transform(Row row) throws Exception { case STRING: case TEXT: case BLOB: + case OBJECT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -151,6 +153,7 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception transformDouble(columns, builder); return; case BLOB: + case OBJECT: case TEXT: case DATE: case STRING: 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 d554cfe25720a..0c5ccd73b492e 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 @@ -98,6 +98,7 @@ public void beforeStart(UDFParameters parameters, UDTFConfigurations configurati case TIMESTAMP: case DATE: case BLOB: + case OBJECT: default: throw new UDFInputSeriesDataTypeNotValidException( 0, @@ -170,6 +171,7 @@ public void transform(Row row, PointCollector collector) case DATE: case STRING: case BLOB: + case OBJECT: case INT64: case INT32: case FLOAT: @@ -346,6 +348,7 @@ public void getValues(Row row) throws IOException, UDFInputSeriesDataTypeNotVali break; case STRING: case BLOB: + case OBJECT: case DATE: case TIMESTAMP: default: 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 824740b8c9b04..3de46df72a061 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 @@ -138,6 +138,7 @@ public void transform(RowWindow rowWindow, PointCollector collector) transformDouble(rowWindow, collector); break; case BLOB: + case OBJECT: case DATE: case STRING: case TIMESTAMP: 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 de4f8edcec5d9..947cbdb2c0134 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 @@ -91,6 +91,7 @@ public void transform(Row row, PointCollector collector) case TIMESTAMP: case DATE: case BLOB: + case OBJECT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -119,6 +120,7 @@ public Object transform(Row row) throws IOException { return transformer.transform(row.getDouble(0)); case DATE: case BLOB: + case OBJECT: case STRING: case TIMESTAMP: case TEXT: @@ -155,6 +157,7 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception case STRING: case TIMESTAMP: case BLOB: + case OBJECT: case DATE: default: // This will not happen. 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 6812d2d2d8ee9..077db957f6e2e 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 @@ -57,6 +57,7 @@ protected void doTransform(Row row, PointCollector collector) break; case DATE: case BLOB: + case OBJECT: case STRING: case TIMESTAMP: case BOOLEAN: 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 1183e5397fefc..fcb68817f153e 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 @@ -59,6 +59,7 @@ protected void doTransform(Row row, PointCollector collector) case STRING: case TIMESTAMP: case BLOB: + case OBJECT: case DATE: default: // This will not happen. 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 66b5cd1add8e8..b0c95bd0d23b8 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 @@ -145,6 +145,7 @@ public void transform(Column[] columns, ColumnBuilder builder) throws Exception transformDouble(columns, builder); return; case BLOB: + case OBJECT: case DATE: case STRING: case TIMESTAMP: 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 49faa398f1a78..a888c998d8d88 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 @@ -107,6 +107,7 @@ public void transform(Row row, PointCollector collector) transformString(row.getTime(), row.getString(0)); break; case BLOB: + case OBJECT: case BOOLEAN: default: // This will not happen. @@ -180,6 +181,7 @@ public void terminate(PointCollector collector) } break; case BLOB: + case OBJECT: case BOOLEAN: default: // This will not happen. 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 1838b85022b3a..b8d94308ed966 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 @@ -53,6 +53,7 @@ protected void constructPQ() throws UDFInputSeriesDataTypeNotValidException { break; case BOOLEAN: case BLOB: + case OBJECT: 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 c990021932948..d4487fdfd953a 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 @@ -71,6 +71,7 @@ protected void updatePreviousValue(Row row) case STRING: case DATE: case BLOB: + case OBJECT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/TableBuiltinScalarFunction.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/TableBuiltinScalarFunction.java index e20ca4dd0737f..5d22ff58d813d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/TableBuiltinScalarFunction.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/TableBuiltinScalarFunction.java @@ -111,6 +111,7 @@ public enum TableBuiltinScalarFunction { SPOOKY_HASH_V2_64("spooky_hash_v2_64"), LPAD("lpad"), RPAD("rpad"), + READ_OBJECT("read_object"), ; private final String functionName; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/utils/UDFDataTypeTransformer.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/utils/UDFDataTypeTransformer.java index 83792ebf66aff..cf9ecf89f96c1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/utils/UDFDataTypeTransformer.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/utils/UDFDataTypeTransformer.java @@ -30,6 +30,7 @@ import org.apache.tsfile.read.common.type.FloatType; import org.apache.tsfile.read.common.type.IntType; import org.apache.tsfile.read.common.type.LongType; +import org.apache.tsfile.read.common.type.ObjectType; import org.apache.tsfile.read.common.type.StringType; import org.apache.tsfile.read.common.type.TimestampType; @@ -82,6 +83,8 @@ public static Type transformReadTypeToUDFDataType(org.apache.tsfile.read.common. return Type.BLOB; case STRING: return Type.STRING; + case OBJECT: + return Type.OBJECT; default: throw new IllegalArgumentException("Invalid input: " + type); } @@ -112,6 +115,8 @@ public static org.apache.tsfile.read.common.type.Type transformUDFDataTypeToRead return BlobType.BLOB; case STRING: return StringType.STRING; + case OBJECT: + return ObjectType.OBJECT; default: throw new IllegalArgumentException("Invalid input: " + type); } @@ -139,6 +144,8 @@ private static Type getUDFDataType(byte type) { return Type.BLOB; case 11: return Type.STRING; + case 12: + return Type.OBJECT; default: throw new IllegalArgumentException("Invalid input: " + type); } 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 7b34111aa1fb8..a069c993abef2 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 @@ -156,6 +156,7 @@ public static BatchData deserializeBatchData(ByteBuffer buffer) { } break; case BLOB: + case OBJECT: case STRING: case TEXT: for (int i = 0; i < length; i++) { @@ -200,6 +201,7 @@ public static BatchData deserializeBatchData(ByteBuffer buffer) { values[j] = new TsPrimitiveType.TsFloat(buffer.getFloat()); break; case BLOB: + case OBJECT: case STRING: case TEXT: int len = buffer.getInt(); @@ -297,6 +299,7 @@ public static void serializeTVPairs( dataOutputStream.writeInt(timeValuePairs.size()); switch (timeValuePairs.get(0).getValue().getDataType()) { case BLOB: + case OBJECT: case STRING: case TEXT: serializeTextTVPairs(timeValuePairs, dataOutputStream); @@ -337,6 +340,7 @@ public static void serializeTVPair( switch (dataType) { case STRING: case BLOB: + case OBJECT: case TEXT: dataOutputStream.writeLong(timeValuePair.getTimestamp()); if (timeValuePair.getTimestamp() != Long.MIN_VALUE) { @@ -488,6 +492,7 @@ public static List deserializeTVPairs(ByteBuffer buffer) { deserializeBooleanTVPairs(buffer, ret, size, dataType); break; case BLOB: + case OBJECT: case STRING: case TEXT: deserializeTextTVPairs(buffer, ret, size, dataType); @@ -521,6 +526,7 @@ public static TimeValuePair deserializeTVPair(ByteBuffer buffer) { case BOOLEAN: return new TimeValuePair(time, TsPrimitiveType.getByType(dataType, buffer.get() == 1)); case BLOB: + case OBJECT: case STRING: case TEXT: int bytesLen = buffer.getInt(); diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift index 53d55ce54ca14..7d334059fff01 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift @@ -668,5 +668,5 @@ service IClientRPCService { TSConnectionInfoResp fetchAllConnectionsInfo(); /** For other node's call */ - common.TSStatus testConnectionEmptyRPC() + common.TSStatus testConnectionEmptyRPC(); } \ No newline at end of file diff --git a/pom.xml b/pom.xml index 43a59cda47353..07c9734aea221 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 0.14.1 1.9 1.5.6-3 - 2.2.0-251027-SNAPSHOT + 2.2.0-251030-SNAPSHOT - 1.3.15 + 1.3.16 1.8.0 3.6.0 1.8 From 0fae41e2665a428aa49bf5a1781715479c0f5d34 Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Fri, 31 Oct 2025 14:13:19 +0800 Subject: [PATCH 021/180] Use the correct value in showConfiguration to override the value read from the configuration file (#16681) (cherry picked from commit 274fc9ed4431b70ac3cbb1b5ca55a0b9374d9e6e) --- .../conf/SystemPropertiesUtils.java | 2 + .../apache/iotdb/db/conf/IoTDBStartCheck.java | 4 ++ .../org/apache/iotdb/db/service/DataNode.java | 4 ++ .../commons/conf/ConfigurationFileUtils.java | 62 ++++++++++++++++++- .../commons/file/SystemPropertiesHandler.java | 2 + 5 files changed, 73 insertions(+), 1 deletion(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java index 3b2b51d42fd06..529f15d06cd44 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.conf.ConfigurationFileUtils; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.exception.BadNodeUrlException; import org.apache.iotdb.commons.file.SystemPropertiesHandler; @@ -87,6 +88,7 @@ public static boolean isRestarted() { */ public static void checkSystemProperties() throws IOException { Properties systemProperties = systemPropertiesHandler.read(); + ConfigurationFileUtils.updateAppliedProperties(systemProperties, false); final String format = "[SystemProperties] The parameter \"{}\" can't be modified after first startup." + " Your configuration: {} will be forced update to: {}"; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java index 3c1746a72ccb3..e2aa1950af6a5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java @@ -415,4 +415,8 @@ public void checkEncryptMagicString() throws IOException, ConfigurationException } } } + + public Properties getProperties() { + return properties; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java index cebf7dab9f38e..c4b08f8f05a8a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java @@ -35,6 +35,7 @@ import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.concurrent.IoTDBDefaultThreadExceptionHandler; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.conf.ConfigurationFileUtils; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.consensus.DataRegionId; @@ -270,6 +271,8 @@ protected void start() { // Serialize mutable system properties IoTDBStartCheck.getInstance().serializeMutableSystemPropertiesIfNecessary(); + ConfigurationFileUtils.updateAppliedProperties( + IoTDBStartCheck.getInstance().getProperties(), false); logger.info("IoTDB configuration: {}", config.getConfigMessage()); logger.info("Congratulations, IoTDB DataNode is set up successfully. Now, enjoy yourself!"); @@ -439,6 +442,7 @@ private void pullAndCheckSystemConfigurations() throws StartupException { throw new StartupException(e.getMessage()); } + ConfigurationFileUtils.updateAppliedPropertiesFromCN(configurationResp); // init initTimestampPrecision(); long endTime = System.currentTimeMillis(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/ConfigurationFileUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/ConfigurationFileUtils.java index b0591d07dd2b7..088656582698f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/ConfigurationFileUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/ConfigurationFileUtils.java @@ -20,6 +20,7 @@ package org.apache.iotdb.commons.conf; import org.apache.iotdb.commons.auth.entity.PrivilegeType; +import org.apache.iotdb.confignode.rpc.thrift.TSystemConfigurationResp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,7 +92,7 @@ public class ConfigurationFileUtils { hidedParameters.add("trust_store_pwd"); } - public static void updateAppliedProperties(TrimProperties properties, boolean isHotReloading) { + public static void updateAppliedProperties(Properties properties, boolean isHotReloading) { try { loadConfigurationDefaultValueFromTemplate(); } catch (IOException e) { @@ -114,6 +115,65 @@ public static void updateAppliedProperties(TrimProperties properties, boolean is } } + public static void updateAppliedPropertiesFromCN(TSystemConfigurationResp resp) { + if (resp.getGlobalConfig().isSetTimestampPrecision()) { + lastAppliedProperties.put( + "timestamp_precision", resp.getGlobalConfig().getTimestampPrecision()); + } + if (resp.getGlobalConfig().isSetTimePartitionInterval()) { + lastAppliedProperties.put( + "time_partition_interval", + String.valueOf(resp.getGlobalConfig().getTimePartitionInterval())); + } + if (resp.getGlobalConfig().isSetTimePartitionOrigin()) { + lastAppliedProperties.put( + "time_partition_origin", String.valueOf(resp.getGlobalConfig().getTimePartitionOrigin())); + } + if (resp.getGlobalConfig().isSetSchemaEngineMode()) { + lastAppliedProperties.put("schema_engine_mode", resp.getGlobalConfig().getSchemaEngineMode()); + } + if (resp.getGlobalConfig().isSetTagAttributeTotalSize()) { + lastAppliedProperties.put( + "tag_attribute_total_size", + String.valueOf(resp.getGlobalConfig().getTagAttributeTotalSize())); + } + if (resp.getGlobalConfig().isSetSeriesPartitionExecutorClass()) { + lastAppliedProperties.put( + "series_partition_executor_class", + resp.getGlobalConfig().getSeriesPartitionExecutorClass()); + } + if (resp.getGlobalConfig().isSetSeriesPartitionSlotNum()) { + lastAppliedProperties.put( + "series_slot_num", String.valueOf(resp.getGlobalConfig().getSeriesPartitionSlotNum())); + } + if (resp.getGlobalConfig().isSetDataRegionConsensusProtocolClass()) { + lastAppliedProperties.put( + "data_region_consensus_protocol_class", + resp.getGlobalConfig().getDataRegionConsensusProtocolClass()); + } + if (resp.getGlobalConfig().isSetSchemaRegionConsensusProtocolClass()) { + lastAppliedProperties.put( + "schema_region_consensus_protocol_class", + resp.getGlobalConfig().getSchemaRegionConsensusProtocolClass()); + } + if (resp.getGlobalConfig().isSetReadConsistencyLevel()) { + lastAppliedProperties.put( + "read_consistency_level", resp.getGlobalConfig().getReadConsistencyLevel()); + } + if (resp.getGlobalConfig().isSetDiskSpaceWarningThreshold()) { + lastAppliedProperties.put( + "disk_space_warning_threshold", + String.valueOf(resp.getGlobalConfig().getDiskSpaceWarningThreshold())); + } + } + + // This method may not be used in the current version directly, but should not be removed to + // reduce conflicts + @SuppressWarnings("unused") + public static void updateAppliedProperties(String key, String value) { + lastAppliedProperties.put(key, value); + } + public static Map getAppliedProperties() { return lastAppliedProperties; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/file/SystemPropertiesHandler.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/file/SystemPropertiesHandler.java index 5264f4735f69b..dfbb2104cfb8d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/file/SystemPropertiesHandler.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/file/SystemPropertiesHandler.java @@ -19,6 +19,7 @@ package org.apache.iotdb.commons.file; +import org.apache.iotdb.commons.conf.ConfigurationFileUtils; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.ratis.util.AutoCloseableLock; @@ -80,6 +81,7 @@ public void put(Object... keyOrValue) throws IOException { } public void overwrite(Properties properties) throws IOException { + ConfigurationFileUtils.updateAppliedProperties(properties, false); try (AutoCloseableLock ignore = AutoCloseableLock.acquire(lock.writeLock())) { if (!formalFile.exists()) { writeWithoutLock(properties, formalFile); From cc8a718932170116952d7d477f48f4746df17946 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 31 Oct 2025 17:23:47 +0800 Subject: [PATCH 022/180] Fixed the bug that the attribute update container may add extra calculated memory to schema engine (#16682) (cherry picked from commit 685141a6405e79533906c6bf2700707296828777) --- .../attribute/update/DeviceAttributeCacheUpdater.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/attribute/update/DeviceAttributeCacheUpdater.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/attribute/update/DeviceAttributeCacheUpdater.java index 64d92335b1b14..fd798cc3a7da2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/attribute/update/DeviceAttributeCacheUpdater.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/attribute/update/DeviceAttributeCacheUpdater.java @@ -440,17 +440,23 @@ private void updateMemory(final long size) { if (size > 0) { requestMemory(size); } else { - releaseMemory(size); + releaseMemory(-size); } } private void requestMemory(final long size) { + if (size < 0) { + throw new UnsupportedOperationException("requestMemory size must not be negative"); + } if (regionStatistics != null) { regionStatistics.requestMemory(size); } } private void releaseMemory(final long size) { + if (size < 0) { + throw new UnsupportedOperationException("releaseMemory size must not be negative"); + } if (regionStatistics != null) { regionStatistics.releaseMemory(size); } From 65ec0e2b3913f6a9740ed57869916cca584d168a Mon Sep 17 00:00:00 2001 From: Weihao Li <60659567+Wei-hao-Li@users.noreply.github.com> Date: Mon, 3 Nov 2025 16:51:52 +0800 Subject: [PATCH 023/180] Add more optimizers for union (#16689) (cherry picked from commit 69cda2da662029b4f088ad5d7376d2f6902506a0) --- .../iterative/rule/PushLimitThroughUnion.java | 107 +++++++++++++++++ .../rule/PushProjectionThroughUnion.java | 111 +++++++++++++++++ .../iterative/rule/PushTopKThroughUnion.java | 102 ++++++++++++++++ .../rule/RemoveEmptyUnionBranches.java | 113 ++++++++++++++++++ .../optimizations/LogicalOptimizeFactory.java | 24 ++-- ...eUnionTest.java => UnionOptimizeTest.java} | 62 +++++++++- 6 files changed, 511 insertions(+), 8 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushLimitThroughUnion.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushProjectionThroughUnion.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushTopKThroughUnion.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/RemoveEmptyUnionBranches.java rename iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/{MergeUnionTest.java => UnionOptimizeTest.java} (64%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushLimitThroughUnion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushLimitThroughUnion.java new file mode 100644 index 0000000000000..0c2c11ac72085 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushLimitThroughUnion.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule; + +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LimitNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.UnionNode; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Capture; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Captures; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Pattern; + +import com.google.common.collect.ImmutableList; + +import java.util.Optional; + +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.Limit.requiresPreSortedInputs; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.limit; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.source; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.union; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.QueryCardinalityUtil.isAtMost; +import static org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Capture.newCapture; + +/** + * Transforms: + * + *
+ * - Limit
+ *    - Union
+ *       - relation1
+ *       - relation2
+ *       ..
+ * 
+ * + * Into: + * + *
+ * - Limit
+ *    - Union
+ *       - Limit
+ *          - relation1
+ *       - Limit
+ *          - relation2
+ *       ..
+ * 
+ * + * Applies to LimitNode without ties only to avoid optimizer loop. + */ +public class PushLimitThroughUnion implements Rule { + private static final Capture CHILD = newCapture(); + + private static final Pattern PATTERN = + limit() + .matching(limit -> !limit.isWithTies()) + .with(requiresPreSortedInputs().equalTo(false)) + .with(source().matching(union().capturedAs(CHILD))); + + @Override + public Pattern getPattern() { + return PATTERN; + } + + @Override + public Result apply(LimitNode parent, Captures captures, Context context) { + UnionNode unionNode = captures.get(CHILD); + ImmutableList.Builder builder = ImmutableList.builder(); + boolean shouldApply = false; + for (PlanNode child : unionNode.getChildren()) { + // This check is to ensure that we don't fire the optimizer if it was previously applied. + if (isAtMost(child, context.getLookup(), parent.getCount())) { + builder.add(child); + } else { + shouldApply = true; + builder.add( + new LimitNode( + context.getIdAllocator().genPlanNodeId(), + child, + parent.getCount(), + Optional.empty())); + } + } + + if (!shouldApply) { + return Result.empty(); + } + + return Result.ofPlanNode( + parent.replaceChildren(ImmutableList.of(unionNode.replaceChildren(builder.build())))); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushProjectionThroughUnion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushProjectionThroughUnion.java new file mode 100644 index 0000000000000..4678ec16b48da --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushProjectionThroughUnion.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule; + +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Assignments; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ProjectNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.UnionNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Capture; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Captures; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Pattern; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableListMultimap; +import org.apache.tsfile.read.common.type.Type; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.iotdb.db.queryengine.plan.relational.planner.ExpressionSymbolInliner.inlineSymbols; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.project; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.source; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.union; +import static org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Capture.newCapture; + +public class PushProjectionThroughUnion implements Rule { + private static final Capture CHILD = newCapture(); + + private static final Pattern PATTERN = + project() + .matching(PushProjectionThroughUnion::nonTrivialProjection) + .with(source().matching(union().capturedAs(CHILD))); + + @Override + public Pattern getPattern() { + return PATTERN; + } + + @Override + public Result apply(ProjectNode parent, Captures captures, Context context) { + UnionNode child = captures.get(CHILD); + + // OutputLayout of the resultant Union, will be same as the layout of the Project + List outputLayout = parent.getOutputSymbols(); + + // Mapping from the output symbol to ordered list of symbols from each of the children + ImmutableListMultimap.Builder mappings = ImmutableListMultimap.builder(); + + // sources for the resultant UnionNode + ImmutableList.Builder outputSources = ImmutableList.builder(); + + for (int i = 0; i < child.getChildren().size(); i++) { + Map outputToInput = + child.sourceSymbolMap(i); // Map: output of union -> input of this child to the union + Assignments.Builder assignments = + Assignments.builder(); // assignments for the new ProjectNode + + // mapping from current ProjectNode to new ProjectNode, used to identify the output layout + Map projectSymbolMapping = new HashMap<>(); + + // Translate the assignments in the ProjectNode using symbols of the source of the UnionNode + for (Map.Entry entry : parent.getAssignments().entrySet()) { + Expression translatedExpression = inlineSymbols(outputToInput, entry.getValue()); + Type type = context.getSymbolAllocator().getTypes().getTableModelType(entry.getKey()); + Symbol symbol = context.getSymbolAllocator().newSymbol(translatedExpression, type); + assignments.put(symbol, translatedExpression); + projectSymbolMapping.put(entry.getKey(), symbol); + } + outputSources.add( + new ProjectNode( + context.getIdAllocator().genPlanNodeId(), + child.getChildren().get(i), + assignments.build())); + outputLayout.forEach(symbol -> mappings.put(symbol, projectSymbolMapping.get(symbol))); + } + + return Result.ofPlanNode( + new UnionNode( + parent.getPlanNodeId(), + outputSources.build(), + mappings.build(), + ImmutableList.copyOf(mappings.build().keySet()))); + } + + private static boolean nonTrivialProjection(ProjectNode project) { + return !project.getAssignments().getExpressions().stream() + .allMatch(SymbolReference.class::isInstance); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushTopKThroughUnion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushTopKThroughUnion.java new file mode 100644 index 0000000000000..2064e7b6863fd --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushTopKThroughUnion.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule; + +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.UnionNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.SymbolMapper; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Capture; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Captures; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Pattern; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.Collections; +import java.util.Set; + +import static com.google.common.collect.Iterables.getLast; +import static com.google.common.collect.Sets.intersection; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.source; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.topK; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.union; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.QueryCardinalityUtil.isAtMost; +import static org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Capture.newCapture; + +public class PushTopKThroughUnion implements Rule { + private static final Capture CHILD = newCapture(); + + private static final Pattern PATTERN = + topK().with(source().matching(union().capturedAs(CHILD))); + + @Override + public Pattern getPattern() { + return PATTERN; + } + + @Override + public Result apply(TopKNode topKNode, Captures captures, Context context) { + UnionNode unionNode = captures.get(CHILD); + + ImmutableList.Builder children = ImmutableList.builder(); + + boolean shouldApply = false; + for (PlanNode child : unionNode.getChildren()) { + SymbolMapper.Builder symbolMapper = SymbolMapper.builder(); + Set sourceOutputSymbols = ImmutableSet.copyOf(child.getOutputSymbols()); + // This check is to ensure that we don't fire the optimizer if it was previously applied, + // which is the same as PushLimitThroughUnion. + if (isAtMost(child, context.getLookup(), topKNode.getCount())) { + children.add(child); + } else { + shouldApply = true; + for (Symbol unionOutput : unionNode.getOutputSymbols()) { + Set inputSymbols = + ImmutableSet.copyOf(unionNode.getSymbolMapping().get(unionOutput)); + Symbol unionInput = getLast(intersection(inputSymbols, sourceOutputSymbols)); + symbolMapper.put(unionOutput, unionInput); + } + children.add( + symbolMapper + .build() + .map( + topKNode, + Collections.singletonList(child), + context.getIdAllocator().genPlanNodeId())); + } + } + + if (!shouldApply) { + return Result.empty(); + } + + return Result.ofPlanNode( + topKNode.replaceChildren( + Collections.singletonList( + new UnionNode( + unionNode.getPlanNodeId(), + children.build(), + unionNode.getSymbolMapping(), + unionNode.getOutputSymbols())))); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/RemoveEmptyUnionBranches.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/RemoveEmptyUnionBranches.java new file mode 100644 index 0000000000000..1b4021f71163e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/RemoveEmptyUnionBranches.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule; + +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Assignments; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ProjectNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.UnionNode; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Captures; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Pattern; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.ListMultimap; + +import java.util.List; + +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.union; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.QueryCardinalityUtil.isEmpty; + +/** + * Removes branches from a UnionNode that are guaranteed to produce 0 rows. + * + *

If there's only one branch left, it replaces the UnionNode with a projection to preserve the + * outputs of the union. + * + *

If all branches are empty, now we do the same process with one branch left case. + */ +public class RemoveEmptyUnionBranches implements Rule { + private static final Pattern PATTERN = union(); + + @Override + public Pattern getPattern() { + return PATTERN; + } + + @Override + public Result apply(UnionNode node, Captures captures, Context context) { + int emptyBranches = 0; + ImmutableList.Builder newChildrenBuilder = ImmutableList.builder(); + ImmutableListMultimap.Builder outputsToInputsBuilder = + ImmutableListMultimap.builder(); + for (int i = 0; i < node.getChildren().size(); i++) { + PlanNode child = node.getChildren().get(i); + if (!isEmpty(child, context.getLookup())) { + newChildrenBuilder.add(child); + + for (Symbol column : node.getOutputSymbols()) { + outputsToInputsBuilder.put(column, node.getSymbolMapping().get(column).get(i)); + } + } else { + emptyBranches++; + } + } + + if (emptyBranches == 0) { + return Result.empty(); + } + + // restore after ValuesNode is introduced + /* + if (emptyBranches == node.getChildren().size()) { + return Result.ofPlanNode(new ValuesNode(node.getId(), node.getOutputSymbols(), ImmutableList.of())); + }*/ + + // Now we do the same process with one branch left case, choose the first child as preserved. + if (emptyBranches == node.getChildren().size()) { + Assignments.Builder assignments = Assignments.builder(); + for (Symbol column : node.getOutputSymbols()) { + assignments.put(column, node.getSymbolMapping().get(column).get(0).toSymbolReference()); + } + + return Result.ofPlanNode( + new ProjectNode(node.getPlanNodeId(), node.getChildren().get(0), assignments.build())); + } + + List newChildren = newChildrenBuilder.build(); + ListMultimap outputsToInputs = outputsToInputsBuilder.build(); + + if (newChildren.size() == 1) { + Assignments.Builder assignments = Assignments.builder(); + + outputsToInputs + .entries() + .forEach(entry -> assignments.put(entry.getKey(), entry.getValue().toSymbolReference())); + + return Result.ofPlanNode( + new ProjectNode(node.getPlanNodeId(), newChildren.get(0), assignments.build())); + } + + return Result.ofPlanNode( + new UnionNode(node.getPlanNodeId(), newChildren, outputsToInputs, node.getOutputSymbols())); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/LogicalOptimizeFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/LogicalOptimizeFactory.java index cdf99e5543470..d70f67114b0f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/LogicalOptimizeFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/LogicalOptimizeFactory.java @@ -68,7 +68,11 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.PruneWindowColumns; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.PushLimitThroughOffset; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.PushLimitThroughProject; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.PushLimitThroughUnion; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.PushProjectionThroughUnion; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.PushTopKThroughUnion; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.RemoveDuplicateConditions; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.RemoveEmptyUnionBranches; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.RemoveRedundantEnforceSingleRowNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.RemoveRedundantExists; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.RemoveRedundantIdentityProjections; @@ -141,8 +145,7 @@ public LogicalOptimizeFactory(PlannerContext plannerContext) { IterativeOptimizer columnPruningOptimizer = new IterativeOptimizer(plannerContext, ruleStats, columnPruningRules); - // Set> projectionPushdownRules = ImmutableSet.of( - // new PushProjectionThroughUnion(), + Set> projectionPushdownRules = ImmutableSet.of(new PushProjectionThroughUnion()); // new PushProjectionThroughExchange(), // // Dereference pushdown rules // new PushDownDereferencesThroughMarkDistinct(typeAnalyzer), @@ -181,7 +184,10 @@ public LogicalOptimizeFactory(PlannerContext plannerContext) { new IterativeOptimizer(plannerContext, ruleStats, simplifyOptimizerRules); Set> limitPushdownRules = - ImmutableSet.of(new PushLimitThroughOffset(), new PushLimitThroughProject()); + ImmutableSet.of( + new PushLimitThroughOffset(), + new PushLimitThroughProject(), + new PushLimitThroughUnion()); ImmutableList.Builder optimizerBuilder = ImmutableList.builder(); @@ -199,12 +205,13 @@ public LogicalOptimizeFactory(PlannerContext plannerContext) { ruleStats, ImmutableSet.>builder() .addAll(columnPruningRules) - // .addAll(projectionPushdownRules). + .addAll(projectionPushdownRules) // addAll(newUnwrapRowSubscript().rules()). // addAll(new PushCastIntoRow().rules()) .addAll( ImmutableSet.of( new ImplementTableFunctionSource(), + new RemoveEmptyUnionBranches(), new MergeFilters(), new InlineProjections(plannerContext), new RemoveRedundantIdentityProjections(), @@ -242,13 +249,13 @@ public LogicalOptimizeFactory(PlannerContext plannerContext) { plannerContext, ruleStats, ImmutableSet.>builder() - // .addAll(projectionPushdownRules) + .addAll(projectionPushdownRules) .addAll(columnPruningRules) .addAll(limitPushdownRules) .addAll( ImmutableSet.of( new MergeUnion(), - // new RemoveEmptyUnionBranches(), + new RemoveEmptyUnionBranches(), new MergeFilters(), new RemoveTrivialFilters(), new MergeLimits(), @@ -331,7 +338,10 @@ public LogicalOptimizeFactory(PlannerContext plannerContext) { new IterativeOptimizer( plannerContext, ruleStats, - ImmutableSet.of(new MergeLimitWithSort(), new MergeLimitOverProjectWithSort())), + ImmutableSet.of( + new MergeLimitWithSort(), + new MergeLimitOverProjectWithSort(), + new PushTopKThroughUnion())), new ParallelizeGrouping()); this.planOptimizers = optimizerBuilder.build(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/MergeUnionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/UnionOptimizeTest.java similarity index 64% rename from iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/MergeUnionTest.java rename to iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/UnionOptimizeTest.java index d74e237082c05..4fdbdb27b3da5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/MergeUnionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/UnionOptimizeTest.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.analyzer; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; +import org.apache.iotdb.db.queryengine.plan.relational.planner.PlanTester; import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolAllocator; import org.apache.iotdb.db.queryengine.plan.relational.planner.TableLogicalPlanner; @@ -31,11 +32,14 @@ import static org.apache.iotdb.db.queryengine.plan.relational.analyzer.TestUtils.SESSION_INFO; import static org.apache.iotdb.db.queryengine.plan.relational.analyzer.TestUtils.TEST_MATADATA; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanAssert.assertPlan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.limit; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.output; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.project; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.tableScan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.topK; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.union; -public class MergeUnionTest { +public class UnionOptimizeTest { @Test public void simpleLeftDeepMerge() { @@ -95,4 +99,60 @@ public void bushyTreeMerge() { tableScan("testdb.t3"), tableScan("testdb.t4"))))); } + + @Test + public void pushLimitThroughUnionTest() { + assertPlan( + new PlanTester() + .createPlan("(select tag1 from t1) union all (select tag1 from t2) limit 1"), + output( + limit(1, (union(limit(1, tableScan("testdb.t1")), limit(1, tableScan("testdb.t2"))))))); + } + + @Test + public void pushProjectionThroughUnionTest() { + assertPlan( + new PlanTester() + .createPlan("select s1 + 1 from ((select s1 from t1) union all (select s1 from t2)) "), + output((union(project(tableScan("testdb.t1")), project(tableScan("testdb.t2")))))); + } + + @Test + public void pushTopKThroughUnionTest() { + assertPlan( + new PlanTester() + .createPlan( + "(select tag1 from t1) union all (select tag1 from t2) order by tag1 limit 10"), + output(topK((union(topK(tableScan("testdb.t1")), topK(tableScan("testdb.t2"))))))); + } + + @Test + public void removeEmptyUnionBranchesTest1() { + // Normal case + assertPlan( + new PlanTester() + .createPlan( + "(select tag1 from t1 limit 0) union all (select tag1 from t2) union all (select tag1 from t3)"), + output((union(tableScan("testdb.t2"), tableScan("testdb.t3"))))); + } + + @Test + public void removeEmptyUnionBranchesTest2() { + // One non-empty branch + assertPlan( + new PlanTester() + .createPlan( + "(select tag1 from t1 limit 0) union all (select tag1 from t2 limit 0) union all (select tag1 from t3 limit 1)"), + output(limit(1, tableScan("testdb.t3")))); + } + + @Test + public void removeEmptyUnionBranchesTest3() { + // All branches are empty + assertPlan( + new PlanTester() + .createPlan( + "(select tag1 from t1 limit 0) union all (select tag1 from t2 limit 0) union all (select tag1 from t3 limit 0)"), + output(limit(0, tableScan("testdb.t1")))); + } } From eae73aae64aa050c795c2ec1598b3fd7c680582b Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Tue, 4 Nov 2025 10:22:33 +0800 Subject: [PATCH 024/180] SeriesScanUtil throws exception when using filters that could not match any time range (#16691) (cherry picked from commit 2488001e8087ef76821ae1782cd54ff1d8b416f7) --- .../java/org/apache/iotdb/db/it/IoTDBFilterIT.java | 14 ++++++++++++++ .../it/query/recent/IoTDBTableAggregationIT.java | 11 +++++++++++ .../execution/operator/source/SeriesScanUtil.java | 5 +++++ .../dataregion/read/QueryDataSource.java | 5 +++++ 4 files changed, 35 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFilterIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFilterIT.java index 306346e2bdc51..b8dd88ea9d715 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFilterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFilterIT.java @@ -24,6 +24,7 @@ import org.apache.iotdb.itbase.category.ClusterIT; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -238,4 +239,17 @@ public void testFilterWithUDTF() { fail(throwable.getMessage()); } } + + @Test + public void testFilterWithEmptySatisfiedTimeRanges() { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement(); + ResultSet resultSet = + statement.executeQuery("select count(*) from root.** where time >= 0 and time < 0")) { + Assert.assertTrue(resultSet.next()); + Assert.assertEquals(0, resultSet.getInt(1)); + } catch (SQLException throwable) { + fail(throwable.getMessage()); + } + } } diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableAggregationIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableAggregationIT.java index c6e5fc138ce3a..0143bdb6e3b56 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableAggregationIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableAggregationIT.java @@ -5480,4 +5480,15 @@ public void emptyBlockInStreamOperatorTest() { retArray, DATABASE_NAME); } + + @Test + public void emptyTimeRangeQueryTest() { + String[] expectedHeader = new String[] {"_col0"}; + String[] retArray = new String[] {"0,"}; + tableResultSetEqualTest( + "select count(*) from table1 where time >= 0 and time < -1", + expectedHeader, + retArray, + DATABASE_NAME); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java index b49b9b199d1a4..e2fbb4be4ecac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java @@ -219,6 +219,11 @@ public void initQueryDataSource(QueryDataSource dataSource) { orderUtils.setCurSeqFileIndex(dataSource); curUnseqFileIndex = 0; + if (dataSource.isEmpty()) { + // no satisfied resources + return; + } + if (satisfiedTimeRange == null) { long startTime = Long.MAX_VALUE; long endTime = Long.MIN_VALUE; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java index 10f843c81734b..2816493401671 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java @@ -100,6 +100,11 @@ public List getUnseqResources() { return unseqResources; } + public boolean isEmpty() { + return (seqResources == null || seqResources.isEmpty()) + && (unseqResources == null || unseqResources.isEmpty()); + } + @Override public IQueryDataSource clone() { QueryDataSource queryDataSource = From eab51a69660ded2842a6f27db8b8a30e853cbe6a Mon Sep 17 00:00:00 2001 From: JackieTien97 Date: Tue, 4 Nov 2025 12:16:34 +0800 Subject: [PATCH 025/180] Add object type in CompatibleResolver --- .../queryengine/plan/relational/type/CompatibleResolver.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/type/CompatibleResolver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/type/CompatibleResolver.java index a33f93692db5e..2f71195203f72 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/type/CompatibleResolver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/type/CompatibleResolver.java @@ -34,6 +34,7 @@ import static org.apache.tsfile.read.common.type.FloatType.FLOAT; import static org.apache.tsfile.read.common.type.IntType.INT32; import static org.apache.tsfile.read.common.type.LongType.INT64; +import static org.apache.tsfile.read.common.type.ObjectType.OBJECT; import static org.apache.tsfile.read.common.type.StringType.STRING; import static org.apache.tsfile.read.common.type.TimestampType.TIMESTAMP; import static org.apache.tsfile.read.common.type.UnknownType.UNKNOWN; @@ -89,6 +90,9 @@ public class CompatibleResolver { addCondition(BLOB, BLOB, BLOB); addCondition(BLOB, UNKNOWN, BLOB); + addCondition(OBJECT, OBJECT, OBJECT); + addCondition(OBJECT, UNKNOWN, OBJECT); + addCondition(UNKNOWN, INT32, INT32); addCondition(UNKNOWN, INT64, INT64); addCondition(UNKNOWN, FLOAT, FLOAT); @@ -99,6 +103,7 @@ public class CompatibleResolver { addCondition(UNKNOWN, TEXT, TEXT); addCondition(UNKNOWN, STRING, STRING); addCondition(UNKNOWN, BLOB, BLOB); + addCondition(UNKNOWN, OBJECT, OBJECT); addCondition(UNKNOWN, UNKNOWN, UNKNOWN); } From 6ea56d99f7df7c10f3b987131b8d3639109cdf33 Mon Sep 17 00:00:00 2001 From: JackieTien97 Date: Wed, 5 Nov 2025 11:09:50 +0800 Subject: [PATCH 026/180] Support In UDF Record --- .../java/org/apache/iotdb/ObjectExample.java | 148 ------------------ .../org/apache/iotdb/ObjectReadExample.java | 114 -------------- .../apache/iotdb/jdbc/IoTDBJDBCResultSet.java | 2 +- .../metadata/TableMetadataImpl.java | 18 ++- .../commons/udf/access/RecordIterator.java | 19 ++- 5 files changed, 29 insertions(+), 272 deletions(-) delete mode 100644 example/session/src/main/java/org/apache/iotdb/ObjectExample.java delete mode 100644 example/session/src/main/java/org/apache/iotdb/ObjectReadExample.java diff --git a/example/session/src/main/java/org/apache/iotdb/ObjectExample.java b/example/session/src/main/java/org/apache/iotdb/ObjectExample.java deleted file mode 100644 index 3339d3aae5a4c..0000000000000 --- a/example/session/src/main/java/org/apache/iotdb/ObjectExample.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb; - -import org.apache.iotdb.isession.ITableSession; -import org.apache.iotdb.rpc.IoTDBConnectionException; -import org.apache.iotdb.rpc.StatementExecutionException; -import org.apache.iotdb.session.TableSessionBuilder; - -import org.apache.tsfile.enums.ColumnCategory; -import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.write.record.Tablet; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -public class ObjectExample { - private static final String LOCAL_URL = "127.0.0.1:6667"; - - public static void main(String[] args) { - - // don't specify database in constructor - try (ITableSession session = - new TableSessionBuilder() - .nodeUrls(Collections.singletonList(LOCAL_URL)) - .username("root") - .password("root") - .build()) { - session.executeNonQueryStatement("CREATE DATABASE IF NOT EXISTS test1"); - session.executeNonQueryStatement("use test1"); - - // insert table data by tablet - List columnNameList = - Arrays.asList("region_id", "plant_id", "device_id", "temperature", "file"); - List dataTypeList = - Arrays.asList( - TSDataType.STRING, - TSDataType.STRING, - TSDataType.STRING, - TSDataType.FLOAT, - TSDataType.OBJECT); - List columnTypeList = - new ArrayList<>( - Arrays.asList( - ColumnCategory.TAG, - ColumnCategory.TAG, - ColumnCategory.TAG, - ColumnCategory.FIELD, - ColumnCategory.FIELD)); - Tablet tablet = new Tablet("tsfile_table", columnNameList, dataTypeList, columnTypeList, 1); - int rowIndex = tablet.getRowSize(); - tablet.addTimestamp(rowIndex, 1); - tablet.addValue(rowIndex, 0, "1"); - tablet.addValue(rowIndex, 1, "5"); - tablet.addValue(rowIndex, 2, "3"); - tablet.addValue(rowIndex, 3, 37.6F); - tablet.addValue( - rowIndex, - 4, - true, - 0, - Files.readAllBytes( - Paths.get( - "/Users/jackietien/Downloads/2_1746622362350_fa24aa15233f4e76bcda789a5771f43f"))); - session.insert(tablet); - tablet.reset(); - - tablet = new Tablet("tsfile_table", columnNameList, dataTypeList, columnTypeList, 1); - rowIndex = tablet.getRowSize(); - tablet.addTimestamp(rowIndex, 2); - tablet.addValue(rowIndex, 0, "1"); - tablet.addValue(rowIndex, 1, "5"); - tablet.addValue(rowIndex, 2, "3"); - tablet.addValue(rowIndex, 3, 37.6F); - tablet.addValue( - rowIndex, - 4, - true, - 0, - Files.readAllBytes( - Paths.get( - "/Users/jackietien/Downloads/2_1746622367063_8fb5ac8e21724140874195b60b878664"))); - session.insert(tablet); - tablet.reset(); - - tablet = new Tablet("tiff_table", columnNameList, dataTypeList, columnTypeList, 1); - rowIndex = tablet.getRowSize(); - tablet.addTimestamp(rowIndex, 1); - tablet.addValue(rowIndex, 0, "1"); - tablet.addValue(rowIndex, 1, "5"); - tablet.addValue(rowIndex, 2, "3"); - tablet.addValue(rowIndex, 3, 37.6F); - tablet.addValue( - rowIndex, - 4, - true, - 0, - Files.readAllBytes(Paths.get("/Users/jackietien/Downloads/1751891240130.tiff"))); - session.insert(tablet); - tablet.reset(); - - tablet = new Tablet("tiff_table", columnNameList, dataTypeList, columnTypeList, 1); - rowIndex = tablet.getRowSize(); - tablet.addTimestamp(rowIndex, 2); - tablet.addValue(rowIndex, 0, "1"); - tablet.addValue(rowIndex, 1, "5"); - tablet.addValue(rowIndex, 2, "4"); - tablet.addValue(rowIndex, 3, 37.6F); - tablet.addValue( - rowIndex, - 4, - true, - 0, - Files.readAllBytes(Paths.get("/Users/jackietien/Downloads/1751891242743.tiff"))); - session.insert(tablet); - tablet.reset(); - - } catch (IoTDBConnectionException e) { - e.printStackTrace(); - } catch (StatementExecutionException e) { - e.printStackTrace(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } -} diff --git a/example/session/src/main/java/org/apache/iotdb/ObjectReadExample.java b/example/session/src/main/java/org/apache/iotdb/ObjectReadExample.java deleted file mode 100644 index 344939175db78..0000000000000 --- a/example/session/src/main/java/org/apache/iotdb/ObjectReadExample.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb; - -import org.apache.iotdb.isession.ITableSession; -import org.apache.iotdb.isession.SessionDataSet; -import org.apache.iotdb.rpc.IoTDBConnectionException; -import org.apache.iotdb.rpc.StatementExecutionException; -import org.apache.iotdb.session.TableSessionBuilder; - -import org.apache.commons.codec.digest.DigestUtils; -import org.apache.tsfile.utils.Binary; - -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Collections; - -public class ObjectReadExample { - private static final String LOCAL_URL = "127.0.0.1:6667"; - - public static void main(String[] args) { - - // don't specify database in constructor - try (ITableSession session = - new TableSessionBuilder() - .nodeUrls(Collections.singletonList(LOCAL_URL)) - .username("root") - .password("root") - .database("test1") - .thriftMaxFrameSize(256 * 1024 * 1024) - .build()) { - try (SessionDataSet dataSet = - session.executeQueryStatement( - "select READ_OBJECT(file) from tsfile_table where time = 1")) { - SessionDataSet.DataIterator iterator = dataSet.iterator(); - while (iterator.next()) { - Binary binary = iterator.getBlob(1); - System.out.println(DigestUtils.md5Hex(binary.getValues())); - } - } - - try (SessionDataSet dataSet = - session.executeQueryStatement( - "select READ_OBJECT(file) from tsfile_table where time = 2")) { - SessionDataSet.DataIterator iterator = dataSet.iterator(); - while (iterator.next()) { - Binary binary = iterator.getBlob(1); - System.out.println(DigestUtils.md5Hex(binary.getValues())); - } - } - - try (SessionDataSet dataSet = - session.executeQueryStatement("select READ_OBJECT(file) from tsfile_table")) { - SessionDataSet.DataIterator iterator = dataSet.iterator(); - while (iterator.next()) { - Binary binary = iterator.getBlob(1); - System.out.println(DigestUtils.md5Hex(binary.getValues())); - } - } - - try (SessionDataSet dataSet = - session.executeQueryStatement( - "select geo_penetrate(file, '0,3,7501,7504') from tsfile_table")) { - SessionDataSet.DataIterator iterator = dataSet.iterator(); - while (iterator.next()) { - Binary binary = iterator.getBlob(1); - ByteBuffer byteBuffer = ByteBuffer.wrap(binary.getValues()); - float[] res = new float[byteBuffer.limit() / Float.BYTES]; - for (int i = 0; i < res.length; i++) { - res[i] = byteBuffer.getFloat(); - } - System.out.println(Arrays.toString(res)); - } - } - - try (SessionDataSet dataSet = - session.executeQueryStatement( - "select geo_penetrate(file, '0,3,7501,7504', 'UNCOMPRESSED_TIFF') from tiff_table")) { - SessionDataSet.DataIterator iterator = dataSet.iterator(); - while (iterator.next()) { - Binary binary = iterator.getBlob(1); - ByteBuffer byteBuffer = ByteBuffer.wrap(binary.getValues()); - float[] res = new float[byteBuffer.limit() / Float.BYTES]; - for (int i = 0; i < res.length; i++) { - res[i] = byteBuffer.getFloat(); - } - System.out.println(Arrays.toString(res)); - } - } - - } catch (IoTDBConnectionException e) { - e.printStackTrace(); - } catch (StatementExecutionException e) { - e.printStackTrace(); - } - } -} diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBJDBCResultSet.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBJDBCResultSet.java index 3157b6c98c42a..77ae98d488769 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBJDBCResultSet.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBJDBCResultSet.java @@ -360,7 +360,7 @@ public byte[] getBytes(int columnIndex) throws SQLException { return null; } - if (dataType.equals(TSDataType.BLOB)) { + if (dataType.equals(TSDataType.BLOB) || dataType.equals(TSDataType.OBJECT)) { Binary binary = ioTDBRpcDataSet.getBinary(columnIndex); return binary == null ? null : binary.getValues(); } else { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java index e4742b873b191..cc9af5c9b7249 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java @@ -1331,8 +1331,13 @@ && isIntegerNumber(argumentTypes.get(2)))) { Collections.emptyMap()); try { ScalarFunctionAnalysis scalarFunctionAnalysis = scalarFunction.analyze(functionArguments); - return UDFDataTypeTransformer.transformUDFDataTypeToReadType( - scalarFunctionAnalysis.getOutputDataType()); + Type returnType = + UDFDataTypeTransformer.transformUDFDataTypeToReadType( + scalarFunctionAnalysis.getOutputDataType()); + if (returnType == ObjectType.OBJECT) { + throw new SemanticException("OBJECT type is not supported as return type"); + } + return returnType; } catch (Exception e) { throw new SemanticException("Invalid function parameters: " + e.getMessage()); } finally { @@ -1349,8 +1354,13 @@ && isIntegerNumber(argumentTypes.get(2)))) { try { AggregateFunctionAnalysis aggregateFunctionAnalysis = aggregateFunction.analyze(functionArguments); - return UDFDataTypeTransformer.transformUDFDataTypeToReadType( - aggregateFunctionAnalysis.getOutputDataType()); + Type returnType = + UDFDataTypeTransformer.transformUDFDataTypeToReadType( + aggregateFunctionAnalysis.getOutputDataType()); + if (returnType == ObjectType.OBJECT) { + throw new SemanticException("OBJECT type is not supported as return type"); + } + return returnType; } catch (Exception e) { throw new SemanticException("Invalid function parameters: " + e.getMessage()); } finally { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/access/RecordIterator.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/access/RecordIterator.java index 3c38383f0b02a..6f5813955dd87 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/access/RecordIterator.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/access/RecordIterator.java @@ -25,13 +25,17 @@ import org.apache.tsfile.block.column.Column; import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.read.common.type.ObjectType; import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.BytesUtils; import org.apache.tsfile.utils.DateUtils; import java.time.LocalDate; import java.util.Iterator; import java.util.List; +import static org.apache.tsfile.read.common.type.BlobType.BLOB; + public class RecordIterator implements Iterator { private final List childrenColumns; @@ -70,7 +74,7 @@ private static class RecordImpl implements Record { private final List childrenColumns; private final List dataTypes; - private int index; + private final int index; private RecordImpl( List childrenColumns, @@ -113,10 +117,15 @@ public Binary getBinary(int columnIndex) { @Override public String getString(int columnIndex) { - return childrenColumns - .get(columnIndex) - .getBinary(index) - .getStringValue(TSFileConfig.STRING_CHARSET); + Binary binary = childrenColumns.get(columnIndex).getBinary(index); + org.apache.tsfile.read.common.type.Type type = dataTypes.get(columnIndex); + if (type == ObjectType.OBJECT) { + return BytesUtils.parseObjectByteArrayToString(binary.getValues()); + } else if (type == BLOB) { + return BytesUtils.parseBlobByteArrayToString(binary.getValues()); + } else { + return binary.getStringValue(TSFileConfig.STRING_CHARSET); + } } @Override From 25082b21bf58a9146bfff26969a801f59f2c9fcb Mon Sep 17 00:00:00 2001 From: JackieTien97 Date: Wed, 5 Nov 2025 18:41:42 +0800 Subject: [PATCH 027/180] disable object type in UDTF --- .../executor/ClusterConfigTaskExecutor.java | 21 +++++++++++++++++++ .../analyzer/StatementAnalyzer.java | 15 ++++++++----- 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index 8770ed2873ce0..22cc5dffc98da 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -324,6 +324,7 @@ import org.apache.iotdb.udf.api.relational.ScalarFunction; import org.apache.iotdb.udf.api.relational.TableFunction; import org.apache.iotdb.udf.api.relational.table.specification.ParameterSpecification; +import org.apache.iotdb.udf.api.relational.table.specification.ScalarParameterSpecification; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.SettableFuture; @@ -368,6 +369,7 @@ import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_RESULT_NODES; import static org.apache.iotdb.db.protocol.client.ConfigNodeClient.MSG_RECONNECTION_FAIL; import static org.apache.iotdb.db.utils.constant.SqlConstant.ROOT; +import static org.apache.iotdb.udf.api.type.Type.OBJECT; public class ClusterConfigTaskExecutor implements IConfigTaskExecutor { @@ -671,6 +673,16 @@ public SettableFuture createFunction( + "'.", TSStatusCode.UDF_LOAD_CLASS_ERROR.getStatusCode())); return future; + } else if (checkObjectScalarParameter(specification)) { + future.setException( + new IoTDBException( + "Failed to create function '" + + udfName + + "', because there is an argument with OBJECT type '" + + specification.getName() + + "'.", + TSStatusCode.UDF_LOAD_CLASS_ERROR.getStatusCode())); + return future; } } } @@ -714,6 +726,15 @@ public SettableFuture createFunction( return future; } + private boolean checkObjectScalarParameter(ParameterSpecification parameterSpecification) { + if (parameterSpecification instanceof ScalarParameterSpecification) { + ScalarParameterSpecification scalarParameterSpecification = + (ScalarParameterSpecification) parameterSpecification; + return scalarParameterSpecification.getType() == OBJECT; + } + return false; + } + @Override public SettableFuture dropFunction(Model model, String udfName) { SettableFuture future = SettableFuture.create(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java index 6a2b7eb34c165..01b216fe875da 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java @@ -221,6 +221,7 @@ import com.google.common.collect.Streams; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.read.common.type.BinaryType; +import org.apache.tsfile.read.common.type.ObjectType; import org.apache.tsfile.read.common.type.RowType; import org.apache.tsfile.read.common.type.StringType; import org.apache.tsfile.read.common.type.TimestampType; @@ -4753,11 +4754,15 @@ public Scope visitTableFunctionInvocation(TableFunctionInvocation node, Optional i -> i.getFields().stream() .map( - f -> - Field.newUnqualified( - f.getName(), - UDFDataTypeTransformer.transformUDFDataTypeToReadType(f.getType()), - TsTableColumnCategory.FIELD)) + f -> { + Type type = + UDFDataTypeTransformer.transformUDFDataTypeToReadType(f.getType()); + if (type == ObjectType.OBJECT) { + throw new SemanticException( + "OBJECT type is not supported as return type"); + } + return Field.newUnqualified(f.getName(), type, TsTableColumnCategory.FIELD); + }) .forEach(fields::add)); // next, columns derived from table arguments, in order of argument declarations From 391429b61926b44d65b05f98f3fccd0181253698 Mon Sep 17 00:00:00 2001 From: Haonan Date: Thu, 6 Nov 2025 19:32:36 +0800 Subject: [PATCH 028/180] [To force_ci/object_type] Support insert object by sql & add IT (#16683) --- .../it/query/old/IoTDBSimpleQueryTableIT.java | 63 +++++- .../it/session/IoTDBSessionRelationalIT.java | 183 ++++++++++++++++++ .../org/apache/iotdb/rpc/TSStatusCode.java | 1 + iotdb-core/datanode/pom.xml | 5 - .../dataregion/DataExecutionVisitor.java | 2 +- .../plan/node/write/InsertRowNode.java | 11 +- .../plan/node/write/InsertTabletNode.java | 2 +- .../planner/plan/node/write/ObjectNode.java | 25 --- .../node/write/RelationalInsertRowsNode.java | 45 ++++- .../write/RelationalInsertTabletNode.java | 2 - .../plan/relational/sql/util/AstUtil.java | 30 +++ .../FragmentInstanceDispatcherImpl.java | 2 +- .../execute/utils/CompactionUtils.java | 9 +- .../apache/iotdb/db/utils/TabletDecoder.java | 1 + .../CompactionDeleteObjectFileTest.java | 122 ------------ 15 files changed, 338 insertions(+), 165 deletions(-) delete mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionDeleteObjectFileTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java index e1571b001291d..d89c7a6b3e43b 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java @@ -657,19 +657,26 @@ public void testNewDataType() { statement.execute("CREATE DATABASE test"); statement.execute("USE " + DATABASE_NAME); statement.execute( - "CREATE TABLE table1(device STRING TAG, s4 DATE FIELD, s5 TIMESTAMP FIELD, s6 BLOB FIELD, s7 STRING FIELD)"); + "CREATE TABLE table1(device STRING TAG, " + + "s4 DATE FIELD, s5 TIMESTAMP FIELD, s6 BLOB FIELD, s7 STRING FIELD, s8 OBJECT FIELD)"); for (int i = 1; i <= 10; i++) { statement.execute( String.format( - "insert into table1(time, device, s4, s5, s6, s7) values(%d, 'd1', '%s', %d, %s, '%s')", - i, LocalDate.of(2024, 5, i % 31 + 1), i, "X'cafebabe'", i)); + "insert into table1(time, device, s4, s5, s6, s7, s8) " + + "values(%d, 'd1', '%s', %d, %s, '%s', %s)", + i, + LocalDate.of(2024, 5, i % 31 + 1), + i, + "X'cafebabe'", + i, + "to_object(true, 0, X'cafebabe')")); } try (ResultSet resultSet = statement.executeQuery("select * from table1")) { final ResultSetMetaData metaData = resultSet.getMetaData(); final int columnCount = metaData.getColumnCount(); - assertEquals(6, columnCount); + assertEquals(7, columnCount); HashMap columnType = new HashMap<>(); for (int i = 3; i <= columnCount; i++) { if (metaData.getColumnLabel(i).equals("s4")) { @@ -680,6 +687,8 @@ public void testNewDataType() { columnType.put(i, TSDataType.BLOB); } else if (metaData.getColumnLabel(i).equals("s7")) { columnType.put(i, TSDataType.TEXT); + } else if (metaData.getColumnLabel(i).equals("s8")) { + columnType.put(i, TSDataType.OBJECT); } } byte[] byteArray = new byte[] {(byte) 0xCA, (byte) 0xFE, (byte) 0xBA, (byte) 0xBE}; @@ -689,12 +698,58 @@ public void testNewDataType() { long timestamp = resultSet.getLong(4); byte[] blob = resultSet.getBytes(5); String text = resultSet.getString(6); + String objectSizeString = resultSet.getString(7); assertEquals(2024 - 1900, date.getYear()); assertEquals(5 - 1, date.getMonth()); assertEquals(time % 31 + 1, date.getDate()); assertEquals(time, timestamp); assertArrayEquals(byteArray, blob); assertEquals(String.valueOf(time), text); + assertEquals("(Object) 4 B", objectSizeString); + } + } + try (ResultSet resultSet = statement.executeQuery("select read_object(s8) from table1")) { + final ResultSetMetaData metaData = resultSet.getMetaData(); + final int columnCount = metaData.getColumnCount(); + assertEquals(1, columnCount); + byte[] byteArray = new byte[] {(byte) 0xCA, (byte) 0xFE, (byte) 0xBA, (byte) 0xBE}; + while (resultSet.next()) { + byte[] blob = resultSet.getBytes(1); + assertArrayEquals(byteArray, blob); + } + } + + } catch (SQLException e) { + fail(); + } + } + + @Test + public void testObjectDataType() { + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE test"); + statement.execute("USE " + DATABASE_NAME); + statement.execute("CREATE TABLE table1(device STRING TAG, s8 OBJECT FIELD)"); + statement.execute( + "insert into table1(time, device, s8) values(1, 'd1', to_object(false, 0, X'cafe'))"); + statement.execute( + "insert into table1(time, device, s8) values(1, 'd1', to_object(true, 2, X'babe'))"); + + try (ResultSet resultSet = statement.executeQuery("select * from table1")) { + while (resultSet.next()) { + String objectSizeString = resultSet.getString(3); + assertEquals("(Object) 4 B", objectSizeString); + } + } + try (ResultSet resultSet = statement.executeQuery("select read_object(s8) from table1")) { + final ResultSetMetaData metaData = resultSet.getMetaData(); + final int columnCount = metaData.getColumnCount(); + assertEquals(1, columnCount); + byte[] byteArray = new byte[] {(byte) 0xCA, (byte) 0xFE, (byte) 0xBA, (byte) 0xBE}; + while (resultSet.next()) { + byte[] blob = resultSet.getBytes(1); + assertArrayEquals(byteArray, blob); } } diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/session/IoTDBSessionRelationalIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/session/IoTDBSessionRelationalIT.java index 8f76adb3be815..bba5681b58bb0 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/session/IoTDBSessionRelationalIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/session/IoTDBSessionRelationalIT.java @@ -35,6 +35,7 @@ import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.StatementExecutionException; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.session.TableSessionBuilder; import org.apache.tsfile.enums.ColumnCategory; @@ -43,6 +44,7 @@ import org.apache.tsfile.file.metadata.TableSchema; import org.apache.tsfile.read.common.RowRecord; import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.BytesUtils; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.MeasurementSchema; @@ -60,6 +62,8 @@ import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; import java.time.LocalDate; import java.util.ArrayList; import java.util.Arrays; @@ -847,6 +851,183 @@ public void insertRelationalTabletWithCacheLeaderTest() } } + @Test + public void insertObjectTest() + throws IoTDBConnectionException, StatementExecutionException, IOException { + String testObject = + System.getProperty("user.dir") + + File.separator + + "target" + + File.separator + + "test-classes" + + File.separator + + "ainode-example" + + File.separator + + "model.pt"; + File object = new File(testObject); + + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + session.executeNonQueryStatement("USE \"db1\""); + // insert table data by tablet + List columnNameList = + Arrays.asList("region_id", "plant_id", "device_id", "temperature", "file"); + List dataTypeList = + Arrays.asList( + TSDataType.STRING, + TSDataType.STRING, + TSDataType.STRING, + TSDataType.FLOAT, + TSDataType.OBJECT); + List columnTypeList = + new ArrayList<>( + Arrays.asList( + ColumnCategory.TAG, + ColumnCategory.TAG, + ColumnCategory.TAG, + ColumnCategory.FIELD, + ColumnCategory.FIELD)); + Tablet tablet = new Tablet("object_table", columnNameList, dataTypeList, columnTypeList, 1); + int rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, 1); + tablet.addValue(rowIndex, 0, "1"); + tablet.addValue(rowIndex, 1, "5"); + tablet.addValue(rowIndex, 2, "3"); + tablet.addValue(rowIndex, 3, 37.6F); + tablet.addValue(rowIndex, 4, true, 0, Files.readAllBytes(Paths.get(testObject))); + session.insert(tablet); + tablet.reset(); + + try (SessionDataSet dataSet = + session.executeQueryStatement("select file from object_table where time = 1")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + Assert.assertEquals( + BytesUtils.parseObjectByteArrayToString(BytesUtils.longToBytes(object.length())), + iterator.getString(1)); + } + } + + try (SessionDataSet dataSet = + session.executeQueryStatement( + "select READ_OBJECT(file) from object_table where time = 1")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + Binary binary = iterator.getBlob(1); + Assert.assertArrayEquals(Files.readAllBytes(Paths.get(testObject)), binary.getValues()); + } + } + } + } + + @Test + public void insertObjectSegmentsTest() + throws IoTDBConnectionException, StatementExecutionException, IOException { + String testObject = + System.getProperty("user.dir") + + File.separator + + "target" + + File.separator + + "test-classes" + + File.separator + + "ainode-example" + + File.separator + + "model.pt"; + byte[] objectBytes = Files.readAllBytes(Paths.get(testObject)); + List objectSegments = new ArrayList<>(); + for (int i = 0; i < objectBytes.length; i += 512) { + objectSegments.add(Arrays.copyOfRange(objectBytes, i, Math.min(i + 512, objectBytes.length))); + } + + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + session.executeNonQueryStatement("USE \"db1\""); + // insert table data by tablet + List columnNameList = + Arrays.asList("region_id", "plant_id", "device_id", "temperature", "file"); + List dataTypeList = + Arrays.asList( + TSDataType.STRING, + TSDataType.STRING, + TSDataType.STRING, + TSDataType.FLOAT, + TSDataType.OBJECT); + List columnTypeList = + new ArrayList<>( + Arrays.asList( + ColumnCategory.TAG, + ColumnCategory.TAG, + ColumnCategory.TAG, + ColumnCategory.FIELD, + ColumnCategory.FIELD)); + Tablet tablet = new Tablet("object_table", columnNameList, dataTypeList, columnTypeList, 1); + for (int i = 0; i < objectSegments.size() - 1; i++) { + int rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, 1); + tablet.addValue(rowIndex, 0, "1"); + tablet.addValue(rowIndex, 1, "5"); + tablet.addValue(rowIndex, 2, "3"); + tablet.addValue(rowIndex, 3, 37.6F); + tablet.addValue(rowIndex, 4, false, i * 512L, objectSegments.get(i)); + session.insert(tablet); + tablet.reset(); + } + + try (SessionDataSet dataSet = + session.executeQueryStatement("select file from object_table where time = 1")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + assertNull(iterator.getString(1)); + } + } + + // insert segment with wrong offset + try { + int rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, 1); + tablet.addValue(rowIndex, 0, "1"); + tablet.addValue(rowIndex, 1, "5"); + tablet.addValue(rowIndex, 2, "3"); + tablet.addValue(rowIndex, 3, 37.6F); + tablet.addValue(rowIndex, 4, false, 512L, objectSegments.get(1)); + session.insert(tablet); + } catch (StatementExecutionException e) { + Assert.assertEquals(TSStatusCode.OBJECT_INSERT_ERROR.getStatusCode(), e.getStatusCode()); + Assert.assertEquals( + String.format( + "741: The file length %d is not equal to the offset %d", + ((objectSegments.size() - 1) * 512), 512L), + e.getMessage()); + } finally { + tablet.reset(); + } + + // last segment + int rowIndex = tablet.getRowSize(); + tablet.addTimestamp(rowIndex, 1); + tablet.addValue(rowIndex, 0, "1"); + tablet.addValue(rowIndex, 1, "5"); + tablet.addValue(rowIndex, 2, "3"); + tablet.addValue(rowIndex, 3, 37.6F); + tablet.addValue( + rowIndex, + 4, + true, + (objectSegments.size() - 1) * 512L, + objectSegments.get(objectSegments.size() - 1)); + session.insert(tablet); + tablet.reset(); + + try (SessionDataSet dataSet = + session.executeQueryStatement( + "select READ_OBJECT(file) from object_table where time = 1")) { + SessionDataSet.DataIterator iterator = dataSet.iterator(); + while (iterator.next()) { + Binary binary = iterator.getBlob(1); + Assert.assertArrayEquals(Files.readAllBytes(Paths.get(testObject)), binary.getValues()); + } + } + } + } + @Test public void autoCreateNontagColumnTest() throws IoTDBConnectionException, StatementExecutionException { @@ -1628,6 +1809,7 @@ public void insertRelationalTabletWithAutoCastTest() int testNum = 14; Set dataTypes = new HashSet<>(); Collections.addAll(dataTypes, TSDataType.values()); + dataTypes.remove(TSDataType.OBJECT); dataTypes.remove(TSDataType.VECTOR); dataTypes.remove(TSDataType.UNKNOWN); @@ -1718,6 +1900,7 @@ public void insertRelationalRowWithAutoCastTest() int testNum = 17; Set dataTypes = new HashSet<>(); Collections.addAll(dataTypes, TSDataType.values()); + dataTypes.remove(TSDataType.OBJECT); dataTypes.remove(TSDataType.VECTOR); dataTypes.remove(TSDataType.UNKNOWN); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index c9ab41e90908e..84534adfadbed 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -148,6 +148,7 @@ public enum TSStatusCode { // OBJECT OBJECT_NOT_EXISTS(740), + OBJECT_INSERT_ERROR(741), // Arithmetic NUMERIC_VALUE_OUT_OF_RANGE(750), diff --git a/iotdb-core/datanode/pom.xml b/iotdb-core/datanode/pom.xml index e8409508a035d..626d8265eed2e 100644 --- a/iotdb-core/datanode/pom.xml +++ b/iotdb-core/datanode/pom.xml @@ -371,11 +371,6 @@ 1.3.0 test - - org.gdal - gdal - 3.11.0 - diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java index 643f62830c293..0ae796f3913b8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java @@ -302,7 +302,7 @@ public TSStatus visitWriteObjectFile(ObjectNode node, DataRegion dataRegion) { return StatusUtils.OK; } catch (final Exception e) { LOGGER.error("Error in executing plan node: {}", node, e); - return new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + return RpcUtils.getStatus(TSStatusCode.OBJECT_INSERT_ERROR.getStatusCode(), e.getMessage()); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java index 04bef0b577c3b..760d36f1f5cd1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java @@ -367,6 +367,7 @@ private void putDataTypesAndValues(ByteBuffer buffer) { case TEXT: case STRING: case BLOB: + case OBJECT: ReadWriteIOUtils.write((Binary) values[i], buffer); break; default: @@ -426,6 +427,7 @@ private void putDataTypesAndValues(DataOutputStream stream) throws IOException { case TEXT: case STRING: case BLOB: + case OBJECT: ReadWriteIOUtils.write((Binary) values[i], stream); break; default: @@ -520,6 +522,7 @@ private void fillDataTypesAndValues(ByteBuffer buffer) { case TEXT: case STRING: case BLOB: + case OBJECT: values[i] = ReadWriteIOUtils.readBinary(buffer); break; default: @@ -589,6 +592,7 @@ private int serializeMeasurementsAndValuesSize() { case TEXT: case STRING: case BLOB: + case OBJECT: size += ReadWriteIOUtils.sizeToWrite((Binary) values[i]); break; default: @@ -668,6 +672,7 @@ private void putDataTypesAndValues(IWALByteBufferView buffer) { case TEXT: case BLOB: case STRING: + case OBJECT: WALWriteUtils.write((Binary) values[i], buffer); break; default: @@ -759,6 +764,7 @@ public void fillDataTypesAndValuesFromWAL(DataInputStream stream) throws IOExcep case TEXT: case STRING: case BLOB: + case OBJECT: values[i] = ReadWriteIOUtils.readBinary(stream); break; default: @@ -849,6 +855,7 @@ public void fillDataTypesAndValuesFromWAL(ByteBuffer buffer) { case TEXT: case STRING: case BLOB: + case OBJECT: values[i] = ReadWriteIOUtils.readBinary(buffer); break; default: @@ -889,7 +896,9 @@ public R accept(PlanVisitor visitor, C context) { } public TimeValuePair composeTimeValuePair(int columnIndex) { - if (columnIndex >= values.length || Objects.isNull(dataTypes[columnIndex])) { + if (columnIndex >= values.length + || Objects.isNull(dataTypes[columnIndex]) + || dataTypes[columnIndex] == TSDataType.OBJECT) { return null; } Object value = values[columnIndex]; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java index 52386b7e077d1..3d3c25c0613a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java @@ -106,7 +106,7 @@ public boolean shouldCheckTTL() { } shouldCheckTTL = true; for (MeasurementSchema measurementSchema : measurementSchemas) { - if (measurementSchema.getType() == TSDataType.OBJECT) { + if (measurementSchema != null && measurementSchema.getType() == TSDataType.OBJECT) { shouldCheckTTL = false; break; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ObjectNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ObjectNode.java index ebe839bbc0c6d..10fb9bc3443a2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ObjectNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ObjectNode.java @@ -68,10 +68,6 @@ public class ObjectNode extends SearchNode implements WALEntryValue { private boolean isGeneratedByRemoteConsensusLeader; - private Long time = null; - - private String table = null; - public ObjectNode(boolean isEOF, long offset, byte[] content, String filePath) { super(new PlanNodeId("")); this.isEOF = isEOF; @@ -89,27 +85,6 @@ public ObjectNode(boolean isEOF, long offset, int contentLength, String filePath this.contentLength = contentLength; } - public long getTimestamp() { - calculateTimeAndTableName(); - return time; - } - - public String getTable() { - calculateTimeAndTableName(); - return table; - } - - private void calculateTimeAndTableName() { - if (time != null && table != null) { - return; - } - File file = new File(filePath); - String fileName = new File(filePath).getName(); - String timeStr = fileName.substring(0, fileName.length() - ".bin".length()); - time = Long.parseLong(timeStr); - table = file.getParentFile().getParentFile().getParentFile().getParentFile().getName(); - } - public boolean isEOF() { return isEOF; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertRowsNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertRowsNode.java index 77020d9220dc1..83f6bbec63e09 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertRowsNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertRowsNode.java @@ -27,13 +27,19 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.generator.TsFileNameGenerator; +import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.IDeviceID.Factory; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.BytesUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataInputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -159,6 +165,7 @@ public String getTableName() { @Override public List splitByPartition(IAnalysis analysis) { + List writePlanNodeList = new ArrayList<>(); Map splitMap = new HashMap<>(); List redirectInfo = new ArrayList<>(); for (int i = 0; i < getInsertRowNodeList().size(); i++) { @@ -172,6 +179,9 @@ public List splitByPartition(IAnalysis analysis) { insertRowNode.getDeviceID(), TimePartitionUtils.getTimePartitionSlot(insertRowNode.getTime()), analysis.getDatabaseName()); + // handle object type + handleObjectValue(insertRowNode, dataRegionReplicaSet, writePlanNodeList); + // Collect redirectInfo redirectInfo.add(dataRegionReplicaSet.getDataNodeLocations().get(0).getClientRpcEndPoint()); RelationalInsertRowsNode tmpNode = splitMap.get(dataRegionReplicaSet); @@ -185,8 +195,41 @@ public List splitByPartition(IAnalysis analysis) { } } analysis.setRedirectNodeList(redirectInfo); + writePlanNodeList.addAll(splitMap.values()); + + return writePlanNodeList; + } - return new ArrayList<>(splitMap.values()); + private void handleObjectValue( + InsertRowNode insertRowNode, + TRegionReplicaSet dataRegionReplicaSet, + List writePlanNodeList) { + for (int j = 0; j < insertRowNode.getDataTypes().length; j++) { + if (insertRowNode.getDataTypes()[j] == TSDataType.OBJECT) { + Object[] values = insertRowNode.getValues(); + byte[] binary = ((Binary) values[j]).getValues(); + ByteBuffer buffer = ByteBuffer.wrap(binary); + boolean isEoF = buffer.get() == 1; + long offset = buffer.getLong(); + byte[] content = ReadWriteIOUtils.readBytes(buffer, buffer.remaining()); + String relativePath = + TsFileNameGenerator.generateObjectFilePath( + dataRegionReplicaSet.getRegionId().getId(), + insertRowNode.getTime(), + insertRowNode.getDeviceID(), + insertRowNode.getMeasurements()[j]); + ObjectNode objectNode = new ObjectNode(isEoF, offset, content, relativePath); + objectNode.setDataRegionReplicaSet(dataRegionReplicaSet); + byte[] filePathBytes = relativePath.getBytes(StandardCharsets.UTF_8); + byte[] valueBytes = new byte[filePathBytes.length + Long.BYTES]; + System.arraycopy( + BytesUtils.longToBytes(offset + content.length), 0, valueBytes, 0, Long.BYTES); + System.arraycopy(filePathBytes, 0, valueBytes, Long.BYTES, filePathBytes.length); + ((Binary) values[j]).setValues(valueBytes); + insertRowNode.setValues(values); + writePlanNodeList.add(objectNode); + } + } } public RelationalInsertRowsNode emptyClone() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java index ee93712e77d02..e3a114211e16e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java @@ -64,8 +64,6 @@ public class RelationalInsertTabletNode extends InsertTabletNode { private boolean singleDevice; - private Object[] convertedColumns; - public RelationalInsertTabletNode( PlanNodeId id, PartialPath devicePath, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/util/AstUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/util/AstUtil.java index cb17a90c04f3c..700d5e40beb89 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/util/AstUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/util/AstUtil.java @@ -21,16 +21,22 @@ import org.apache.iotdb.commons.utils.CommonDateTimeUtils; 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.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; 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.Node; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NullLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StringLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.TableExpressionType; import com.google.common.graph.SuccessorsFunction; import com.google.common.graph.Traverser; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.BytesUtils; import java.time.ZoneId; import java.util.List; @@ -113,6 +119,30 @@ public static Object expressionToTsValue(Expression expression) { throw new SemanticException( String.format("Cannot insert identifier %s, please use string literal", expression)); } + if (expression instanceof FunctionCall + && "to_object".equals(((FunctionCall) expression).getName().toString())) { + List arguments = ((FunctionCall) expression).getArguments(); + if (arguments.size() == 3 + && arguments.get(0).getExpressionType() == TableExpressionType.BOOLEAN_LITERAL + && arguments.get(1).getExpressionType() == TableExpressionType.LONG_LITERAL + && arguments.get(2).getExpressionType() == TableExpressionType.BINARY_LITERAL) { + boolean isEOF = + (boolean) + ((BooleanLiteral) ((FunctionCall) expression).getArguments().get(0)).getTsValue(); + long offset = + (long) ((LongLiteral) ((FunctionCall) expression).getArguments().get(1)).getTsValue(); + byte[] content = + ((Binary) + ((BinaryLiteral) ((FunctionCall) expression).getArguments().get(2)) + .getTsValue()) + .getValues(); + byte[] val = new byte[content.length + 9]; + val[0] = (byte) (isEOF ? 1 : 0); + System.arraycopy(BytesUtils.longToBytes(offset), 0, val, 1, 8); + System.arraycopy(content, 0, val, 9, content.length); + return new Binary(val); + } + } throw new SemanticException("Unsupported expression: " + expression); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java index 7e61bea8a4a87..18ab1cdc27544 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java @@ -328,7 +328,7 @@ private Future dispatchWrite(List if (dispatchFailures.isEmpty()) { return immediateFuture(new FragInstanceDispatchResult(true)); } - if (instances.size() == 1) { + if (instances.size() == 1 || dispatchFailures.size() == 1) { return immediateFuture(new FragInstanceDispatchResult(dispatchFailures.get(0))); } else { List failureStatusList = new ArrayList<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java index 1668355db311e..b057583782848 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java @@ -512,14 +512,19 @@ public static ModEntry convertTtlToDeletion(IDeviceID deviceID, long timeLowerBo } } - public static void removeDeletedObjectFiles(TsFileResource resource) - throws IOException, IllegalPathException { + public static void removeDeletedObjectFiles(TsFileResource resource) { + // check for compaction recovery + if (!resource.tsFileExists()) { + return; + } try (MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(Collections.singletonList(resource))) { while (deviceIterator.hasNextDevice()) { deviceIterator.nextDevice(); deviceIterator.getReaderAndChunkMetadataForCurrentAlignedSeries(); } + } catch (Exception e) { + logger.warn("Failed to remove object files from file {}", resource.getTsFilePath(), e); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TabletDecoder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TabletDecoder.java index 335d7a23b42a6..bd1eba9cfa8fd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TabletDecoder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TabletDecoder.java @@ -192,6 +192,7 @@ private Object decodeColumn(ByteBuffer uncompressed, int columnIndex) { case STRING: case BLOB: case TEXT: + case OBJECT: Binary[] binaryCol = new Binary[rowSize]; if (encoding == TSEncoding.PLAIN) { // PlainEncoder uses var int, which may cause compatibility problem diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionDeleteObjectFileTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionDeleteObjectFileTest.java deleted file mode 100644 index a4b35c01da6d8..0000000000000 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionDeleteObjectFileTest.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.storageengine.dataregion.compaction; - -import org.apache.iotdb.commons.exception.MetadataException; -import org.apache.iotdb.commons.schema.table.TsTable; -import org.apache.iotdb.commons.schema.table.column.FieldColumnSchema; -import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; -import org.apache.iotdb.db.exception.StorageEngineException; -import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; -import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer; -import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.SettleCompactionTask; -import org.apache.iotdb.db.storageengine.dataregion.modification.DeletionPredicate; -import org.apache.iotdb.db.storageengine.dataregion.modification.IDPredicate; -import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; -import org.apache.iotdb.db.storageengine.dataregion.modification.TableDeletionEntry; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; - -import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.exception.write.WriteProcessException; -import org.apache.tsfile.file.metadata.StringArrayDeviceID; -import org.apache.tsfile.file.metadata.enums.CompressionType; -import org.apache.tsfile.file.metadata.enums.TSEncoding; -import org.apache.tsfile.read.common.TimeRange; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -public class CompactionDeleteObjectFileTest extends AbstractCompactionTest { - @Before - public void setUp() - throws IOException, WriteProcessException, MetadataException, InterruptedException { - super.setUp(); - } - - @After - public void tearDown() throws IOException, StorageEngineException { - super.tearDown(); - } - - @Test - public void test1() throws IOException { - createTable("tsfile_table", 100); - File dir = new File("/Users/shuww/Downloads/0708/1_副本"); - List resources = new ArrayList<>(); - for (File file : dir.listFiles()) { - if (!file.getName().endsWith(".tsfile")) { - continue; - } - TsFileResource resource = new TsFileResource(file); - - try (ModificationFile modificationFile = resource.getExclusiveModFile()) { - modificationFile.write( - new TableDeletionEntry( - new DeletionPredicate( - "tsfile_table", - new IDPredicate.FullExactMatch( - new StringArrayDeviceID(new String[] {"tsfile_table", "1", "5", "3"})), - Arrays.asList("file")), - new TimeRange(-1, 0))); - modificationFile.write( - new TableDeletionEntry( - new DeletionPredicate( - "tsfile_table", - new IDPredicate.FullExactMatch( - new StringArrayDeviceID(new String[] {"tsfile_table", "1", "5", "3"})), - Arrays.asList("file")), - new TimeRange(2, 2))); - } - resource.deserialize(); - resources.add(resource); - } - - // InnerSpaceCompactionTask task = - // new InnerSpaceCompactionTask( - // 0, tsFileManager, resources, true, new ReadChunkCompactionPerformer(), 0); - SettleCompactionTask task = - new SettleCompactionTask( - 0, - tsFileManager, - resources, - Collections.emptyList(), - true, - new FastCompactionPerformer(false), - 0); - task.start(); - } - - public void createTable(String tableName, long ttl) { - TsTable tsTable = new TsTable(tableName); - tsTable.addColumnSchema(new TagColumnSchema("id_column", TSDataType.STRING)); - tsTable.addColumnSchema( - new FieldColumnSchema("s1", TSDataType.STRING, TSEncoding.PLAIN, CompressionType.LZ4)); - tsTable.addProp(TsTable.TTL_PROPERTY, ttl + ""); - DataNodeTableCache.getInstance().preUpdateTable("Downloads", tsTable, null); - DataNodeTableCache.getInstance().commitUpdateTable("Downloads", tableName, null); - } -} From 0fc8e176e72f2341f82cf69750dc22b15d5cee15 Mon Sep 17 00:00:00 2001 From: Jackie Tien Date: Tue, 4 Nov 2025 16:40:50 +0800 Subject: [PATCH 029/180] Fix wrong push limit down to AggTableScanNode (#16696) (cherry picked from commit 8420beaaf5470b695c5d2afcb7491d226a7a8202) --- .../it/query/recent/IoTDBTableAggregationIT.java | 13 +++++++++++++ .../distribute/TableDistributedPlanGenerator.java | 2 +- .../iterative/rule/PushDownOffsetIntoTableScan.java | 4 +++- .../planner/node/AggregationTableScanNode.java | 10 ++++++++++ .../optimizations/PushLimitOffsetIntoTableScan.java | 2 +- 5 files changed, 28 insertions(+), 3 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableAggregationIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableAggregationIT.java index 0143bdb6e3b56..88367283eb761 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableAggregationIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableAggregationIT.java @@ -5491,4 +5491,17 @@ public void emptyTimeRangeQueryTest() { retArray, DATABASE_NAME); } + + @Test + public void orderByLimitTest() { + String[] expectedHeader = + new String[] {"province", "city", "region", "device_id", "_col4", "_col5"}; + String[] retArray = new String[] {"beijing,beijing,chaoyang,d09,2024-09-24T06:00:00.000Z,2,"}; + + tableResultSetEqualTest( + "select province, city, region, device_id, date_bin(1h, time), count(s1) from table1 where s1 >= 40 group by 1,2,3,4,5 order by province, city, region, device_id, date_bin(1h, time) limit 1", + expectedHeader, + retArray, + DATABASE_NAME); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java index 850ef0a81e31f..069d1df746958 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java @@ -423,7 +423,7 @@ public List visitTopK(TopKNode node, PlanContext context) { private boolean canTopKEliminated(OrderingScheme orderingScheme, long k, PlanNode child) { // if DeviceTableScanNode has limit <= K and with same order, we can directly return // DeviceTableScanNode - if (child instanceof DeviceTableScanNode) { + if (child instanceof DeviceTableScanNode && !(child instanceof AggregationTableScanNode)) { DeviceTableScanNode tableScanNode = (DeviceTableScanNode) child; if (canSortEliminated(orderingScheme, nodeOrderingMap.get(child.getPlanNodeId()))) { if (tableScanNode.getPushDownLimit() <= 0) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushDownOffsetIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushDownOffsetIntoTableScan.java index d683dbe3fdb80..77143f9037428 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushDownOffsetIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushDownOffsetIntoTableScan.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.OffsetNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode; @@ -53,7 +54,8 @@ public Pattern getPattern() { @Override public Result apply(OffsetNode parent, Captures captures, Context context) { TableScanNode tableScanNode = captures.get(CHILD); - if (tableScanNode instanceof DeviceTableScanNode + if ((tableScanNode instanceof DeviceTableScanNode + && !(tableScanNode instanceof AggregationTableScanNode)) && !((DeviceTableScanNode) tableScanNode).isPushLimitToEachDevice()) { tableScanNode.setPushDownOffset(parent.getCount()); // consider case that there is no limit diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/AggregationTableScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/AggregationTableScanNode.java index b6840d7200a2a..56d39f2f77dcb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/AggregationTableScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/AggregationTableScanNode.java @@ -623,4 +623,14 @@ public void setDeviceCountMap(Map deviceCountMap) { public Map getDeviceCountMap() { return deviceCountMap; } + + @Override + public void setPushDownLimit(long pushDownLimit) { + throw new IllegalStateException("Should never push down limit to AggregationTableScanNode."); + } + + @Override + public void setPushDownOffset(long pushDownOffset) { + throw new IllegalStateException("Should never push down offset to AggregationTableScanNode."); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java index 2993c8f3696e6..2bd9b778aec2e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java @@ -50,7 +50,7 @@ import static org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.PushPredicateIntoTableScan.containsDiffFunction; /** - * Optimization phase: Distributed plan planning. + * Optimization phase: Logical plan planning. * *

The LIMIT OFFSET condition can be pushed down to the DeviceTableScanNode, when the following * conditions are met: From 4dbb3c620bcb25775e0fffc1e74b26b7f3e0e044 Mon Sep 17 00:00:00 2001 From: Yongzao Date: Tue, 4 Nov 2025 17:19:47 +0800 Subject: [PATCH 030/180] More robust IoTDBRegionReconstructForIoTV1IT (#16697) (cherry picked from commit b6c6c7a11387fdc5c5a1037f4e0e5b641a42fdd4) --- .../pass/commit/IoTDBRegionReconstructForIoTV1IT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/regionmigration/pass/commit/IoTDBRegionReconstructForIoTV1IT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/regionmigration/pass/commit/IoTDBRegionReconstructForIoTV1IT.java index 42dee93d3318e..3dc4468e52691 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/regionmigration/pass/commit/IoTDBRegionReconstructForIoTV1IT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/regionmigration/pass/commit/IoTDBRegionReconstructForIoTV1IT.java @@ -199,6 +199,7 @@ public void normal1C3DTest() throws Exception { if (System.currentTimeMillis() - start > 60_000L) { fail("Cannot execute query within 60s"); } + TimeUnit.SECONDS.sleep(1); continue; } if (resultSet.hasNext()) { From 1bb15b2254ac9a55c74dfd628e1017dac7993702 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 6 Nov 2025 20:40:50 +0800 Subject: [PATCH 031/180] Implemented the alter encoding compression function for tree model (#16672) * ifpermitted * reconstruct * final-prev * complete-dn * partial * very-partial * fix * partial-set * fix * test * fix * shop * fix * some * partial * bishop * fix * fix * grasia * fix * main * fix * partial * fix * minor * fix * fix * spotless * test * part * fix * bug-fix * fix * Revert "fix" This reverts commit da7a080906737af14c6022294935e963bae88ad2. * Reapply "fix" This reverts commit 27de58a8ffb335b85e617c60cbbc6e753f7141e4. (cherry picked from commit f096b8210ff2fa04566b9667ba5509bf7a344816) --- .../IoTDBAlterEncodingCompressorIT.java | 187 ++++++++++ .../manual/IoTDBPipeInclusionIT.java | 9 +- .../apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 | 6 +- .../org/apache/iotdb/db/qp/sql/SqlLexer.g4 | 4 + .../client/async/CnToDnAsyncRequestType.java | 2 + ...oDnInternalServiceAsyncRequestManager.java | 6 + .../rpc/DataNodeAsyncRequestRPCHandler.java | 1 + .../consensus/request/ConfigPhysicalPlan.java | 4 + .../request/ConfigPhysicalPlanType.java | 1 + .../request/ConfigPhysicalPlanVisitor.java | 8 + .../PipeAlterEncodingCompressorPlan.java | 113 ++++++ .../confignode/manager/ConfigManager.java | 18 + .../iotdb/confignode/manager/IManager.java | 3 + .../confignode/manager/ProcedureManager.java | 41 +++ .../protocol/IoTDBConfigNodeReceiver.java | 56 ++- ...PipeConfigPhysicalPlanTSStatusVisitor.java | 11 + .../IoTDBConfigNodeSyncClientManager.java | 6 +- .../protocol/IoTDBConfigRegionAirGapSink.java | 2 + .../sink/protocol/IoTDBConfigRegionSink.java | 6 +- .../source/ConfigRegionListeningFilter.java | 3 + ...igPhysicalPlanTreePatternParseVisitor.java | 25 ++ .../executor/ConfigPlanExecutor.java | 1 + .../AlterEncodingCompressorProcedure.java | 331 ++++++++++++++++++ .../schema/DataNodeTSStatusTaskExecutor.java | 74 ++++ .../schema/DeactivateTemplateProcedure.java | 26 +- .../schema/DeleteLogicalViewProcedure.java | 26 +- .../schema/DeleteTimeSeriesProcedure.java | 46 +-- .../AbstractAlterOrDropTableProcedure.java | 29 +- .../state/AlterEncodingCompressorState.java | 25 ++ .../procedure/store/ProcedureFactory.java | 9 + .../procedure/store/ProcedureType.java | 2 + .../thrift/ConfigNodeRPCServiceProcessor.java | 6 + .../receiver/PipeEnrichedProcedureTest.java | 28 ++ .../AlterEncodingCompressorProcedureTest.java | 62 ++++ .../schemaregion/SchemaExecutionVisitor.java | 15 + .../IoTDBDataNodeAsyncClientManager.java | 9 +- .../IoTDBDataNodeSyncClientManager.java | 6 +- .../airgap/IoTDBDataNodeAirGapSink.java | 2 + .../async/IoTDBDataRegionAsyncSink.java | 6 +- .../thrift/sync/IoTDBDataNodeSyncSink.java | 6 +- .../db/protocol/client/ConfigNodeClient.java | 7 + .../impl/DataNodeInternalRPCServiceImpl.java | 36 +- .../config/TreeConfigTaskVisitor.java | 14 +- .../executor/ClusterConfigTaskExecutor.java | 57 +++ .../config/executor/IConfigTaskExecutor.java | 4 + .../metadata/AlterEncodingCompressorTask.java | 47 +++ .../queryengine/plan/parser/ASTVisitor.java | 68 +++- .../plan/planner/plan/node/PlanNodeType.java | 4 + .../plan/planner/plan/node/PlanVisitor.java | 5 + .../write/AlterEncodingCompressorNode.java | 146 ++++++++ .../security/TreeAccessCheckVisitor.java | 59 +++- .../plan/statement/StatementType.java | 1 + .../plan/statement/StatementVisitor.java | 10 +- .../AlterEncodingCompressorStatement.java | 126 +++++++ .../metadata/DeleteDatabaseStatement.java | 2 +- .../metadata/ShowDatabaseStatement.java | 2 +- .../schemaregion/ISchemaRegion.java | 3 + .../schemaregion/SchemaRegionPlanType.java | 2 + .../schemaregion/SchemaRegionPlanVisitor.java | 6 + .../impl/SchemaRegionMemoryImpl.java | 37 +- .../impl/SchemaRegionPBTreeImpl.java | 8 + .../visitor/SchemaRegionPlanDeserializer.java | 7 + .../visitor/SchemaRegionPlanSerializer.java | 8 + .../impl/mem/MTreeBelowSGMemoryImpl.java | 33 ++ .../req/SchemaRegionWritePlanFactory.java | 3 + .../commons/exception/MetadataException.java | 2 +- .../iotdb/commons/path/PathPatternNode.java | 7 + .../iotdb/commons/path/PathPatternTree.java | 13 +- .../pipe/receiver/IoTDBFileReceiver.java | 6 + .../pipe/sink/client/IoTDBClientManager.java | 5 +- .../sink/client/IoTDBSyncClientManager.java | 9 +- .../common/PipeTransferHandshakeConstant.java | 1 + .../commons/pipe/sink/protocol/IoTDBSink.java | 3 +- .../pipe/sink/protocol/IoTDBSslSyncSink.java | 6 +- .../iotdb/commons/utils/SerializeUtils.java | 18 + .../src/main/thrift/confignode.thrift | 12 + .../src/main/thrift/datanode.thrift | 13 + 77 files changed, 1848 insertions(+), 163 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAlterEncodingCompressorIT.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeAlterEncodingCompressorPlan.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeTSStatusTaskExecutor.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/AlterEncodingCompressorState.java create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedureTest.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/AlterEncodingCompressorTask.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/AlterEncodingCompressorNode.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/AlterEncodingCompressorStatement.java diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAlterEncodingCompressorIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAlterEncodingCompressorIT.java new file mode 100644 index 0000000000000..42c0275ab35b2 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAlterEncodingCompressorIT.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.it.schema; + +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.itbase.category.ClusterIT; +import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.apache.iotdb.util.AbstractSchemaIT; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runners.Parameterized; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; + +import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + +@Category({LocalStandaloneIT.class, ClusterIT.class}) +public class IoTDBAlterEncodingCompressorIT extends AbstractSchemaIT { + + public IoTDBAlterEncodingCompressorIT(SchemaTestMode schemaTestMode) { + super(schemaTestMode); + } + + @Parameterized.BeforeParam + public static void before() throws Exception { + setUpEnvironment(); + EnvFactory.getEnv().initClusterEnvironment(); + } + + @Parameterized.AfterParam + public static void after() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + tearDownEnvironment(); + } + + @After + public void tearDown() throws Exception { + clearSchema(); + } + + @Test + public void alterEncodingAndCompressorTest() throws Exception { + if (schemaTestMode.equals(SchemaTestMode.PBTree)) { + return; + } + try (final Connection connection = EnvFactory.getEnv().getConnection(); + final Statement statement = connection.createStatement()) { + statement.execute("create timeSeries root.vehicle.wind.a int32"); + + try { + statement.execute("alter timeSeries root.nonExist.** set encoding=PLAIN"); + fail(); + } catch (final SQLException e) { + Assert.assertEquals( + "508: Timeseries [root.nonExist.**] does not exist or is represented by device template", + e.getMessage()); + } + + try { + statement.execute("alter timeSeries if exists root.nonExist.** set encoding=PLAIN"); + } catch (final SQLException e) { + fail( + "Alter encoding & compressor shall not fail when timeSeries not exists if set if exists"); + } + + try { + statement.execute("alter timeSeries if exists root.vehicle.** set encoding=aaa"); + fail(); + } catch (final SQLException e) { + Assert.assertEquals("701: Unsupported encoding: AAA", e.getMessage()); + } + + try { + statement.execute("alter timeSeries if exists root.vehicle.** set compressor=aaa"); + fail(); + } catch (final SQLException e) { + Assert.assertEquals("701: Unsupported compressor: AAA", e.getMessage()); + } + + try { + statement.execute("alter timeSeries if exists root.vehicle.** set falseKey=aaa"); + fail(); + } catch (final SQLException e) { + Assert.assertEquals("701: property falsekey is unsupported yet.", e.getMessage()); + } + + try { + statement.execute("alter timeSeries if exists root.vehicle.** set encoding=DICTIONARY"); + fail(); + } catch (final SQLException e) { + Assert.assertTrue(e.getMessage().contains("encoding DICTIONARY does not support INT32")); + } + + statement.execute("alter timeSeries root.** set encoding=Plain, compressor=LZMA2"); + + try (final ResultSet resultSet = statement.executeQuery("SHOW TIMESERIES")) { + while (resultSet.next()) { + assertEquals("PLAIN", resultSet.getString(5)); + assertEquals("LZMA2", resultSet.getString(6)); + } + } + + statement.execute("create user IoTDBUser '!@#$!dfdfzvd343'"); + statement.execute("grant write on root.vehicle.wind.a to user IoTDBUser"); + statement.execute("create timeSeries root.vehicle.wind.b int32"); + } + + try (final Connection connection = + EnvFactory.getEnv().getConnection("IoTDBUser", "!@#$!dfdfzvd343"); + final Statement statement = connection.createStatement()) { + try { + statement.execute("alter timeSeries root.vechile.** set encoding=PLAIN, compressor=LZMA2"); + fail(); + } catch (final SQLException e) { + Assert.assertEquals( + "803: No permissions for this operation, please add privilege WRITE_SCHEMA on [root.vechile.**]", + e.getMessage()); + } + + try { + statement.execute( + "alter timeSeries root.vechile.wind.a, root.__audit.** set encoding=PLAIN, compressor=LZMA2"); + fail(); + } catch (final SQLException e) { + Assert.assertEquals( + "803: 'AUDIT' permission is needed to alter the encoding and compressor of database root.__audit", + e.getMessage()); + } + + try { + statement.execute( + "alter timeSeries if permitted root.vehicle.**, root.__audit.** set encoding=GORILLA, compressor=GZIP"); + } catch (final SQLException e) { + fail("Alter encoding & compressor shall not fail when no privileges if set if permitted"); + } + } + + try (final Connection connection = EnvFactory.getEnv().getConnection(); + final Statement statement = connection.createStatement()) { + try (final ResultSet resultSet = + statement.executeQuery("SHOW TIMESERIES root.__audit.**._0.password")) { + while (resultSet.next()) { + assertEquals("PLAIN", resultSet.getString(5)); + assertEquals("LZMA2", resultSet.getString(6)); + } + } + + try (final ResultSet resultSet = + statement.executeQuery("SHOW TIMESERIES root.vehicle.wind.b")) { + resultSet.next(); + assertEquals("TS_2DIFF", resultSet.getString(5)); + assertEquals("LZ4", resultSet.getString(6)); + } + + try (final ResultSet resultSet = + statement.executeQuery("SHOW TIMESERIES root.vehicle.wind.a")) { + resultSet.next(); + assertEquals("GORILLA", resultSet.getString(5)); + assertEquals("GZIP", resultSet.getString(6)); + } + } + } +} diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java index 84c90a667eecc..4778f1f9b9bfd 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java @@ -79,9 +79,10 @@ public void testPureSchemaInclusion() throws Exception { Arrays.asList( // TODO: add database creation after the database auto creating on receiver can be // banned - "create timeseries root.ln.wf01.wt01.status with datatype=BOOLEAN,encoding=PLAIN", - "ALTER timeseries root.ln.wf01.wt01.status ADD TAGS tag3=v3", - "ALTER timeseries root.ln.wf01.wt01.status ADD ATTRIBUTES attr4=v4"), + "create timeSeries root.ln.wf01.wt01.status with datatype=BOOLEAN,encoding=PLAIN", + "ALTER timeSeries root.ln.wf01.wt01.status ADD TAGS tag3=v3", + "ALTER timeSeries root.ln.wf01.wt01.status ADD ATTRIBUTES attr4=v4", + "ALTER timeSeries root.** set compressor=ZSTD"), null); TestUtils.assertDataEventuallyOnEnv( @@ -89,7 +90,7 @@ public void testPureSchemaInclusion() throws Exception { "show timeseries root.ln.**", "Timeseries,Alias,Database,DataType,Encoding,Compression,Tags,Attributes,Deadband,DeadbandParameters,ViewType,", Collections.singleton( - "root.ln.wf01.wt01.status,null,root.ln,BOOLEAN,PLAIN,LZ4,{\"tag3\":\"v3\"},{\"attr4\":\"v4\"},null,null,BASE,")); + "root.ln.wf01.wt01.status,null,root.ln,BOOLEAN,PLAIN,ZSTD,{\"tag3\":\"v3\"},{\"attr4\":\"v4\"},null,null,BASE,")); TestUtils.executeNonQueries( senderEnv, diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 index 58f77da6c33e2..7fd2bd490339e 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 @@ -39,7 +39,7 @@ ddlStatement // Database : createDatabase | dropDatabase | dropPartition | alterDatabase | showDatabases | countDatabases // Timeseries & Path - | createTimeseries | dropTimeseries | alterTimeseries + | createTimeseries | dropTimeseries | alterTimeseries | alterEncodingCompressor | showDevices | showTimeseries | showChildPaths | showChildNodes | countDevices | countTimeseries | countNodes // Device Template | createSchemaTemplate | createTimeseriesUsingSchemaTemplate | dropSchemaTemplate | dropTimeseriesOfSchemaTemplate @@ -177,6 +177,10 @@ alterClause | UPSERT aliasClause? tagClause? attributeClause? ; +alterEncodingCompressor + : ALTER TIMESERIES (IF EXISTS)? (IF PERMITTED)? prefixPath (COMMA prefixPath)* SET attributePair (COMMA attributePair)* + ; + aliasClause : ALIAS operator_eq alias ; diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 index 0efaf9fac3879..7a0c196939862 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 @@ -610,6 +610,10 @@ PATHS : P A T H S ; +PERMITTED + : P E R M I T T E D + ; + PIPE : P I P E ; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnAsyncRequestType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnAsyncRequestType.java index fc7ff81c75bea..0d6d439583f94 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnAsyncRequestType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnAsyncRequestType.java @@ -90,6 +90,8 @@ public enum CnToDnAsyncRequestType { DELETE_DATA_FOR_DELETE_SCHEMA, DELETE_TIMESERIES, + ALTER_ENCODING_COMPRESSOR, + CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE, ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE, DEACTIVATE_TEMPLATE, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java index 8ccd0104eb928..cde9a174492d3 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java @@ -48,6 +48,7 @@ import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.ConsumerGroupPushMetaRPCHandler; import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.TopicPushMetaRPCHandler; import org.apache.iotdb.mpp.rpc.thrift.TActiveTriggerInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TAlterEncodingCompressorReq; import org.apache.iotdb.mpp.rpc.thrift.TAlterViewReq; import org.apache.iotdb.mpp.rpc.thrift.TCheckSchemaRegionUsingTemplateReq; import org.apache.iotdb.mpp.rpc.thrift.TCheckTimeSeriesExistenceReq; @@ -309,6 +310,11 @@ protected void initActionMapBuilder() { CnToDnAsyncRequestType.DELETE_TIMESERIES, (req, client, handler) -> client.deleteTimeSeries((TDeleteTimeSeriesReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnAsyncRequestType.ALTER_ENCODING_COMPRESSOR, + (req, client, handler) -> + client.alterEncodingCompressor( + (TAlterEncodingCompressorReq) req, (SchemaUpdateRPCHandler) handler)); actionMapBuilder.put( CnToDnAsyncRequestType.CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE, (req, client, handler) -> diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java index 46b9f44d8cd1a..301bfc71f97d8 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java @@ -83,6 +83,7 @@ public static DataNodeAsyncRequestRPCHandler buildHandler( case ROLLBACK_SCHEMA_BLACK_LIST: case DELETE_DATA_FOR_DELETE_SCHEMA: case DELETE_TIMESERIES: + case ALTER_ENCODING_COMPRESSOR: case CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE: case ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE: case DEACTIVATE_TEMPLATE: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java index eb6d37064fd3b..e0b2c144c0eac 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java @@ -62,6 +62,7 @@ import org.apache.iotdb.confignode.consensus.request.write.partition.CreateSchemaPartitionPlan; import org.apache.iotdb.confignode.consensus.request.write.partition.RemoveRegionLocationPlan; import org.apache.iotdb.confignode.consensus.request.write.partition.UpdateRegionLocationPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeAlterEncodingCompressorPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeCreateTableOrViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteDevicesPlan; @@ -552,6 +553,9 @@ public static ConfigPhysicalPlan create(final ByteBuffer buffer) throws IOExcept case PipeDeleteDevices: plan = new PipeDeleteDevicesPlan(); break; + case PipeAlterEncodingCompressor: + plan = new PipeAlterEncodingCompressorPlan(); + break; case UpdateTriggersOnTransferNodes: plan = new UpdateTriggersOnTransferNodesPlan(); break; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java index 77ad500dd2b04..81ef37f93daf9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java @@ -307,6 +307,7 @@ public enum ConfigPhysicalPlanType { PipeSetTTL((short) 1705), PipeCreateTableOrView((short) 1706), PipeDeleteDevices((short) 1707), + PipeAlterEncodingCompressor((short) 1708), /** Subscription */ CreateTopic((short) 1800), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanVisitor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanVisitor.java index 03aaf86123a58..1167ae96541e0 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanVisitor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanVisitor.java @@ -24,6 +24,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; import org.apache.iotdb.confignode.consensus.request.write.database.DeleteDatabasePlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeAlterEncodingCompressorPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeCreateTableOrViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteDevicesPlan; @@ -193,6 +194,8 @@ public R process(final ConfigPhysicalPlan plan, final C context) { return visitRenameTable((RenameTablePlan) plan, context); case RenameView: return visitRenameView((RenameViewPlan) plan, context); + case PipeAlterEncodingCompressor: + return visitPipeAlterEncodingCompressor((PipeAlterEncodingCompressorPlan) plan, context); default: return visitPlan(plan, context); } @@ -504,4 +507,9 @@ public R visitRenameTable(final RenameTablePlan renameTablePlan, final C context public R visitRenameView(final RenameViewPlan renameViewPlan, final C context) { return visitRenameTable(renameViewPlan, context); } + + public R visitPipeAlterEncodingCompressor( + final PipeAlterEncodingCompressorPlan pipeAlterEncodingCompressorPlan, final C context) { + return visitPlan(pipeAlterEncodingCompressorPlan, context); + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeAlterEncodingCompressorPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeAlterEncodingCompressorPlan.java new file mode 100644 index 0000000000000..bd31a7c57b2eb --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeAlterEncodingCompressorPlan.java @@ -0,0 +1,113 @@ +/* + * 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.confignode.consensus.request.write.pipe.payload; + +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import javax.annotation.Nonnull; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class PipeAlterEncodingCompressorPlan extends ConfigPhysicalPlan { + + private ByteBuffer patternTreeBytes; + private byte encoding; + private byte compressor; + private boolean mayAlterAudit; + + public PipeAlterEncodingCompressorPlan() { + super(ConfigPhysicalPlanType.PipeAlterEncodingCompressor); + } + + public PipeAlterEncodingCompressorPlan( + final @Nonnull ByteBuffer patternTreeBytes, + final byte encoding, + final byte compressor, + final boolean mayAlterAudit) { + super(ConfigPhysicalPlanType.PipeAlterEncodingCompressor); + this.patternTreeBytes = patternTreeBytes; + this.encoding = encoding; + this.compressor = compressor; + this.mayAlterAudit = mayAlterAudit; + } + + public void setPatternTreeBytes(ByteBuffer patternTreeBytes) { + this.patternTreeBytes = patternTreeBytes; + } + + public ByteBuffer getPatternTreeBytes() { + patternTreeBytes.rewind(); + return patternTreeBytes; + } + + public byte getEncoding() { + return encoding; + } + + public byte getCompressor() { + return compressor; + } + + public void setMayAlterAudit(final boolean mayAlterAudit) { + this.mayAlterAudit = mayAlterAudit; + } + + public boolean isMayAlterAudit() { + return mayAlterAudit; + } + + @Override + protected void serializeImpl(final DataOutputStream stream) throws IOException { + stream.writeShort(getType().getPlanType()); + ReadWriteIOUtils.write(patternTreeBytes, stream); + ReadWriteIOUtils.write(encoding, stream); + ReadWriteIOUtils.write(compressor, stream); + } + + @Override + protected void deserializeImpl(final ByteBuffer buffer) throws IOException { + patternTreeBytes = ByteBuffer.wrap(ReadWriteIOUtils.readBinary(buffer).getValues()); + encoding = ReadWriteIOUtils.readByte(buffer); + compressor = ReadWriteIOUtils.readByte(buffer); + } + + @Override + public int hashCode() { + return Objects.hash(patternTreeBytes, encoding, compressor); + } + + @Override + public String toString() { + return "PipeAlterEncodingCompressorPlan{" + + "patternTreeBytes=" + + patternTreeBytes + + ", encoding=" + + encoding + + ", compressor=" + + compressor + + '}'; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index 233c3be881bde..65883f3065d3f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -143,6 +143,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TAINodeRegisterReq; import org.apache.iotdb.confignode.rpc.thrift.TAINodeRestartReq; import org.apache.iotdb.confignode.rpc.thrift.TAINodeRestartResp; +import org.apache.iotdb.confignode.rpc.thrift.TAlterEncodingCompressorReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterLogicalViewReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterOrDropTableReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterPipeReq; @@ -2235,6 +2236,23 @@ public TSStatus alterSchemaTemplate(TAlterSchemaTemplateReq req) { } } + @Override + public TSStatus alterEncodingCompressor(final TAlterEncodingCompressorReq req) { + TSStatus status = confirmLeader(); + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return procedureManager.alterEncodingCompressor( + req.getQueryId(), + PathPatternTree.deserialize(req.pathPatternTree), + req.getEncoding(), + req.getCompressor(), + req.isIfExists(), + req.isIsGeneratedByPipe(), + req.isMayAlterAudit()); + } else { + return status; + } + } + @Override public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) { TSStatus status = confirmLeader(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java index 217ecf347f526..3415c2d2d6e9a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java @@ -62,6 +62,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TAINodeRegisterReq; import org.apache.iotdb.confignode.rpc.thrift.TAINodeRestartReq; import org.apache.iotdb.confignode.rpc.thrift.TAINodeRestartResp; +import org.apache.iotdb.confignode.rpc.thrift.TAlterEncodingCompressorReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterLogicalViewReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterOrDropTableReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterPipeReq; @@ -716,6 +717,8 @@ TDataPartitionTableResp getOrCreateDataPartition( TSStatus alterSchemaTemplate(TAlterSchemaTemplateReq req); + TSStatus alterEncodingCompressor(TAlterEncodingCompressorReq req); + /** Delete timeseries. */ TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index 3dd3a2c68ffbc..cb4703081348a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -84,6 +84,7 @@ import org.apache.iotdb.confignode.procedure.impl.region.RegionMigrationPlan; import org.apache.iotdb.confignode.procedure.impl.region.RegionOperationProcedure; import org.apache.iotdb.confignode.procedure.impl.region.RemoveRegionPeerProcedure; +import org.apache.iotdb.confignode.procedure.impl.schema.AlterEncodingCompressorProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterLogicalViewProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.DeactivateTemplateProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.DeleteDatabaseProcedure; @@ -307,6 +308,46 @@ public TSStatus deleteDatabases( } } + public TSStatus alterEncodingCompressor( + final String queryId, + final PathPatternTree patternTree, + final byte encoding, + final byte compressor, + final boolean ifExists, + final boolean isGeneratedByPipe, + final boolean mayAlterAudit) { + AlterEncodingCompressorProcedure procedure = null; + synchronized (this) { + ProcedureType type; + AlterEncodingCompressorProcedure alterEncodingCompressorProcedure; + for (Procedure runningProcedure : executor.getProcedures().values()) { + type = ProcedureFactory.getProcedureType(runningProcedure); + if (type == null || !type.equals(ProcedureType.ALTER_ENCODING_COMPRESSOR_PROCEDURE)) { + continue; + } + alterEncodingCompressorProcedure = ((AlterEncodingCompressorProcedure) runningProcedure); + if (queryId.equals(alterEncodingCompressorProcedure.getQueryId())) { + procedure = alterEncodingCompressorProcedure; + break; + } + } + + if (procedure == null) { + procedure = + new AlterEncodingCompressorProcedure( + isGeneratedByPipe, + queryId, + patternTree, + ifExists, + encoding, + compressor, + mayAlterAudit); + this.executor.submitProcedure(procedure); + } + } + return waitingProcedureFinished(procedure); + } + public TSStatus deleteTimeSeries( String queryId, PathPatternTree patternTree, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java index e4ccb60332c02..dc2e491c96cc2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java @@ -26,6 +26,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.path.PathPatternTreeUtils; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; @@ -38,6 +39,7 @@ import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferFileSealReqV1; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferFileSealReqV2; import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; +import org.apache.iotdb.commons.schema.table.Audit; import org.apache.iotdb.commons.schema.table.TreeViewSchema; import org.apache.iotdb.commons.schema.table.TsTable; import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema; @@ -53,6 +55,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; import org.apache.iotdb.confignode.consensus.request.write.database.DeleteDatabasePlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeAlterEncodingCompressorPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeCreateTableOrViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteDevicesPlan; @@ -281,7 +284,7 @@ private TSStatus executePlanAndClassifyExceptions(final ConfigPhysicalPlan plan) return result; } - private TSStatus checkPermission(final ConfigPhysicalPlan plan) { + private TSStatus checkPermission(final ConfigPhysicalPlan plan) throws IOException { TSStatus status = loginIfNecessary(); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { return status; @@ -342,6 +345,45 @@ private TSStatus checkPermission(final ConfigPhysicalPlan plan) { .getAllPathPatterns()), PrivilegeType.WRITE_SCHEMA)) .getStatus(); + case PipeAlterEncodingCompressor: + // Judge here in the future + if (configManager + .checkUserPrivileges(username, new PrivilegeUnion(PrivilegeType.AUDIT)) + .getStatus() + .getCode() + != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + ((PipeAlterEncodingCompressorPlan) plan).setMayAlterAudit(false); + } + if (skipIfNoPrivileges.get()) { + final PathPatternTree pathPatternTree = + PathPatternTree.deserialize( + ByteBuffer.wrap( + configManager + .fetchAuthizedPatternTree(username, PrivilegeType.WRITE_SCHEMA.ordinal()) + .getPathPatternTree())); + if (((PipeAlterEncodingCompressorPlan) plan).isMayAlterAudit()) { + pathPatternTree.appendPathPattern(Audit.TREE_MODEL_AUDIT_DATABASE_PATH_PATTERN, true); + } + ((PipeAlterEncodingCompressorPlan) plan) + .setPatternTreeBytes( + PathPatternTreeUtils.intersectWithFullPathPrefixTree( + PathPatternTree.deserialize( + ((PipeAlterEncodingCompressorPlan) plan).getPatternTreeBytes()), + pathPatternTree) + .serialize()); + return StatusUtils.OK; + } else { + return configManager + .checkUserPrivileges( + username, + new PrivilegeUnion( + new ArrayList<>( + PathPatternTree.deserialize( + ((PipeAlterEncodingCompressorPlan) plan).getPatternTreeBytes()) + .getAllPathPatterns()), + PrivilegeType.WRITE_SCHEMA)) + .getStatus(); + } case PipeDeleteLogicalView: return configManager .checkUserPrivileges( @@ -648,6 +690,18 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce queryId, ((PipeDeactivateTemplatePlan) plan).getTemplateSetInfo(), shouldMarkAsPipeRequest.get()); + case PipeAlterEncodingCompressor: + return configManager + .getProcedureManager() + .alterEncodingCompressor( + queryId, + PathPatternTree.deserialize( + ((PipeAlterEncodingCompressorPlan) plan).getPatternTreeBytes()), + ((PipeAlterEncodingCompressorPlan) plan).getEncoding(), + ((PipeAlterEncodingCompressorPlan) plan).getCompressor(), + true, + shouldMarkAsPipeRequest.get(), + ((PipeAlterEncodingCompressorPlan) plan).isMayAlterAudit()); case UpdateTriggerStateInTable: // TODO: Record complete message in trigger return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/visitor/PipeConfigPhysicalPlanTSStatusVisitor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/visitor/PipeConfigPhysicalPlanTSStatusVisitor.java index e1b6ce7c7078d..cc8b57deb8e5f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/visitor/PipeConfigPhysicalPlanTSStatusVisitor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/visitor/PipeConfigPhysicalPlanTSStatusVisitor.java @@ -27,6 +27,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; import org.apache.iotdb.confignode.consensus.request.write.database.DeleteDatabasePlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeAlterEncodingCompressorPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeCreateTableOrViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteDevicesPlan; @@ -579,4 +580,14 @@ private TSStatus visitCommonTablePlan(final ConfigPhysicalPlan plan, final TSSta } return visitPlan(plan, context); } + + @Override + public TSStatus visitPipeAlterEncodingCompressor( + final PipeAlterEncodingCompressorPlan plan, final TSStatus context) { + if (context.getCode() == TSStatusCode.PATH_NOT_EXIST.getStatusCode()) { + return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(context.getMessage()); + } + return visitPlan(plan, context); + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/client/IoTDBConfigNodeSyncClientManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/client/IoTDBConfigNodeSyncClientManager.java index eb3eb1638e03b..c579fad73c123 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/client/IoTDBConfigNodeSyncClientManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/client/IoTDBConfigNodeSyncClientManager.java @@ -47,7 +47,8 @@ public IoTDBConfigNodeSyncClientManager( boolean shouldReceiverConvertOnTypeMismatch, String loadTsFileStrategy, boolean validateTsFile, - boolean shouldMarkAsPipeRequest) { + boolean shouldMarkAsPipeRequest, + final boolean skipIfNoPrivileges) { super( endPoints, useSSL, @@ -60,7 +61,8 @@ public IoTDBConfigNodeSyncClientManager( shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, validateTsFile, - shouldMarkAsPipeRequest); + shouldMarkAsPipeRequest, + skipIfNoPrivileges); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionAirGapSink.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionAirGapSink.java index e6ea5efa19d72..83c03ca9cd0e6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionAirGapSink.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionAirGapSink.java @@ -86,6 +86,8 @@ protected byte[] generateHandShakeV2Payload() throws IOException { params.put( PipeTransferHandshakeConstant.HANDSHAKE_KEY_MARK_AS_PIPE_REQUEST, Boolean.toString(shouldMarkAsPipeRequest)); + params.put( + PipeTransferHandshakeConstant.HANDSHAKE_KEY_SKIP_IF, Boolean.toString(skipIfNoPrivileges)); return PipeTransferConfigNodeHandshakeV2Req.toTPipeTransferBytes(params); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionSink.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionSink.java index df941720f3800..5866839af17b8 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionSink.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionSink.java @@ -76,7 +76,8 @@ protected IoTDBSyncClientManager constructClient( final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, final boolean validateTsFile, - final boolean shouldMarkAsPipeRequest) { + final boolean shouldMarkAsPipeRequest, + final boolean skipIfNoPrivileges) { return new IoTDBConfigNodeSyncClientManager( nodeUrls, useSSL, @@ -88,7 +89,8 @@ protected IoTDBSyncClientManager constructClient( shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, validateTsFile, - shouldMarkAsPipeRequest); + shouldMarkAsPipeRequest, + skipIfNoPrivileges); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/ConfigRegionListeningFilter.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/ConfigRegionListeningFilter.java index edec328a72373..a9a08611643f8 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/ConfigRegionListeningFilter.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/ConfigRegionListeningFilter.java @@ -91,6 +91,9 @@ public class ConfigRegionListeningFilter { Arrays.asList( ConfigPhysicalPlanType.UnsetTemplate, ConfigPhysicalPlanType.PipeUnsetTemplate))); + OPTION_PLAN_MAP.put( + new PartialPath("schema.timeseries.ordinary.alter"), + Collections.singletonList(ConfigPhysicalPlanType.PipeAlterEncodingCompressor)); OPTION_PLAN_MAP.put( new PartialPath("schema.timeseries.ordinary.drop"), Collections.singletonList(ConfigPhysicalPlanType.PipeDeleteTimeSeries)); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitor.java index 339ea27558988..8e9a3d2cc0fe8 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitor.java @@ -30,6 +30,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; import org.apache.iotdb.confignode.consensus.request.write.database.DeleteDatabasePlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeAlterEncodingCompressorPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; @@ -253,6 +254,30 @@ public Optional visitPipeDeleteLogicalView( } } + @Override + public Optional visitPipeAlterEncodingCompressor( + final PipeAlterEncodingCompressorPlan pipeAlterEncodingCompressorPlan, + final UnionIoTDBTreePattern pattern) { + try { + final PathPatternTree intersectedTree = + pattern.getIntersection( + PathPatternTree.deserialize(pipeAlterEncodingCompressorPlan.getPatternTreeBytes())); + return !intersectedTree.isEmpty() + ? Optional.of( + new PipeAlterEncodingCompressorPlan( + intersectedTree.serialize(), + pipeAlterEncodingCompressorPlan.getEncoding(), + pipeAlterEncodingCompressorPlan.getCompressor(), + pipeAlterEncodingCompressorPlan.isMayAlterAudit())) + : Optional.empty(); + } catch (final IOException e) { + LOGGER.warn( + "Serialization failed for the alter encoding time series plan in pipe transmission, skip transfer", + e); + return Optional.empty(); + } + } + @Override public Optional visitPipeDeactivateTemplate( final PipeDeactivateTemplatePlan pipeDeactivateTemplatePlan, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index d86169e14ef9a..fe8b28c4da2e5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -679,6 +679,7 @@ public TSStatus executeNonQueryPlan(ConfigPhysicalPlan physicalPlan) case PipeDeleteLogicalView: case PipeDeactivateTemplate: case PipeDeleteDevices: + case PipeAlterEncodingCompressor: // Pipe payload, used to trigger plan extraction. // Will not be actually executed. return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java new file mode 100644 index 0000000000000..a76e3494d051a --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java @@ -0,0 +1,331 @@ +/* + * 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.confignode.procedure.impl.schema; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.utils.SerializeUtils; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.confignode.client.async.CnToDnAsyncRequestType; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeAlterEncodingCompressorPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.manager.ClusterManager; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; +import org.apache.iotdb.confignode.procedure.exception.ProcedureException; +import org.apache.iotdb.confignode.procedure.impl.StateMachineProcedure; +import org.apache.iotdb.confignode.procedure.state.AlterEncodingCompressorState; +import org.apache.iotdb.confignode.procedure.store.ProcedureType; +import org.apache.iotdb.consensus.exception.ConsensusException; +import org.apache.iotdb.db.exception.metadata.PathNotExistException; +import org.apache.iotdb.mpp.rpc.thrift.TAlterEncodingCompressorReq; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.iotdb.confignode.procedure.impl.schema.DeleteTimeSeriesProcedure.invalidateCache; +import static org.apache.iotdb.confignode.procedure.impl.schema.DeleteTimeSeriesProcedure.preparePatternTreeBytesData; + +public class AlterEncodingCompressorProcedure + extends StateMachineProcedure { + private static final Logger LOGGER = LoggerFactory.getLogger(AlterEncodingCompressorState.class); + private String queryId; + private PathPatternTree patternTree; + private boolean ifExists; + private byte encoding; + private byte compressor; + private boolean mayAlterAudit; + + private transient ByteBuffer patternTreeBytes; + private transient String requestMessage; + + public AlterEncodingCompressorProcedure(final boolean isGeneratedByPipe) { + super(isGeneratedByPipe); + } + + public AlterEncodingCompressorProcedure( + final boolean isGeneratedByPipe, + final String queryId, + final PathPatternTree pathPatternTree, + final boolean ifExists, + final byte encoding, + final byte compressor, + final boolean mayAlterAudit) { + super(isGeneratedByPipe); + this.queryId = queryId; + setPatternTree(pathPatternTree); + this.ifExists = ifExists; + this.encoding = encoding; + this.compressor = compressor; + this.mayAlterAudit = mayAlterAudit; + } + + public String getQueryId() { + return queryId; + } + + @TestOnly + public PathPatternTree getPatternTree() { + return patternTree; + } + + public void setPatternTree(final PathPatternTree patternTree) { + this.patternTree = patternTree; + requestMessage = patternTree.getAllPathPatterns().toString(); + patternTreeBytes = preparePatternTreeBytesData(patternTree); + } + + @Override + protected Flow executeFromState( + final ConfigNodeProcedureEnv env, final AlterEncodingCompressorState state) + throws InterruptedException { + final long startTime = System.currentTimeMillis(); + try { + switch (state) { + case ALTER_SCHEMA_REGION: + if (LOGGER.isInfoEnabled()) { + LOGGER.info( + "Alter encoding {} & compressor {} in schema region for timeSeries {}", + SerializeUtils.deserializeEncodingNullable(encoding), + SerializeUtils.deserializeCompressorNullable(compressor), + requestMessage); + } + alterEncodingCompressorInSchemaRegion(env); + break; + case CLEAR_CACHE: + LOGGER.info("Invalidate cache of timeSeries {}", requestMessage); + invalidateCache(env, patternTreeBytes, requestMessage, this::setFailure); + collectPayload4Pipe(env); + return Flow.NO_MORE_STATE; + default: + setFailure(new ProcedureException("Unrecognized state " + state)); + return Flow.NO_MORE_STATE; + } + return Flow.HAS_MORE_STATE; + } finally { + LOGGER.info( + "AlterEncodingCompressor-[{}] costs {}ms", + state, + (System.currentTimeMillis() - startTime)); + } + } + + private void alterEncodingCompressorInSchemaRegion(final ConfigNodeProcedureEnv env) { + final Map relatedSchemaRegionGroup = + env.getConfigManager().getRelatedSchemaRegionGroup(patternTree, mayAlterAudit); + + if (relatedSchemaRegionGroup.isEmpty()) { + if (!ifExists) { + setFailure( + new ProcedureException( + new PathNotExistException( + patternTree.getAllPathPatterns().stream() + .map(PartialPath::getFullPath) + .collect(Collectors.toList()), + false))); + } + return; + } + + final DataNodeTSStatusTaskExecutor alterEncodingCompressorTask = + new DataNodeTSStatusTaskExecutor( + env, + env.getConfigManager().getRelatedSchemaRegionGroup(patternTree, mayAlterAudit), + false, + CnToDnAsyncRequestType.ALTER_ENCODING_COMPRESSOR, + ((dataNodeLocation, consensusGroupIdList) -> + new TAlterEncodingCompressorReq(consensusGroupIdList, patternTreeBytes, ifExists) + .setCompressor(compressor) + .setEncoding(encoding))) { + + private final Map failureMap = new HashMap<>(); + + @Override + protected List processResponseOfOneDataNode( + final TDataNodeLocation dataNodeLocation, + final List consensusGroupIdList, + final TSStatus response) { + final List failedRegionList = new ArrayList<>(); + if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + failureMap.remove(dataNodeLocation); + return failedRegionList; + } + + if (response.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { + final List subStatus = response.getSubStatus(); + for (int i = 0; i < subStatus.size(); i++) { + if (subStatus.get(i).getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && !(subStatus.get(i).getCode() == TSStatusCode.PATH_NOT_EXIST.getStatusCode() + && ifExists)) { + failedRegionList.add(consensusGroupIdList.get(i)); + } + } + } else if (!(response.getCode() == TSStatusCode.PATH_NOT_EXIST.getStatusCode() + && ifExists)) { + failedRegionList.addAll(consensusGroupIdList); + } + if (!failedRegionList.isEmpty()) { + failureMap.put(dataNodeLocation, response); + } else { + failureMap.remove(dataNodeLocation); + } + return failedRegionList; + } + + @Override + protected void onAllReplicasetFailure( + final TConsensusGroupId consensusGroupId, + final Set dataNodeLocationSet) { + setFailure( + new ProcedureException( + new MetadataException( + String.format( + "Alter encoding compressor %s in schema regions failed. Failures: %s", + requestMessage, failureMap)))); + interruptTask(); + } + }; + alterEncodingCompressorTask.execute(); + setNextState(AlterEncodingCompressorState.CLEAR_CACHE); + } + + private void collectPayload4Pipe(final ConfigNodeProcedureEnv env) { + TSStatus result; + try { + result = + env.getConfigManager() + .getConsensusManager() + .write( + isGeneratedByPipe + ? new PipeEnrichedPlan( + new PipeAlterEncodingCompressorPlan( + patternTreeBytes, encoding, compressor, mayAlterAudit)) + : new PipeAlterEncodingCompressorPlan( + patternTreeBytes, encoding, compressor, mayAlterAudit)); + } catch (final ConsensusException e) { + LOGGER.warn(ClusterManager.CONSENSUS_WRITE_ERROR, e); + result = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); + result.setMessage(e.getMessage()); + } + if (result.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + throw new PipeException(result.getMessage()); + } + } + + @Override + protected void rollbackState( + final ConfigNodeProcedureEnv env, + final AlterEncodingCompressorState alterEncodingCompressorState) + throws IOException, InterruptedException, ProcedureException { + // Not supported now + } + + @Override + protected AlterEncodingCompressorState getState(final int stateId) { + return AlterEncodingCompressorState.values()[stateId]; + } + + @Override + protected int getStateId(final AlterEncodingCompressorState alterEncodingCompressorState) { + return alterEncodingCompressorState.ordinal(); + } + + @Override + protected AlterEncodingCompressorState getInitialState() { + return AlterEncodingCompressorState.ALTER_SCHEMA_REGION; + } + + @Override + public void serialize(final DataOutputStream stream) throws IOException { + stream.writeShort( + isGeneratedByPipe + ? ProcedureType.PIPE_ENRICHED_ALTER_ENCODING_COMPRESSOR_PROCEDURE.getTypeCode() + : ProcedureType.ALTER_ENCODING_COMPRESSOR_PROCEDURE.getTypeCode()); + super.serialize(stream); + ReadWriteIOUtils.write(queryId, stream); + patternTree.serialize(stream); + ReadWriteIOUtils.write(ifExists, stream); + ReadWriteIOUtils.write(encoding, stream); + ReadWriteIOUtils.write(compressor, stream); + ReadWriteIOUtils.write(mayAlterAudit, stream); + } + + @Override + public void deserialize(final ByteBuffer byteBuffer) { + super.deserialize(byteBuffer); + queryId = ReadWriteIOUtils.readString(byteBuffer); + setPatternTree(PathPatternTree.deserialize(byteBuffer)); + ifExists = ReadWriteIOUtils.readBoolean(byteBuffer); + encoding = ReadWriteIOUtils.readByte(byteBuffer); + compressor = ReadWriteIOUtils.readByte(byteBuffer); + mayAlterAudit = ReadWriteIOUtils.readBoolean(byteBuffer); + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) { + return false; + } + final AlterEncodingCompressorProcedure that = (AlterEncodingCompressorProcedure) o; + return this.getProcId() == that.getProcId() + && this.getCurrentState().equals(that.getCurrentState()) + && this.getCycles() == getCycles() + && Objects.equals(this.queryId, that.queryId) + && this.isGeneratedByPipe == that.isGeneratedByPipe + && this.patternTree.equals(that.patternTree) + && this.encoding == that.encoding + && this.compressor == that.compressor + && this.mayAlterAudit == that.mayAlterAudit; + } + + @Override + public int hashCode() { + return Objects.hash( + getProcId(), + getCurrentState(), + getCycles(), + queryId, + isGeneratedByPipe, + patternTree, + ifExists, + encoding, + compressor, + mayAlterAudit); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeTSStatusTaskExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeTSStatusTaskExecutor.java new file mode 100644 index 0000000000000..700cb3a9a49ab --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeTSStatusTaskExecutor.java @@ -0,0 +1,74 @@ +/* + * 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.confignode.procedure.impl.schema; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.confignode.client.async.CnToDnAsyncRequestType; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; +import org.apache.iotdb.rpc.TSStatusCode; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +public abstract class DataNodeTSStatusTaskExecutor + extends DataNodeRegionTaskExecutor { + + protected DataNodeTSStatusTaskExecutor( + final ConfigNodeProcedureEnv env, + final Map targetRegionGroup, + final boolean executeOnAllReplicaset, + final CnToDnAsyncRequestType dataNodeRequestType, + final BiFunction, Q> dataNodeRequestGenerator) { + super( + env, + targetRegionGroup, + executeOnAllReplicaset, + dataNodeRequestType, + dataNodeRequestGenerator); + } + + @Override + protected List processResponseOfOneDataNode( + final TDataNodeLocation dataNodeLocation, + final List consensusGroupIdList, + final TSStatus response) { + final List failedRegionList = new ArrayList<>(); + if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return failedRegionList; + } + + if (response.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { + final List subStatus = response.getSubStatus(); + for (int i = 0; i < subStatus.size(); i++) { + if (subStatus.get(i).getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + failedRegionList.add(consensusGroupIdList.get(i)); + } + } + } else { + failedRegionList.addAll(consensusGroupIdList); + } + return failedRegionList; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java index 1df9da1509dc7..7e1ca07086add 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java @@ -428,8 +428,7 @@ public int hashCode() { getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, queryId, templateSetInfo); } - private class DeactivateTemplateRegionTaskExecutor - extends DataNodeRegionTaskExecutor { + private class DeactivateTemplateRegionTaskExecutor extends DataNodeTSStatusTaskExecutor { private final String taskName; @@ -459,29 +458,6 @@ private class DeactivateTemplateRegionTaskExecutor this.taskName = taskName; } - @Override - protected List processResponseOfOneDataNode( - TDataNodeLocation dataNodeLocation, - List consensusGroupIdList, - TSStatus response) { - List failedRegionList = new ArrayList<>(); - if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return failedRegionList; - } - - if (response.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - List subStatus = response.getSubStatus(); - for (int i = 0; i < subStatus.size(); i++) { - if (subStatus.get(i).getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - failedRegionList.add(consensusGroupIdList.get(i)); - } - } - } else { - failedRegionList.addAll(consensusGroupIdList); - } - return failedRegionList; - } - @Override protected void onAllReplicasetFailure( TConsensusGroupId consensusGroupId, Set dataNodeLocationSet) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java index 6b448e3257918..1cfead14fcf32 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java @@ -335,8 +335,7 @@ public int hashCode() { getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, patternTree); } - private class DeleteLogicalViewRegionTaskExecutor - extends DataNodeRegionTaskExecutor { + private class DeleteLogicalViewRegionTaskExecutor extends DataNodeTSStatusTaskExecutor { private final String taskName; @@ -350,29 +349,6 @@ private class DeleteLogicalViewRegionTaskExecutor this.taskName = taskName; } - @Override - protected List processResponseOfOneDataNode( - final TDataNodeLocation dataNodeLocation, - final List consensusGroupIdList, - final TSStatus response) { - final List failedRegionList = new ArrayList<>(); - if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return failedRegionList; - } - - if (response.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - final List subStatus = response.getSubStatus(); - for (int i = 0; i < subStatus.size(); i++) { - if (subStatus.get(i).getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - failedRegionList.add(consensusGroupIdList.get(i)); - } - } - } else { - failedRegionList.addAll(consensusGroupIdList); - } - return failedRegionList; - } - @Override protected void onAllReplicasetFailure( final TConsensusGroupId consensusGroupId, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java index 6aef790d78655..594b6b035bbbb 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java @@ -60,6 +60,7 @@ import java.util.Objects; import java.util.Set; import java.util.function.BiFunction; +import java.util.function.Consumer; import java.util.stream.Collectors; public class DeleteTimeSeriesProcedure @@ -121,7 +122,8 @@ protected Flow executeFromState( } case CLEAN_DATANODE_SCHEMA_CACHE: LOGGER.info("Invalidate cache of timeSeries {}", requestMessage); - invalidateCache(env); + invalidateCache(env, patternTreeBytes, requestMessage, this::setFailure); + setNextState(DeleteTimeSeriesState.DELETE_DATA); break; case DELETE_DATA: LOGGER.info("Delete data of timeSeries {}", requestMessage); @@ -196,7 +198,11 @@ protected List processResponseOfOneDataNode( : 0; } - private void invalidateCache(final ConfigNodeProcedureEnv env) { + public static void invalidateCache( + final ConfigNodeProcedureEnv env, + final ByteBuffer patternTreeBytes, + final String requestMessage, + final Consumer setFailure) { final Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); final DataNodeAsyncRequestContext clientHandler = @@ -210,13 +216,11 @@ private void invalidateCache(final ConfigNodeProcedureEnv env) { // All dataNodes must clear the related schemaEngine cache if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { LOGGER.error("Failed to invalidate schemaEngine cache of timeSeries {}", requestMessage); - setFailure( + setFailure.accept( new ProcedureException(new MetadataException("Invalidate schemaEngine cache failed"))); return; } } - - setNextState(DeleteTimeSeriesState.DELETE_DATA); } private void deleteData(final ConfigNodeProcedureEnv env) { @@ -344,7 +348,7 @@ public void setPatternTree(final PathPatternTree patternTree) { patternTreeBytes = preparePatternTreeBytesData(patternTree); } - private ByteBuffer preparePatternTreeBytesData(final PathPatternTree patternTree) { + public static ByteBuffer preparePatternTreeBytesData(final PathPatternTree patternTree) { final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); try { @@ -383,7 +387,9 @@ public void deserialize(final ByteBuffer byteBuffer) { @Override public boolean equals(final Object o) { - if (this == o) return true; + if (this == o) { + return true; + } if (o == null || getClass() != o.getClass()) { return false; } @@ -401,8 +407,7 @@ public int hashCode() { getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, patternTree); } - private class DeleteTimeSeriesRegionTaskExecutor - extends DataNodeRegionTaskExecutor { + private class DeleteTimeSeriesRegionTaskExecutor extends DataNodeTSStatusTaskExecutor { private final String taskName; @@ -432,29 +437,6 @@ private class DeleteTimeSeriesRegionTaskExecutor this.taskName = taskName; } - @Override - protected List processResponseOfOneDataNode( - final TDataNodeLocation dataNodeLocation, - final List consensusGroupIdList, - final TSStatus response) { - final List failedRegionList = new ArrayList<>(); - if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return failedRegionList; - } - - if (response.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - final List subStatus = response.getSubStatus(); - for (int i = 0; i < subStatus.size(); i++) { - if (subStatus.get(i).getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - failedRegionList.add(consensusGroupIdList.get(i)); - } - } - } else { - failedRegionList.addAll(consensusGroupIdList); - } - return failedRegionList; - } - @Override protected void onAllReplicasetFailure( final TConsensusGroupId consensusGroupId, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java index d818300ae7593..fe92c802ce687 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java @@ -29,9 +29,8 @@ import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.impl.StateMachineProcedure; -import org.apache.iotdb.confignode.procedure.impl.schema.DataNodeRegionTaskExecutor; +import org.apache.iotdb.confignode.procedure.impl.schema.DataNodeTSStatusTaskExecutor; import org.apache.iotdb.confignode.procedure.impl.schema.SchemaUtils; -import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; @@ -42,7 +41,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -186,7 +184,7 @@ public void deserialize(final ByteBuffer byteBuffer) { } } - protected class TableRegionTaskExecutor extends DataNodeRegionTaskExecutor { + protected class TableRegionTaskExecutor extends DataNodeTSStatusTaskExecutor { private final String taskName; @@ -200,29 +198,6 @@ protected TableRegionTaskExecutor( this.taskName = taskName; } - @Override - protected List processResponseOfOneDataNode( - final TDataNodeLocation dataNodeLocation, - final List consensusGroupIdList, - final TSStatus response) { - final List failedRegionList = new ArrayList<>(); - if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return failedRegionList; - } - - if (response.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - final List subStatus = response.getSubStatus(); - for (int i = 0; i < subStatus.size(); i++) { - if (subStatus.get(i).getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - failedRegionList.add(consensusGroupIdList.get(i)); - } - } - } else { - failedRegionList.addAll(consensusGroupIdList); - } - return failedRegionList; - } - @Override protected void onAllReplicasetFailure( final TConsensusGroupId consensusGroupId, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/AlterEncodingCompressorState.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/AlterEncodingCompressorState.java new file mode 100644 index 0000000000000..4045971d2f198 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/AlterEncodingCompressorState.java @@ -0,0 +1,25 @@ +/* + * 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.confignode.procedure.state; + +public enum AlterEncodingCompressorState { + ALTER_SCHEMA_REGION, + CLEAR_CACHE +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java index 6ea7796a00d78..e023171f4fa88 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java @@ -44,6 +44,7 @@ import org.apache.iotdb.confignode.procedure.impl.region.ReconstructRegionProcedure; import org.apache.iotdb.confignode.procedure.impl.region.RegionMigrateProcedure; import org.apache.iotdb.confignode.procedure.impl.region.RemoveRegionPeerProcedure; +import org.apache.iotdb.confignode.procedure.impl.schema.AlterEncodingCompressorProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterLogicalViewProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.DeactivateTemplateProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.DeleteDatabaseProcedure; @@ -140,6 +141,9 @@ public Procedure create(ByteBuffer buffer) throws IOException { case NOTIFY_REGION_MIGRATION_PROCEDURE: procedure = new NotifyRegionMigrationProcedure(); break; + case ALTER_ENCODING_COMPRESSOR_PROCEDURE: + procedure = new AlterEncodingCompressorProcedure(false); + break; case DELETE_TIMESERIES_PROCEDURE: procedure = new DeleteTimeSeriesProcedure(false); break; @@ -343,6 +347,9 @@ public Procedure create(ByteBuffer buffer) throws IOException { case PIPE_ENRICHED_RENAME_VIEW_PROCEDURE: procedure = new RenameViewProcedure(true); break; + case PIPE_ENRICHED_ALTER_ENCODING_COMPRESSOR_PROCEDURE: + procedure = new AlterEncodingCompressorProcedure(true); + break; case REMOVE_AI_NODE_PROCEDURE: procedure = new RemoveAINodeProcedure(); break; @@ -425,6 +432,8 @@ public static ProcedureType getProcedureType(final Procedure procedure) { return ProcedureType.REMOVE_REGION_PEER_PROCEDURE; } else if (procedure instanceof CreateRegionGroupsProcedure) { return ProcedureType.CREATE_REGION_GROUPS; + } else if (procedure instanceof AlterEncodingCompressorProcedure) { + return ProcedureType.ALTER_ENCODING_COMPRESSOR_PROCEDURE; } else if (procedure instanceof DeleteTimeSeriesProcedure) { return ProcedureType.DELETE_TIMESERIES_PROCEDURE; } else if (procedure instanceof ReconstructRegionProcedure) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java index 620736b9cbaa0..65ac1fb24ad5a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java @@ -46,6 +46,7 @@ public enum ProcedureType { /** Timeseries */ DELETE_TIMESERIES_PROCEDURE((short) 300), + ALTER_ENCODING_COMPRESSOR_PROCEDURE((short) 301), /** Trigger */ CREATE_TRIGGER_PROCEDURE((short) 400), @@ -146,6 +147,7 @@ public enum ProcedureType { PIPE_ENRICHED_SET_VIEW_PROPERTIES_PROCEDURE((short) 145), PIPE_ENRICHED_RENAME_VIEW_COLUMN_PROCEDURE((short) 146), PIPE_ENRICHED_RENAME_VIEW_PROCEDURE((short) 147), + PIPE_ENRICHED_ALTER_ENCODING_COMPRESSOR_PROCEDURE((short) 148), /** Subscription */ CREATE_TOPIC_PROCEDURE((short) 1500), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index 2269f22910752..bcc9f5068a5b7 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -92,6 +92,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TAINodeRestartReq; import org.apache.iotdb.confignode.rpc.thrift.TAINodeRestartResp; import org.apache.iotdb.confignode.rpc.thrift.TAddConsensusGroupReq; +import org.apache.iotdb.confignode.rpc.thrift.TAlterEncodingCompressorReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterLogicalViewReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterOrDropTableReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterPipeReq; @@ -1141,6 +1142,11 @@ public TSStatus alterSchemaTemplate(TAlterSchemaTemplateReq req) { return configManager.alterSchemaTemplate(req); } + @Override + public TSStatus alterEncodingCompressor(final TAlterEncodingCompressorReq req) throws TException { + return configManager.alterEncodingCompressor(req); + } + @Override public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) { return configManager.deleteTimeSeries(req); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/receiver/PipeEnrichedProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/receiver/PipeEnrichedProcedureTest.java index 9a3054cacda9f..3893dff1607e2 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/receiver/PipeEnrichedProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/receiver/PipeEnrichedProcedureTest.java @@ -33,6 +33,7 @@ import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; import org.apache.iotdb.confignode.consensus.request.write.auth.AuthorTreePlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.procedure.impl.schema.AlterEncodingCompressorProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterLogicalViewProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.DeactivateTemplateProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.DeleteDatabaseProcedure; @@ -760,4 +761,31 @@ public void setViewPropertiesTest() throws IOException { Assert.assertEquals(setViewPropertiesProcedure, deserializedProcedure); } + + @Test + public void alterEncodingCompressorTest() throws IllegalPathException, IOException { + final String queryId = "1"; + final PathPatternTree patternTree = new PathPatternTree(); + patternTree.appendPathPattern(new PartialPath("root.sg1.**")); + patternTree.appendPathPattern(new PartialPath("root.sg2.*.s1")); + patternTree.constructTree(); + final AlterEncodingCompressorProcedure alterEncodingCompressorProcedure = + new AlterEncodingCompressorProcedure( + false, queryId, patternTree, false, (byte) 0, (byte) 0, false); + + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); + alterEncodingCompressorProcedure.serialize(dataOutputStream); + + final ByteBuffer byteBuffer = ByteBuffer.wrap(byteArrayOutputStream.toByteArray()); + + Assert.assertEquals( + ProcedureType.ALTER_ENCODING_COMPRESSOR_PROCEDURE.getTypeCode(), byteBuffer.getShort()); + + final AlterEncodingCompressorProcedure deserializedProcedure = + new AlterEncodingCompressorProcedure(false); + deserializedProcedure.deserialize(byteBuffer); + + Assert.assertEquals(alterEncodingCompressorProcedure, deserializedProcedure); + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedureTest.java new file mode 100644 index 0000000000000..b6f2541b21580 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedureTest.java @@ -0,0 +1,62 @@ +/* + * 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.confignode.procedure.impl.schema; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.confignode.procedure.store.ProcedureType; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +public class AlterEncodingCompressorProcedureTest { + @Test + public void serializeDeserializeTest() throws IllegalPathException, IOException { + final String queryId = "1"; + final PathPatternTree patternTree = new PathPatternTree(); + patternTree.appendPathPattern(new PartialPath("root.sg1.**")); + patternTree.appendPathPattern(new PartialPath("root.sg2.*.s1")); + patternTree.constructTree(); + final AlterEncodingCompressorProcedure alterEncodingCompressorProcedure = + new AlterEncodingCompressorProcedure( + false, queryId, patternTree, false, (byte) 0, (byte) 0, false); + + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); + alterEncodingCompressorProcedure.serialize(dataOutputStream); + + final ByteBuffer byteBuffer = ByteBuffer.wrap(byteArrayOutputStream.toByteArray()); + + Assert.assertEquals( + ProcedureType.ALTER_ENCODING_COMPRESSOR_PROCEDURE.getTypeCode(), byteBuffer.getShort()); + + final AlterEncodingCompressorProcedure deserializedProcedure = + new AlterEncodingCompressorProcedure(false); + deserializedProcedure.deserialize(byteBuffer); + + Assert.assertEquals(alterEncodingCompressorProcedure, deserializedProcedure); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/schemaregion/SchemaExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/schemaregion/SchemaExecutionVisitor.java index 6b9ba0a261200..8d3c215583858 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/schemaregion/SchemaExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/schemaregion/SchemaExecutionVisitor.java @@ -34,6 +34,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.ActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterTimeSeriesNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.BatchActivateTemplateNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.ConstructSchemaBlackListNode; @@ -554,6 +555,20 @@ public TSStatus visitDeleteTimeseries( } } + @Override + public TSStatus visitAlterEncodingCompressor( + final AlterEncodingCompressorNode node, final ISchemaRegion schemaRegion) { + try { + schemaRegion.alterEncodingCompressor(node); + return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS); + } catch (final MetadataException e) { + if (e.getErrorCode() != TSStatusCode.PATH_NOT_EXIST.getStatusCode()) { + logger.error(e.getMessage(), e); + } + return RpcUtils.getStatus(e.getErrorCode(), e.getMessage()); + } + } + @Override public TSStatus visitPreDeactivateTemplate( final PreDeactivateTemplateNode node, final ISchemaRegion schemaRegion) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeAsyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeAsyncClientManager.java index 88bc4fb4d1e0d..5ee7f22bc0711 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeAsyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeAsyncClientManager.java @@ -102,7 +102,8 @@ public IoTDBDataNodeAsyncClientManager( final String loadTsFileStrategy, final boolean validateTsFile, final boolean shouldMarkAsPipeRequest, - final boolean isTSFileUsed) { + final boolean isTSFileUsed, + final boolean skipIfNoPrivileges) { super( endPoints, useLeaderCache, @@ -111,7 +112,8 @@ public IoTDBDataNodeAsyncClientManager( shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, validateTsFile, - shouldMarkAsPipeRequest); + shouldMarkAsPipeRequest, + skipIfNoPrivileges); endPointSet = new HashSet<>(endPoints); @@ -306,6 +308,9 @@ public void onError(final Exception e) { params.put( PipeTransferHandshakeConstant.HANDSHAKE_KEY_MARK_AS_PIPE_REQUEST, Boolean.toString(shouldMarkAsPipeRequest)); + params.put( + PipeTransferHandshakeConstant.HANDSHAKE_KEY_SKIP_IF, + Boolean.toString(skipIfNoPrivileges)); client.setTimeoutDynamically(PipeConfig.getInstance().getPipeConnectorHandshakeTimeoutMs()); client.pipeTransfer(PipeTransferDataNodeHandshakeV2Req.toTPipeTransferReq(params), callback); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeSyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeSyncClientManager.java index b610e8f4f832d..b8eff41a9eab1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeSyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeSyncClientManager.java @@ -57,7 +57,8 @@ public IoTDBDataNodeSyncClientManager( final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, final boolean validateTsFile, - final boolean shouldMarkAsPipeRequest) { + final boolean shouldMarkAsPipeRequest, + final boolean skipIfNoPrivileges) { super( endPoints, useSSL, @@ -70,7 +71,8 @@ public IoTDBDataNodeSyncClientManager( shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, validateTsFile, - shouldMarkAsPipeRequest); + shouldMarkAsPipeRequest, + skipIfNoPrivileges); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataNodeAirGapSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataNodeAirGapSink.java index 800994d7a428c..b593df5661206 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataNodeAirGapSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataNodeAirGapSink.java @@ -70,6 +70,8 @@ protected byte[] generateHandShakeV2Payload() throws IOException { params.put( PipeTransferHandshakeConstant.HANDSHAKE_KEY_MARK_AS_PIPE_REQUEST, Boolean.toString(shouldMarkAsPipeRequest)); + params.put( + PipeTransferHandshakeConstant.HANDSHAKE_KEY_SKIP_IF, Boolean.toString(skipIfNoPrivileges)); return PipeTransferDataNodeHandshakeV2Req.toTPipeTransferBytes(params); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java index 76f1af26846fb..567715e7292f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java @@ -163,7 +163,8 @@ public void customize( loadTsFileStrategy, loadTsFileValidation, shouldMarkAsPipeRequest, - false); + false, + skipIfNoPrivileges); transferTsFileClientManager = new IoTDBDataNodeAsyncClientManager( @@ -178,7 +179,8 @@ public void customize( loadTsFileStrategy, loadTsFileValidation, shouldMarkAsPipeRequest, - isSplitTSFileBatchModeEnabled); + isSplitTSFileBatchModeEnabled, + skipIfNoPrivileges); if (isTabletBatchModeEnabled) { tabletBatchBuilder = new PipeTransferBatchReqBuilder(parameters); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataNodeSyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataNodeSyncSink.java index 8ab1f5b2e2232..72486d6c4e7cd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataNodeSyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataNodeSyncSink.java @@ -52,7 +52,8 @@ protected IoTDBSyncClientManager constructClient( final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, final boolean validateTsFile, - final boolean shouldMarkAsPipeRequest) { + final boolean shouldMarkAsPipeRequest, + final boolean skipIfNoPrivileges) { clientManager = new IoTDBDataNodeSyncClientManager( nodeUrls, @@ -66,7 +67,8 @@ protected IoTDBSyncClientManager constructClient( shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, validateTsFile, - shouldMarkAsPipeRequest); + shouldMarkAsPipeRequest, + skipIfNoPrivileges); return clientManager; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java index 6ce425143f2d6..6854a3191fad3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java @@ -52,6 +52,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TAINodeRestartReq; import org.apache.iotdb.confignode.rpc.thrift.TAINodeRestartResp; import org.apache.iotdb.confignode.rpc.thrift.TAddConsensusGroupReq; +import org.apache.iotdb.confignode.rpc.thrift.TAlterEncodingCompressorReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterLogicalViewReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterOrDropTableReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterPipeReq; @@ -1105,6 +1106,12 @@ public TSStatus alterSchemaTemplate(TAlterSchemaTemplateReq req) throws TExcepti () -> client.alterSchemaTemplate(req), status -> !updateConfigNodeLeader(status)); } + @Override + public TSStatus alterEncodingCompressor(TAlterEncodingCompressorReq req) throws TException { + return executeRemoteCallWithRetry( + () -> client.alterEncodingCompressor(req), status -> !updateConfigNodeLeader(status)); + } + @Override public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) throws TException { return executeRemoteCallWithRetry( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index ed199001ef838..3f050b1184f47 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -85,6 +85,7 @@ import org.apache.iotdb.commons.trigger.TriggerInformation; import org.apache.iotdb.commons.udf.UDFInformation; import org.apache.iotdb.commons.utils.PathUtils; +import org.apache.iotdb.commons.utils.SerializeUtils; import org.apache.iotdb.commons.utils.StatusUtils; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.consensus.common.Peer; @@ -145,6 +146,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.load.LoadTsFilePieceNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.ConstructSchemaBlackListNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.DeactivateTemplateNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.DeleteTimeSeriesNode; @@ -205,6 +207,7 @@ import org.apache.iotdb.metrics.utils.SystemMetric; import org.apache.iotdb.mpp.rpc.thrift.IDataNodeRPCService; import org.apache.iotdb.mpp.rpc.thrift.TActiveTriggerInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TAlterEncodingCompressorReq; import org.apache.iotdb.mpp.rpc.thrift.TAlterViewReq; import org.apache.iotdb.mpp.rpc.thrift.TAttributeUpdateReq; import org.apache.iotdb.mpp.rpc.thrift.TAuditLogReq; @@ -786,11 +789,11 @@ public TSStatus deleteTimeSeries(final TDeleteTimeSeriesReq req) throws TExcepti return executeInternalSchemaTask( req.getSchemaRegionIdList(), consensusGroupId -> { - String storageGroup = + String database = schemaEngine .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId())) .getDatabaseFullPath(); - PathPatternTree filteredPatternTree = filterPathPatternTree(patternTree, storageGroup); + PathPatternTree filteredPatternTree = filterPathPatternTree(patternTree, database); if (filteredPatternTree.isEmpty()) { return RpcUtils.SUCCESS_STATUS; } @@ -806,6 +809,35 @@ public TSStatus deleteTimeSeries(final TDeleteTimeSeriesReq req) throws TExcepti }); } + @Override + public TSStatus alterEncodingCompressor(final TAlterEncodingCompressorReq req) throws TException { + final PathPatternTree patternTree = + PathPatternTree.deserialize(ByteBuffer.wrap(req.getPathPatternTree())); + return executeInternalSchemaTask( + req.getSchemaRegionIdList(), + consensusGroupId -> { + final String database = + schemaEngine + .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId())) + .getDatabaseFullPath(); + final PathPatternTree filteredPatternTree = filterPathPatternTree(patternTree, database); + if (filteredPatternTree.isEmpty()) { + return RpcUtils.SUCCESS_STATUS; + } + final RegionWriteExecutor executor = new RegionWriteExecutor(); + return executor + .execute( + new SchemaRegionId(consensusGroupId.getId()), + new AlterEncodingCompressorNode( + new PlanNodeId(""), + filteredPatternTree, + req.isIfExists(), + SerializeUtils.deserializeEncodingNullable(req.getEncoding()), + SerializeUtils.deserializeCompressorNullable(req.getCompressor()))) + .getStatus(); + }); + } + @Override public TSStatus constructSchemaBlackListWithTemplate(TConstructSchemaBlackListWithTemplateReq req) throws TException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java index 465326ff6394e..1fdf636c33d7c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java @@ -30,6 +30,7 @@ import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.AlterEncodingCompressorTask; import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.CountDatabaseTask; import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.CountTimeSlotListTask; import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.CreateContinuousQueryTask; @@ -123,6 +124,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.queryengine.plan.statement.StatementNode; import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterEncodingCompressorStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDatabaseStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountTimeSlotListStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CreateContinuousQueryStatement; @@ -250,14 +252,13 @@ public IConfigTask visitAlterDatabase( } @Override - public IConfigTask visitDeleteStorageGroup( + public IConfigTask visitDeleteDatabase( DeleteDatabaseStatement statement, MPPQueryContext context) { return new DeleteStorageGroupTask(statement); } @Override - public IConfigTask visitShowStorageGroup( - ShowDatabaseStatement statement, MPPQueryContext context) { + public IConfigTask visitShowDatabase(ShowDatabaseStatement statement, MPPQueryContext context) { return new ShowDatabaseTask(statement); } @@ -703,6 +704,13 @@ public IConfigTask visitDropSubscription( return new DropSubscriptionTask(dropSubscriptionStatement); } + @Override + public IConfigTask visitAlterEncodingCompressor( + AlterEncodingCompressorStatement alterEncodingCompressorStatement, MPPQueryContext context) { + return new AlterEncodingCompressorTask( + context.getQueryId().getId(), alterEncodingCompressorStatement); + } + @Override public IConfigTask visitDeleteTimeSeries( DeleteTimeSeriesStatement deleteTimeSeriesStatement, MPPQueryContext context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index 22cc5dffc98da..71b6f58f83e06 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -75,8 +75,10 @@ import org.apache.iotdb.commons.udf.service.UDFExecutableManager; import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.PathUtils; +import org.apache.iotdb.commons.utils.SerializeUtils; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.rpc.thrift.TAINodeRemoveReq; +import org.apache.iotdb.confignode.rpc.thrift.TAlterEncodingCompressorReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterLogicalViewReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterOrDropTableReq; import org.apache.iotdb.confignode.rpc.thrift.TAlterPipeReq; @@ -244,6 +246,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowCluster; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowDB; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Use; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterEncodingCompressorStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDatabaseStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountTimeSlotListStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CreateContinuousQueryStatement; @@ -2795,6 +2798,60 @@ public SettableFuture showTopics( return future; } + @Override + public SettableFuture alterEncodingCompressor( + final String queryId, + final AlterEncodingCompressorStatement alterEncodingCompressorStatement) { + final SettableFuture future = SettableFuture.create(); + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); + try { + alterEncodingCompressorStatement.getPatternTree().serialize(dataOutputStream); + } catch (final IOException ignored) { + // memory operation, won't happen + } + final TAlterEncodingCompressorReq req = + new TAlterEncodingCompressorReq( + queryId, + ByteBuffer.wrap(byteArrayOutputStream.toByteArray()), + SerializeUtils.serializeNullable(alterEncodingCompressorStatement.getEncoding()), + SerializeUtils.serializeNullable(alterEncodingCompressorStatement.getCompressor()), + alterEncodingCompressorStatement.ifExists(), + alterEncodingCompressorStatement.isWithAudit()); + try (final ConfigNodeClient client = + CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + TSStatus tsStatus; + do { + try { + tsStatus = client.alterEncodingCompressor(req); + } catch (final TTransportException e) { + if (e.getType() == TTransportException.TIMED_OUT + || e.getCause() instanceof SocketTimeoutException) { + // time out mainly caused by slow execution, wait until + tsStatus = RpcUtils.getStatus(TSStatusCode.OVERLAP_WITH_EXISTING_TASK); + } else { + throw e; + } + } + // keep waiting until task ends + } while (TSStatusCode.OVERLAP_WITH_EXISTING_TASK.getStatusCode() == tsStatus.getCode()); + + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) { + if (tsStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { + future.setException( + new BatchProcessException(tsStatus.subStatus.toArray(new TSStatus[0]))); + } else { + future.setException(new IoTDBException(tsStatus)); + } + } else { + future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); + } + } catch (final ClientManagerException | TException e) { + future.setException(e); + } + return future; + } + @Override public SettableFuture deleteTimeSeries( final String queryId, final DeleteTimeSeriesStatement deleteTimeSeriesStatement) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java index 3250e18de1422..926fd37cf3ab3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java @@ -45,6 +45,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowCluster; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowDB; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Use; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterEncodingCompressorStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDatabaseStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountTimeSlotListStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CreateContinuousQueryStatement; @@ -238,6 +239,9 @@ SettableFuture dropSubscription( SettableFuture showTopics(ShowTopicsStatement showTopicsStatement); + SettableFuture alterEncodingCompressor( + String queryId, AlterEncodingCompressorStatement alterEncodingCompressorStatement); + SettableFuture deleteTimeSeries( String queryId, DeleteTimeSeriesStatement deleteTimeSeriesStatement); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/AlterEncodingCompressorTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/AlterEncodingCompressorTask.java new file mode 100644 index 0000000000000..21a878397bf6b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/AlterEncodingCompressorTask.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.execution.config.metadata; + +import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult; +import org.apache.iotdb.db.queryengine.plan.execution.config.IConfigTask; +import org.apache.iotdb.db.queryengine.plan.execution.config.executor.IConfigTaskExecutor; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterEncodingCompressorStatement; + +import com.google.common.util.concurrent.ListenableFuture; + +public class AlterEncodingCompressorTask implements IConfigTask { + + private final String queryId; + + private final AlterEncodingCompressorStatement alterEncodingCompressorStatement; + + public AlterEncodingCompressorTask( + final String queryId, + final AlterEncodingCompressorStatement alterEncodingCompressorStatement) { + this.queryId = queryId; + this.alterEncodingCompressorStatement = alterEncodingCompressorStatement; + } + + @Override + public ListenableFuture execute(final IConfigTaskExecutor configTaskExecutor) + throws InterruptedException { + return configTaskExecutor.alterEncodingCompressor(queryId, alterEncodingCompressorStatement); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java index 1cb0b96698411..c4e8664cf6d6f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java @@ -30,6 +30,7 @@ import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.schema.cache.CacheClearOptions; import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; import org.apache.iotdb.commons.schema.filter.SchemaFilter; @@ -143,6 +144,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.literal.Literal; import org.apache.iotdb.db.queryengine.plan.statement.literal.LongLiteral; import org.apache.iotdb.db.queryengine.plan.statement.literal.StringLiteral; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterEncodingCompressorStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterTimeSeriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDatabaseStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDevicesStatement; @@ -285,6 +287,7 @@ import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -296,6 +299,9 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static java.util.stream.Collectors.toList; +import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_TIMESERIES_COMPRESSION; +import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_TIMESERIES_COMPRESSOR; +import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_TIMESERIES_ENCODING; import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_RESULT_NODES; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.FIELD; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TAG; @@ -457,13 +463,11 @@ private void checkPropsInCreateTimeSeries(CreateTimeSeriesStatement createTimeSe final IoTDBDescriptor ioTDBDescriptor = IoTDBDescriptor.getInstance(); createTimeSeriesStatement.setEncoding( ioTDBDescriptor.getDefaultEncodingByType(createTimeSeriesStatement.getDataType())); - if (props != null - && props.containsKey(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase())) { - String encodingString = - props.get(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase()).toUpperCase(); + if (props != null && props.containsKey(COLUMN_TIMESERIES_ENCODING.toLowerCase())) { + String encodingString = props.get(COLUMN_TIMESERIES_ENCODING.toLowerCase()).toUpperCase(); try { createTimeSeriesStatement.setEncoding(TSEncoding.valueOf(encodingString)); - props.remove(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase()); + props.remove(COLUMN_TIMESERIES_ENCODING.toLowerCase()); } catch (Exception e) { throw new SemanticException(String.format("Unsupported encoding: %s", encodingString)); } @@ -520,13 +524,12 @@ public void parseAttributeClausesForCreateAlignedTimeSeries( } TSEncoding encoding = IoTDBDescriptor.getInstance().getDefaultEncodingByType(dataType); - if (props.containsKey(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase())) { - String encodingString = - props.get(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase()).toUpperCase(); + if (props.containsKey(COLUMN_TIMESERIES_ENCODING.toLowerCase())) { + String encodingString = props.get(COLUMN_TIMESERIES_ENCODING.toLowerCase()).toUpperCase(); try { encoding = TSEncoding.valueOf(encodingString); createAlignedTimeSeriesStatement.addEncoding(encoding); - props.remove(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase()); + props.remove(COLUMN_TIMESERIES_ENCODING.toLowerCase()); } catch (Exception e) { throw new SemanticException(String.format("unsupported encoding: %s", encodingString)); } @@ -660,6 +663,46 @@ public void parseAliasClause( } } + @Override + public Statement visitAlterEncodingCompressor( + final IoTDBSqlParser.AlterEncodingCompressorContext ctx) { + final PathPatternTree tree = new PathPatternTree(); + ctx.prefixPath().forEach(path -> tree.appendPathPattern(parsePrefixPath(path))); + tree.constructTree(); + TSEncoding encoding = null; + CompressionType compressor = null; + for (final IoTDBSqlParser.AttributePairContext pair : ctx.attributePair()) { + final String key = parseAttributeKey(pair.key).toLowerCase(Locale.ENGLISH); + final String value = parseAttributeValue(pair.value).toUpperCase(Locale.ENGLISH); + switch (key) { + case COLUMN_TIMESERIES_ENCODING: + try { + encoding = TSEncoding.valueOf(value); + } catch (final Exception e) { + throw new SemanticException(String.format("Unsupported encoding: %s", value)); + } + break; + case COLUMN_TIMESERIES_COMPRESSOR: + case COLUMN_TIMESERIES_COMPRESSION: + try { + compressor = CompressionType.valueOf(value); + } catch (final Exception e) { + throw new SemanticException(String.format("Unsupported compressor: %s", value)); + } + break; + default: + throw new SemanticException(String.format("property %s is unsupported yet.", key)); + } + } + + return new AlterEncodingCompressorStatement( + tree, + encoding, + compressor, + Objects.nonNull(ctx.EXISTS()), + Objects.nonNull(ctx.PERMITTED())); + } + // Drop Timeseries ====================================================================== @Override @@ -3843,13 +3886,12 @@ void parseAttributeClauseForSchemaTemplate( } TSEncoding encoding = IoTDBDescriptor.getInstance().getDefaultEncodingByType(dataType); - if (props.containsKey(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase())) { - String encodingString = - props.get(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase()).toUpperCase(); + if (props.containsKey(COLUMN_TIMESERIES_ENCODING.toLowerCase())) { + String encodingString = props.get(COLUMN_TIMESERIES_ENCODING.toLowerCase()).toUpperCase(); try { encoding = TSEncoding.valueOf(encodingString); encodings.add(encoding); - props.remove(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase()); + props.remove(COLUMN_TIMESERIES_ENCODING.toLowerCase()); } catch (Exception e) { throw new SemanticException(String.format("Unsupported encoding: %s", encodingString)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java index 61b0b1d4a568a..902102797c984 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java @@ -39,6 +39,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.TimeSeriesCountNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.TimeSeriesSchemaScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.ActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterTimeSeriesNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.BatchActivateTemplateNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.ConstructSchemaBlackListNode; @@ -262,6 +263,7 @@ public enum PlanNodeType { CONTINUOUS_SAME_SEARCH_INDEX_SEPARATOR((short) 97), LAST_QUERY_SCAN((short) 98), + ALTER_ENCODING_COMPRESSOR((short) 99), CREATE_OR_UPDATE_TABLE_DEVICE((short) 902), TABLE_DEVICE_QUERY_SCAN((short) 903), @@ -593,6 +595,8 @@ public static PlanNode deserialize(ByteBuffer buffer, short nodeType) { "You should never see ContinuousSameSearchIndexSeparatorNode in this function, because ContinuousSameSearchIndexSeparatorNode should never be used in network transmission."); case 98: return LastQueryScanNode.deserialize(buffer); + case 99: + return AlterEncodingCompressorNode.deserialize(buffer); case 902: return CreateOrUpdateTableDeviceNode.deserialize(buffer); case 903: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java index 629b06f547855..bbc83cce18b61 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java @@ -36,6 +36,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.TimeSeriesCountNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.TimeSeriesSchemaScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.ActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterTimeSeriesNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.BatchActivateTemplateNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.ConstructSchemaBlackListNode; @@ -497,6 +498,10 @@ public R visitDeleteTimeseries(DeleteTimeSeriesNode node, C context) { return visitPlan(node, context); } + public R visitAlterEncodingCompressor(AlterEncodingCompressorNode node, C context) { + return visitPlan(node, context); + } + public R visitConstructSchemaBlackList(ConstructSchemaBlackListNode node, C context) { return visitPlan(node, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/AlterEncodingCompressorNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/AlterEncodingCompressorNode.java new file mode 100644 index 0000000000000..2e4c711483eaa --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/AlterEncodingCompressorNode.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write; + +import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.utils.SerializeUtils; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegionPlan; +import org.apache.iotdb.db.schemaengine.schemaregion.SchemaRegionPlanType; +import org.apache.iotdb.db.schemaengine.schemaregion.SchemaRegionPlanVisitor; + +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; + +public class AlterEncodingCompressorNode extends PlanNode implements ISchemaRegionPlan { + + public static final AlterEncodingCompressorNode MOCK_INSTANCE = + new AlterEncodingCompressorNode(new PlanNodeId(""), null, false, null, null); + + private final PathPatternTree patternTree; + private final boolean ifExists; + private final TSEncoding encoding; + private final CompressionType compressionType; + + public AlterEncodingCompressorNode( + final PlanNodeId id, + final PathPatternTree patternTree, + final boolean ifExists, + final TSEncoding encoding, + final CompressionType compressionType) { + super(id); + this.patternTree = patternTree; + this.ifExists = ifExists; + this.encoding = encoding; + this.compressionType = compressionType; + } + + public PathPatternTree getPatternTree() { + return patternTree; + } + + public boolean isIfExists() { + return ifExists; + } + + public TSEncoding getEncoding() { + return encoding; + } + + public CompressionType getCompressionType() { + return compressionType; + } + + @Override + public List getChildren() { + return null; + } + + @Override + public void addChild(final PlanNode child) {} + + @Override + public PlanNode clone() { + return null; + } + + @Override + public int allowedChildCount() { + return 0; + } + + @Override + public List getOutputColumnNames() { + return null; + } + + @Override + public R accept(final PlanVisitor visitor, final C context) { + return visitor.visitAlterEncodingCompressor(this, context); + } + + @Override + protected void serializeAttributes(final ByteBuffer byteBuffer) { + PlanNodeType.ALTER_ENCODING_COMPRESSOR.serialize(byteBuffer); + patternTree.serialize(byteBuffer); + ReadWriteIOUtils.write(ifExists, byteBuffer); + ReadWriteIOUtils.write(SerializeUtils.serializeNullable(encoding), byteBuffer); + ReadWriteIOUtils.write(SerializeUtils.serializeNullable(compressionType), byteBuffer); + } + + @Override + protected void serializeAttributes(final DataOutputStream stream) throws IOException { + PlanNodeType.ALTER_ENCODING_COMPRESSOR.serialize(stream); + patternTree.serialize(stream); + ReadWriteIOUtils.write(ifExists, stream); + ReadWriteIOUtils.write(SerializeUtils.serializeNullable(encoding), stream); + ReadWriteIOUtils.write(SerializeUtils.serializeNullable(compressionType), stream); + } + + public static AlterEncodingCompressorNode deserialize(final ByteBuffer byteBuffer) { + final PathPatternTree patternTree = PathPatternTree.deserialize(byteBuffer); + final boolean ifExists = ReadWriteIOUtils.readBoolean(byteBuffer); + final TSEncoding encoding = + SerializeUtils.deserializeEncodingNullable(ReadWriteIOUtils.readByte(byteBuffer)); + final CompressionType compressor = + SerializeUtils.deserializeCompressorNullable(ReadWriteIOUtils.readByte(byteBuffer)); + final PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); + return new AlterEncodingCompressorNode(planNodeId, patternTree, ifExists, encoding, compressor); + } + + @Override + public SchemaRegionPlanType getPlanType() { + return SchemaRegionPlanType.ALTER_ENCODING_COMPRESSOR; + } + + @Override + public R accept(final SchemaRegionPlanVisitor visitor, final C context) { + return visitor.visitAlterEncodingCompressor(this, context); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java index d939a85d7b457..9f4c2cf9fb695 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java @@ -27,7 +27,10 @@ import org.apache.iotdb.commons.auth.entity.PrivilegeType; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.path.PathPatternTreeUtils; +import org.apache.iotdb.commons.schema.table.Audit; +import org.apache.iotdb.commons.utils.StatusUtils; import org.apache.iotdb.db.audit.DNAuditLogger; import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.queryengine.plan.statement.AuthorType; @@ -42,6 +45,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.internal.InternalBatchActivateTemplateStatement; import org.apache.iotdb.db.queryengine.plan.statement.internal.InternalCreateMultiTimeSeriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.internal.InternalCreateTimeSeriesStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterEncodingCompressorStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterTimeSeriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDatabaseStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDevicesStatement; @@ -168,6 +172,7 @@ import static org.apache.iotdb.commons.schema.table.Audit.TREE_MODEL_AUDIT_DATABASE_PATH; import static org.apache.iotdb.commons.schema.table.Audit.includeByAuditTreeDB; import static org.apache.iotdb.db.auth.AuthorityChecker.SUCCEED; +import static org.apache.iotdb.db.auth.AuthorityChecker.getAuthorizedPathTree; import static org.apache.iotdb.db.queryengine.plan.relational.security.AccessControlImpl.READ_ONLY_DB_ERROR_MSG; public class TreeAccessCheckVisitor extends StatementVisitor { @@ -935,7 +940,7 @@ public TSStatus visitAlterDatabase( } @Override - public TSStatus visitShowStorageGroup( + public TSStatus visitShowDatabase( ShowDatabaseStatement showDatabaseStatement, TreeAccessCheckContext context) { context .setAuditLogOperation(AuditLogOperation.QUERY) @@ -973,7 +978,7 @@ public TSStatus visitCountStorageGroup( } @Override - public TSStatus visitDeleteStorageGroup( + public TSStatus visitDeleteDatabase( DeleteDatabaseStatement statement, TreeAccessCheckContext context) { context .setAuditLogOperation(AuditLogOperation.DDL) @@ -1415,6 +1420,56 @@ public TSStatus visitAlterTimeSeries( return checkTimeSeriesPermission(context, statement::getPaths, PrivilegeType.WRITE_SCHEMA); } + @Override + public TSStatus visitAlterEncodingCompressor( + final AlterEncodingCompressorStatement alterEncodingCompressorStatement, + final TreeAccessCheckContext context) { + context.setAuditLogOperation(AuditLogOperation.DDL); + final boolean audit = + checkHasGlobalAuth( + context, PrivilegeType.AUDIT, alterEncodingCompressorStatement.getPaths()::toString); + if (audit) { + alterEncodingCompressorStatement.setWithAudit(true); + } + if (alterEncodingCompressorStatement.ifPermitted()) { + try { + final PathPatternTree authTree = + getAuthorizedPathTree(context.getUsername(), PrivilegeType.WRITE_SCHEMA); + if (audit) { + authTree.appendPathPattern(Audit.TREE_MODEL_AUDIT_DATABASE_PATH_PATTERN, true); + authTree.constructTree(); + } + alterEncodingCompressorStatement.setPatternTree( + PathPatternTreeUtils.intersectWithFullPathPrefixTree( + alterEncodingCompressorStatement.getPatternTree(), authTree)); + return StatusUtils.OK; + } catch (final AuthException e) { + recordObjectAuthenticationAuditLog( + context.setResult(false), + () -> + alterEncodingCompressorStatement.getPaths().stream() + .distinct() + .collect(Collectors.toList()) + .toString()); + return new TSStatus(e.getCode().getStatusCode()); + } + } + // Check audit privilege + if (!audit) { + for (final PartialPath path : alterEncodingCompressorStatement.getPaths()) { + if (includeByAuditTreeDB(path)) { + return new TSStatus(TSStatusCode.NO_PERMISSION.getStatusCode()) + .setMessage( + String.format( + "'AUDIT' permission is needed to alter the encoding and compressor of database %s", + TREE_MODEL_AUDIT_DATABASE)); + } + } + } + return checkTimeSeriesPermission( + context, alterEncodingCompressorStatement::getPaths, PrivilegeType.WRITE_SCHEMA); + } + @Override public TSStatus visitDeleteTimeSeries( DeleteTimeSeriesStatement statement, TreeAccessCheckContext context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementType.java index 4284a484d573a..d75e2e09b7c28 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementType.java @@ -58,6 +58,7 @@ public enum StatementType { CREATE_MULTI_TIME_SERIES, DELETE_TIME_SERIES, ALTER_TIME_SERIES, + ALTER_ENCODING_COMPRESSOR, CHANGE_ALIAS, CHANGE_TAG_OFFSET, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java index 3bafdf8bfe0f0..7ce212c32970a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java @@ -34,6 +34,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.internal.InternalCreateMultiTimeSeriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.internal.InternalCreateTimeSeriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.internal.SeriesSchemaFetchStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterEncodingCompressorStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterTimeSeriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDatabaseStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountDevicesStatement; @@ -196,11 +197,16 @@ public R visitAlterTimeSeries(AlterTimeSeriesStatement alterTimeSeriesStatement, return visitStatement(alterTimeSeriesStatement, context); } + public R visitAlterEncodingCompressor( + AlterEncodingCompressorStatement alterEncodingCompressorStatement, C context) { + return visitStatement(alterEncodingCompressorStatement, context); + } + public R visitDeleteTimeSeries(DeleteTimeSeriesStatement deleteTimeSeriesStatement, C context) { return visitStatement(deleteTimeSeriesStatement, context); } - public R visitDeleteStorageGroup(DeleteDatabaseStatement deleteDatabaseStatement, C context) { + public R visitDeleteDatabase(DeleteDatabaseStatement deleteDatabaseStatement, C context) { return visitStatement(deleteDatabaseStatement, context); } @@ -384,7 +390,7 @@ public R visitAuthor(AuthorStatement authorStatement, C context) { return visitStatement(authorStatement, context); } - public R visitShowStorageGroup(ShowDatabaseStatement showDatabaseStatement, C context) { + public R visitShowDatabase(ShowDatabaseStatement showDatabaseStatement, C context) { return visitStatement(showDatabaseStatement, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/AlterEncodingCompressorStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/AlterEncodingCompressorStatement.java new file mode 100644 index 0000000000000..a8dd88841a437 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/AlterEncodingCompressorStatement.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.statement.metadata; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.db.queryengine.plan.analyze.QueryType; +import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement; +import org.apache.iotdb.db.queryengine.plan.statement.Statement; +import org.apache.iotdb.db.queryengine.plan.statement.StatementType; +import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; + +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; + +import java.util.List; +import java.util.Objects; + +public class AlterEncodingCompressorStatement extends Statement implements IConfigStatement { + + private PathPatternTree patternTree; + private final TSEncoding encoding; + private final CompressionType compressor; + private final boolean ifExists; + private final boolean ifPermitted; + private boolean withAudit = false; + + public AlterEncodingCompressorStatement( + final PathPatternTree pathPatternTree, + final TSEncoding encoding, + final CompressionType compressor, + final boolean ifExists, + final boolean ifPermitted) { + statementType = StatementType.ALTER_ENCODING_COMPRESSOR; + this.patternTree = pathPatternTree; + this.encoding = encoding; + this.compressor = compressor; + this.ifExists = ifExists; + this.ifPermitted = ifPermitted; + } + + public TSEncoding getEncoding() { + return encoding; + } + + public CompressionType getCompressor() { + return compressor; + } + + public PathPatternTree getPatternTree() { + return patternTree; + } + + public void setPatternTree(final PathPatternTree patternTree) { + this.patternTree = patternTree; + } + + public boolean ifExists() { + return ifExists; + } + + public boolean ifPermitted() { + return ifPermitted; + } + + public void setWithAudit(final boolean withAudit) { + this.withAudit = withAudit; + } + + public boolean isWithAudit() { + return withAudit; + } + + @Override + public List getPaths() { + return patternTree.getAllPathPatterns(); + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final AlterEncodingCompressorStatement that = (AlterEncodingCompressorStatement) obj; + return Objects.equals(this.patternTree, that.patternTree) + && Objects.equals(this.encoding, that.encoding) + && Objects.equals(this.compressor, that.compressor) + && Objects.equals(this.ifExists, that.ifExists) + && Objects.equals(this.ifPermitted, that.ifPermitted); + } + + @Override + public int hashCode() { + return Objects.hash(patternTree, encoding, compressor, ifExists, ifPermitted); + } + + @Override + public R accept(final StatementVisitor visitor, final C context) { + return visitor.visitAlterEncodingCompressor(this, context); + } + + @Override + public QueryType getQueryType() { + return QueryType.WRITE; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/DeleteDatabaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/DeleteDatabaseStatement.java index 9861331974b38..6b538267555f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/DeleteDatabaseStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/DeleteDatabaseStatement.java @@ -63,7 +63,7 @@ public List getPrefixPath() { @Override public R accept(StatementVisitor visitor, C context) { - return visitor.visitDeleteStorageGroup(this, context); + return visitor.visitDeleteDatabase(this, context); } public void setPrefixPath(List prefixPathList) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java index d91e1f65f0d85..e1a536622429b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java @@ -116,7 +116,7 @@ public void buildTSBlock( @Override public R accept(final StatementVisitor visitor, C context) { - return visitor.visitShowStorageGroup(this, context); + return visitor.visitShowDatabase(this, context); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/ISchemaRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/ISchemaRegion.java index a9b002bbd6d15..916e52b4acabc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/ISchemaRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/ISchemaRegion.java @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.db.exception.metadata.SchemaQuotaExceededException; import org.apache.iotdb.db.queryengine.common.schematree.ClusterSchemaTree; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.cache.TableId; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.ConstructTableDevicesBlackListNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode; @@ -208,6 +209,8 @@ Pair constructSchemaBlackList(final PathPatternTree patternTree) */ void deleteTimeseriesInBlackList(final PathPatternTree patternTree) throws MetadataException; + void alterEncodingCompressor(final AlterEncodingCompressorNode node) throws MetadataException; + // endregion // region Interfaces for Logical View diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanType.java index 9808a19143b5c..b0d41c725eeb0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanType.java @@ -39,6 +39,8 @@ public enum SchemaRegionPlanType { ACTIVATE_TEMPLATE_IN_CLUSTER((byte) 63), PRE_DELETE_TIMESERIES_IN_CLUSTER((byte) 64), ROLLBACK_PRE_DELETE_TIMESERIES((byte) 65), + ALTER_ENCODING_COMPRESSOR((byte) 66), + // endregion PRE_DEACTIVATE_TEMPLATE((byte) 0), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanVisitor.java index b0514d7d5428e..0cf087a40e117 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanVisitor.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.ConstructTableDevicesBlackListNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.DeleteTableDeviceNode; @@ -179,4 +180,9 @@ public R visitDropTableAttribute( final TableAttributeColumnDropNode dropTableAttributePlan, final C context) { return visitSchemaRegionPlan(dropTableAttributePlan, context); } + + public R visitAlterEncodingCompressor( + final AlterEncodingCompressorNode alterEncodingCompressorNode, final C context) { + return visitSchemaRegionPlan(alterEncodingCompressorNode, context); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java index 12873ab563cf2..3a80abba2b8c7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java @@ -38,6 +38,7 @@ import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.metadata.PathNotExistException; import org.apache.iotdb.db.exception.metadata.SchemaDirCreationFailureException; import org.apache.iotdb.db.exception.metadata.SchemaQuotaExceededException; import org.apache.iotdb.db.exception.metadata.SeriesOverflowException; @@ -49,6 +50,7 @@ import org.apache.iotdb.db.queryengine.execution.relational.ColumnTransformerBuilder; import org.apache.iotdb.db.queryengine.plan.analyze.TypeProvider; import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.CreateTimeSeriesNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.InputLocation; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; @@ -941,21 +943,36 @@ public Set fetchSchemaBlackList(PathPatternTree patternTree) } @Override - public void deleteTimeseriesInBlackList(PathPatternTree patternTree) throws MetadataException { - for (PartialPath pathPattern : patternTree.getAllPathPatterns()) { - for (PartialPath path : mTree.getPreDeletedTimeSeries(pathPattern)) { + public void deleteTimeseriesInBlackList(final PathPatternTree patternTree) + throws MetadataException { + for (final PartialPath pathPattern : patternTree.getAllPathPatterns()) { + for (final PartialPath path : mTree.getPreDeletedTimeSeries(pathPattern)) { try { deleteSingleTimeseriesInBlackList(path); writeToMLog( SchemaRegionWritePlanFactory.getDeleteTimeSeriesPlan( Collections.singletonList(path))); - } catch (IOException e) { + } catch (final IOException e) { throw new MetadataException(e); } } } } + @Override + public void alterEncodingCompressor(final AlterEncodingCompressorNode node) + throws MetadataException { + boolean exist = false; + for (final PartialPath pathPattern : node.getPatternTree().getAllPathPatterns()) { + exist |= + mTree.alterEncodingCompressor(pathPattern, node.getEncoding(), node.getCompressionType()); + } + if (!exist) { + throw new PathNotExistException(node.getPatternTree().getAllPathPatterns().toString(), false); + } + writeToMLog(node); + } + @Override public void createLogicalView(final ICreateLogicalViewPlan plan) throws MetadataException { if (!regionStatistics.isAllowToCreateNewSeries()) { @@ -2060,5 +2077,17 @@ public RecoverOperationResult visitDropTableAttribute( return new RecoverOperationResult(e); } } + + @Override + public RecoverOperationResult visitAlterEncodingCompressor( + final AlterEncodingCompressorNode alterEncodingCompressorNode, + final SchemaRegionMemoryImpl context) { + try { + alterEncodingCompressor(alterEncodingCompressorNode); + return RecoverOperationResult.SUCCESS; + } catch (final MetadataException e) { + return new RecoverOperationResult(e); + } + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java index 0e180b2029b28..58e8fd786a5ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java @@ -39,6 +39,7 @@ import org.apache.iotdb.db.exception.metadata.SchemaDirCreationFailureException; import org.apache.iotdb.db.exception.metadata.SchemaQuotaExceededException; import org.apache.iotdb.db.queryengine.common.schematree.ClusterSchemaTree; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.cache.TableId; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.ConstructTableDevicesBlackListNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode; @@ -940,6 +941,13 @@ public void deleteTimeseriesInBlackList(PathPatternTree patternTree) throws Meta } } + @Override + public void alterEncodingCompressor(final AlterEncodingCompressorNode node) + throws MetadataException { + throw new UnsupportedOperationException( + "PBTree does not support altering encoding and compressor yet."); + } + @Override public void createLogicalView(ICreateLogicalViewPlan plan) throws MetadataException { while (!regionStatistics.isAllowToCreateNewSeries()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/logfile/visitor/SchemaRegionPlanDeserializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/logfile/visitor/SchemaRegionPlanDeserializer.java index 258f271cc9d7c..163ccb4e59de8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/logfile/visitor/SchemaRegionPlanDeserializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/logfile/visitor/SchemaRegionPlanDeserializer.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.path.PathDeserializeUtil; import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.ConstructTableDevicesBlackListNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.DeleteTableDeviceNode; @@ -475,5 +476,11 @@ public ISchemaRegionPlan visitDropTableAttribute( final TableAttributeColumnDropNode dropTableAttributePlan, final ByteBuffer buffer) { return (TableAttributeColumnDropNode) PlanNodeType.deserialize(buffer); } + + @Override + public ISchemaRegionPlan visitAlterEncodingCompressor( + final AlterEncodingCompressorNode alterEncodingCompressorNode, final ByteBuffer buffer) { + return (AlterEncodingCompressorNode) PlanNodeType.deserialize(buffer); + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/logfile/visitor/SchemaRegionPlanSerializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/logfile/visitor/SchemaRegionPlanSerializer.java index 63854468b347d..b7b7d9758ca4b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/logfile/visitor/SchemaRegionPlanSerializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/logfile/visitor/SchemaRegionPlanSerializer.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.ConstructTableDevicesBlackListNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.DeleteTableDeviceNode; @@ -545,6 +546,13 @@ public SchemaRegionPlanSerializationResult visitDropTableAttribute( return visitPlanNode(dropTableAttributePlan, outputStream); } + @Override + public SchemaRegionPlanSerializationResult visitAlterEncodingCompressor( + final AlterEncodingCompressorNode alterEncodingCompressorNode, + final DataOutputStream outputStream) { + return visitPlanNode(alterEncodingCompressorNode, outputStream); + } + private SchemaRegionPlanSerializationResult visitPlanNode( final PlanNode planNode, final DataOutputStream outputStream) { try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java index 387d8c6643177..53e8c2617bcb3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java @@ -73,6 +73,7 @@ import org.apache.iotdb.db.schemaengine.schemaregion.utils.filter.DeviceFilterVisitor; import org.apache.iotdb.db.schemaengine.template.Template; import org.apache.iotdb.db.storageengine.rescon.quotas.DataNodeSpaceQuotaManager; +import org.apache.iotdb.db.utils.SchemaUtils; import org.apache.iotdb.rpc.TSStatusCode; import com.google.common.util.concurrent.ListenableFuture; @@ -676,6 +677,38 @@ protected Void collectMeasurement(final IMeasurementMNode node) { return result; } + public boolean alterEncodingCompressor( + final PartialPath pathPattern, final TSEncoding encoding, final CompressionType compressor) + throws MetadataException { + final boolean[] exist = {false}; + try (final MeasurementUpdater collector = + new MeasurementUpdater( + rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) { + @Override + protected void updateMeasurement(final IMeasurementMNode node) + throws MetadataException { + if (node.isLogicalView()) { + return; + } + exist[0] = true; + final IMeasurementSchema schema = node.getSchema(); + if (Objects.nonNull(encoding)) { + SchemaUtils.checkDataTypeWithEncoding(node.getDataType(), encoding); + } + node.setSchema( + new MeasurementSchema( + schema.getMeasurementName(), + schema.getType(), + Objects.nonNull(encoding) ? encoding : schema.getEncodingType(), + Objects.nonNull(compressor) ? compressor : schema.getCompressor(), + schema.getProps())); + } + }) { + collector.traverse(); + } + return exist[0]; + } + // TODO: seems useless public Set getDevicesOfPreDeletedTimeSeries(final PartialPath pathPattern) throws MetadataException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/SchemaRegionWritePlanFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/SchemaRegionWritePlanFactory.java index 5fe5ec993cd48..8a87a6f38293d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/SchemaRegionWritePlanFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/SchemaRegionWritePlanFactory.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.AlterEncodingCompressorNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.ConstructTableDevicesBlackListNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.DeleteTableDeviceNode; @@ -87,6 +88,8 @@ public static ISchemaRegionPlan getEmptyPlan(final SchemaRegionPlanType planType return new PreDeleteTimeSeriesPlanImpl(); case ROLLBACK_PRE_DELETE_TIMESERIES: return new RollbackPreDeleteTimeSeriesPlanImpl(); + case ALTER_ENCODING_COMPRESSOR: + return AlterEncodingCompressorNode.MOCK_INSTANCE; case PRE_DEACTIVATE_TEMPLATE: return new PreDeactivateTemplatePlanImpl(); case ROLLBACK_PRE_DEACTIVATE_TEMPLATE: diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/MetadataException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/MetadataException.java index 3a83a6310f544..5229c97163025 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/MetadataException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/MetadataException.java @@ -45,7 +45,7 @@ public MetadataException(String msg, boolean isUserException) { } public MetadataException(String message, Throwable cause) { - super(message + cause.getMessage(), TSStatusCode.METADATA_ERROR.getStatusCode()); + super(message, cause, TSStatusCode.METADATA_ERROR.getStatusCode()); } public MetadataException(IoTDBException exception) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternNode.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternNode.java index 67616c5070b49..2003fdfcc95d1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternNode.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternNode.java @@ -251,6 +251,13 @@ void serializeChildren(final DataOutputStream outputStream) throws IOException { } } + void clear() { + if (Objects.nonNull(valueSet)) { + valueSet.clear(); + } + children.clear(); + } + public static > PathPatternNode deserializeNode( final ByteBuffer buffer, final T serializer, final Consumer nodeNameProcessor) { final PathPatternNode node = diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java index f7703dd6d98dc..9f16d306b43d2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java @@ -97,9 +97,17 @@ public void appendFullPath(PartialPath devicePath, String measurement) { appendBranchWithoutPrune(root, pathNodes, 0); } + public void appendPathPattern(final PartialPath pathPattern) { + appendPathPattern(pathPattern, false); + } + /** Add a pathPattern (may contain wildcards) to pathPatternList. */ - public void appendPathPattern(PartialPath pathPattern) { + public void appendPathPattern(final PartialPath pathPattern, final boolean isReload) { if (useWildcard) { + // This does not guarantee multi-thread safety + if (isReload && (pathPatternList == null || pathPatternList.isEmpty())) { + pathPatternList = getAllPathPatterns(); + } boolean isExist = false; for (PartialPath path : pathPatternList) { if (path.include(pathPattern)) { @@ -113,6 +121,9 @@ public void appendPathPattern(PartialPath pathPattern) { pathPatternList.removeIf(pathPattern::include); pathPatternList.add(pathPattern); } + if (isReload) { + root.clear(); + } } else { appendBranchWithoutPrune(root, pathPattern.getNodes(), 0); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java index 0baf3a8b992da..c0ee96e2b1050 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java @@ -95,6 +95,7 @@ public abstract class IoTDBFileReceiver implements IoTDBReceiver { protected final AtomicBoolean validateTsFile = new AtomicBoolean(true); protected final AtomicBoolean shouldMarkAsPipeRequest = new AtomicBoolean(true); + protected final AtomicBoolean skipIfNoPrivileges = new AtomicBoolean(false); @Override public IoTDBSinkRequestVersion getVersion() { @@ -347,6 +348,11 @@ protected TPipeTransferResp handleTransferHandshakeV2(final PipeTransferHandshak .getOrDefault( PipeTransferHandshakeConstant.HANDSHAKE_KEY_MARK_AS_PIPE_REQUEST, "true"))); + skipIfNoPrivileges.set( + Boolean.parseBoolean( + req.getParams() + .getOrDefault(PipeTransferHandshakeConstant.HANDSHAKE_KEY_SKIP_IF, "false"))); + // Handle the handshake request as a v1 request. // Here we construct a fake "dataNode" request to valid from v1 validation logic, though // it may not require the actual type of the v1 request. diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBClientManager.java index 3f99e50e0c115..3e009b279d769 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBClientManager.java @@ -49,6 +49,7 @@ public abstract class IoTDBClientManager { protected final String loadTsFileStrategy; protected final boolean shouldMarkAsPipeRequest; + protected final boolean skipIfNoPrivileges; // This flag indicates whether the receiver supports mods transferring if // it is a DataNode receiver. The flag is useless for configNode receiver. @@ -69,7 +70,8 @@ protected IoTDBClientManager( final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, final boolean validateTsFile, - final boolean shouldMarkAsPipeRequest) { + final boolean shouldMarkAsPipeRequest, + final boolean skipIfNoPrivileges) { this.endPointList = endPointList; this.useLeaderCache = useLeaderCache; @@ -80,6 +82,7 @@ protected IoTDBClientManager( this.loadTsFileStrategy = loadTsFileStrategy; this.validateTsFile = validateTsFile; this.shouldMarkAsPipeRequest = shouldMarkAsPipeRequest; + this.skipIfNoPrivileges = skipIfNoPrivileges; } public boolean supportModsIfIsDataNodeReceiver() { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBSyncClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBSyncClientManager.java index 24ad3c4991581..4c5d51f83c557 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBSyncClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBSyncClientManager.java @@ -79,7 +79,8 @@ protected IoTDBSyncClientManager( boolean shouldReceiverConvertOnTypeMismatch, String loadTsFileStrategy, boolean validateTsFile, - boolean shouldMarkAsPipeRequest) { + boolean shouldMarkAsPipeRequest, + final boolean skipIfNoPrivileges) { super( endPoints, useLeaderCache, @@ -88,7 +89,8 @@ protected IoTDBSyncClientManager( shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, validateTsFile, - shouldMarkAsPipeRequest); + shouldMarkAsPipeRequest, + skipIfNoPrivileges); this.useSSL = useSSL; this.trustStorePath = trustStorePath; @@ -245,6 +247,9 @@ public void sendHandshakeReq(final Pair clientAndStatu params.put( PipeTransferHandshakeConstant.HANDSHAKE_KEY_MARK_AS_PIPE_REQUEST, Boolean.toString(shouldMarkAsPipeRequest)); + params.put( + PipeTransferHandshakeConstant.HANDSHAKE_KEY_SKIP_IF, + Boolean.toString(skipIfNoPrivileges)); // Try to handshake by PipeTransferHandshakeV2Req. TPipeTransferResp resp = client.pipeTransfer(buildHandshakeV2Req(params)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferHandshakeConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferHandshakeConstant.java index 8f6120131c71b..710fca828e938 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferHandshakeConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferHandshakeConstant.java @@ -31,6 +31,7 @@ public class PipeTransferHandshakeConstant { public static final String HANDSHAKE_KEY_PASSWORD = "password"; public static final String HANDSHAKE_KEY_VALIDATE_TSFILE = "validateTsFile"; public static final String HANDSHAKE_KEY_MARK_AS_PIPE_REQUEST = "markAsPipeRequest"; + public static final String HANDSHAKE_KEY_SKIP_IF = "skipIf"; private PipeTransferHandshakeConstant() { // Utility class diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSink.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSink.java index e255e1c3a4ddd..9dda99c22d3fa 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSink.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSink.java @@ -171,6 +171,7 @@ public abstract class IoTDBSink implements PipeConnector { protected boolean loadTsFileValidation; protected boolean shouldMarkAsPipeRequest; + protected boolean skipIfNoPrivileges; private boolean isRpcCompressionEnabled; private final List compressors = new ArrayList<>(); @@ -439,7 +440,7 @@ public void customize( .filter(s -> !s.isEmpty()) .map(String::toLowerCase) .collect(Collectors.toSet()); - boolean skipIfNoPrivileges = skipIfOptionSet.remove(CONNECTOR_IOTDB_SKIP_IF_NO_PRIVILEGES); + skipIfNoPrivileges = skipIfOptionSet.remove(CONNECTOR_IOTDB_SKIP_IF_NO_PRIVILEGES); if (!skipIfOptionSet.isEmpty()) { throw new PipeParameterNotValidException( String.format("Parameters in set %s are not allowed in 'skipif'", skipIfOptionSet)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSslSyncSink.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSslSyncSink.java index 0eac53701fb85..b6b8e52f1fad8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSslSyncSink.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSslSyncSink.java @@ -140,7 +140,8 @@ public void customize( shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, loadTsFileValidation, - shouldMarkAsPipeRequest); + shouldMarkAsPipeRequest, + skipIfNoPrivileges); } protected abstract IoTDBSyncClientManager constructClient( @@ -157,7 +158,8 @@ protected abstract IoTDBSyncClientManager constructClient( final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, final boolean validateTsFile, - final boolean shouldMarkAsPipeRequest); + final boolean shouldMarkAsPipeRequest, + final boolean skipIfNoPrivileges); @Override public void handshake() throws Exception { 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 a069c993abef2..bbc3a101edbb2 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 @@ -23,6 +23,8 @@ import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.common.BatchData; import org.apache.tsfile.read.common.BatchData.BatchDataType; @@ -46,6 +48,22 @@ private SerializeUtils() { // util class } + public static byte serializeNullable(final TSEncoding encoding) { + return encoding == null ? -1 : encoding.serialize(); + } + + public static TSEncoding deserializeEncodingNullable(final byte encoding) { + return encoding == -1 ? null : TSEncoding.deserialize(encoding); + } + + public static byte serializeNullable(final CompressionType compressor) { + return compressor == null ? -1 : compressor.serialize(); + } + + public static CompressionType deserializeCompressorNullable(final byte compressor) { + return compressor == -1 ? null : CompressionType.deserialize(compressor); + } + public static void serialize(String str, DataOutputStream dataOutputStream) { try { byte[] strBytes = str.getBytes(TSFileConfig.STRING_CHARSET); diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 9a4c87594be29..3ee3ca89bdcc1 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -919,6 +919,16 @@ struct TPipeConfigTransferResp { 2: optional binary body } +struct TAlterEncodingCompressorReq { + 1: required string queryId + 2: required binary pathPatternTree + 3: required byte encoding + 4: required byte compressor + 5: required bool ifExists + 6: required bool mayAlterAudit + 7: optional bool isGeneratedByPipe +} + struct TDeleteTimeSeriesReq { 1: required string queryId 2: required binary pathPatternTree @@ -1882,6 +1892,8 @@ service IConfigNodeRPCService { common.TSStatus alterSchemaTemplate(TAlterSchemaTemplateReq req) + common.TSStatus alterEncodingCompressor(TAlterEncodingCompressorReq req) + /** * Generate a set of DeleteTimeSeriesProcedure to delete some specific TimeSeries * diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index ce2cbd0f15c4b..9416633a2c41b 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -464,6 +464,14 @@ struct TDeleteTimeSeriesReq { 3: optional bool isGeneratedByPipe } +struct TAlterEncodingCompressorReq { + 1: required list schemaRegionIdList + 2: required binary pathPatternTree + 3: required bool ifExists + 4: optional byte encoding + 5: optional byte compressor +} + struct TConstructSchemaBlackListWithTemplateReq { 1: required list schemaRegionIdList 2: required map> templateSetInfo @@ -1072,6 +1080,11 @@ service IDataNodeRPCService { */ common.TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) + /** + * Alter matched timeseries to specific encoding and compressor in target schemaRegions + */ + common.TSStatus alterEncodingCompressor(TAlterEncodingCompressorReq req) + /** * Construct schema black list in target schemaRegion to block R/W on matched timeseries represent by template */ From 0971b8fa67fc36c2abb0d52f662f0d2fdf8ad173 Mon Sep 17 00:00:00 2001 From: LimJiaWenBrenda <88846228+LJW21-02@users.noreply.github.com> Date: Fri, 7 Nov 2025 10:25:38 +0800 Subject: [PATCH 032/180] Support SSL-related parameters and interactive password input in export/import tools (#16698) * Add ssl param + Support interactive pw input * fix spotless * Changed params processing order: ts > tpw > pw (cherry picked from commit ababbf337e2e642bf1225d0bb288a08c4768f409) --- .../apache/iotdb/tool/common/Constants.java | 12 ++++++ .../apache/iotdb/tool/common/OptionsUtil.java | 30 +++++++++++++ .../iotdb/tool/data/AbstractDataTool.java | 41 +++++++++++++++++- .../iotdb/tool/data/ExportDataTable.java | 10 +++-- .../iotdb/tool/data/ExportDataTree.java | 29 +++++++------ .../iotdb/tool/data/ImportDataTable.java | 10 +++-- .../iotdb/tool/data/ImportDataTree.java | 10 +++-- .../iotdb/tool/schema/AbstractSchemaTool.java | 43 +++++++++++++++++-- .../iotdb/tool/schema/ExportSchemaTable.java | 10 +++-- .../iotdb/tool/schema/ExportSchemaTree.java | 12 +++++- .../iotdb/tool/schema/ImportSchemaTable.java | 10 +++-- .../iotdb/tool/schema/ImportSchemaTree.java | 10 +++-- 12 files changed, 190 insertions(+), 37 deletions(-) diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/common/Constants.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/common/Constants.java index bca36d0241636..0d4c5f218b73b 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/common/Constants.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/common/Constants.java @@ -56,6 +56,18 @@ public class Constants { public static final String USERNAME_DESC = "Username (optional)"; public static final String USERNAME_DEFAULT_VALUE = "root"; + public static final String USE_SSL_ARGS = "usessl"; + public static final String USE_SSL_NAME = "use_ssl"; + public static final String USE_SSL_DESC = "Use SSL statement. (optional)"; + + public static final String TRUST_STORE_ARGS = "ts"; + public static final String TRUST_STORE_NAME = "trust_store"; + public static final String TRUST_STORE_DESC = "Trust store. (optional)"; + + public static final String TRUST_STORE_PWD_ARGS = "tpw"; + public static final String TRUST_STORE_PWD_NAME = "trust_store_password"; + public static final String TRUST_STORE_PWD_DESC = "Trust store password. (optional)"; + public static final String FILE_TYPE_ARGS = "ft"; public static final String FILE_TYPE_NAME = "file_type"; public static final String FILE_TYPE_ARGS_NAME = "format"; diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/common/OptionsUtil.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/common/OptionsUtil.java index 655bee2068466..d128a01bcc86d 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/common/OptionsUtil.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/common/OptionsUtil.java @@ -103,6 +103,36 @@ public static Options createCommonOptions(Options options) { .build(); options.addOption(opPassword); + Option opUseSSL = + Option.builder(USE_SSL_ARGS) + .longOpt(USE_SSL_NAME) + .optionalArg(true) + .argName(USE_SSL_NAME) + .hasArg() + .desc(USE_SSL_DESC) + .build(); + options.addOption(opUseSSL); + + Option opTrustStore = + Option.builder(TRUST_STORE_ARGS) + .longOpt(TRUST_STORE_NAME) + .optionalArg(true) + .argName(TRUST_STORE_NAME) + .hasArg() + .desc(TRUST_STORE_DESC) + .build(); + options.addOption(opTrustStore); + + Option opTrustStorePwd = + Option.builder(TRUST_STORE_PWD_ARGS) + .longOpt(TRUST_STORE_PWD_NAME) + .optionalArg(true) + .argName(TRUST_STORE_PWD_NAME) + .hasArg() + .desc(TRUST_STORE_PWD_DESC) + .build(); + options.addOption(opTrustStorePwd); + return options; } diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/AbstractDataTool.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/AbstractDataTool.java index 852c671400442..f5f11bc8d51ac 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/AbstractDataTool.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/AbstractDataTool.java @@ -19,7 +19,10 @@ package org.apache.iotdb.tool.data; +import org.apache.iotdb.cli.type.ExitType; +import org.apache.iotdb.cli.utils.CliContext; import org.apache.iotdb.cli.utils.IoTPrinter; +import org.apache.iotdb.cli.utils.JlineUtils; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; import org.apache.iotdb.commons.utils.PathUtils; @@ -50,6 +53,7 @@ import org.apache.tsfile.read.common.Field; import org.apache.tsfile.read.common.RowRecord; import org.apache.tsfile.utils.Binary; +import org.jline.reader.LineReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,6 +90,9 @@ public abstract class AbstractDataTool { protected static String endTime; protected static String username; protected static String password; + protected static Boolean useSsl; + protected static String trustStore; + protected static String trustStorePwd; protected static Boolean aligned; protected static String database; protected static String startTime; @@ -142,7 +149,8 @@ protected static String checkRequiredArg( return str; } - protected static void parseBasicParams(CommandLine commandLine) throws ArgsErrorException { + protected static void parseBasicParams(CommandLine commandLine) + throws ArgsErrorException, IOException { host = checkRequiredArg( Constants.HOST_ARGS, Constants.HOST_NAME, commandLine, Constants.HOST_DEFAULT_VALUE); @@ -155,7 +163,36 @@ protected static void parseBasicParams(CommandLine commandLine) throws ArgsError Constants.USERNAME_NAME, commandLine, Constants.USERNAME_DEFAULT_VALUE); - password = commandLine.getOptionValue(Constants.PW_ARGS, Constants.PW_DEFAULT_VALUE); + CliContext cliCtx = new CliContext(System.in, System.out, System.err, ExitType.SYSTEM_EXIT); + LineReader lineReader = JlineUtils.getLineReader(cliCtx, username, host, port); + cliCtx.setLineReader(lineReader); + String useSslStr = commandLine.getOptionValue(Constants.USE_SSL_ARGS); + useSsl = Boolean.parseBoolean(useSslStr); + if (useSsl) { + String givenTS = commandLine.getOptionValue(Constants.TRUST_STORE_ARGS); + if (givenTS != null) { + trustStore = givenTS; + } else { + trustStore = cliCtx.getLineReader().readLine("please input your trust_store:", '\0'); + } + String givenTPW = commandLine.getOptionValue(Constants.TRUST_STORE_PWD_ARGS); + if (givenTPW != null) { + trustStorePwd = givenTPW; + } else { + trustStorePwd = cliCtx.getLineReader().readLine("please input your trust_store_pwd:", '\0'); + } + } + boolean hasPw = commandLine.hasOption(Constants.PW_ARGS); + if (hasPw) { + String inputPassword = commandLine.getOptionValue(Constants.PW_ARGS); + if (inputPassword != null) { + password = inputPassword; + } else { + password = cliCtx.getLineReader().readLine("please input your password:", '\0'); + } + } else { + password = Constants.PW_DEFAULT_VALUE; + } } protected static void printHelpOptions( diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ExportDataTable.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ExportDataTable.java index 63760001f7462..dcf05baa14e66 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ExportDataTable.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ExportDataTable.java @@ -63,14 +63,18 @@ public class ExportDataTable extends AbstractExportData { @Override public void init() throws IoTDBConnectionException, StatementExecutionException { - tableSession = + TableSessionBuilder tableSessionBuilder = new TableSessionBuilder() .nodeUrls(Collections.singletonList(host + ":" + port)) .username(username) .password(password) .database(database) - .thriftMaxFrameSize(rpcMaxFrameSize) - .build(); + .thriftMaxFrameSize(rpcMaxFrameSize); + if (useSsl) { + tableSessionBuilder = + tableSessionBuilder.useSSL(true).trustStore(trustStore).trustStorePwd(trustStorePwd); + } + tableSession = tableSessionBuilder.build(); SessionDataSet sessionDataSet = tableSession.executeQueryStatement("show databases", timeout); List databases = new ArrayList<>(); while (sessionDataSet.hasNext()) { diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ExportDataTree.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ExportDataTree.java index 6751215ef0ed5..beee7f49c929a 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ExportDataTree.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ExportDataTree.java @@ -67,18 +67,23 @@ public class ExportDataTree extends AbstractExportData { @Override public void init() throws IoTDBConnectionException, StatementExecutionException, TException { - session = - new Session( - host, - Integer.parseInt(port), - username, - password, - SessionConfig.DEFAULT_FETCH_SIZE, - null, - SessionConfig.DEFAULT_INITIAL_BUFFER_CAPACITY, - rpcMaxFrameSize, - SessionConfig.DEFAULT_REDIRECTION_MODE, - SessionConfig.DEFAULT_VERSION); + Session.Builder sessionBuilder = + new Session.Builder() + .host(host) + .port(Integer.parseInt(port)) + .username(username) + .password(password) + .fetchSize(SessionConfig.DEFAULT_FETCH_SIZE) + .zoneId(null) + .thriftDefaultBufferSize(SessionConfig.DEFAULT_INITIAL_BUFFER_CAPACITY) + .thriftMaxFrameSize(rpcMaxFrameSize) + .enableRedirection(SessionConfig.DEFAULT_REDIRECTION_MODE) + .version(SessionConfig.DEFAULT_VERSION); + if (useSsl) { + sessionBuilder = + sessionBuilder.useSSL(true).trustStore(trustStore).trustStorePwd(trustStorePwd); + } + session = sessionBuilder.build(); session.open(false); timestampPrecision = session.getTimestampPrecision(); if (timeZoneID != null) { diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ImportDataTable.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ImportDataTable.java index e5740f12ccbaf..f6e84362d90d7 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ImportDataTable.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ImportDataTable.java @@ -66,7 +66,7 @@ public class ImportDataTable extends AbstractImportData { private static Map columnCategory = new HashMap<>(); public void init() throws InterruptedException { - sessionPool = + TableSessionPoolBuilder tableSessionPoolBuilder = new TableSessionPoolBuilder() .nodeUrls(Collections.singletonList(host + ":" + port)) .user(username) @@ -75,8 +75,12 @@ public void init() throws InterruptedException { .enableThriftCompression(false) .enableRedirection(false) .enableAutoFetch(false) - .database(database) - .build(); + .database(database); + if (useSsl) { + tableSessionPoolBuilder = + tableSessionPoolBuilder.useSSL(true).trustStore(trustStore).trustStorePwd(trustStorePwd); + } + sessionPool = tableSessionPoolBuilder.build(); final File file = new File(targetPath); if (!file.isFile() && !file.isDirectory()) { ioTPrinter.println(String.format("Source file or directory %s does not exist", targetPath)); diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ImportDataTree.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ImportDataTree.java index 687ed554b294d..95aa92631422a 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ImportDataTree.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/data/ImportDataTree.java @@ -61,7 +61,7 @@ public class ImportDataTree extends AbstractImportData { public void init() throws InterruptedException, IoTDBConnectionException, StatementExecutionException { - sessionPool = + SessionPool.Builder sessionPoolBuilder = new SessionPool.Builder() .host(host) .port(Integer.parseInt(port)) @@ -70,8 +70,12 @@ public void init() .maxSize(threadNum + 1) .enableIoTDBRpcCompression(false) .enableRedirection(false) - .enableAutoFetch(false) - .build(); + .enableAutoFetch(false); + if (useSsl) { + sessionPoolBuilder = + sessionPoolBuilder.useSSL(true).trustStore(trustStore).trustStorePwd(trustStorePwd); + } + sessionPool = sessionPoolBuilder.build(); sessionPool.setEnableQueryRedirection(false); if (timeZoneID != null) { sessionPool.setTimeZone(timeZoneID); diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/AbstractSchemaTool.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/AbstractSchemaTool.java index eebf6d66ab556..09442bdab878f 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/AbstractSchemaTool.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/AbstractSchemaTool.java @@ -19,7 +19,10 @@ package org.apache.iotdb.tool.schema; +import org.apache.iotdb.cli.type.ExitType; +import org.apache.iotdb.cli.utils.CliContext; import org.apache.iotdb.cli.utils.IoTPrinter; +import org.apache.iotdb.cli.utils.JlineUtils; import org.apache.iotdb.exception.ArgsErrorException; import org.apache.iotdb.session.Session; import org.apache.iotdb.tool.common.Constants; @@ -29,6 +32,7 @@ import org.apache.commons.csv.CSVPrinter; import org.apache.commons.csv.QuoteMode; import org.apache.tsfile.external.commons.lang3.StringUtils; +import org.jline.reader.LineReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,6 +50,9 @@ public abstract class AbstractSchemaTool { protected static String database; protected static String username; protected static String password; + protected static Boolean useSsl; + protected static String trustStore; + protected static String trustStorePwd; protected static Session session; protected static String queryPath; protected static int threadNum = 8; @@ -80,7 +87,8 @@ protected static String checkRequiredArg( return str; } - protected static void parseBasicParams(CommandLine commandLine) throws ArgsErrorException { + protected static void parseBasicParams(CommandLine commandLine) + throws ArgsErrorException, IOException { host = checkRequiredArg( Constants.HOST_ARGS, Constants.HOST_NAME, commandLine, Constants.HOST_DEFAULT_VALUE); @@ -93,9 +101,36 @@ protected static void parseBasicParams(CommandLine commandLine) throws ArgsError Constants.USERNAME_NAME, commandLine, Constants.USERNAME_DEFAULT_VALUE); - password = - checkRequiredArg( - Constants.PW_ARGS, Constants.PW_NAME, commandLine, Constants.PW_DEFAULT_VALUE); + CliContext cliCtx = new CliContext(System.in, System.out, System.err, ExitType.SYSTEM_EXIT); + LineReader lineReader = JlineUtils.getLineReader(cliCtx, username, host, port); + cliCtx.setLineReader(lineReader); + String useSslStr = commandLine.getOptionValue(Constants.USE_SSL_ARGS); + useSsl = Boolean.parseBoolean(useSslStr); + if (useSsl) { + String givenTS = commandLine.getOptionValue(Constants.TRUST_STORE_ARGS); + if (givenTS != null) { + trustStore = givenTS; + } else { + trustStore = cliCtx.getLineReader().readLine("please input your trust_store:", '\0'); + } + String givenTPW = commandLine.getOptionValue(Constants.TRUST_STORE_PWD_ARGS); + if (givenTPW != null) { + trustStorePwd = givenTPW; + } else { + trustStorePwd = cliCtx.getLineReader().readLine("please input your trust_store_pwd:", '\0'); + } + } + boolean hasPw = commandLine.hasOption(Constants.PW_ARGS); + if (hasPw) { + String inputPassword = commandLine.getOptionValue(Constants.PW_ARGS); + if (inputPassword != null) { + password = inputPassword; + } else { + password = cliCtx.getLineReader().readLine("please input your password:", '\0'); + } + } else { + password = Constants.PW_DEFAULT_VALUE; + } } /** diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ExportSchemaTable.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ExportSchemaTable.java index bf6c77766b205..2ea66f360e1ce 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ExportSchemaTable.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ExportSchemaTable.java @@ -52,7 +52,7 @@ public class ExportSchemaTable extends AbstractExportSchema { private static Map tableCommentList = new HashMap<>(); public void init() throws InterruptedException { - sessionPool = + TableSessionPoolBuilder tableSessionPoolBuilder = new TableSessionPoolBuilder() .nodeUrls(Collections.singletonList(host + ":" + port)) .user(username) @@ -61,8 +61,12 @@ public void init() throws InterruptedException { .enableThriftCompression(false) .enableRedirection(false) .enableAutoFetch(false) - .database(database) - .build(); + .database(database); + if (useSsl) { + tableSessionPoolBuilder = + tableSessionPoolBuilder.useSSL(true).trustStore(trustStore).trustStorePwd(trustStorePwd); + } + sessionPool = tableSessionPoolBuilder.build(); checkDatabase(); try { parseTablesBySelectSchema(String.format(Constants.EXPORT_SCHEMA_TABLES_SELECT, database)); diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ExportSchemaTree.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ExportSchemaTree.java index b047654f57ee3..c5fbdbc49d1f4 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ExportSchemaTree.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ExportSchemaTree.java @@ -43,7 +43,17 @@ public class ExportSchemaTree extends AbstractExportSchema { public void init() throws InterruptedException, IoTDBConnectionException, StatementExecutionException { - session = new Session(host, Integer.parseInt(port), username, password); + Session.Builder sessionBuilder = + new Session.Builder() + .host(host) + .port(Integer.parseInt(port)) + .username(username) + .password(password); + if (useSsl) { + sessionBuilder = + sessionBuilder.useSSL(true).trustStore(trustStore).trustStorePwd(trustStorePwd); + } + session = sessionBuilder.build(); session.open(false); } diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ImportSchemaTable.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ImportSchemaTable.java index 138c0b946c5af..89d9c56ad9fa0 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ImportSchemaTable.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ImportSchemaTable.java @@ -48,7 +48,7 @@ public class ImportSchemaTable extends AbstractImportSchema { private static ITableSessionPool sessionPool; public void init() throws InterruptedException { - sessionPool = + TableSessionPoolBuilder tableSessionPoolBuilder = new TableSessionPoolBuilder() .nodeUrls(Collections.singletonList(host + ":" + port)) .user(username) @@ -57,8 +57,12 @@ public void init() throws InterruptedException { .enableThriftCompression(false) .enableRedirection(false) .enableAutoFetch(false) - .database(database) - .build(); + .database(database); + if (useSsl) { + tableSessionPoolBuilder = + tableSessionPoolBuilder.useSSL(true).trustStore(trustStore).trustStorePwd(trustStorePwd); + } + sessionPool = tableSessionPoolBuilder.build(); final File file = new File(targetPath); if (!file.isFile() && !file.isDirectory()) { ioTPrinter.println(String.format("Source file or directory %s does not exist", targetPath)); diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ImportSchemaTree.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ImportSchemaTree.java index bbeb31513d3dc..c5edb7d52e929 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ImportSchemaTree.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/schema/ImportSchemaTree.java @@ -61,7 +61,7 @@ public class ImportSchemaTree extends AbstractImportSchema { public void init() throws InterruptedException, IoTDBConnectionException, StatementExecutionException { - sessionPool = + SessionPool.Builder sessionPoolBuilder = new SessionPool.Builder() .host(host) .port(Integer.parseInt(port)) @@ -70,8 +70,12 @@ public void init() .maxSize(threadNum + 1) .enableIoTDBRpcCompression(false) .enableRedirection(false) - .enableAutoFetch(false) - .build(); + .enableAutoFetch(false); + if (useSsl) { + sessionPoolBuilder = + sessionPoolBuilder.useSSL(true).trustStore(trustStore).trustStorePwd(trustStorePwd); + } + sessionPool = sessionPoolBuilder.build(); sessionPool.setEnableQueryRedirection(false); final File file = new File(targetPath); if (!file.isFile() && !file.isDirectory()) { From 0e15626c69151bcadb4c206e0048d3a0716c4bba Mon Sep 17 00:00:00 2001 From: alpass163 Date: Fri, 7 Nov 2025 17:05:08 +0800 Subject: [PATCH 033/180] implement the intersect (distinct | all ) for table model (#16700) (cherry picked from commit 4ebb89b91e8e433c7997c2fe11ffeb4e2afdf197) --- .../query/recent/IoTDBIntersectTableIT.java | 153 ++++++++ .../planner/plan/node/PlanGraphPrinter.java | 10 + .../plan/planner/plan/node/PlanNodeType.java | 4 + .../plan/planner/plan/node/PlanVisitor.java | 5 + .../relational/planner/RelationPlanner.java | 24 +- .../iterative/rule/ImplementIntersectAll.java | 95 +++++ .../ImplementIntersectDistinctAsUnion.java | 88 +++++ .../rule/PruneDistinctAggregation.java | 9 + .../rule/SetOperationNodeTranslator.java | 355 ++++++++++++++++++ .../planner/node/IntersectNode.java | 98 +++++ .../relational/planner/node/Patterns.java | 22 +- .../optimizations/LogicalOptimizeFactory.java | 13 + .../UnaliasSymbolReferences.java | 31 ++ .../relational/analyzer/IntersectTest.java | 121 ++++++ .../relational/grammar/sql/RelationalSql.g4 | 5 +- 15 files changed, 1015 insertions(+), 18 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBIntersectTableIT.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/ImplementIntersectAll.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/ImplementIntersectDistinctAsUnion.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/SetOperationNodeTranslator.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/IntersectNode.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/IntersectTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBIntersectTableIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBIntersectTableIT.java new file mode 100644 index 0000000000000..2bc8d6bddb695 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBIntersectTableIT.java @@ -0,0 +1,153 @@ +/* + * 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.relational.it.query.recent; + +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.TableClusterIT; +import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import static org.apache.iotdb.db.it.utils.TestUtils.prepareTableData; +import static org.apache.iotdb.db.it.utils.TestUtils.tableAssertTestFail; +import static org.apache.iotdb.db.it.utils.TestUtils.tableResultSetEqualTest; + +@RunWith(IoTDBTestRunner.class) +@Category({TableLocalStandaloneIT.class, TableClusterIT.class}) +public class IoTDBIntersectTableIT { + protected static final String DATABASE_NAME = "test"; + protected static final String[] createSqls = + new String[] { + "CREATE DATABASE " + DATABASE_NAME, + "USE " + DATABASE_NAME, + // table1: ('d1', 1, 1) * 2, ('d1', 2, 2) *1 + "create table table1(device STRING TAG, s1 INT32 FIELD, s2 INT32 FIELD)", + "insert into table1 values (1, 'd1', 1, 1)", + "insert into table1 values (2, 'd1', 1, 1)", + "insert into table1 values (3, 'd1', 2, 2)", + // table2: ('d1', 1, 1.0) * 3, ('d1', 3, 3.0) *1 + "create table table2(device STRING TAG, s1 INT64 FIELD, s2 DOUBLE FIELD)", + "insert into table2 values (1, 'd1', 1, 1.0)", + "insert into table2 values (2, 'd1', 1, 1.0)", + "insert into table2 values (3, 'd1', 1, 1.0)", + "insert into table2 values (4, 'd1', 3, 3.0)", + // table3: use for testing alias + "create table table3(device STRING TAG, s1_testName INT64 FIELD, s2_testName DOUBLE FIELD)", + "insert into table3 values (1, 'd1', 1, 1.0)", + "insert into table3 values (2, 'd1', 1, 1.0)", + "insert into table3 values (3, 'd1', 1, 1.0)", + "insert into table3 values (4, 'd1', 3, 3.0)", + // table4: test type compatible + "create table table4(device STRING TAG, s1 TEXT FIELD, s2 DOUBLE FIELD)" + }; + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv().initClusterEnvironment(); + prepareTableData(createSqls); + } + + @AfterClass + public static void tearDown() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Test + public void normalTest() { + String[] expectedHeader = new String[] {"device", "s1", "s2"}; + + // --- INTERSECT (DISTINCT) --- + // table1 and table2, expected one tuple : ('d1', 1, 1.0) + String[] retArray = + new String[] { + "d1,1,1.0,", + }; + tableResultSetEqualTest( + "select device, s1, s2 from table1 intersect select device, s1, s2 from table2", + expectedHeader, + retArray, + DATABASE_NAME); + tableResultSetEqualTest( + "select device, s1, s2 from table1 intersect distinct select device, s1, s2 from table2", + expectedHeader, + retArray, + DATABASE_NAME); + + // --- INTERSECT ALL --- + // (1, 1.0) shows twice in table1, shows three times in table2 + // expected: min(2, 3) = 2 tuple + retArray = new String[] {"d1,1,1.0,", "d1,1,1.0,"}; + tableResultSetEqualTest( + "select device, s1, s2 from table1 intersect all select device, s1, s2 from table2", + expectedHeader, + retArray, + DATABASE_NAME); + // test table3, the column name is different + tableResultSetEqualTest( + "select device, s1, s2 from table1 intersect all select device, s1_testName, s2_testName from table3", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void mappingTest() { + // table1 (aliased): (s1 as col_a) -> (1), (1), (2) + // table2 (aliased): (s2 as col_a) -> (1.0), (1.0), (1.0), (3.0) + // common value: (1.0) + + String[] expectedHeader = new String[] {"col_a"}; + + // --- INTERSECT (DISTINCT) with alias --- + String[] retArray = new String[] {"1.0,"}; + tableResultSetEqualTest( + "select col_a from ((select s1 as col_a, device as col_b from table1) intersect (select s2 as col_a, device as col_b from table2)) order by col_a", + expectedHeader, + retArray, + DATABASE_NAME); + + // --- INTERSECT ALL with alias --- + retArray = new String[] {"1.0,", "1.0,"}; + tableResultSetEqualTest( + "select col_a from ((select s1 as col_a, device as col_b from table1) intersect all (select s2 as col_a, device as col_b from table2)) order by col_a", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void exceptionTest() { + // type is incompatible (INT32 vs TEXT) + tableAssertTestFail( + "(select * from table1) intersect all (select * from table4)", + "has incompatible types: INT32, TEXT", + DATABASE_NAME); + + tableAssertTestFail( + "(select * from table1) intersect all (select time from table4)", + "INTERSECT query has different number of fields: 4, 1", + DATABASE_NAME); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java index 6ffc534a712f5..82e03d8789b1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java @@ -75,6 +75,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GapFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GroupNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntersectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LinearFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.MarkDistinctNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.PatternRecognitionNode; @@ -1106,6 +1107,15 @@ public List visitUnion(UnionNode node, GraphContext context) { return render(node, boxValue, context); } + @Override + public List visitIntersect(IntersectNode node, GraphContext context) { + List boxValue = new ArrayList<>(); + boxValue.add(String.format("Intersect-%s", node.getPlanNodeId().getId())); + boxValue.add(String.format("OutputSymbols: %s", node.getOutputSymbols())); + boxValue.add(String.format("isDistinct: %s", node.isDistinct())); + return render(node, boxValue, context); + } + private List render(PlanNode node, List nodeBoxString, GraphContext context) { Box box = new Box(nodeBoxString); List> children = new ArrayList<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java index 902102797c984..c4b77def0539f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java @@ -123,6 +123,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GapFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GroupNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntersectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LinearFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.MarkDistinctNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.PatternRecognitionNode; @@ -313,6 +314,7 @@ public enum PlanNodeType { TABLE_WINDOW_FUNCTION((short) 1032), TABLE_INTO_NODE((short) 1033), TABLE_UNION_NODE((short) 1034), + TABLE_INTERSECT_NODE((short) 1035), RELATIONAL_INSERT_TABLET((short) 2000), RELATIONAL_INSERT_ROW((short) 2001), @@ -707,6 +709,8 @@ public static PlanNode deserialize(ByteBuffer buffer, short nodeType) { buffer); case 1034: return UnionNode.deserialize(buffer); + case 1035: + return IntersectNode.deserialize(buffer); case 2000: return RelationalInsertTabletNode.deserialize(buffer); case 2001: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java index bbc83cce18b61..3e237650a6a77 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java @@ -127,6 +127,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GapFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GroupNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntersectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LinearFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.MarkDistinctNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.PatternRecognitionNode; @@ -845,4 +846,8 @@ public R visitPatternRecognition(PatternRecognitionNode node, C context) { public R visitUnion(UnionNode node, C context) { return visitPlan(node, context); } + + public R visitIntersect(IntersectNode node, C context) { + return visitPlan(node, context); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java index af823554b7b52..ce573ce1e2d7c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java @@ -57,6 +57,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntersectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.Measure; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.PatternRecognitionNode; @@ -1145,6 +1146,24 @@ protected RelationPlan visitUnion(Union node, Void context) { planNode, analysis.getScope(node), planNode.getOutputSymbols(), outerContext); } + @Override + protected RelationPlan visitIntersect(Intersect node, Void context) { + Preconditions.checkArgument( + !node.getRelations().isEmpty(), "No relations specified for intersect"); + SetOperationPlan setOperationPlan = process(node); + + PlanNode intersectNode = + new IntersectNode( + idAllocator.genPlanNodeId(), + setOperationPlan.getChildren(), + setOperationPlan.getSymbolMapping(), + ImmutableList.copyOf(setOperationPlan.getSymbolMapping().keySet()), + node.isDistinct()); + + return new RelationPlan( + intersectNode, analysis.getScope(node), intersectNode.getOutputSymbols(), outerContext); + } + private SetOperationPlan process(SetOperation node) { RelationType outputFields = analysis.getOutputDescriptor(node); List outputs = @@ -1191,11 +1210,6 @@ protected RelationPlan visitValues(Values node, Void context) { throw new IllegalStateException("Values is not supported in current version."); } - @Override - protected RelationPlan visitIntersect(Intersect node, Void context) { - throw new IllegalStateException("Intersect is not supported in current version."); - } - @Override protected RelationPlan visitExcept(Except node, Void context) { throw new IllegalStateException("Except is not supported in current version."); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/ImplementIntersectAll.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/ImplementIntersectAll.java new file mode 100644 index 0000000000000..af31281c918aa --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/ImplementIntersectAll.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule; + +import org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Assignments; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntersectNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ProjectNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.QualifiedName; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Captures; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Pattern; + +import com.google.common.collect.ImmutableList; + +import static java.util.Objects.requireNonNull; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.Intersect.distinct; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns.intersect; + +public class ImplementIntersectAll implements Rule { + + private static final Pattern PATTERN = intersect().with(distinct().equalTo(false)); + + private final Metadata metadata; + + public ImplementIntersectAll(Metadata metadata) { + this.metadata = requireNonNull(metadata, "metadata is null"); + } + + @Override + public Pattern getPattern() { + return PATTERN; + } + + @Override + public Result apply(IntersectNode node, Captures captures, Context context) { + + SetOperationNodeTranslator translator = + new SetOperationNodeTranslator( + metadata, context.getSymbolAllocator(), context.getIdAllocator()); + + // 1. translate the intersect(all) node to other planNodes + SetOperationNodeTranslator.TranslationResult translationResult = + translator.makeSetContainmentPlanForAll(node); + + // 2. add the filter node above the result node from translation process + // filter condition : row_number <= least(countA, countB...) + Expression minCount = translationResult.getCountSymbols().get(0).toSymbolReference(); + for (int i = 1; i < translationResult.getCountSymbols().size(); i++) { + minCount = + new FunctionCall( + QualifiedName.of(TableBuiltinScalarFunction.LEAST.getFunctionName()), + ImmutableList.of( + minCount, translationResult.getCountSymbols().get(i).toSymbolReference())); + } + + FilterNode filterNode = + new FilterNode( + context.getIdAllocator().genPlanNodeId(), + translationResult.getPlanNode(), + new ComparisonExpression( + ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, + translationResult.getRowNumberSymbol().toSymbolReference(), + minCount)); + + // 3. add the project node to remove the redundant columns + return Result.ofPlanNode( + new ProjectNode( + context.getIdAllocator().genPlanNodeId(), + filterNode, + Assignments.identity(node.getOutputSymbols()))); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/ImplementIntersectDistinctAsUnion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/ImplementIntersectDistinctAsUnion.java new file mode 100644 index 0000000000000..0a8f8c498ae85 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/ImplementIntersectDistinctAsUnion.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule; + +import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Assignments; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntersectNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.Patterns; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ProjectNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.GenericLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Captures; +import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Pattern; + +import com.google.common.collect.ImmutableList; + +import static java.util.Objects.requireNonNull; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.ir.IrUtils.and; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression.Operator.GREATER_THAN_OR_EQUAL; + +public class ImplementIntersectDistinctAsUnion implements Rule { + + private static final Pattern PATTERN = + Patterns.intersect().with(Patterns.Intersect.distinct().equalTo(true)); + + private final Metadata metadata; + + @Override + public Pattern getPattern() { + return PATTERN; + } + + public ImplementIntersectDistinctAsUnion(Metadata metadata) { + this.metadata = requireNonNull(metadata, "metadata is null"); + } + + @Override + public Result apply(IntersectNode node, Captures captures, Context context) { + + SetOperationNodeTranslator translator = + new SetOperationNodeTranslator( + metadata, context.getSymbolAllocator(), context.getIdAllocator()); + + SetOperationNodeTranslator.TranslationResult result = + translator.makeSetContainmentPlanForDistinct(node); + + // add the filterNode above the aggregation node + Expression predicate = + and( + result.getCountSymbols().stream() + .map( + symbol -> + new ComparisonExpression( + GREATER_THAN_OR_EQUAL, + symbol.toSymbolReference(), + new GenericLiteral("INT64", "1"))) + .collect(ImmutableList.toImmutableList())); + + FilterNode filterNode = + new FilterNode(context.getIdAllocator().genPlanNodeId(), result.getPlanNode(), predicate); + + return Result.ofPlanNode( + new ProjectNode( + context.getIdAllocator().genPlanNodeId(), + filterNode, + Assignments.identity(node.getOutputSymbols()))); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneDistinctAggregation.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneDistinctAggregation.java index 5b3a410147890..26fd027801625 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneDistinctAggregation.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneDistinctAggregation.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Lookup; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntersectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.UnionNode; import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Captures; import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Pattern; @@ -100,6 +101,14 @@ public PlanNode visitUnion(UnionNode node, Boolean context) { return rewriteChildren(node, context); } + @Override + public PlanNode visitIntersect(IntersectNode node, Boolean context) { + if (node.isDistinct()) { + return rewriteChildren(node, context); + } + return visitPlan(node, context); + } + /*@Override public PlanNode visitUnion(UnionNode node, Boolean context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/SetOperationNodeTranslator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/SetOperationNodeTranslator.java new file mode 100644 index 0000000000000..3088860564e1e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/SetOperationNodeTranslator.java @@ -0,0 +1,355 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule; + +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.function.BoundSignature; +import org.apache.iotdb.db.queryengine.plan.relational.function.FunctionId; +import org.apache.iotdb.db.queryengine.plan.relational.function.FunctionKind; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.FunctionNullability; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ResolvedFunction; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Assignments; +import org.apache.iotdb.db.queryengine.plan.relational.planner.DataOrganizationSpecification; +import org.apache.iotdb.db.queryengine.plan.relational.planner.OrderingScheme; +import org.apache.iotdb.db.queryengine.plan.relational.planner.SortOrder; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolAllocator; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GroupNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ProjectNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SetOperationNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.UnionNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.WindowNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Cast; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FrameBound; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NullLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WindowFrame; +import org.apache.iotdb.db.utils.constant.SqlConstant; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import org.apache.tsfile.read.common.type.Type; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.Iterables.concat; +import static java.util.Objects.requireNonNull; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode.singleGroupingSet; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.Util.getResolvedBuiltInAggregateFunction; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral.TRUE_LITERAL; +import static org.apache.iotdb.db.queryengine.plan.relational.type.TypeSignatureTranslator.toSqlType; +import static org.apache.iotdb.db.utils.constant.SqlConstant.COUNT; +import static org.apache.tsfile.read.common.type.BooleanType.BOOLEAN; +import static org.apache.tsfile.read.common.type.LongType.INT64; + +public class SetOperationNodeTranslator { + + private static final String MARKER = "marker"; + private static final String COUNT_MARKER = "count"; + private static final String ROW_NUMBER_SYMBOL = "row_number"; + private final SymbolAllocator symbolAllocator; + private final QueryId idAllocator; + private final Metadata metadata; + + public SetOperationNodeTranslator( + Metadata metadata, SymbolAllocator symbolAllocator, QueryId idAllocator) { + + this.metadata = requireNonNull(metadata, "metadata is null"); + this.symbolAllocator = requireNonNull(symbolAllocator, "symbolAllocator is null"); + this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); + } + + /** for intersect distinct and except distinct , use true and false for markers */ + public TranslationResult makeSetContainmentPlanForDistinct(SetOperationNode node) { + + checkArgument(!(node instanceof UnionNode), "Cannot simplify a UnionNode"); + List markers = allocateSymbols(node.getChildren().size(), MARKER, BOOLEAN); + + // 1. add the marker column to the origin planNode + List projectNodesWithMarkers = appendMarkers(markers, node.getChildren(), node); + + // 2. add the union node over all new projection nodes. + // The outputs of the union must have the same name as the original intersect node + UnionNode union = + union( + projectNodesWithMarkers, + ImmutableList.copyOf(concat(node.getOutputSymbols(), markers))); + + // 3. add the aggregation node above the union node + List aggregationOutputs = allocateSymbols(markers.size(), COUNT, INT64); + AggregationNode aggregation = + computeCounts(union, node.getOutputSymbols(), markers, aggregationOutputs); + + return new TranslationResult(aggregation, aggregationOutputs); + } + + /** for intersect all and except all, use true and false for markers */ + public TranslationResult makeSetContainmentPlanForAll(SetOperationNode node) { + + checkArgument(!(node instanceof UnionNode), "Cannot simplify a UnionNode"); + List markers = allocateSymbols(node.getChildren().size(), MARKER, BOOLEAN); + + // for every child of SetOperation node, add the marker column for the child + List projectNodesWithMarkers = appendMarkers(markers, node.getChildren(), node); + + UnionNode union = + union( + projectNodesWithMarkers, + ImmutableList.copyOf(concat(node.getOutputSymbols(), markers))); + List countOutputs = allocateSymbols(markers.size(), COUNT_MARKER, INT64); + Symbol rowNumberSymbol = symbolAllocator.newSymbol(ROW_NUMBER_SYMBOL, INT64); + WindowNode windowNode = + appendCounts(union, node.getOutputSymbols(), markers, countOutputs, rowNumberSymbol); + + ProjectNode projectNode = + new ProjectNode( + idAllocator.genPlanNodeId(), + windowNode, + Assignments.identity( + ImmutableList.copyOf( + concat( + node.getOutputSymbols(), + countOutputs, + ImmutableList.of(rowNumberSymbol))))); + + return new TranslationResult(projectNode, countOutputs, Optional.of(rowNumberSymbol)); + } + + /** + * only for transforming the intersection (all) node, add the window node and group node above the + * union node + */ + private WindowNode appendCounts( + UnionNode union, + List originOutputSymbols, + List markers, + List countOutputs, + Symbol rowNumberSymbol) { + + checkArgument( + markers.size() == countOutputs.size(), + "the size of markers should be same as the size of count output symbols"); + + // Add group node above the union node to assist partitioning, preparing for the window node + ImmutableMap.Builder sortOrderings = ImmutableMap.builder(); + ImmutableList.Builder sortSymbolBuilder = ImmutableList.builder(); + for (Symbol originalColumn : originOutputSymbols) { + sortSymbolBuilder.add(originalColumn); + sortOrderings.put(originalColumn, SortOrder.ASC_NULLS_LAST); + } + ImmutableList sortSymbols = sortSymbolBuilder.build(); + GroupNode groupNode = + new GroupNode( + idAllocator.genPlanNodeId(), + union, + new OrderingScheme(sortSymbols, sortOrderings.build()), + sortSymbols.size()); + + // build the windowFunctions for count(marker) and row_number + ImmutableMap.Builder windowFunctions = ImmutableMap.builder(); + WindowNode.Frame windowFunctionFrame = + new WindowNode.Frame( + WindowFrame.Type.ROWS, + FrameBound.Type.UNBOUNDED_PRECEDING, + Optional.empty(), + Optional.empty(), + FrameBound.Type.UNBOUNDED_FOLLOWING, + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty()); + + ResolvedFunction countFunction = + getResolvedBuiltInAggregateFunction( + metadata, SqlConstant.COUNT, Collections.singletonList(BOOLEAN)); + for (int i = 0; i < markers.size(); i++) { + windowFunctions.put( + countOutputs.get(i), + new WindowNode.Function( + countFunction, + ImmutableList.of(markers.get(i).toSymbolReference()), + windowFunctionFrame, + false)); + } + + List argumentTypes = ImmutableList.of(); + ResolvedFunction rowNumberFunction = + new ResolvedFunction( + new BoundSignature( + SqlConstant.ROW_NUMBER, + metadata.getFunctionReturnType(SqlConstant.ROW_NUMBER, argumentTypes), + argumentTypes), + FunctionId.NOOP_FUNCTION_ID, + FunctionKind.WINDOW, + true, + FunctionNullability.getAggregationFunctionNullability(argumentTypes.size())); + + windowFunctions.put( + rowNumberSymbol, + new WindowNode.Function(rowNumberFunction, ImmutableList.of(), windowFunctionFrame, false)); + + // add the windowNode above the group node + return new WindowNode( + idAllocator.genPlanNodeId(), + groupNode, + new DataOrganizationSpecification(originOutputSymbols, Optional.empty()), + windowFunctions.buildOrThrow(), + Optional.empty(), + ImmutableSet.of(), + 0); + } + + /** get an array of markers, used for the new columns */ + private List allocateSymbols(int count, String nameHint, Type type) { + ImmutableList.Builder symbolsBuilder = ImmutableList.builder(); + for (int i = 0; i < count; i++) { + symbolsBuilder.add(symbolAllocator.newSymbol(nameHint, type)); + } + return symbolsBuilder.build(); + } + + /** + * Builds projection nodes with marker columns for each child of the set operation. Each child + * gets TRUE for its own marker and NULL (cast to BOOLEAN) for others. This is used in the + * implementation of INTERSECT and EXCEPT set operations. + */ + private List appendMarkers( + List markers, List children, SetOperationNode node) { + ImmutableList.Builder projectionsWithMarker = ImmutableList.builder(); + + Map> symbolMapping = node.getSymbolMapping().asMap(); + for (int childIndex = 0; childIndex < children.size(); childIndex++) { + + // add the original symbols to projection node + Assignments.Builder assignments = Assignments.builder(); + for (Symbol outputSymbol : node.getOutputSymbols()) { + Collection inputSymbols = symbolMapping.get(outputSymbol); + Symbol sourceSymbol = Iterables.get(inputSymbols, childIndex); + + Symbol newProjectedSymbol = symbolAllocator.newSymbol(outputSymbol); + assignments.put(newProjectedSymbol, sourceSymbol.toSymbolReference()); + } + + // add the new marker symbol to the new projection node + for (int j = 0; j < markers.size(); j++) { + Expression expression = + j == childIndex ? TRUE_LITERAL : new Cast(new NullLiteral(), toSqlType(BOOLEAN)); + assignments.put(symbolAllocator.newSymbol(markers.get(j).getName(), BOOLEAN), expression); + } + + projectionsWithMarker.add( + new ProjectNode( + idAllocator.genPlanNodeId(), children.get(childIndex), assignments.build())); + } + + return projectionsWithMarker.build(); + } + + private UnionNode union(List projectNodesWithMarkers, List outputs) { + + ImmutableListMultimap.Builder outputsToInputs = ImmutableListMultimap.builder(); + + for (PlanNode projectionNode : projectNodesWithMarkers) { + List outputSymbols = projectionNode.getOutputSymbols(); + for (int i = 0; i < outputSymbols.size(); i++) { + outputsToInputs.put(outputs.get(i), outputSymbols.get(i)); + } + } + + return new UnionNode( + idAllocator.genPlanNodeId(), projectNodesWithMarkers, outputsToInputs.build(), outputs); + } + + /** add the aggregation node above the union node */ + private AggregationNode computeCounts( + UnionNode unionNode, + List originalColumns, + List markers, + List aggregationOutputs) { + + ImmutableMap.Builder aggregations = ImmutableMap.builder(); + + ResolvedFunction resolvedFunction = + getResolvedBuiltInAggregateFunction(metadata, COUNT, Collections.singletonList(BOOLEAN)); + + for (int i = 0; i < markers.size(); i++) { + Symbol countMarker = aggregationOutputs.get(i); + aggregations.put( + countMarker, + new AggregationNode.Aggregation( + resolvedFunction, + ImmutableList.of(markers.get(i).toSymbolReference()), + false, + Optional.empty(), + Optional.empty(), + Optional.empty())); + } + + return AggregationNode.singleAggregation( + idAllocator.genPlanNodeId(), + unionNode, + aggregations.buildOrThrow(), + singleGroupingSet(originalColumns)); + } + + public static class TranslationResult { + + private final PlanNode planNode; + private final List countSymbols; + private final Optional rowNumberSymbol; + + public TranslationResult(PlanNode planNode, List countSymbols) { + this(planNode, countSymbols, Optional.empty()); + } + + public TranslationResult( + PlanNode planNode, List countSymbols, Optional rowNumberSymbol) { + this.planNode = requireNonNull(planNode, "planNode is null"); + this.countSymbols = + ImmutableList.copyOf(requireNonNull(countSymbols, "countSymbols is null")); + this.rowNumberSymbol = requireNonNull(rowNumberSymbol, "rowNumberSymbol is null"); + } + + public List getCountSymbols() { + return countSymbols; + } + + public Symbol getRowNumberSymbol() { + checkState(rowNumberSymbol.isPresent(), "rowNumberSymbol is empty"); + return rowNumberSymbol.get(); + } + + public PlanNode getPlanNode() { + return this.planNode; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/IntersectNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/IntersectNode.java new file mode 100644 index 0000000000000..3f7a086497180 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/IntersectNode.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.node; + +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; + +import com.google.common.collect.ListMultimap; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; + +public class IntersectNode extends SetOperationNode { + + private final boolean distinct; + + public IntersectNode( + PlanNodeId id, + List children, + ListMultimap outputToInputs, + List outputs, + boolean distinct) { + super(id, children, outputToInputs, outputs); + this.distinct = distinct; + } + + private IntersectNode( + PlanNodeId id, + ListMultimap outputToInputs, + List outputs, + boolean distinct) { + super(id, outputToInputs, outputs); + this.distinct = distinct; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitIntersect(this, context); + } + + public boolean isDistinct() { + return distinct; + } + + @Override + public PlanNode clone() { + return new IntersectNode(getPlanNodeId(), getSymbolMapping(), getOutputSymbols(), distinct); + } + + @Override + public List getOutputColumnNames() { + throw new UnsupportedOperationException(); + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + throw new UnsupportedOperationException( + "IntersectNode should never be serialized in current version"); + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + throw new UnsupportedOperationException( + "IntersectNode should never be serialized in current version"); + } + + public static IntersectNode deserialize(ByteBuffer byteBuffer) { + throw new UnsupportedOperationException( + "IntersectNode should never be deserialized in current version"); + } + + @Override + public PlanNode replaceChildren(List newChildren) { + return new IntersectNode( + getPlanNodeId(), newChildren, getSymbolMapping(), getOutputSymbols(), isDistinct()); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/Patterns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/Patterns.java index 3ebd933d28c81..d08ac9ecf7221 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/Patterns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/Patterns.java @@ -189,6 +189,10 @@ public static Pattern union() { return typeOf(UnionNode.class); } + public static Pattern intersect() { + return typeOf(IntersectNode.class); + } + /*public static Pattern tableWriterNode() { return typeOf(TableWriterNode.class); @@ -249,10 +253,6 @@ public static Pattern distinctLimit() return typeOf(DistinctLimitNode.class); } - public static Pattern intersect() - { - return typeOf(IntersectNode.class); - } public static Pattern except() { @@ -359,6 +359,12 @@ public static Property requiresPreSortedInputs() { } } + public static final class Intersect { + public static Property distinct() { + return property("distinct", IntersectNode::isDistinct); + } + } + /*public static final class Sample { public static Property sampleRatio() @@ -415,13 +421,7 @@ public static Property getFilteringSource() } }*/ - /*public static final class Intersect - { - public static Property distinct() - { - return property("distinct", IntersectNode::isDistinct); - } - } + /* public static final class Except { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/LogicalOptimizeFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/LogicalOptimizeFactory.java index d70f67114b0f9..65015715945c5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/LogicalOptimizeFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/LogicalOptimizeFactory.java @@ -26,6 +26,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.RuleStatsRecorder; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.CanonicalizeExpressions; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.ImplementIntersectAll; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.ImplementIntersectDistinctAsUnion; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.ImplementPatternRecognition; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.ImplementTableFunctionSource; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.rule.InlineProjections; @@ -275,6 +277,17 @@ public LogicalOptimizeFactory(PlannerContext plannerContext) { // new MergeExcept new PruneDistinctAggregation())) .build()), + new IterativeOptimizer( + plannerContext, + ruleStats, + ImmutableSet.>builder() + .add( + new ImplementIntersectDistinctAsUnion(metadata), + // new ImplementExceptDistinctAsUnion(metadata) + new ImplementIntersectAll(metadata) + // new ImplementExceptAll(metadata))), + ) + .build()), columnPruningOptimizer, inlineProjectionLimitFiltersOptimizer, new IterativeOptimizer( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java index 95f0f6cb0f2a2..ff3a01be84999 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java @@ -41,6 +41,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GapFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GroupNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntersectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LimitNode; @@ -1015,6 +1016,36 @@ public PlanAndMappings visitUnion(UnionNode node, UnaliasContext context) { mapping); } + @Override + public PlanAndMappings visitIntersect(IntersectNode node, UnaliasContext context) { + + List rewrittenSources = + node.getChildren().stream() + .map(source -> source.accept(this, context)) + .collect(toImmutableList()); + + List inputMappers = + rewrittenSources.stream() + .map(source -> symbolMapper(new HashMap<>(source.getMappings()))) + .collect(toImmutableList()); + + Map mapping = new HashMap<>(context.getCorrelationMapping()); + SymbolMapper outputMapper = symbolMapper(mapping); + + ListMultimap newOutputToInputs = + rewriteOutputToInputsMap(node.getSymbolMapping(), outputMapper, inputMappers); + List newOutputs = outputMapper.mapAndDistinct(node.getOutputSymbols()); + + return new PlanAndMappings( + new IntersectNode( + node.getPlanNodeId(), + rewrittenSources.stream().map(PlanAndMappings::getRoot).collect(toImmutableList()), + newOutputToInputs, + newOutputs, + node.isDistinct()), + mapping); + } + private ListMultimap rewriteOutputToInputsMap( ListMultimap oldMapping, SymbolMapper outputMapper, diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/IntersectTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/IntersectTest.java new file mode 100644 index 0000000000000..13d18c89ddfaa --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/IntersectTest.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.analyzer; + +import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; +import org.apache.iotdb.db.queryengine.plan.relational.planner.PlanTester; + +import org.junit.Test; + +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanAssert.assertPlan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.aggregation; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.filter; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.output; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.project; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.sort; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.tableScan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.union; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.window; + +/** tests for intersect (distinct) and intersect all */ +public class IntersectTest { + + @Test + public void intersectTest() { + + PlanTester planTester = new PlanTester(); + LogicalQueryPlan actualLogicalQueryPlan = + planTester.createPlan("select tag1 from t1 intersect select tag1 from t2"); + // just verify the Logical plan: `Output - project - filter - aggregation - union - 2*(project + // - tableScan)` + assertPlan( + actualLogicalQueryPlan, + output( + project( + filter( + aggregation( + union( + project(tableScan("testdb.t1")), project(tableScan("testdb.t2")))))))); + } + + @Test + public void intersectAllTest() { + + PlanTester planTester = new PlanTester(); + LogicalQueryPlan actualLogicalQueryPlan = + planTester.createPlan("select tag1 from t1 intersect all select tag1 from t2"); + assertPlan( + actualLogicalQueryPlan, + output( + project( + filter( + project( + window( + sort( + union( + project(tableScan("testdb.t1")), + project(tableScan("testdb.t2")))))))))); + } + + @Test + public void typeCompatibleTest() { + // use CAST if types of according columns is not compatible + // s1 is INT64, s3 is DOUBLE + + PlanTester planTester = new PlanTester(); + LogicalQueryPlan actualLogicalQueryPlan = + planTester.createPlan("select s1, s3 from table2 intersect all select s1, s1 from table3 "); + + assertPlan( + actualLogicalQueryPlan, + output( + project( + filter( + project( + window( + sort( + union( + project(tableScan("testdb.table2")), + project(tableScan("testdb.table3")))))))))); + } + + /** the priority of intersect is higher than that of the union */ + @Test + public void setOperationPriority() { + + PlanTester planTester = new PlanTester(); + LogicalQueryPlan actualLogicalQueryPlan = + planTester.createPlan( + "select tag1 from t1 union select tag1 from t2 intersect select tag1 from t3"); + + assertPlan( + actualLogicalQueryPlan, + output( + aggregation( + union( + tableScan("testdb.t1"), + project( + filter( + aggregation( + union( + project(tableScan("testdb.t2")), + project(tableScan("testdb.t3")))))))))); + } +} diff --git a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 index 59231b6de2bcc..7357196219f51 100644 --- a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 +++ b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 @@ -930,8 +930,9 @@ rowCount ; queryTerm - : queryPrimary #queryTermDefault - | left=queryTerm operator=(INTERSECT | UNION | EXCEPT) setQuantifier? right=queryTerm #setOperation + : queryPrimary #queryTermDefault + | left=queryTerm operator=INTERSECT setQuantifier? right=queryTerm #setOperation + | left=queryTerm operator=(UNION | EXCEPT) setQuantifier? right=queryTerm #setOperation ; queryPrimary From 96d4eae4e151b4ce1f1578b4b762bd84ebc5772d Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Fri, 7 Nov 2025 17:09:09 +0800 Subject: [PATCH 034/180] Fix the error message of ConfigExecution (#16680) (cherry picked from commit c406a81f53c050c7b497af335a2bd72f465436e0) --- .../plan/execution/config/ConfigExecution.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/ConfigExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/ConfigExecution.java index 1ae57fe38d5d6..1880924f297ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/ConfigExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/ConfigExecution.java @@ -166,28 +166,30 @@ private void fail(final Throwable cause) { if (cause instanceof IoTDBException) { if (Objects.nonNull(((IoTDBException) cause).getStatus())) { status = ((IoTDBException) cause).getStatus(); + errorCode = status.getCode(); } else { errorCode = ((IoTDBException) cause).getErrorCode(); } } else if (cause instanceof IoTDBRuntimeException) { if (Objects.nonNull(((IoTDBRuntimeException) cause).getStatus())) { status = ((IoTDBRuntimeException) cause).getStatus(); + errorCode = status.getCode(); } else { errorCode = ((IoTDBRuntimeException) cause).getErrorCode(); } } - if (Objects.nonNull(status) && isUserException(status) - || !userExceptionCodes.contains(errorCode)) { - LOGGER.warn( - "Failures happened during running ConfigExecution when executing {}.", - Objects.nonNull(task) ? task.getClass().getSimpleName() : null, - cause); - } else { + if ((Objects.nonNull(status) && isUserException(status)) + || userExceptionCodes.contains(errorCode)) { LOGGER.info( "Failures happened during running ConfigExecution when executing {}, message: {}, status: {}", Objects.nonNull(task) ? task.getClass().getSimpleName() : null, cause.getMessage(), errorCode); + } else { + LOGGER.warn( + "Failures happened during running ConfigExecution when executing {}.", + Objects.nonNull(task) ? task.getClass().getSimpleName() : null, + cause); } stateMachine.transitionToFailed(cause); final ConfigTaskResult result; From 5d9baf5b755df07dae990da1a2f666c4d0ba8a45 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 7 Nov 2025 18:36:50 +0800 Subject: [PATCH 035/180] Fixed the null error message of fetch schema failed when the regions is removing (#16713) (cherry picked from commit 9ae8233091f1b655929c3343ee103389349e0d56) --- .../execution/executor/RegionReadExecutor.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionReadExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionReadExecutor.java index e2fb4c9598814..2e63db2e2e75c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionReadExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionReadExecutor.java @@ -100,13 +100,15 @@ public RegionExecutionResult execute( }); return resp; } - } catch (ConsensusGroupNotExistException e) { + } catch (final ConsensusGroupNotExistException e) { LOGGER.warn("Execute FragmentInstance in ConsensusGroup {} failed.", groupId, e); - RegionExecutionResult resp = + final String errorMsg = String.format(ERROR_MSG_FORMAT, e.getMessage()); + final RegionExecutionResult resp = RegionExecutionResult.create( false, - String.format(ERROR_MSG_FORMAT, e.getMessage()), - new TSStatus(TSStatusCode.CONSENSUS_GROUP_NOT_EXIST.getStatusCode())); + errorMsg, + new TSStatus(TSStatusCode.CONSENSUS_GROUP_NOT_EXIST.getStatusCode()) + .setMessage(errorMsg)); resp.setReadNeedRetry(true); return resp; } catch (Throwable e) { From d0de0a67a72e7fc44259d784bf507aaeff8e7cd3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 10 Nov 2025 09:42:10 +0800 Subject: [PATCH 036/180] Pipe: Fixed the NPE of commit interval (#16714) * fix * fix (cherry picked from commit a9ba7243ba9667ff0e88d32e067299ecba519f19) --- .../pipe/agent/task/progress/interval/PipeCommitInterval.java | 4 +++- .../iotdb/commons/pipe/datastructure/PipeCommitQueueTest.java | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/interval/PipeCommitInterval.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/interval/PipeCommitInterval.java index 46a3d3e2a865c..885df4727dac7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/interval/PipeCommitInterval.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/interval/PipeCommitInterval.java @@ -20,6 +20,7 @@ package org.apache.iotdb.commons.pipe.agent.task.progress.interval; import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.datastructure.interval.Interval; @@ -40,7 +41,8 @@ public PipeCommitInterval( final PipeTaskMeta pipeTaskMeta) { super(start, end); this.pipeTaskMeta = pipeTaskMeta; - this.currentIndex = currentIndex; + this.currentIndex = + Objects.nonNull(currentIndex) ? currentIndex : MinimumProgressIndex.INSTANCE; this.onCommittedHooks = onCommittedHooks; } diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/datastructure/PipeCommitQueueTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/datastructure/PipeCommitQueueTest.java index 61b217c35d472..6a894b2104176 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/datastructure/PipeCommitQueueTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/datastructure/PipeCommitQueueTest.java @@ -57,6 +57,8 @@ public void testCommitQueue() { Assert.assertEquals(0, pipeCommitQueue.size()); Assert.assertEquals(new IoTProgressIndex(0, 4L), pipeTaskMeta.getProgressIndex()); Assert.assertEquals(1, commitHookTestSet.size()); + // Test null progressIndex + pipeCommitQueue.offer(new TestEnrichedEvent(6, null)); } private class TestEnrichedEvent extends EnrichedEvent { From ba7a7a6fc1bc38535249c63b55225a3270f62fc5 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 10 Nov 2025 09:43:01 +0800 Subject: [PATCH 037/180] Follow fix for encoding & compressor (#16712) * fix * fix (cherry picked from commit 3ec8979e42aaa3bae6505e7bbf0d24dc406b4904) --- .../PipeAlterEncodingCompressorPlan.java | 19 ++++++++++++++++++- .../AlterEncodingCompressorProcedure.java | 7 +++++-- 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeAlterEncodingCompressorPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeAlterEncodingCompressorPlan.java index bd31a7c57b2eb..08ec62be80ca6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeAlterEncodingCompressorPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeAlterEncodingCompressorPlan.java @@ -85,6 +85,7 @@ protected void serializeImpl(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(patternTreeBytes, stream); ReadWriteIOUtils.write(encoding, stream); ReadWriteIOUtils.write(compressor, stream); + ReadWriteIOUtils.write(mayAlterAudit, stream); } @Override @@ -92,11 +93,27 @@ protected void deserializeImpl(final ByteBuffer buffer) throws IOException { patternTreeBytes = ByteBuffer.wrap(ReadWriteIOUtils.readBinary(buffer).getValues()); encoding = ReadWriteIOUtils.readByte(buffer); compressor = ReadWriteIOUtils.readByte(buffer); + mayAlterAudit = ReadWriteIOUtils.readBoolean(buffer); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final PipeAlterEncodingCompressorPlan that = (PipeAlterEncodingCompressorPlan) o; + return this.patternTreeBytes.equals(that.patternTreeBytes) + && this.encoding == that.encoding + && this.compressor == that.compressor + && this.mayAlterAudit == that.mayAlterAudit; } @Override public int hashCode() { - return Objects.hash(patternTreeBytes, encoding, compressor); + return Objects.hash(patternTreeBytes, encoding, compressor, mayAlterAudit); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java index a76e3494d051a..627547ce72236 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java @@ -63,7 +63,8 @@ public class AlterEncodingCompressorProcedure extends StateMachineProcedure { - private static final Logger LOGGER = LoggerFactory.getLogger(AlterEncodingCompressorState.class); + private static final Logger LOGGER = + LoggerFactory.getLogger(AlterEncodingCompressorProcedure.class); private String queryId; private PathPatternTree patternTree; private boolean ifExists; @@ -298,7 +299,9 @@ public void deserialize(final ByteBuffer byteBuffer) { @Override public boolean equals(final Object o) { - if (this == o) return true; + if (this == o) { + return true; + } if (o == null || getClass() != o.getClass()) { return false; } From 0657671d20f5cb2694d3ab5fe1b09a47f3f231c6 Mon Sep 17 00:00:00 2001 From: Haonan Date: Mon, 10 Nov 2025 09:43:43 +0800 Subject: [PATCH 038/180] Avoid unnecessary ssl error log caused by jdk bug (#16709) (cherry picked from commit 9d4c41092a85d819ae18aba0fbdcb0e5ee203d46) --- .../iotdb/rpc/NettyTNonblockingTransport.java | 9 +++++---- .../iotdb/rpc/TElasticFramedTransport.java | 20 +++++++++++++------ 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/NettyTNonblockingTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/NettyTNonblockingTransport.java index b42b901cc1831..549e1bb2e0b26 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/NettyTNonblockingTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/NettyTNonblockingTransport.java @@ -168,10 +168,11 @@ protected void initChannel(SocketChannel ch) throws Exception { "SSL handshake completed successfully for {}:{}", host, port); } } else { - if (!future - .cause() - .getMessage() - .contains("SslHandler removed before handshake completed")) { + if (future.cause().getMessage() != null + && !future + .cause() + .getMessage() + .contains("SslHandler removed before handshake completed")) { logger.warn( "SSL handshake failed for {}:{}", host, port, future.cause()); } else if (logger.isDebugEnabled()) { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java index c366d8a9333ff..6008988a80998 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java @@ -132,13 +132,21 @@ public int read(byte[] buf, int off, int len) throws TTransportException { if (e.getCause() instanceof SocketTimeoutException) { throw new TTransportException(TTransportException.TIMED_OUT, e.getCause()); } - // When client with SSL shut down due to time out. Some unnecessary error logs may be printed. - // Adding this workaround to avoid the problem. - if (e.getCause() instanceof SSLHandshakeException - && e.getCause().getCause() != null - && e.getCause().getCause() instanceof EOFException) { - throw new TTransportException(TTransportException.END_OF_FILE, e.getCause()); + if (e.getCause() instanceof SSLHandshakeException) { + // There is an unsolved JDK bug https://bugs.openjdk.org/browse/JDK-8221218. + // Adding this workaround to avoid the error log printed. + if (e.getMessage() + .contains("Insufficient buffer remaining for AEAD cipher fragment (2).")) { + throw new TTransportException(TTransportException.END_OF_FILE, e.getCause()); + } + // When client with SSL shutdown due to time out. Some unnecessary error logs may be + // printed. + // Adding this workaround to avoid the problem. + if (e.getCause().getCause() != null && e.getCause().getCause() instanceof EOFException) { + throw new TTransportException(TTransportException.END_OF_FILE, e.getCause()); + } } + if (e.getCause() instanceof SSLException && e.getMessage().contains("Unsupported or unrecognized SSL message")) { SocketAddress remoteAddress = null; From cbe57655b1e518b8f6904c5c7f8c2de5b5098e3d Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Mon, 10 Nov 2025 15:55:59 +0800 Subject: [PATCH 039/180] Active Load: Ignore java.io.UncheckedIOException exceptions thrown during the file scanning process. (#16705) * Active Load: Ignore java.io.UncheckedIOException exceptions thrown during the file scanning process. * update (cherry picked from commit 2de3bb6f534bb94f816f893c9091b0c95fb1d2a8) --- .../db/storageengine/load/active/ActiveLoadDirScanner.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java index 8abfa61870808..5ac33dfbdd4bd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java @@ -32,6 +32,7 @@ import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.Path; @@ -125,6 +126,8 @@ private void scan() throws IOException { parentFile.getAbsoluteFile(), listeningDirFile.getAbsoluteFile()), isGeneratedByPipe); }); + } catch (UncheckedIOException e) { + LOGGER.debug("The file has been deleted. Ignore this exception."); } catch (final Exception e) { LOGGER.warn("Exception occurred during scanning dir: {}", listeningDir, e); } From 3a8d6d5278874f8ff7a86998e84fdf89046ba026 Mon Sep 17 00:00:00 2001 From: libo Date: Mon, 10 Nov 2025 16:03:44 +0800 Subject: [PATCH 040/180] =?UTF-8?q?Delete=20the=20tsfile=20and=20related?= =?UTF-8?q?=20attachments=20When=20only=20one=20table=20and=20dat=E2=80=A6?= =?UTF-8?q?=20(#16687)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Delete the tsfile and related attachments When only one table and data is cleared fully in the tsfile * Adjust logic is only effective in the table model, and refers to the tree model to delete files related to tsfile. * Supply an IT test to validate if involved supporting files about tsfile are deleted after delete all data in a table. * Resolve problem reported that file is not exist in the situation of 1C3D. * File may be not exist in the situation of 1C3D so that verify if it is exists first before scan the directory. * Fix the logic that verify only on table When multiple devices in same one table. * Don't delete file until all devices matched by time. * Avoid to delete the file when the device that need be deleted and other devices are all in the same one tsfile. * Increment a break to avoid extra I/O in the same file. * After delete all datas from a table, validate multiple devices involved files are deleted by different time range if it's success. * Add some logs and trace cause. * Debug * Change into logger * Decrement interval time * Clean environment before run testCompletelyDeleteTable * Recover all workflow yml files, log level need change into "debug" level. * Add a verification due to performance considerations. * Add some verification due to performance considerations. (cherry picked from commit f67526420d0f0fa5f01a2e8fa5929acd98173ab3) --- .../it/db/it/IoTDBDeletionTableIT.java | 372 ++++++++++++++++++ .../storageengine/dataregion/DataRegion.java | 121 +++++- 2 files changed, 492 insertions(+), 1 deletion(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java index 316ae614681cd..997a80a76f1bd 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java @@ -23,6 +23,7 @@ import org.apache.iotdb.isession.ITableSession; import org.apache.iotdb.isession.SessionDataSet; import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.ManualIT; import org.apache.iotdb.itbase.category.TableClusterIT; @@ -34,6 +35,7 @@ import org.apache.tsfile.read.common.RowRecord; import org.apache.tsfile.read.common.TimeRange; +import org.awaitility.Awaitility; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -47,8 +49,12 @@ import org.slf4j.LoggerFactory; import java.io.BufferedWriter; +import java.io.File; import java.io.FileWriter; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; @@ -63,8 +69,10 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -87,6 +95,17 @@ public class IoTDBDeletionTableIT { "INSERT INTO test.vehicle%d(time, deviceId, s0,s1,s2,s3,s4" + ") VALUES(%d,'d%d',%d,%d,%f,%s,%b)"; + private final String insertDeletionTemplate = + "INSERT INTO deletion.vehicle%d(time, deviceId, s0,s1,s2,s3,s4" + + ") VALUES(%d,'d%d',%d,%d,%f,%s,%b)"; + + private static String sequenceDataDir = "data" + File.separator + "sequence"; + private static String unsequenceDataDir = "data" + File.separator + "unsequence"; + + private static final String RESOURCE = ".resource"; + private static final String MODS = ".mods"; + private static final String TSFILE = ".tsfile"; + @BeforeClass public static void setUpClass() { Locale.setDefault(Locale.ENGLISH); @@ -470,6 +489,26 @@ public void testFullDeleteWithoutWhereClause() throws SQLException { } } + @Test + public void testFullDeleteWithoutWhereClauseByDifferentTime() throws SQLException { + prepareMultiDeviceDifferentTimeData(5, 2); + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("use deletion"); + statement.execute("DELETE FROM vehicle5"); + try (ResultSet set = statement.executeQuery("SELECT s0 FROM vehicle5")) { + int cnt = 0; + while (set.next()) { + cnt++; + } + assertEquals(0, cnt); + } + cleanData(5); + } catch (Exception e) { + fail(e.getMessage()); + } + } + @Test public void testDeleteWithSpecificDevice() throws SQLException { prepareData(6, 1); @@ -2003,6 +2042,226 @@ public void testDeletionReadPerformance() throws SQLException, IOException { } } + @Test + public void testCompletelyDeleteTable() throws SQLException { + int testNum = 1; + cleanDeletionDatabase(); + prepareDeletionDatabase(); + prepareMultiDeviceDifferentTimeData(testNum, 1); + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("use deletion"); + + statement.execute("DROP TABLE vehicle" + testNum); + + statement.execute("flush"); + + statement.execute( + String.format( + "CREATE TABLE vehicle%d(deviceId STRING TAG, s0 INT32 FIELD, s1 INT64 FIELD, s2 FLOAT FIELD, s3 TEXT FIELD, s4 BOOLEAN FIELD)", + testNum)); + + try (ResultSet set = statement.executeQuery("SELECT * FROM vehicle" + testNum)) { + assertFalse(set.next()); + } + + prepareData(testNum, 1); + + statement.execute("DELETE FROM vehicle" + testNum + " WHERE time <= 1000"); + + Awaitility.await() + .atMost(5, TimeUnit.MINUTES) + .pollDelay(500, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .until( + () -> { + AtomicBoolean completelyDeleteSuccess = new AtomicBoolean(true); + boolean allPass = true; + for (DataNodeWrapper wrapper : EnvFactory.getEnv().getDataNodeWrapperList()) { + String dataNodeDir = wrapper.getDataNodeDir(); + + if (Paths.get( + dataNodeDir + + File.separator + + sequenceDataDir + + File.separator + + "deletion") + .toFile() + .exists()) { + try (Stream s = + Files.walk( + Paths.get( + dataNodeDir + + File.separator + + sequenceDataDir + + File.separator + + "deletion"))) { + s.forEach( + source -> { + if (source.toString().endsWith(RESOURCE) + || source.toString().endsWith(MODS) + || source.toString().endsWith(TSFILE)) { + if (source.toFile().length() > 0) { + LOGGER.error( + "[testCompletelyDeleteTable] undeleted seq file : {}", + source.toFile().getAbsolutePath()); + completelyDeleteSuccess.set(false); + } + } + }); + } + } + + if (Paths.get( + dataNodeDir + + File.separator + + unsequenceDataDir + + File.separator + + "deletion") + .toFile() + .exists()) { + try (Stream s = + Files.walk( + Paths.get( + dataNodeDir + + File.separator + + unsequenceDataDir + + File.separator + + "deletion"))) { + s.forEach( + source -> { + if (source.toString().endsWith(RESOURCE) + || source.toString().endsWith(MODS) + || source.toString().endsWith(TSFILE)) { + if (source.toFile().length() > 0) { + LOGGER.error( + "[testCompletelyDeleteTable] undeleted unseq file: {}", + source.toFile().getAbsolutePath()); + completelyDeleteSuccess.set(false); + } + } + }); + } + } + + allPass = allPass && completelyDeleteSuccess.get(); + } + return allPass; + }); + } + cleanData(testNum); + } + + @Test + public void testMultiDeviceCompletelyDeleteTable() throws SQLException { + int testNum = 1; + cleanDeletionDatabase(); + prepareDeletionDatabase(); + prepareMultiDeviceDifferentTimeData(testNum, 2); + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("use deletion"); + + statement.execute("DROP TABLE vehicle" + testNum); + + statement.execute("flush"); + + statement.execute( + String.format( + "CREATE TABLE vehicle%d(deviceId STRING TAG, s0 INT32 FIELD, s1 INT64 FIELD, s2 FLOAT FIELD, s3 TEXT FIELD, s4 BOOLEAN FIELD)", + testNum)); + + try (ResultSet set = statement.executeQuery("SELECT * FROM vehicle" + testNum)) { + assertFalse(set.next()); + } + + prepareData(testNum, 2); + + statement.execute("DELETE FROM vehicle" + testNum + " WHERE time <= 1000"); + + Awaitility.await() + .atMost(5, TimeUnit.MINUTES) + .pollDelay(2, TimeUnit.SECONDS) + .pollInterval(2, TimeUnit.SECONDS) + .until( + () -> { + AtomicBoolean completelyDeleteSuccess = new AtomicBoolean(true); + boolean allPass = true; + for (DataNodeWrapper wrapper : EnvFactory.getEnv().getDataNodeWrapperList()) { + String dataNodeDir = wrapper.getDataNodeDir(); + + if (Paths.get( + dataNodeDir + + File.separator + + sequenceDataDir + + File.separator + + "deletion") + .toFile() + .exists()) { + try (Stream s = + Files.walk( + Paths.get( + dataNodeDir + + File.separator + + sequenceDataDir + + File.separator + + "deletion"))) { + s.forEach( + source -> { + if (source.toString().endsWith(RESOURCE) + || source.toString().endsWith(MODS) + || source.toString().endsWith(TSFILE)) { + if (source.toFile().length() > 0) { + LOGGER.error( + "[testMultiDeviceCompletelyDeleteTable] undeleted unseq file: {}", + source.toFile().getAbsolutePath()); + completelyDeleteSuccess.set(false); + } + } + }); + } + } + + if (Paths.get( + dataNodeDir + + File.separator + + unsequenceDataDir + + File.separator + + "deletion") + .toFile() + .exists()) { + try (Stream s = + Files.walk( + Paths.get( + dataNodeDir + + File.separator + + unsequenceDataDir + + File.separator + + "deletion"))) { + s.forEach( + source -> { + if (source.toString().endsWith(RESOURCE) + || source.toString().endsWith(MODS) + || source.toString().endsWith(TSFILE)) { + if (source.toFile().length() > 0) { + LOGGER.error( + "[testMultiDeviceCompletelyDeleteTable] undeleted unseq file: {}", + source.toFile().getAbsolutePath()); + completelyDeleteSuccess.set(false); + } + } + }); + } + } + + allPass = allPass && completelyDeleteSuccess.get(); + } + return allPass; + }); + } + cleanData(testNum); + } + private static void prepareDatabase() { try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { @@ -2015,6 +2274,40 @@ private static void prepareDatabase() { } } + private static void prepareDeletionDatabase() { + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE IF NOT EXISTS deletion"); + } catch (Exception e) { + fail(e.getMessage()); + } + } + + private void cleanDeletionDatabase() { + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("DROP DATABASE IF EXISTS deletion"); + for (DataNodeWrapper wrapper : EnvFactory.getEnv().getDataNodeWrapperList()) { + String dataNodeDir = wrapper.getDataNodeDir(); + File targetFile = + Paths.get(dataNodeDir + File.separator + sequenceDataDir + File.separator + "deletion") + .toFile(); + if (targetFile.exists()) { + targetFile.delete(); + } + + targetFile = + Paths.get(dataNodeDir + File.separator + sequenceDataDir + File.separator + "deletion") + .toFile(); + if (targetFile.exists()) { + targetFile.delete(); + } + } + } catch (Exception e) { + fail(e.getMessage()); + } + } + private void prepareData(int testNum, int deviceNum) throws SQLException { try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { @@ -2062,6 +2355,85 @@ private void prepareData(int testNum, int deviceNum) throws SQLException { } } + private void prepareMultiDeviceDifferentTimeData(int testNum, int deviceNum) throws SQLException { + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("use deletion"); + statement.execute( + String.format( + "CREATE TABLE IF NOT EXISTS vehicle%d(deviceId STRING TAG, s0 INT32 FIELD, s1 INT64 FIELD, s2 FLOAT FIELD, s3 TEXT FIELD, s4 BOOLEAN FIELD)", + testNum)); + + for (int d = 0; d < deviceNum; d++) { + // prepare seq file + for (int i = 201 * (d + 1); i <= 300 * (d + 1); i++) { + statement.execute( + String.format( + insertDeletionTemplate, + testNum, + i, + d, + i, + i, + (double) i, + "'" + i + "'", + i % 2 == 0)); + } + } + + statement.execute("flush"); + + for (int d = 0; d < deviceNum; d++) { + // prepare unseq File + for (int i = 1 * (d + 1); i <= 100 * (d + 1); i++) { + statement.execute( + String.format( + insertDeletionTemplate, + testNum, + i, + d, + i, + i, + (double) i, + "'" + i + "'", + i % 2 == 0)); + } + } + statement.execute("flush"); + + for (int d = 0; d < deviceNum; d++) { + // prepare BufferWrite cache + for (int i = 301 * (d + 1); i <= 400 * (d + 1); i++) { + statement.execute( + String.format( + insertDeletionTemplate, + testNum, + i, + d, + i, + i, + (double) i, + "'" + i + "'", + i % 2 == 0)); + } + // prepare Overflow cache + for (int i = 101 * (d + 1); i <= 200 * (d + 1); i++) { + statement.execute( + String.format( + insertDeletionTemplate, + testNum, + i, + d, + i, + i, + (double) i, + "'" + i + "'", + i % 2 == 0)); + } + } + } + } + private void cleanData(int testNum) throws SQLException { try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { 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 eec169aa807a5..2c0c39032c437 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 @@ -2941,12 +2941,110 @@ private void deleteDataInUnsealedFiles( private void deleteDataInSealedFiles(Collection sealedTsFiles, ModEntry deletion) throws IOException { Set involvedModificationFiles = new HashSet<>(); + List deletedByMods = new ArrayList<>(); + List deletedByFiles = new ArrayList<>(); for (TsFileResource sealedTsFile : sealedTsFiles) { if (canSkipDelete(sealedTsFile, deletion)) { continue; } - involvedModificationFiles.add(sealedTsFile.getModFileForWrite()); + ITimeIndex timeIndex = sealedTsFile.getTimeIndex(); + + if ((timeIndex instanceof ArrayDeviceTimeIndex) + && (deletion.getType() == ModEntry.ModType.TABLE_DELETION)) { + ArrayDeviceTimeIndex deviceTimeIndex = (ArrayDeviceTimeIndex) timeIndex; + Set devicesInFile = deviceTimeIndex.getDevices(); + boolean onlyOneTable = false; + + if (deletion instanceof TableDeletionEntry) { + TableDeletionEntry tableDeletionEntry = (TableDeletionEntry) deletion; + String tableName = tableDeletionEntry.getTableName(); + long matchSize = + devicesInFile.stream() + .filter( + device -> { + if (logger.isDebugEnabled()) { + logger.debug( + "device is {}, deviceTable is {}, tableDeletionEntry.getPredicate().matches(device) is {}", + device, + device.getTableName(), + tableDeletionEntry.getPredicate().matches(device)); + } + return tableName.equals(device.getTableName()) + && tableDeletionEntry.getPredicate().matches(device); + }) + .count(); + onlyOneTable = matchSize == devicesInFile.size(); + if (logger.isDebugEnabled()) { + logger.debug( + "tableName is {}, matchSize is {}, onlyOneTable is {}", + tableName, + matchSize, + onlyOneTable); + } + } + + if (onlyOneTable) { + int matchSize = 0; + for (IDeviceID device : devicesInFile) { + Optional optStart = deviceTimeIndex.getStartTime(device); + Optional optEnd = deviceTimeIndex.getEndTime(device); + if (!optStart.isPresent() || !optEnd.isPresent()) { + continue; + } + + long fileStartTime = optStart.get(); + long fileEndTime = optEnd.get(); + + if (logger.isDebugEnabled()) { + logger.debug( + "tableName is {}, device is {}, deletionStartTime is {}, deletionEndTime is {}, fileStartTime is {}, fileEndTime is {}", + device.getTableName(), + device, + deletion.getStartTime(), + deletion.getEndTime(), + fileStartTime, + fileEndTime); + } + if (isFileFullyMatchedByTime(deletion, fileStartTime, fileEndTime)) { + ++matchSize; + } else { + deletedByMods.add(sealedTsFile); + break; + } + } + if (matchSize == devicesInFile.size()) { + deletedByFiles.add(sealedTsFile); + } + + if (logger.isDebugEnabled()) { + logger.debug("expect is {}, actual is {}", devicesInFile.size(), matchSize); + for (TsFileResource tsFileResource : deletedByFiles) { + logger.debug( + "delete tsFileResource is {}", tsFileResource.getTsFile().getAbsolutePath()); + } + } + } else { + involvedModificationFiles.add(sealedTsFile.getModFileForWrite()); + } + } else { + involvedModificationFiles.add(sealedTsFile.getModFileForWrite()); + } + } + + for (TsFileResource tsFileResource : deletedByMods) { + if (tsFileResource.isClosed() + || !tsFileResource.getProcessor().deleteDataInMemory(deletion)) { + involvedModificationFiles.add(tsFileResource.getModFileForWrite()); + } // else do nothing + } + + if (!deletedByFiles.isEmpty()) { + deleteTsFileCompletely(deletedByFiles); + if (logger.isDebugEnabled()) { + logger.debug( + "deleteTsFileCompletely execute successful, all tsfile are deleted successfully"); + } } if (involvedModificationFiles.isEmpty()) { @@ -2984,6 +3082,27 @@ private void deleteDataInSealedFiles(Collection sealedTsFiles, M involvedModificationFiles.size()); } + private boolean isFileFullyMatchedByTime( + ModEntry deletion, long fileStartTime, long fileEndTime) { + return fileStartTime >= deletion.getStartTime() && fileEndTime <= deletion.getEndTime(); + } + + /** Delete completely TsFile and related supporting files */ + private void deleteTsFileCompletely(List tsfileResourceList) { + for (TsFileResource tsFileResource : tsfileResourceList) { + tsFileManager.remove(tsFileResource, tsFileResource.isSeq()); + tsFileResource.writeLock(); + try { + FileMetrics.getInstance() + .deleteTsFile(tsFileResource.isSeq(), Collections.singletonList(tsFileResource)); + tsFileResource.remove(); + logger.info("Remove tsfile {} directly when delete data", tsFileResource.getTsFilePath()); + } finally { + tsFileResource.writeUnlock(); + } + } + } + private void deleteDataDirectlyInFile(List tsfileResourceList, ModEntry modEntry) throws IOException { List deletedByMods = new ArrayList<>(); From 602d4650aaf0f9a26f41f2898fb51f420fbb344e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 10 Nov 2025 17:27:18 +0800 Subject: [PATCH 041/180] Fixed the potential non-consistency caused by initialize of DataNodeTableCache & Cleared the storageGroups in compatible codes (#16693) * fix * refactor * may-complete * fix * refactor * fix * Update IoTDBFlushQueryIT.java * bug-fix * fix (cherry picked from commit 639de5255ce51c4c104160faafad3fa26ad60e82) --- .../env/cluster/config/MppCommonConfig.java | 4 +- .../cluster/config/MppSharedCommonConfig.java | 6 +- .../iotdb/itbase/constant/TestConstant.java | 14 ++-- .../apache/iotdb/itbase/env/CommonConfig.java | 2 +- .../it/IoTDBConfigNodeSnapshotIT.java | 31 ++++---- .../database/IoTDBDatabaseSetAndDeleteIT.java | 66 ++++++++-------- .../load/IoTDBConfigNodeSwitchLeaderIT.java | 2 +- .../IoTDBAutoRegionGroupExtensionIT.java | 8 +- .../IoTDBCustomRegionGroupExtensionIT.java | 2 +- .../partition/IoTDBPartitionCreationIT.java | 4 +- .../it/partition/IoTDBPartitionDurableIT.java | 4 +- .../it/partition/IoTDBPartitionGetterIT.java | 41 +++++----- .../it/utils/ConfigNodeTestUtils.java | 16 ++-- .../apache/iotdb/db/it/IoTDBExampleIT.java | 4 +- .../apache/iotdb/db/it/IoTDBFlushQueryIT.java | 5 +- .../iotdb/db/it/IoTDBRestServiceIT.java | 8 +- .../iotdb/db/it/IoTDBSetConfigurationIT.java | 2 +- .../iotdb/db/it/IoTDBSimpleQueryIT.java | 2 +- .../it/IoTDBSyntaxConventionIdentifierIT.java | 2 +- .../iotdb/db/it/path/IoTDBQuotedPathIT.java | 4 +- .../db/it/schema/IoTDBAutoCreateSchemaIT.java | 14 ++-- .../it/schema/IoTDBDeactivateTemplateIT.java | 2 +- .../db/it/schema/IoTDBDeleteTimeSeriesIT.java | 2 +- .../db/it/schema/IoTDBMetadataFetchIT.java | 2 +- .../apache/iotdb/db/it/schema/IoTDBTagIT.java | 2 +- .../it/query/old/IoTDBSimpleQueryTableIT.java | 2 +- .../session/it/IoTDBSessionSimpleIT.java | 22 +++--- .../it/IoTDBSessionSyntaxConventionIT.java | 6 +- .../write/database/PreDeleteDatabasePlan.java | 22 +++--- .../confignode/manager/ConfigManager.java | 28 +++---- .../iotdb/confignode/manager/IManager.java | 16 ++-- .../confignode/manager/ProcedureManager.java | 27 +++++++ .../confignode/manager/load/LoadManager.java | 2 +- .../manager/load/balancer/RegionBalancer.java | 2 +- .../manager/schema/ClusterSchemaManager.java | 26 ++++++- .../persistence/partition/PartitionInfo.java | 2 +- .../impl/schema/DeleteDatabaseProcedure.java | 28 ++++--- ...oupState.java => DeleteDatabaseState.java} | 2 +- .../impl/DataNodeInternalRPCServiceImpl.java | 48 ++++++------ .../plan/analyze/AnalyzeVisitor.java | 2 +- .../config/TreeConfigTaskVisitor.java | 3 +- .../queryengine/plan/parser/ASTVisitor.java | 18 ++--- .../fetcher/TableDeviceSchemaFetcher.java | 6 +- .../security/TreeAccessCheckVisitor.java | 2 +- .../plan/statement/StatementVisitor.java | 2 +- .../metadata/CountDatabaseStatement.java | 2 +- .../impl/mem/MTreeBelowSGMemoryImpl.java | 78 +++++++++---------- .../table/DataNodeTableCache.java | 17 ++++ .../schema/table/NonCommittableTsTable.java | 43 ++++++++++ .../iotdb/commons/schema/table/TsTable.java | 24 +++--- 50 files changed, 384 insertions(+), 295 deletions(-) rename iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/{DeleteStorageGroupState.java => DeleteDatabaseState.java} (96%) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/NonCommittableTsTable.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 1df43c16206df..6cb7e00824266 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 @@ -580,8 +580,8 @@ public CommonConfig setSubscriptionEnabled(boolean subscriptionEnabled) { } @Override - public CommonConfig setDefaultStorageGroupLevel(int defaultStorageGroupLevel) { - setProperty("default_database_level", String.valueOf(defaultStorageGroupLevel)); + public CommonConfig setDefaultDatabaseLevel(int defaultDatabaseLevel) { + setProperty("default_database_level", String.valueOf(defaultDatabaseLevel)); return this; } 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 504b1ae60e234..1b4801f569226 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 @@ -602,9 +602,9 @@ public CommonConfig setSubscriptionEnabled(boolean subscriptionEnabled) { } @Override - public CommonConfig setDefaultStorageGroupLevel(int defaultStorageGroupLevel) { - dnConfig.setDefaultStorageGroupLevel(defaultStorageGroupLevel); - cnConfig.setDefaultStorageGroupLevel(defaultStorageGroupLevel); + public CommonConfig setDefaultDatabaseLevel(int defaultDatabaseLevel) { + dnConfig.setDefaultDatabaseLevel(defaultDatabaseLevel); + cnConfig.setDefaultDatabaseLevel(defaultDatabaseLevel); return this; } diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/constant/TestConstant.java b/integration-test/src/main/java/org/apache/iotdb/itbase/constant/TestConstant.java index 8a9d11516c5ae..03b09bc819be0 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/constant/TestConstant.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/constant/TestConstant.java @@ -168,13 +168,12 @@ public static String recordToInsert(TSRecord record) { } public static String getTestTsFilePath( - String logicalStorageGroupName, - long VirtualStorageGroupId, + String logicalDatabaseName, + long VirtualDatabaseId, long TimePartitionId, long tsFileVersion) { String filePath = - String.format( - TEST_TSFILE_PATH, logicalStorageGroupName, VirtualStorageGroupId, TimePartitionId); + String.format(TEST_TSFILE_PATH, logicalDatabaseName, VirtualDatabaseId, TimePartitionId); String fileName = System.currentTimeMillis() + FilePathUtils.FILE_NAME_SEPARATOR @@ -184,11 +183,8 @@ public static String getTestTsFilePath( } public static String getTestTsFileDir( - String logicalStorageGroupName, long VirtualStorageGroupId, long TimePartitionId) { + String logicalDatabaseName, long VirtualDatabaseId, long TimePartitionId) { return String.format( - TestConstant.TEST_TSFILE_PATH, - logicalStorageGroupName, - VirtualStorageGroupId, - TimePartitionId); + TestConstant.TEST_TSFILE_PATH, logicalDatabaseName, VirtualDatabaseId, TimePartitionId); } } 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 9767e1c089e55..f27bdc8d66da8 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 @@ -186,7 +186,7 @@ CommonConfig setSubscriptionPrefetchTsFileBatchMaxSizeInBytes( CommonConfig setSubscriptionEnabled(boolean subscriptionEnabled); - default CommonConfig setDefaultStorageGroupLevel(int defaultStorageGroupLevel) { + default CommonConfig setDefaultDatabaseLevel(int defaultDatabaseLevel) { return this; } diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBConfigNodeSnapshotIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBConfigNodeSnapshotIT.java index d906cd43bda57..a5c757d8dd361 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBConfigNodeSnapshotIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/IoTDBConfigNodeSnapshotIT.java @@ -106,7 +106,7 @@ public void tearDown() { @Test public void testPartitionInfoSnapshot() throws Exception { final String sg = "root.sg"; - final int storageGroupNum = 10; + final int databaseNum = 10; final int seriesPartitionSlotsNum = 10; final int timePartitionSlotsNum = 10; @@ -118,10 +118,10 @@ public void testPartitionInfoSnapshot() throws Exception { Set expectedCQEntries = createCQs(client); - for (int i = 0; i < storageGroupNum; i++) { - String storageGroup = sg + i; - TDatabaseSchema storageGroupSchema = new TDatabaseSchema(storageGroup); - TSStatus status = client.setDatabase(storageGroupSchema); + for (int i = 0; i < databaseNum; i++) { + String database = sg + i; + TDatabaseSchema databaseSchema = new TDatabaseSchema(database); + TSStatus status = client.setDatabase(databaseSchema); assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); for (int j = 0; j < seriesPartitionSlotsNum; j++) { @@ -129,7 +129,7 @@ public void testPartitionInfoSnapshot() throws Exception { // Create SchemaPartition ByteBuffer patternTree = - generatePatternTreeBuffer(new String[] {storageGroup + ".d" + j + ".s"}); + generatePatternTreeBuffer(new String[] {database + ".d" + j + ".s"}); TSchemaPartitionReq schemaPartitionReq = new TSchemaPartitionReq(patternTree); TSchemaPartitionTableResp schemaPartitionTableResp = client.getOrCreateSchemaPartitionTable(schemaPartitionReq); @@ -139,10 +139,8 @@ public void testPartitionInfoSnapshot() throws Exception { schemaPartitionTableResp.getStatus().getCode()); Assert.assertNotNull(schemaPartitionTableResp.getSchemaPartitionTable()); assertEquals(1, schemaPartitionTableResp.getSchemaPartitionTableSize()); - Assert.assertNotNull( - schemaPartitionTableResp.getSchemaPartitionTable().get(storageGroup)); - assertEquals( - 1, schemaPartitionTableResp.getSchemaPartitionTable().get(storageGroup).size()); + Assert.assertNotNull(schemaPartitionTableResp.getSchemaPartitionTable().get(database)); + assertEquals(1, schemaPartitionTableResp.getSchemaPartitionTable().get(database).size()); for (int k = 0; k < timePartitionSlotsNum; k++) { TTimePartitionSlot timePartitionSlot = @@ -151,9 +149,9 @@ public void testPartitionInfoSnapshot() throws Exception { // Create DataPartition Map> partitionSlotsMap = new HashMap<>(); - partitionSlotsMap.put(storageGroup, new HashMap<>()); + partitionSlotsMap.put(database, new HashMap<>()); partitionSlotsMap - .get(storageGroup) + .get(database) .put( seriesPartitionSlot, new TTimeSlotList() @@ -167,19 +165,18 @@ public void testPartitionInfoSnapshot() throws Exception { dataPartitionTableResp.getStatus().getCode()); Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable()); assertEquals(1, dataPartitionTableResp.getDataPartitionTableSize()); - Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable().get(storageGroup)); - assertEquals( - 1, dataPartitionTableResp.getDataPartitionTable().get(storageGroup).size()); + Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable().get(database)); + assertEquals(1, dataPartitionTableResp.getDataPartitionTable().get(database).size()); Assert.assertNotNull( dataPartitionTableResp .getDataPartitionTable() - .get(storageGroup) + .get(database) .get(seriesPartitionSlot)); assertEquals( 1, dataPartitionTableResp .getDataPartitionTable() - .get(storageGroup) + .get(database) .get(seriesPartitionSlot) .size()); } diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java index e08339823d6db..959f37d594d51 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java @@ -74,17 +74,17 @@ public void testSetAndQueryDatabase() throws Exception { try (SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { // set Database0 by default values - TDatabaseSchema storageGroupSchema0 = new TDatabaseSchema(sg0); - status = client.setDatabase(storageGroupSchema0); + TDatabaseSchema databaseSchema0 = new TDatabaseSchema(sg0); + status = client.setDatabase(databaseSchema0); Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); // set Database1 by specific values - TDatabaseSchema storageGroupSchema1 = + TDatabaseSchema databaseSchema1 = new TDatabaseSchema(sg1) .setSchemaReplicationFactor(5) .setDataReplicationFactor(5) .setTimePartitionInterval(2048L); - status = client.setDatabase(storageGroupSchema1); + status = client.setDatabase(databaseSchema1); Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); // test count all Databases @@ -111,21 +111,21 @@ public void testSetAndQueryDatabase() throws Exception { TSStatusCode.SUCCESS_STATUS.getStatusCode(), getResp.getStatus().getCode()); Map schemaMap = getResp.getDatabaseSchemaMap(); Assert.assertEquals(2, schemaMap.size()); - TDatabaseSchema storageGroupSchema = schemaMap.get(sg0); - Assert.assertNotNull(storageGroupSchema); - Assert.assertEquals(sg0, storageGroupSchema.getName()); - Assert.assertEquals(1, storageGroupSchema.getSchemaReplicationFactor()); - Assert.assertEquals(1, storageGroupSchema.getDataReplicationFactor()); - Assert.assertEquals(604800000, storageGroupSchema.getTimePartitionInterval()); - storageGroupSchema = schemaMap.get(sg1); - Assert.assertNotNull(storageGroupSchema); - Assert.assertEquals(sg1, storageGroupSchema.getName()); - Assert.assertEquals(5, storageGroupSchema.getSchemaReplicationFactor()); - Assert.assertEquals(5, storageGroupSchema.getDataReplicationFactor()); - Assert.assertEquals(2048L, storageGroupSchema.getTimePartitionInterval()); + TDatabaseSchema databaseSchema = schemaMap.get(sg0); + Assert.assertNotNull(databaseSchema); + Assert.assertEquals(sg0, databaseSchema.getName()); + Assert.assertEquals(1, databaseSchema.getSchemaReplicationFactor()); + Assert.assertEquals(1, databaseSchema.getDataReplicationFactor()); + Assert.assertEquals(604800000, databaseSchema.getTimePartitionInterval()); + databaseSchema = schemaMap.get(sg1); + Assert.assertNotNull(databaseSchema); + Assert.assertEquals(sg1, databaseSchema.getName()); + Assert.assertEquals(5, databaseSchema.getSchemaReplicationFactor()); + Assert.assertEquals(5, databaseSchema.getDataReplicationFactor()); + Assert.assertEquals(2048L, databaseSchema.getTimePartitionInterval()); // test fail by re-register - status = client.setDatabase(storageGroupSchema0); + status = client.setDatabase(databaseSchema0); Assert.assertEquals(TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode(), status.getCode()); // test Database setter interfaces @@ -149,35 +149,35 @@ public void testSetAndQueryDatabase() throws Exception { TSStatusCode.SUCCESS_STATUS.getStatusCode(), getResp.getStatus().getCode()); schemaMap = getResp.getDatabaseSchemaMap(); Assert.assertEquals(1, schemaMap.size()); - storageGroupSchema = schemaMap.get(sg1); - Assert.assertNotNull(storageGroupSchema); - Assert.assertEquals(sg1, storageGroupSchema.getName()); - Assert.assertEquals(1, storageGroupSchema.getSchemaReplicationFactor()); - Assert.assertEquals(1, storageGroupSchema.getDataReplicationFactor()); - Assert.assertEquals(604800, storageGroupSchema.getTimePartitionInterval()); + databaseSchema = schemaMap.get(sg1); + Assert.assertNotNull(databaseSchema); + Assert.assertEquals(sg1, databaseSchema.getName()); + Assert.assertEquals(1, databaseSchema.getSchemaReplicationFactor()); + Assert.assertEquals(1, databaseSchema.getDataReplicationFactor()); + Assert.assertEquals(604800, databaseSchema.getTimePartitionInterval()); } } @Test - public void testDeleteStorageGroup() throws Exception { + public void testDeleteDatabase() throws Exception { TSStatus status; final String sg0 = "root.sg0"; final String sg1 = "root.sg1"; try (SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { - TDatabaseSchema storageGroupSchema0 = new TDatabaseSchema(sg0); - // set StorageGroup0 by default values - status = client.setDatabase(storageGroupSchema0); + TDatabaseSchema databaseSchema0 = new TDatabaseSchema(sg0); + // set Database0 by default values + status = client.setDatabase(databaseSchema0); Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - // set StorageGroup1 by specific values - TDatabaseSchema storageGroupSchema1 = new TDatabaseSchema(sg1); - status = client.setDatabase(storageGroupSchema1); + // set Database1 by specific values + TDatabaseSchema databaseSchema1 = new TDatabaseSchema(sg1); + status = client.setDatabase(databaseSchema1); Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - TDeleteDatabasesReq deleteStorageGroupsReq = new TDeleteDatabasesReq(); + TDeleteDatabasesReq deleteDatabasesReq = new TDeleteDatabasesReq(); List sgs = Arrays.asList(sg0, sg1); - deleteStorageGroupsReq.setPrefixPathList(sgs); - TSStatus deleteSgStatus = client.deleteDatabases(deleteStorageGroupsReq); + deleteDatabasesReq.setPrefixPathList(sgs); + TSStatus deleteSgStatus = client.deleteDatabases(deleteDatabasesReq); TDatabaseSchemaResp root = client.getMatchedDatabaseSchemas( new TGetDatabaseReq(Arrays.asList("root", "sg*"), ALL_MATCH_SCOPE_BINARY)); diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java index a49d1501af5a5..2246ce4314a6b 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/load/IoTDBConfigNodeSwitchLeaderIT.java @@ -101,7 +101,7 @@ public void basicDataInheritIT() throws Exception { try (SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { - // Set StorageGroups + // Set Databases status = client.setDatabase((new TDatabaseSchema(sg0))); Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); status = client.setDatabase((new TDatabaseSchema(sg1))); diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBAutoRegionGroupExtensionIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBAutoRegionGroupExtensionIT.java index a82b992a4df8a..33481d9faae37 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBAutoRegionGroupExtensionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBAutoRegionGroupExtensionIT.java @@ -100,9 +100,9 @@ public void testAutoRegionGroupExtensionPolicy() throws Exception { try (SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { - setStorageGroupAndCheckRegionGroupDistribution(client); + setDatabaseAndCheckRegionGroupDistribution(client); - // Delete all StorageGroups + // Delete all Databases for (int i = 0; i < TEST_DATABASE_NUM; i++) { String curSg = DATABASE + i; client.deleteDatabase(new TDeleteDatabaseReq(curSg)); @@ -126,11 +126,11 @@ public void testAutoRegionGroupExtensionPolicy() throws Exception { Assert.assertTrue(isAllRegionGroupDeleted); // Re-test for safety - setStorageGroupAndCheckRegionGroupDistribution(client); + setDatabaseAndCheckRegionGroupDistribution(client); } } - private void setStorageGroupAndCheckRegionGroupDistribution(SyncConfigNodeIServiceClient client) + private void setDatabaseAndCheckRegionGroupDistribution(SyncConfigNodeIServiceClient client) throws TException, IllegalPathException, IOException { for (int i = 0; i < TEST_DATABASE_NUM; i++) { diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBCustomRegionGroupExtensionIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBCustomRegionGroupExtensionIT.java index 820e6835c5954..64b35875a203b 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBCustomRegionGroupExtensionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBCustomRegionGroupExtensionIT.java @@ -98,7 +98,7 @@ public void testCustomRegionGroupExtensionPolicy() throws Exception { for (int i = 0; i < testSgNum; i++) { String curSg = sg + i; - /* Set StorageGroup */ + /* Set Database */ TSStatus status = client.setDatabase(new TDatabaseSchema(curSg)); Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionCreationIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionCreationIT.java index 5043b4a842697..954f54e01bd02 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionCreationIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionCreationIT.java @@ -96,10 +96,10 @@ public void setUp() throws Exception { // Init 1C3D environment EnvFactory.getEnv().initClusterEnvironment(1, 3); - setStorageGroup(); + setDatabase(); } - private void setStorageGroup() throws Exception { + private void setDatabase() throws Exception { try (SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { TSStatus status = client.setDatabase(new TDatabaseSchema(sg)); diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionDurableIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionDurableIT.java index ceedc355295b3..8d6e4a5f83113 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionDurableIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionDurableIT.java @@ -104,10 +104,10 @@ public void setUp() throws Exception { // Init 1C3D environment EnvFactory.getEnv().initClusterEnvironment(1, 3); - setStorageGroup(); + setDatabase(); } - private void setStorageGroup() throws Exception { + private void setDatabase() throws Exception { try (SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { TSStatus status = client.setDatabase(new TDatabaseSchema(sg)); diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java index 6ca5d884b0734..f110d4a8d28f0 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java @@ -84,7 +84,7 @@ public class IoTDBPartitionGetterIT { private static final int testDataRegionGroupPerDatabase = 5; private static final String sg = "root.sg"; - private static final int storageGroupNum = 2; + private static final int databaseNum = 2; private static final int testSeriesPartitionSlotNum = 1000; private static final int seriesPartitionBatchSize = 10; private static final int testTimePartitionSlotsNum = 10; @@ -116,8 +116,8 @@ public static void tearDown() { private static void prepareData() throws Exception { try (SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { - /* Set StorageGroups */ - for (int i = 0; i < storageGroupNum; i++) { + /* Set Databases */ + for (int i = 0; i < databaseNum; i++) { TSStatus status = client.setDatabase(new TDatabaseSchema(sg + i)); Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); } @@ -149,13 +149,13 @@ private static void prepareData() throws Exception { } /* Create DataPartitions */ - for (int i = 0; i < storageGroupNum; i++) { - String storageGroup = sg + i; + for (int i = 0; i < databaseNum; i++) { + String database = sg + i; for (int j = 0; j < testSeriesPartitionSlotNum; j += seriesPartitionBatchSize) { for (long k = 0; k < testTimePartitionSlotsNum; k += timePartitionBatchSize) { Map> partitionSlotsMap = ConfigNodeTestUtils.constructPartitionSlotsMap( - storageGroup, + database, j, j + seriesPartitionBatchSize, k, @@ -187,7 +187,7 @@ private static void prepareData() throws Exception { dataPartitionTableResp.getStatus().getCode()); Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable()); ConfigNodeTestUtils.checkDataPartitionTable( - storageGroup, + database, j, j + seriesPartitionBatchSize, k, @@ -229,7 +229,7 @@ public void testGetSchemaPartition() throws Exception { schemaPartitionTableResp.getStatus().getCode()); Assert.assertEquals(0, schemaPartitionTableResp.getSchemaPartitionTableSize()); - // Test getSchemaPartition, when a device path doesn't match any StorageGroup and including + // Test getSchemaPartition, when a device path doesn't match any Database and including // "**", ConfigNode will return all the SchemaPartitions buffer = generatePatternTreeBuffer(new String[] {allPaths}); schemaPartitionReq.setPathPatternTree(buffer); @@ -244,8 +244,8 @@ public void testGetSchemaPartition() throws Exception { Assert.assertEquals(2, schemaPartitionTable.get(sg + i).size()); } - // Test getSchemaPartition, when a device path matches with a StorageGroup and end with "*", - // ConfigNode will return all the SchemaPartitions in this StorageGroup + // Test getSchemaPartition, when a device path matches with a Database and end with "*", + // ConfigNode will return all the SchemaPartitions in this Database buffer = generatePatternTreeBuffer(new String[] {allSg0, d11}); schemaPartitionReq.setPathPatternTree(buffer); schemaPartitionTableResp = client.getSchemaPartitionTable(schemaPartitionReq); @@ -295,17 +295,16 @@ public void testGetDataPartition() throws Exception { } int leastDataRegionGroupNum = (int) - Math.ceil( - (double) totalCpuCoreNum / (double) (storageGroupNum * testReplicationFactor)); + Math.ceil((double) totalCpuCoreNum / (double) (databaseNum * testReplicationFactor)); leastDataRegionGroupNum = Math.min(leastDataRegionGroupNum, testDataRegionGroupPerDatabase); - for (int i = 0; i < storageGroupNum; i++) { - String storageGroup = sg + i; + for (int i = 0; i < databaseNum; i++) { + String database = sg + i; for (int j = 0; j < testSeriesPartitionSlotNum; j += seriesPartitionBatchSize) { for (long k = 0; k < testTimePartitionSlotsNum; k += timePartitionBatchSize) { partitionSlotsMap = ConfigNodeTestUtils.constructPartitionSlotsMap( - storageGroup, + database, j, j + seriesPartitionBatchSize, k, @@ -320,7 +319,7 @@ public void testGetDataPartition() throws Exception { dataPartitionTableResp.getStatus().getCode()); Assert.assertNotNull(dataPartitionTableResp.getDataPartitionTable()); ConfigNodeTestUtils.checkDataPartitionTable( - storageGroup, + database, j, j + seriesPartitionBatchSize, k, @@ -334,10 +333,10 @@ public void testGetDataPartition() throws Exception { // And this number should be greater than or equal to leastDataRegionGroupNum TGetDatabaseReq req = new TGetDatabaseReq( - Arrays.asList(storageGroup.split("\\.")), SchemaConstant.ALL_MATCH_SCOPE_BINARY); - TShowDatabaseResp showStorageGroupResp = client.showDatabase(req); + Arrays.asList(database.split("\\.")), SchemaConstant.ALL_MATCH_SCOPE_BINARY); + TShowDatabaseResp showDatabaseResp = client.showDatabase(req); Assert.assertTrue( - showStorageGroupResp.getDatabaseInfoMap().get(storageGroup).getDataRegionNum() + showDatabaseResp.getDatabaseInfoMap().get(database).getDataRegionNum() >= leastDataRegionGroupNum); } } @@ -394,7 +393,7 @@ public void testGetSlots() throws Exception { TSStatusCode.SUCCESS_STATUS.getStatusCode(), getRegionIdResp.status.getCode()); // Get all RegionIds within database - for (int i = 0; i < storageGroupNum; i++) { + for (int i = 0; i < databaseNum; i++) { String curSg = sg + i; getRegionIdReq = new TGetRegionIdReq(TConsensusGroupType.DataRegion); getRegionIdReq.setDatabase(curSg); @@ -536,7 +535,7 @@ public void testGetSchemaNodeManagementPartition() throws Exception { Assert.assertEquals( TSStatusCode.SUCCESS_STATUS.getStatusCode(), nodeManagementResp.getStatus().getCode()); // +1 for AUDIT database - Assert.assertEquals(storageGroupNum + 1, nodeManagementResp.getMatchedNodeSize()); + Assert.assertEquals(databaseNum + 1, nodeManagementResp.getMatchedNodeSize()); Assert.assertNotNull(nodeManagementResp.getSchemaRegionMap()); Assert.assertEquals(0, nodeManagementResp.getSchemaRegionMapSize()); } diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java index dfc37e52526c5..0697a6c674d24 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java @@ -120,24 +120,24 @@ public static ByteBuffer generatePatternTreeBuffer(String[] paths) } public static Map> constructPartitionSlotsMap( - String storageGroup, + String database, int seriesSlotStart, int seriesSlotEnd, long timeSlotStart, long timeSlotEnd, long timePartitionInterval) { Map> result = new HashMap<>(); - result.put(storageGroup, new HashMap<>()); + result.put(database, new HashMap<>()); for (int i = seriesSlotStart; i < seriesSlotEnd; i++) { TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(i); result - .get(storageGroup) + .get(database) .put(seriesPartitionSlot, new TTimeSlotList().setTimePartitionSlots(new ArrayList<>())); for (long j = timeSlotStart; j < timeSlotEnd; j++) { TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot(j * timePartitionInterval); result - .get(storageGroup) + .get(database) .get(seriesPartitionSlot) .getTimePartitionSlots() .add(timePartitionSlot); @@ -148,7 +148,7 @@ public static Map> constructPar } public static void checkDataPartitionTable( - String storageGroup, + String database, int seriesSlotStart, int seriesSlotEnd, long timeSlotStart, @@ -157,12 +157,12 @@ public static void checkDataPartitionTable( Map>>> dataPartitionTable) { - // Check the existence of StorageGroup - Assert.assertTrue(dataPartitionTable.containsKey(storageGroup)); + // Check the existence of Database + Assert.assertTrue(dataPartitionTable.containsKey(database)); // Check the number of SeriesPartitionSlot Map>> - seriesPartitionTable = dataPartitionTable.get(storageGroup); + seriesPartitionTable = dataPartitionTable.get(database); Assert.assertEquals(seriesSlotEnd - seriesSlotStart, seriesPartitionTable.size()); for (int i = seriesSlotStart; i < seriesSlotEnd; i++) { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBExampleIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBExampleIT.java index 91bf6462033f5..d51a058ec51d3 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBExampleIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBExampleIT.java @@ -58,8 +58,8 @@ public void exampleTest() throws Exception { statement.execute("CREATE DATABASE root.sg"); try (ResultSet resultSet = statement.executeQuery("show databases root.sg")) { if (resultSet.next()) { - String storageGroupPath = resultSet.getString(ColumnHeaderConstant.DATABASE); - Assert.assertEquals("root.sg", storageGroupPath); + String databasePath = resultSet.getString(ColumnHeaderConstant.DATABASE); + Assert.assertEquals("root.sg", databasePath); } else { Assert.fail("This ResultSet is empty."); } diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFlushQueryIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFlushQueryIT.java index ae039b792b1e6..6253e618edacf 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFlushQueryIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFlushQueryIT.java @@ -26,7 +26,6 @@ import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -166,7 +165,6 @@ public void testFlushGivenGroupNoData() { } @Test - @Ignore public void testFlushNotExistGroupNoData() { try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { @@ -175,8 +173,7 @@ public void testFlushNotExistGroupNoData() { statement.execute( "FLUSH root.noexist.nodatagroup1,root.notExistGroup1,root.notExistGroup2"); } catch (SQLException sqe) { - String expectedMsg = - "322: 322: storageGroup root.notExistGroup1,root.notExistGroup2 does not exist"; + String expectedMsg = "500: Database root.notExistGroup1,root.notExistGroup2 does not exist"; sqe.printStackTrace(); assertTrue(sqe.getMessage().contains(expectedMsg)); } diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBRestServiceIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBRestServiceIT.java index 7d6d8774385a2..d84a2493cc25e 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBRestServiceIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBRestServiceIT.java @@ -393,7 +393,7 @@ public void insertAndQuery() { queryShowChildPaths(httpClient); queryShowNodes(httpClient); showAllTTL(httpClient); - showStorageGroup(httpClient); + showDatabase(httpClient); showFunctions(httpClient); showTimeseries(httpClient); @@ -413,7 +413,7 @@ public void insertAndQuery() { queryShowChildPathsV2(httpClient); queryShowNodesV2(httpClient); showAllTTLV2(httpClient); - showStorageGroupV2(httpClient); + showDatabaseV2(httpClient); showFunctionsV2(httpClient); showTimeseriesV2(httpClient); @@ -1094,7 +1094,7 @@ public void showAllTTL(CloseableHttpClient httpClient) { Assert.assertEquals(values2, valuesResult.get(1)); } - public void showStorageGroup(CloseableHttpClient httpClient) { + public void showDatabase(CloseableHttpClient httpClient) { String sql = "{\"sql\":\"SHOW DATABASES root.sg25\"}"; Map map = queryMetaData(httpClient, sql); List columnNamesResult = (List) map.get("columnNames"); @@ -1757,7 +1757,7 @@ public void showAllTTLV2(CloseableHttpClient httpClient) { Assert.assertEquals(values2, valuesResult.get(1)); } - public void showStorageGroupV2(CloseableHttpClient httpClient) { + public void showDatabaseV2(CloseableHttpClient httpClient) { String sql = "{\"sql\":\"SHOW DATABASES root.sg25\"}"; Map map = queryMetaDataV2(httpClient, sql); List columnNamesResult = (List) map.get("column_names"); 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 aceec6d2075bf..96ee0bc30c1c3 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 @@ -188,7 +188,7 @@ public void testSetDefaultSGLevel() throws SQLException { // can start with an illegal value EnvFactory.getEnv().cleanClusterEnvironment(); - EnvFactory.getEnv().getConfig().getCommonConfig().setDefaultStorageGroupLevel(-1); + EnvFactory.getEnv().getConfig().getCommonConfig().setDefaultDatabaseLevel(-1); EnvFactory.getEnv().initClusterEnvironment(); try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java index 19ca146e2e676..c56c0077f29e2 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSimpleQueryIT.java @@ -1040,7 +1040,7 @@ public void testInvalidMaxPointNumber() { } @Test - public void testStorageGroupWithHyphenInName() { + public void testDatabaseWithHyphenInName() { try (final Connection connection = EnvFactory.getEnv().getConnection(); final Statement statement = connection.createStatement()) { statement.setFetchSize(5); diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSyntaxConventionIdentifierIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSyntaxConventionIdentifierIT.java index 8be338625547a..c21e4e347c4db 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSyntaxConventionIdentifierIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBSyntaxConventionIdentifierIT.java @@ -388,7 +388,7 @@ public void testNodeNameIllegal() { } @Test - public void testCreateIllegalStorageGroup() { + public void testCreateIllegalDatabase() { try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/path/IoTDBQuotedPathIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/path/IoTDBQuotedPathIT.java index 4cba7008a229b..ed98ff2eac824 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/path/IoTDBQuotedPathIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/path/IoTDBQuotedPathIT.java @@ -101,13 +101,13 @@ public void test() { } @Test - public void testIllegalStorageGroup() { + public void testIllegalDatabase() { try (final Connection connection = EnvFactory.getEnv().getConnection(); final Statement statement = connection.createStatement()) { statement.execute("CREATE DATABASE root.`\"ln`"); } catch (final SQLException e) { Assert.assertTrue( - e.getMessage().contains("Error StorageGroup name") + e.getMessage().contains("Error Database name") || e.getMessage() .contains( "The database name can only contain english or chinese characters, numbers, backticks and underscores.")); diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAutoCreateSchemaIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAutoCreateSchemaIT.java index 5453fdc9ab1b5..620411b8c78cc 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAutoCreateSchemaIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAutoCreateSchemaIT.java @@ -164,12 +164,12 @@ private void executeSQL(String[] sqls) throws ClassNotFoundException { */ @Test public void testInsertAutoCreate2() throws Exception { - String storageGroup = "root.sg2.a.b.c"; + String database = "root.sg2.a.b.c"; String timeSeriesPrefix = "root.sg2.a.b"; try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { - statement.execute(String.format("CREATE DATABASE %s", storageGroup)); + statement.execute(String.format("CREATE DATABASE %s", database)); try { statement.execute( String.format("INSERT INTO %s(timestamp, c) values(123, \"aabb\")", timeSeriesPrefix)); @@ -177,18 +177,18 @@ public void testInsertAutoCreate2() throws Exception { } // ensure that current database in cache is right. - InsertAutoCreate2Tool(statement, storageGroup, timeSeriesPrefix); + InsertAutoCreate2Tool(statement, database, timeSeriesPrefix); } // todo restart test // EnvironmentUtils.stopDaemon(); // setUp(); // // // ensure that database in cache is right after recovering. - // InsertAutoCreate2Tool(storageGroup, timeSeriesPrefix); + // InsertAutoCreate2Tool(database, timeSeriesPrefix); } - private void InsertAutoCreate2Tool( - Statement statement, String storageGroup, String timeSeriesPrefix) throws SQLException { + private void InsertAutoCreate2Tool(Statement statement, String database, String timeSeriesPrefix) + throws SQLException { Set resultList = new HashSet<>(); try (ResultSet resultSet = statement.executeQuery("show timeseries")) { while (resultSet.next()) { @@ -204,7 +204,7 @@ private void InsertAutoCreate2Tool( resultList.add(resultSet.getString(ColumnHeaderConstant.DATABASE)); } } - Assert.assertTrue(resultList.contains(storageGroup)); + Assert.assertTrue(resultList.contains(database)); } /** diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java index eb02ca1d7ee9e..22572aac6eeb6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeactivateTemplateIT.java @@ -170,7 +170,7 @@ public void deactivateTemplateCrossSchemaRegionTest() throws Exception { } @Test - public void deactivateTemplateCrossStorageGroupTest() throws Exception { + public void deactivateTemplateCrossDatabaseTest() throws Exception { String insertSql = "insert into root.sg%d.d2(time, s1, s2) values(%d, %d, %d)"; try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeleteTimeSeriesIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeleteTimeSeriesIT.java index c278fd89f0ada..0866b37836e23 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeleteTimeSeriesIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBDeleteTimeSeriesIT.java @@ -304,7 +304,7 @@ public void deleteTimeSeriesCrossSchemaRegionTest() throws Exception { } @Test - public void deleteTimeSeriesCrossStorageGroupTest() throws Exception { + public void deleteTimeSeriesCrossDatabaseTest() throws Exception { String[] retArray1 = new String[] {"4,4,4,4"}; String insertSql = "insert into root.sg%d.d1(time, s1, s2) values(%d, %d, %d)"; diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java index cb2d1ff3d477b..1b6818d39f0a5 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java @@ -592,7 +592,7 @@ public void showCountDevices() throws SQLException { } @Test - public void showCountStorageGroup() throws SQLException { + public void showCountDatabase() throws SQLException { try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { String[] sqls = diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java index 8ed8d1ee29aa6..a403758cc364f 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBTagIT.java @@ -1073,7 +1073,7 @@ public void sameNameTest() { } @Test - public void deleteStorageGroupTest() { + public void deleteDatabaseTest() { List ret = Collections.singletonList( "root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY," diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java index d89c7a6b3e43b..b97dc4915a1d1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/IoTDBSimpleQueryTableIT.java @@ -606,7 +606,7 @@ public void testUseSameStatement() throws SQLException { } @Test - public void testStorageGroupWithHyphenInName() { + public void testDatabaseWithHyphenInName() { try (final Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); final Statement statement = connection.createStatement()) { diff --git a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java index e60eb5471c99d..38f6345bae5d1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java @@ -406,11 +406,11 @@ public void chineseCharacterTest() { session.close(); return; } - String storageGroup = "root.存储组1"; + String database = "root.存储组1"; String[] devices = new String[] {"设备1.指标1", "设备1.s2", "d2.s1", "d2.指标2"}; - session.setStorageGroup(storageGroup); + session.setStorageGroup(database); for (String path : devices) { - String fullPath = storageGroup + TsFileConstant.PATH_SEPARATOR + path; + String fullPath = database + TsFileConstant.PATH_SEPARATOR + path; session.createTimeseries( fullPath, TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY); } @@ -418,7 +418,7 @@ public void chineseCharacterTest() { for (String path : devices) { for (int i = 0; i < 10; i++) { String[] ss = path.split("\\."); - StringBuilder deviceId = new StringBuilder(storageGroup); + StringBuilder deviceId = new StringBuilder(database); for (int j = 0; j < ss.length - 1; j++) { deviceId.append(TsFileConstant.PATH_SEPARATOR).append(ss[j]); } @@ -441,7 +441,7 @@ public void chineseCharacterTest() { count++; } assertEquals(10, count); - session.deleteStorageGroup(storageGroup); + session.deleteDatabase(database); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -757,8 +757,8 @@ public void createTimeSeriesWithDoubleTicksTest() { session.close(); return; } - String storageGroup = "root.sg"; - session.setStorageGroup(storageGroup); + String database = "root.sg"; + session.setStorageGroup(database); session.createTimeseries( "root.sg.`my.device.with.colon:`.s", @@ -769,7 +769,7 @@ public void createTimeSeriesWithDoubleTicksTest() { final SessionDataSet dataSet = session.executeQueryStatement("SHOW TIMESERIES"); assertTrue(dataSet.hasNext()); - session.deleteStorageGroup(storageGroup); + session.deleteDatabase(database); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -785,8 +785,8 @@ public void createWrongTimeSeriesTest() { session.close(); return; } - String storageGroup = "root.sg"; - session.setStorageGroup(storageGroup); + String database = "root.sg"; + session.setStorageGroup(database); try { session.createTimeseries( @@ -804,7 +804,7 @@ public void createWrongTimeSeriesTest() { final SessionDataSet dataSet = session.executeQueryStatement("SHOW TIMESERIES root.sg.**"); assertFalse(dataSet.hasNext()); - session.deleteStorageGroup(storageGroup); + session.deleteDatabase(database); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java index ac0c7501d262a..ac65c6be784e6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java @@ -69,8 +69,8 @@ public void tearDown() throws Exception { @Test public void createTimeSeriesTest() { try (ISession session = EnvFactory.getEnv().getSessionConnection()) { - String storageGroup = "root.sg"; - session.setStorageGroup(storageGroup); + String database = "root.sg"; + session.setStorageGroup(database); try { session.createTimeseries( @@ -91,7 +91,7 @@ public void createTimeSeriesTest() { final SessionDataSet dataSet = session.executeQueryStatement("SHOW TIMESERIES root.sg.**"); assertFalse(dataSet.hasNext()); - session.deleteStorageGroup(storageGroup); + session.deleteDatabase(database); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/PreDeleteDatabasePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/PreDeleteDatabasePlan.java index bc9313f92f17b..743f305398c0a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/PreDeleteDatabasePlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/PreDeleteDatabasePlan.java @@ -29,25 +29,25 @@ import java.util.Objects; public class PreDeleteDatabasePlan extends ConfigPhysicalPlan { - private String storageGroup; + private String database; private PreDeleteType preDeleteType; public PreDeleteDatabasePlan() { super(ConfigPhysicalPlanType.PreDeleteDatabase); } - public PreDeleteDatabasePlan(String storageGroup, PreDeleteType preDeleteType) { + public PreDeleteDatabasePlan(String database, PreDeleteType preDeleteType) { this(); - this.storageGroup = storageGroup; + this.database = database; this.preDeleteType = preDeleteType; } - public String getStorageGroup() { - return storageGroup; + public String getDatabase() { + return database; } - public void setStorageGroup(String storageGroup) { - this.storageGroup = storageGroup; + public void setDatabase(String database) { + this.database = database; } public PreDeleteType getPreDeleteType() { @@ -57,13 +57,13 @@ public PreDeleteType getPreDeleteType() { @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); - BasicStructureSerDeUtil.write(storageGroup, stream); + BasicStructureSerDeUtil.write(database, stream); stream.write(preDeleteType.getType()); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { - this.storageGroup = BasicStructureSerDeUtil.readString(buffer); + this.database = BasicStructureSerDeUtil.readString(buffer); this.preDeleteType = buffer.get() == (byte) 1 ? PreDeleteType.ROLLBACK : PreDeleteType.EXECUTE; } @@ -79,12 +79,12 @@ public boolean equals(Object o) { return false; } PreDeleteDatabasePlan that = (PreDeleteDatabasePlan) o; - return storageGroup.equals(that.storageGroup) && preDeleteType == that.preDeleteType; + return database.equals(that.database) && preDeleteType == that.preDeleteType; } @Override public int hashCode() { - return Objects.hash(super.hashCode(), storageGroup, preDeleteType); + return Objects.hash(super.hashCode(), database, preDeleteType); } public enum PreDeleteType { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index 65883f3065d3f..502937713c2e5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -3062,23 +3062,17 @@ public TDescTable4InformationSchemaResp describeTable4InformationSchema() { @Override public TFetchTableResp fetchTables(final Map> fetchTableMap) { final TSStatus status = confirmLeader(); - return status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() - ? clusterSchemaManager.fetchTables( - fetchTableMap.entrySet().stream() - .filter( - entry -> { - entry - .getValue() - .removeIf( - table -> - procedureManager - .checkDuplicateTableTask( - entry.getKey(), null, table, null, null, null) - .getRight()); - return true; - }) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))) - : new TFetchTableResp(status); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return new TFetchTableResp(status); + } + fetchTableMap.forEach( + (key, value) -> + value.removeIf( + table -> + procedureManager + .checkDuplicateTableTask(key, null, table, null, null, null) + .getRight())); + return clusterSchemaManager.fetchTables(fetchTableMap); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java index 3415c2d2d6e9a..e15c33e04b7ec 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java @@ -401,18 +401,18 @@ public interface IManager { TSStatus setTimePartitionInterval(SetTimePartitionIntervalPlan configPhysicalPlan); /** - * Count StorageGroups. + * Count Databases. * - * @return The number of matched StorageGroups + * @return The number of matched Databases */ DataSet countMatchedDatabases(CountDatabasePlan countDatabasePlan); /** - * Get StorageGroupSchemas. + * Get DatabaseSchemas. * - * @return StorageGroupSchemaDataSet + * @return DatabaseSchemaDataSet */ - DataSet getMatchedDatabaseSchemas(GetDatabasePlan getOrCountStorageGroupPlan); + DataSet getMatchedDatabaseSchemas(GetDatabasePlan getOrCountDatabasePlan); /** * Set Database. @@ -429,7 +429,7 @@ public interface IManager { TSStatus alterDatabase(DatabaseSchemaPlan databaseSchemaPlan); /** - * Delete StorageGroups. + * Delete Databases. * * @param tDeleteReq TDeleteDatabaseReq * @return status @@ -660,10 +660,10 @@ TDataPartitionTableResp getOrCreateDataPartition( TShowConfigNodes4InformationSchemaResp showConfigNodes4InformationSchema(); /** - * Show StorageGroup. + * Show Database. * * @param req TShowDatabaseReq - * @return TShowStorageGroupResp + * @return TShowDatabaseResp */ TShowDatabaseResp showDatabase(TGetDatabaseReq req); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index cb4703081348a..2e4227af3fc8b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -2112,6 +2112,33 @@ public TDeleteTableDeviceResp deleteDevices( } } + public Map> getAllExecutingTables() { + final Map> result = new HashMap<>(); + for (final Procedure procedure : executor.getProcedures().values()) { + if (procedure.isFinished()) { + continue; + } + // CreateTableOrViewProcedure is covered by the default process, thus we can ignore it here + // Note that if a table is creating there will not be a working table, and the DN will either + // be updated by commit or fetch the CN tables + // And it won't be committed by other procedures because: + // if the preUpdate of other procedure has failed there will not be any commit here + // if it succeeded then it will go to the normal process and will not leave any problems + if (procedure instanceof AbstractAlterOrDropTableProcedure) { + result + .computeIfAbsent( + ((AbstractAlterOrDropTableProcedure) procedure).getDatabase(), + k -> new ArrayList<>()) + .add(((AbstractAlterOrDropTableProcedure) procedure).getTableName()); + } + if (procedure instanceof DeleteDatabaseProcedure + && ((DeleteDatabaseProcedure) procedure).getDeleteDatabaseSchema().isIsTableModel()) { + result.put(((DeleteDatabaseProcedure) procedure).getDatabase(), null); + } + } + return result; + } + public TSStatus executeWithoutDuplicate( final String database, final TsTable table, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java index 54dd582551db7..993bfc0e40066 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java @@ -103,7 +103,7 @@ protected void setHeartbeatService(IManager configManager, LoadCache loadCache) * @param consensusGroupType TConsensusGroupType of RegionGroup to be allocated * @return CreateRegionGroupsPlan * @throws NotEnoughDataNodeException If there are not enough DataNodes - * @throws DatabaseNotExistsException If some specific StorageGroups don't exist + * @throws DatabaseNotExistsException If some specific Databases don't exist */ public CreateRegionGroupsPlan allocateRegionGroups( final Map allotmentMap, final TConsensusGroupType consensusGroupType) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RegionBalancer.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RegionBalancer.java index 3528cbe136075..10864e9fba9c2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RegionBalancer.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RegionBalancer.java @@ -74,7 +74,7 @@ public RegionBalancer(IManager configManager) { * @param consensusGroupType {@link TConsensusGroupType} of the new RegionGroups * @return CreateRegionGroupsPlan * @throws NotEnoughDataNodeException When the number of DataNodes is not enough for allocation - * @throws DatabaseNotExistsException When some StorageGroups don't exist + * @throws DatabaseNotExistsException When some Databases don't exist */ public CreateRegionGroupsPlan genRegionGroupsAllocationPlan( final Map allotmentMap, final TConsensusGroupType consensusGroupType) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java index bfe6e830cba91..9a04d9e9e59b7 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.schema.SchemaConstant; +import org.apache.iotdb.commons.schema.table.NonCommittableTsTable; import org.apache.iotdb.commons.schema.table.TableNodeStatus; import org.apache.iotdb.commons.schema.table.TreeViewSchema; import org.apache.iotdb.commons.schema.table.TsTable; @@ -1253,8 +1254,31 @@ public TFetchTableResp fetchTables(final Map> fetchTableMap) } public byte[] getAllTableInfoForDataNodeActivation() { + // To guarantee the safety of fetched tables + // If DataNode discovered that the table is being altered, it will fetch it from configNode, and + // if it's still in execution, it can use the table temporarily + // However, if the database is deleting then it must fetch it from configNode, or else the table + // is considered to be non exist + final Map> alteringTables = + configManager.getProcedureManager().getAllExecutingTables(); + final Map> usingTableMap = clusterSchemaInfo.getAllUsingTables(); + final Map> preCreateTableMap = clusterSchemaInfo.getAllPreCreateTables(); + alteringTables.forEach( + (k, v) -> { + final List preCreateList = + preCreateTableMap.computeIfAbsent(k, database -> new ArrayList<>()); + if (Objects.isNull(v)) { + usingTableMap + .remove(k) + .forEach( + table -> preCreateList.add(new NonCommittableTsTable(table.getTableName()))); + } else { + preCreateList.addAll( + v.stream().map(NonCommittableTsTable::new).collect(Collectors.toList())); + } + }); return TsTableInternalRPCUtil.serializeTableInitializationInfo( - clusterSchemaInfo.getAllUsingTables(), clusterSchemaInfo.getAllPreCreateTables()); + usingTableMap, preCreateTableMap); } // endregion diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java index 2067783307cab..b907527416bda 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java @@ -292,7 +292,7 @@ public List getRegionMaintainEntryList() { public TSStatus preDeleteDatabase(final PreDeleteDatabasePlan preDeleteDatabasePlan) { final PreDeleteDatabasePlan.PreDeleteType preDeleteType = preDeleteDatabasePlan.getPreDeleteType(); - final String database = preDeleteDatabasePlan.getStorageGroup(); + final String database = preDeleteDatabasePlan.getDatabase(); final DatabasePartitionTable databasePartitionTable = databasePartitionTables.get(database); if (databasePartitionTable == null) { return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java index 6d14b8be050d0..3ea3f2d06545d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java @@ -37,7 +37,7 @@ import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.impl.StateMachineProcedure; -import org.apache.iotdb.confignode.procedure.state.schema.DeleteStorageGroupState; +import org.apache.iotdb.confignode.procedure.state.schema.DeleteDatabaseState; import org.apache.iotdb.confignode.procedure.store.ProcedureType; import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; import org.apache.iotdb.consensus.exception.ConsensusException; @@ -57,7 +57,7 @@ import java.util.Objects; public class DeleteDatabaseProcedure - extends StateMachineProcedure { + extends StateMachineProcedure { private static final Logger LOG = LoggerFactory.getLogger(DeleteDatabaseProcedure.class); private static final int RETRY_THRESHOLD = 5; @@ -82,8 +82,7 @@ public void setDeleteDatabaseSchema(final TDatabaseSchema deleteDatabaseSchema) } @Override - protected Flow executeFromState( - final ConfigNodeProcedureEnv env, final DeleteStorageGroupState state) + protected Flow executeFromState(final ConfigNodeProcedureEnv env, final DeleteDatabaseState state) throws InterruptedException { if (deleteDatabaseSchema == null) { return Flow.NO_MORE_STATE; @@ -95,14 +94,14 @@ protected Flow executeFromState( "[DeleteDatabaseProcedure] Pre delete database: {}", deleteDatabaseSchema.getName()); env.preDeleteDatabase( PreDeleteDatabasePlan.PreDeleteType.EXECUTE, deleteDatabaseSchema.getName()); - setNextState(DeleteStorageGroupState.INVALIDATE_CACHE); + setNextState(DeleteDatabaseState.INVALIDATE_CACHE); break; case INVALIDATE_CACHE: LOG.info( "[DeleteDatabaseProcedure] Invalidate cache of database: {}", deleteDatabaseSchema.getName()); if (env.invalidateCache(deleteDatabaseSchema.getName())) { - setNextState(DeleteStorageGroupState.DELETE_DATABASE_SCHEMA); + setNextState(DeleteDatabaseState.DELETE_DATABASE_SCHEMA); } else { setFailure(new ProcedureException("[DeleteDatabaseProcedure] Invalidate cache failed")); } @@ -240,8 +239,7 @@ protected Flow executeFromState( } @Override - protected void rollbackState( - final ConfigNodeProcedureEnv env, final DeleteStorageGroupState state) + protected void rollbackState(final ConfigNodeProcedureEnv env, final DeleteDatabaseState state) throws IOException, InterruptedException { switch (state) { case PRE_DELETE_DATABASE: @@ -257,7 +255,7 @@ protected void rollbackState( } @Override - protected boolean isRollbackSupported(final DeleteStorageGroupState state) { + protected boolean isRollbackSupported(final DeleteDatabaseState state) { switch (state) { case PRE_DELETE_DATABASE: case INVALIDATE_CACHE: @@ -268,18 +266,18 @@ protected boolean isRollbackSupported(final DeleteStorageGroupState state) { } @Override - protected DeleteStorageGroupState getState(final int stateId) { - return DeleteStorageGroupState.values()[stateId]; + protected DeleteDatabaseState getState(final int stateId) { + return DeleteDatabaseState.values()[stateId]; } @Override - protected int getStateId(final DeleteStorageGroupState deleteStorageGroupState) { - return deleteStorageGroupState.ordinal(); + protected int getStateId(final DeleteDatabaseState deleteDatabaseState) { + return deleteDatabaseState.ordinal(); } @Override - protected DeleteStorageGroupState getInitialState() { - return DeleteStorageGroupState.PRE_DELETE_DATABASE; + protected DeleteDatabaseState getInitialState() { + return DeleteDatabaseState.PRE_DELETE_DATABASE; } public String getDatabase() { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteDatabaseState.java similarity index 96% rename from iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java rename to iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteDatabaseState.java index bc19a262e6453..cde6b2bdd8111 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteStorageGroupState.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/DeleteDatabaseState.java @@ -19,7 +19,7 @@ package org.apache.iotdb.confignode.procedure.state.schema; -public enum DeleteStorageGroupState { +public enum DeleteDatabaseState { PRE_DELETE_DATABASE, INVALIDATE_CACHE, DELETE_DATABASE_SCHEMA diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 3f050b1184f47..90d7f20fb251c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -655,12 +655,12 @@ public TSStatus constructSchemaBlackList(final TConstructSchemaBlackListReq req) executeSchemaBlackListTask( req.getSchemaRegionIdList(), consensusGroupId -> { - final String storageGroup = + final String database = schemaEngine .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId())) .getDatabaseFullPath(); final PathPatternTree filteredPatternTree = - filterPathPatternTree(patternTree, storageGroup); + filterPathPatternTree(patternTree, database); if (filteredPatternTree.isEmpty()) { return new TSStatus(TSStatusCode.ONLY_LOGICAL_VIEW.getStatusCode()); } @@ -688,11 +688,11 @@ public TSStatus rollbackSchemaBlackList(TRollbackSchemaBlackListReq req) { return executeInternalSchemaTask( req.getSchemaRegionIdList(), consensusGroupId -> { - String storageGroup = + String database = schemaEngine .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId())) .getDatabaseFullPath(); - PathPatternTree filteredPatternTree = filterPathPatternTree(patternTree, storageGroup); + PathPatternTree filteredPatternTree = filterPathPatternTree(patternTree, database); if (filteredPatternTree.isEmpty()) { return RpcUtils.SUCCESS_STATUS; } @@ -889,12 +889,12 @@ private Map> filterTemplateSetInfo( Map> templateSetInfo, TConsensusGroupId consensusGroupId) { Map> result = new HashMap<>(); - PartialPath storageGroupPath = getStorageGroupPath(consensusGroupId); - if (null != storageGroupPath) { - PartialPath storageGroupPattern = storageGroupPath.concatNode(MULTI_LEVEL_PATH_WILDCARD); + PartialPath databasePath = getDatabasePath(consensusGroupId); + if (null != databasePath) { + PartialPath databasePattern = databasePath.concatNode(MULTI_LEVEL_PATH_WILDCARD); templateSetInfo.forEach( (k, v) -> { - if (storageGroupPattern.overlapWith(k) || storageGroupPath.overlapWith(k)) { + if (databasePattern.overlapWith(k) || databasePath.overlapWith(k)) { result.put(k, v); } }); @@ -902,10 +902,10 @@ private Map> filterTemplateSetInfo( return result; } - private PartialPath getStorageGroupPath(TConsensusGroupId consensusGroupId) { - PartialPath storageGroupPath = null; + private PartialPath getDatabasePath(TConsensusGroupId consensusGroupId) { + PartialPath databasePath = null; try { - storageGroupPath = + databasePath = new PartialPath( schemaEngine .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId())) @@ -913,7 +913,7 @@ private PartialPath getStorageGroupPath(TConsensusGroupId consensusGroupId) { } catch (IllegalPathException ignored) { // Won't reach here } - return storageGroupPath; + return databasePath; } @Override @@ -1099,12 +1099,12 @@ public TSStatus constructViewSchemaBlackList(final TConstructViewSchemaBlackList executeInternalSchemaTask( req.getSchemaRegionIdList(), consensusGroupId -> { - final String storageGroup = + final String database = schemaEngine .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId())) .getDatabaseFullPath(); final PathPatternTree filteredPatternTree = - filterPathPatternTree(patternTree, storageGroup); + filterPathPatternTree(patternTree, database); if (filteredPatternTree.isEmpty()) { return RpcUtils.SUCCESS_STATUS; } @@ -1132,12 +1132,11 @@ public TSStatus rollbackViewSchemaBlackList(final TRollbackViewSchemaBlackListRe return executeInternalSchemaTask( req.getSchemaRegionIdList(), consensusGroupId -> { - final String storageGroup = + final String database = schemaEngine .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId())) .getDatabaseFullPath(); - final PathPatternTree filteredPatternTree = - filterPathPatternTree(patternTree, storageGroup); + final PathPatternTree filteredPatternTree = filterPathPatternTree(patternTree, database); if (filteredPatternTree.isEmpty()) { return RpcUtils.SUCCESS_STATUS; } @@ -1157,11 +1156,11 @@ public TSStatus deleteViewSchema(TDeleteViewSchemaReq req) { return executeInternalSchemaTask( req.getSchemaRegionIdList(), consensusGroupId -> { - String storageGroup = + String database = schemaEngine .getSchemaRegion(new SchemaRegionId(consensusGroupId.getId())) .getDatabaseFullPath(); - PathPatternTree filteredPatternTree = filterPathPatternTree(patternTree, storageGroup); + PathPatternTree filteredPatternTree = filterPathPatternTree(patternTree, database); if (filteredPatternTree.isEmpty()) { return RpcUtils.SUCCESS_STATUS; } @@ -2070,12 +2069,11 @@ public TSStatus testConnectionEmptyRPC() throws TException { return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } - private PathPatternTree filterPathPatternTree(PathPatternTree patternTree, String storageGroup) { + private PathPatternTree filterPathPatternTree(PathPatternTree patternTree, String database) { PathPatternTree filteredPatternTree = new PathPatternTree(); try { - PartialPath storageGroupPattern = - new PartialPath(storageGroup).concatNode(MULTI_LEVEL_PATH_WILDCARD); - for (PartialPath pathPattern : patternTree.getOverlappedPathPatterns(storageGroupPattern)) { + PartialPath databasePattern = new PartialPath(database).concatNode(MULTI_LEVEL_PATH_WILDCARD); + for (PartialPath pathPattern : patternTree.getOverlappedPathPatterns(databasePattern)) { filteredPatternTree.appendPathPattern(pathPattern); } filteredPatternTree.constructTree(); @@ -2733,8 +2731,8 @@ public TSStatus notifyRegionMigration(TNotifyRegionMigrationReq req) throws TExc return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } - private TSStatus createNewRegion(ConsensusGroupId regionId, String storageGroup) { - return regionManager.createNewRegion(regionId, storageGroup); + private TSStatus createNewRegion(ConsensusGroupId regionId, String database) { + return regionManager.createNewRegion(regionId, database); } @Override 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 30e8426c707e3..34a289b76c9da 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 @@ -3295,7 +3295,7 @@ public Analysis visitShowCluster( } @Override - public Analysis visitCountStorageGroup( + public Analysis visitCountDatabase( CountDatabaseStatement countDatabaseStatement, MPPQueryContext context) { Analysis analysis = new Analysis(); analysis.setRealStatement(countDatabaseStatement); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java index 1fdf636c33d7c..40724d4df1545 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TreeConfigTaskVisitor.java @@ -263,8 +263,7 @@ public IConfigTask visitShowDatabase(ShowDatabaseStatement statement, MPPQueryCo } @Override - public IConfigTask visitCountStorageGroup( - CountDatabaseStatement statement, MPPQueryContext context) { + public IConfigTask visitCountDatabase(CountDatabaseStatement statement, MPPQueryContext context) { return new CountDatabaseTask(statement); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java index c4e8664cf6d6f..64d606ef72ee4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java @@ -943,7 +943,7 @@ public Statement visitCountNodes(CountNodesContext ctx) { return new CountNodesStatement(path, level); } - // Count StorageGroup ======================================================================== + // Count Database ======================================================================== @Override public Statement visitCountDatabases(CountDatabasesContext ctx) { PartialPath path; @@ -3574,18 +3574,18 @@ private String parseAttributeValue(IoTDBSqlParser.AttributeValueContext ctx) { @Override public Statement visitFlush(final IoTDBSqlParser.FlushContext ctx) { final FlushStatement flushStatement = new FlushStatement(StatementType.FLUSH); - List storageGroups = null; + List databases = null; if (ctx.boolean_literal() != null) { flushStatement.setSeq(Boolean.parseBoolean(ctx.boolean_literal().getText())); } flushStatement.setOnCluster(ctx.LOCAL() == null); if (ctx.prefixPath(0) != null) { - storageGroups = new ArrayList<>(); + databases = new ArrayList<>(); for (final IoTDBSqlParser.PrefixPathContext prefixPathContext : ctx.prefixPath()) { - storageGroups.add(parsePrefixPath(prefixPathContext).getFullPath()); + databases.add(parsePrefixPath(prefixPathContext).getFullPath()); } } - flushStatement.setDatabases(storageGroups); + flushStatement.setDatabases(databases); return flushStatement; } @@ -3741,14 +3741,14 @@ public Statement visitShowRegions(IoTDBSqlParser.ShowRegionsContext ctx) { } if (ctx.OF() != null) { - List storageGroups = null; + List databases = null; if (ctx.prefixPath(0) != null) { - storageGroups = new ArrayList<>(); + databases = new ArrayList<>(); for (IoTDBSqlParser.PrefixPathContext prefixPathContext : ctx.prefixPath()) { - storageGroups.add(parsePrefixPath(prefixPathContext)); + databases.add(parsePrefixPath(prefixPathContext)); } } - showRegionStatement.setDatabases(storageGroups); + showRegionStatement.setDatabases(databases); } else { showRegionStatement.setDatabases(null); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableDeviceSchemaFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableDeviceSchemaFetcher.java index 9b0bea05bce9b..84c8c9c002e29 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableDeviceSchemaFetcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableDeviceSchemaFetcher.java @@ -459,11 +459,11 @@ private boolean tryGetTreeDeviceInCache( } public static IDeviceID convertTagValuesToDeviceID( - final String tableName, final String[] idValues) { + final String tableName, final String[] tagValues) { // Convert to IDeviceID - final String[] deviceIdNodes = new String[idValues.length + 1]; + final String[] deviceIdNodes = new String[tagValues.length + 1]; deviceIdNodes[0] = tableName; - System.arraycopy(idValues, 0, deviceIdNodes, 1, idValues.length); + System.arraycopy(tagValues, 0, deviceIdNodes, 1, tagValues.length); return IDeviceID.Factory.DEFAULT_FACTORY.create(deviceIdNodes); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java index 9f4c2cf9fb695..d24ec3ef0d3d5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java @@ -959,7 +959,7 @@ public TSStatus visitShowDatabase( } @Override - public TSStatus visitCountStorageGroup( + public TSStatus visitCountDatabase( CountDatabaseStatement countDatabaseStatement, TreeAccessCheckContext context) { context .setAuditLogOperation(AuditLogOperation.QUERY) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java index 7ce212c32970a..3f8bb158c09a2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java @@ -402,7 +402,7 @@ public R visitShowDevices(ShowDevicesStatement showDevicesStatement, C context) return visitStatement(showDevicesStatement, context); } - public R visitCountStorageGroup(CountDatabaseStatement countDatabaseStatement, C context) { + public R visitCountDatabase(CountDatabaseStatement countDatabaseStatement, C context) { return visitStatement(countDatabaseStatement, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java index 4ff683f44367f..2d186ce417307 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java @@ -32,7 +32,7 @@ public CountDatabaseStatement(PartialPath partialPath) { @Override public R accept(StatementVisitor visitor, C context) { - return visitor.visitCountStorageGroup(this, context); + return visitor.visitCountDatabase(this, context); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java index 53e8c2617bcb3..15d51928d6406 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java @@ -141,7 +141,7 @@ public class MTreeBelowSGMemoryImpl { private final MemMTreeStore store; @SuppressWarnings("java:S3077") - private volatile IMemMNode storageGroupMNode; + private volatile IMemMNode databaseMNode; private final IMemMNode rootNode; private final Function, Map> tagGetter; @@ -153,38 +153,38 @@ public class MTreeBelowSGMemoryImpl { // region MTree initialization, clear and serialization public MTreeBelowSGMemoryImpl( - final PartialPath storageGroupPath, + final PartialPath databasePath, final Function, Map> tagGetter, final Function, Map> attributeGetter, final MemSchemaRegionStatistics regionStatistics, final SchemaRegionMemMetric metric) { - store = new MemMTreeStore(storageGroupPath, regionStatistics, metric); + store = new MemMTreeStore(databasePath, regionStatistics, metric); this.regionStatistics = regionStatistics; - this.storageGroupMNode = store.getRoot(); - this.rootNode = store.generatePrefix(storageGroupPath); - levelOfSG = storageGroupPath.getNodeLength() - 1; + this.databaseMNode = store.getRoot(); + this.rootNode = store.generatePrefix(databasePath); + levelOfSG = databasePath.getNodeLength() - 1; this.tagGetter = tagGetter; this.attributeGetter = attributeGetter; } private MTreeBelowSGMemoryImpl( - final PartialPath storageGroupPath, + final PartialPath databasePath, final MemMTreeStore store, final Function, Map> tagGetter, final Function, Map> attributeGetter, final MemSchemaRegionStatistics regionStatistics) { this.store = store; this.regionStatistics = regionStatistics; - this.storageGroupMNode = store.getRoot(); - this.rootNode = store.generatePrefix(storageGroupPath); - levelOfSG = storageGroupPath.getNodeLength() - 1; + this.databaseMNode = store.getRoot(); + this.rootNode = store.generatePrefix(databasePath); + levelOfSG = databasePath.getNodeLength() - 1; this.tagGetter = tagGetter; this.attributeGetter = attributeGetter; } public void clear() { store.clear(); - storageGroupMNode = null; + databaseMNode = null; } public synchronized boolean createSnapshot(final File snapshotDir) { @@ -193,7 +193,7 @@ public synchronized boolean createSnapshot(final File snapshotDir) { public static MTreeBelowSGMemoryImpl loadFromSnapshot( final File snapshotDir, - final String storageGroupFullPath, + final String databaseFullPath, final MemSchemaRegionStatistics regionStatistics, final SchemaRegionMemMetric metric, final Consumer> measurementProcess, @@ -203,7 +203,7 @@ public static MTreeBelowSGMemoryImpl loadFromSnapshot( final Function, Map> attributeGetter) throws IOException, IllegalPathException { return new MTreeBelowSGMemoryImpl( - PartialPath.getQualifiedDatabasePartialPath(storageGroupFullPath), + PartialPath.getQualifiedDatabasePartialPath(databaseFullPath), MemMTreeStore.loadFromSnapshot( snapshotDir, measurementProcess, @@ -409,7 +409,7 @@ private IMemMNode checkAndAutoCreateInternalPath(final PartialPath devicePath) if (nodeNames.length == levelOfSG + 1) { return null; } - IMemMNode cur = storageGroupMNode; + IMemMNode cur = databaseMNode; IMemMNode child; String childName; // e.g, path = root.sg.d1.s1, create internal nodes and set cur to sg node, parent of d1 @@ -433,13 +433,12 @@ private IMemMNode checkAndAutoCreateDeviceNode( throws PathAlreadyExistException, ExceedQuotaException { if (deviceParent == null) { // device is sg - return storageGroupMNode; + return databaseMNode; } IMemMNode device = store.getChild(deviceParent, deviceName); if (device == null) { if (IoTDBDescriptor.getInstance().getConfig().isQuotaEnable()) { - if (!DataNodeSpaceQuotaManager.getInstance() - .checkDeviceLimit(storageGroupMNode.getName())) { + if (!DataNodeSpaceQuotaManager.getInstance().checkDeviceLimit(databaseMNode.getName())) { throw new ExceedQuotaException( "The number of devices has reached the upper limit", TSStatusCode.SPACE_QUOTA_EXCEEDED.getStatusCode()); @@ -501,8 +500,7 @@ public Map checkMeasurementExistence( devicePath.getFullPath() + "." + measurementList.get(i), aliasList.get(i))); } if (IoTDBDescriptor.getInstance().getConfig().isQuotaEnable()) { - if (!DataNodeSpaceQuotaManager.getInstance() - .checkTimeSeriesNum(storageGroupMNode.getName())) { + if (!DataNodeSpaceQuotaManager.getInstance().checkTimeSeriesNum(databaseMNode.getName())) { failingMeasurementMap.put( i, new ExceedQuotaException( @@ -747,7 +745,7 @@ public IMemMNode getDeviceNodeWithAutoCreating(final PartialPath deviceId) throws MetadataException { MetaFormatUtils.checkTimeseries(deviceId); final String[] nodeNames = deviceId.getNodes(); - IMemMNode cur = storageGroupMNode; + IMemMNode cur = databaseMNode; IMemMNode child; for (int i = levelOfSG + 1; i < nodeNames.length; i++) { child = cur.getChild(nodeNames[i]); @@ -894,7 +892,7 @@ protected Void collectEntity(final IDeviceMNode node) { */ public IMemMNode getNodeByPath(final PartialPath path) throws PathNotExistException { final String[] nodes = path.getNodes(); - IMemMNode cur = storageGroupMNode; + IMemMNode cur = databaseMNode; IMemMNode next; for (int i = levelOfSG + 1; i < nodes.length; i++) { next = cur.getChild(nodes[i]); @@ -930,7 +928,7 @@ public IMeasurementMNode getMeasurementMNode(final PartialPath path) public void activateTemplate(final PartialPath activatePath, final Template template) throws MetadataException { final String[] nodes = activatePath.getNodes(); - IMemMNode cur = storageGroupMNode; + IMemMNode cur = databaseMNode; for (int i = levelOfSG + 1; i < nodes.length; i++) { cur = cur.getChild(nodes[i]); } @@ -1037,7 +1035,7 @@ protected void updateEntity(final IDeviceMNode node) { public void activateTemplateWithoutCheck( final PartialPath activatePath, final int templateId, final boolean isAligned) { final String[] nodes = activatePath.getNodes(); - IMemMNode cur = storageGroupMNode; + IMemMNode cur = databaseMNode; for (int i = levelOfSG + 1; i < nodes.length; i++) { cur = cur.getChild(nodes[i]); } @@ -1334,18 +1332,16 @@ public void close() {} private IMemMNode getTableDeviceNode(final String table, final Object[] deviceId) throws PathNotExistException { - IMemMNode cur = storageGroupMNode; + IMemMNode cur = databaseMNode; IMemMNode next; next = cur.getChild(table); if (next == null) { throw new PathNotExistException( - storageGroupMNode.getFullPath() + PATH_SEPARATOR + table + Arrays.toString(deviceId), - true); + databaseMNode.getFullPath() + PATH_SEPARATOR + table + Arrays.toString(deviceId), true); } else if (next.isMeasurement()) { throw new PathNotExistException( - storageGroupMNode.getFullPath() + PATH_SEPARATOR + table + Arrays.toString(deviceId), - true); + databaseMNode.getFullPath() + PATH_SEPARATOR + table + Arrays.toString(deviceId), true); } cur = next; @@ -1353,14 +1349,13 @@ private IMemMNode getTableDeviceNode(final String table, final Object[] deviceId next = cur.getChild(deviceId[i] == null ? null : String.valueOf(deviceId[i])); if (next == null) { throw new PathNotExistException( - storageGroupMNode.getFullPath() + PATH_SEPARATOR + table + Arrays.toString(deviceId), - true); + databaseMNode.getFullPath() + PATH_SEPARATOR + table + Arrays.toString(deviceId), true); } else if (next.isMeasurement()) { if (i == deviceId.length - 1) { return next; } else { throw new PathNotExistException( - storageGroupMNode.getFullPath() + PATH_SEPARATOR + table + Arrays.toString(deviceId), + databaseMNode.getFullPath() + PATH_SEPARATOR + table + Arrays.toString(deviceId), true); } } @@ -1628,7 +1623,7 @@ protected Void collectMeasurement(final IMeasurementMNode node) { // region table device management public int getTableDeviceNotExistNum(final String tableName, final List deviceIdList) { - final IMemMNode tableNode = storageGroupMNode.getChild(tableName); + final IMemMNode tableNode = databaseMNode.getChild(tableName); int notExistNum = deviceIdList.size(); if (tableNode == null) { return notExistNum; @@ -1659,11 +1654,10 @@ public void createOrUpdateTableDevice( if (LOGGER.isDebugEnabled()) { LOGGER.debug("Start to create table device {}.{}", tableName, Arrays.toString(devicePath)); } - IMemMNode cur = storageGroupMNode.getChild(tableName); + IMemMNode cur = databaseMNode.getChild(tableName); if (cur == null) { cur = - store.addChild( - storageGroupMNode, tableName, nodeFactory.createInternalMNode(cur, tableName)); + store.addChild(databaseMNode, tableName, nodeFactory.createInternalMNode(cur, tableName)); } for (final String childName : devicePath) { @@ -1776,14 +1770,14 @@ protected void updateEntity(final IDeviceMNode node) { public boolean deleteTableDevice(final String tableName, final IntConsumer attributeDeleter) throws MetadataException { - if (!store.hasChild(storageGroupMNode, tableName)) { + if (!store.hasChild(databaseMNode, tableName)) { return false; } final AtomicInteger memoryReleased = new AtomicInteger(0); try (final MNodeCollector collector = new MNodeCollector( - storageGroupMNode, - new PartialPath(new String[] {storageGroupMNode.getName(), tableName}), + databaseMNode, + new PartialPath(new String[] {databaseMNode.getName(), tableName}), this.store, true, SchemaConstant.ALL_MATCH_SCOPE) { @@ -1805,7 +1799,7 @@ protected Void collectMNode(final IMemMNode node) { }) { collector.traverse(); } - storageGroupMNode.deleteChild(tableName); + databaseMNode.deleteChild(tableName); regionStatistics.resetTableDevice(tableName); store.releaseMemory(memoryReleased.get()); return true; @@ -1813,14 +1807,14 @@ protected Void collectMNode(final IMemMNode node) { public boolean dropTableAttribute(final String tableName, final IntConsumer attributeDropper) throws MetadataException { - if (!store.hasChild(storageGroupMNode, tableName)) { + if (!store.hasChild(databaseMNode, tableName)) { return false; } final AtomicInteger memoryReleased = new AtomicInteger(0); try (final EntityUpdater updater = new EntityUpdater( - storageGroupMNode, - new PartialPath(new String[] {storageGroupMNode.getName(), tableName}), + databaseMNode, + new PartialPath(new String[] {databaseMNode.getName(), tableName}), this.store, true, SchemaConstant.ALL_MATCH_SCOPE) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/DataNodeTableCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/DataNodeTableCache.java index fdbca98cc0041..2001550ab67f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/DataNodeTableCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/DataNodeTableCache.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.schemaengine.table; +import org.apache.iotdb.commons.schema.table.NonCommittableTsTable; import org.apache.iotdb.commons.schema.table.TsTable; import org.apache.iotdb.commons.schema.table.TsTableInternalRPCUtil; import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema; @@ -177,6 +178,14 @@ public void rollbackUpdateTable(String database, final String tableName, final S if (Objects.nonNull(oldName)) { // Equals to commit update final TsTable oldTable = preUpdateTableMap.get(database).get(oldName).getLeft(); + // Cannot be rolled back, consider: + // 1. Fetched a written CN table + // 2. CN rollback because of timeout + // 3. If we roll back here, the flag will be cleared, and it will always be the written + // one + if (oldTable instanceof NonCommittableTsTable) { + return; + } databaseTableMap .computeIfAbsent(database, k -> new ConcurrentHashMap<>()) .put(tableName, oldTable); @@ -207,6 +216,14 @@ public void commitUpdateTable( readWriteLock.writeLock().lock(); try { final TsTable newTable = preUpdateTableMap.get(database).get(tableName).getLeft(); + // Cannot be committed, consider: + // 1. Fetched a non-changed CN table + // 2. CN is changed + // 3. If we commit here, it will always be the non-changed one + // (And it is not committable because it's not real table) + if (newTable instanceof NonCommittableTsTable) { + return; + } final TsTable oldTable = databaseTableMap .computeIfAbsent(database, k -> new ConcurrentHashMap<>()) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/NonCommittableTsTable.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/NonCommittableTsTable.java new file mode 100644 index 0000000000000..5f22c86474f3c --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/NonCommittableTsTable.java @@ -0,0 +1,43 @@ +/* + * 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.commons.schema.table; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.IOException; +import java.io.OutputStream; + +/** + * This table is just for occupation, and notice the dataNode to fetch the newest version from + * configNode. Note that the table cannot be committed or rolled-back, yet it can still be + * pre-updated or invalidated, because the two can update the table to the newest and trustable + * version. + */ +public class NonCommittableTsTable extends TsTable { + public NonCommittableTsTable(final String tableName) { + super(tableName); + } + + @Override + public void serialize(final OutputStream stream) throws IOException { + ReadWriteIOUtils.write(tableName, stream); + ReadWriteIOUtils.write(-1, stream); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java index 00ced1990eaf9..3bc5ab642484c 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java @@ -65,7 +65,7 @@ public class TsTable { public static final String TTL_PROPERTY = "ttl"; public static final Set TABLE_ALLOWED_PROPERTIES = Collections.singleton(TTL_PROPERTY); - private String tableName; + protected String tableName; private final Map columnSchemaMap = new LinkedHashMap<>(); private final Map tagColumnIndexMap = new HashMap<>(); @@ -331,10 +331,13 @@ public void serialize(final OutputStream stream) throws IOException { ReadWriteIOUtils.write(props, stream); } - public static TsTable deserialize(InputStream inputStream) throws IOException { - String name = ReadWriteIOUtils.readString(inputStream); - TsTable table = new TsTable(name); - int columnNum = ReadWriteIOUtils.readInt(inputStream); + public static TsTable deserialize(final InputStream inputStream) throws IOException { + final String name = ReadWriteIOUtils.readString(inputStream); + final int columnNum = ReadWriteIOUtils.readInt(inputStream); + if (columnNum < 0) { + return new NonCommittableTsTable(name); + } + final TsTable table = new TsTable(name); for (int i = 0; i < columnNum; i++) { table.addColumnSchema(TsTableColumnSchemaUtil.deserialize(inputStream)); } @@ -342,10 +345,13 @@ public static TsTable deserialize(InputStream inputStream) throws IOException { return table; } - public static TsTable deserialize(ByteBuffer buffer) { - String name = ReadWriteIOUtils.readString(buffer); - TsTable table = new TsTable(name); - int columnNum = ReadWriteIOUtils.readInt(buffer); + public static TsTable deserialize(final ByteBuffer buffer) { + final String name = ReadWriteIOUtils.readString(buffer); + final int columnNum = ReadWriteIOUtils.readInt(buffer); + if (columnNum < 0) { + return new NonCommittableTsTable(name); + } + final TsTable table = new TsTable(name); for (int i = 0; i < columnNum; i++) { table.addColumnSchema(TsTableColumnSchemaUtil.deserialize(buffer)); } From ce054e88e2f473d21c40616076cec18621cdc90b Mon Sep 17 00:00:00 2001 From: Haonan Date: Mon, 10 Nov 2025 18:15:48 +0800 Subject: [PATCH 042/180] Optimize the configuration logic of dn_thrift_max_frame_size (#16724) * Add thrift max frame size calculate logic * Add thrift max frame size calculate logic * fix review (cherry picked from commit ffa9c562c4eef4ee7f695fb4ef3311c50634a181) --- .../TCompressedElasticFramedTransport.java | 8 +- .../iotdb/rpc/TElasticFramedTransport.java | 99 +++++++++++-------- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 10 +- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 4 - .../conf/iotdb-system.properties.template | 4 +- .../iotdb/commons/conf/IoTDBConstant.java | 1 - 6 files changed, 69 insertions(+), 57 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java index a3b4f38064a88..5b9c81ec58b91 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java @@ -44,13 +44,7 @@ protected TCompressedElasticFramedTransport( protected void readFrame() throws TTransportException { underlying.readAll(i32buf, 0, 4); int size = TFramedTransport.decodeFrameSize(i32buf); - - if (size < 0) { - close(); - throw new TTransportException( - TTransportException.CORRUPTED_DATA, "Read a negative frame size (" + size + ")!"); - } - + checkFrameSize(size); readBuffer.fill(underlying, size); RpcStat.readCompressedBytes.addAndGet(size); try { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java index 6008988a80998..cd5eea309939b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java @@ -169,54 +169,71 @@ public int read(byte[] buf, int off, int len) throws TTransportException { protected void readFrame() throws TTransportException { underlying.readAll(i32buf, 0, 4); int size = TFramedTransport.decodeFrameSize(i32buf); + checkFrameSize(size); + readBuffer.fill(underlying, size); + } - if (size < 0) { - close(); - throw new TTransportException( - TTransportException.CORRUPTED_DATA, "Read a negative frame size (" + size + ")!"); + protected void checkFrameSize(int size) throws TTransportException { + final int HTTP_GET_SIGNATURE = 0x47455420; // "GET " + final int HTTP_POST_SIGNATURE = 0x504F5354; // "POST" + final int TLS_MIN_VERSION = 0x160300; + final int TLS_MAX_VERSION = 0x160303; + final int TLS_LENGTH_HIGH_MAX = 0x02; + + FrameError error = null; + if (size == HTTP_GET_SIGNATURE || size == HTTP_POST_SIGNATURE) { + error = FrameError.HTTP_REQUEST; + } else { + int high24 = size >>> 8; + if (high24 >= TLS_MIN_VERSION + && high24 <= TLS_MAX_VERSION + && (i32buf[3] & 0xFF) <= TLS_LENGTH_HIGH_MAX) { + error = FrameError.TLS_REQUEST; + } else if (size < 0) { + error = FrameError.NEGATIVE_FRAME_SIZE; + } else if (size > thriftMaxFrameSize) { + error = FrameError.FRAME_SIZE_EXCEEDED; + } } - if (size > thriftMaxFrameSize) { - SocketAddress remoteAddress = null; - if (underlying instanceof TSocket) { - remoteAddress = ((TSocket) underlying).getSocket().getRemoteSocketAddress(); - } - close(); - if (size == 1195725856L || size == 1347375956L) { - // if someone sends HTTP GET/POST to this port, the size will be read as the following - throw new TTransportException( - TTransportException.CORRUPTED_DATA, - String.format( - "Singular frame size (%d) detected, you may be sending HTTP GET/POST" - + "%s requests to the Thrift-RPC port, " - + "please confirm that you are using the right port", - size, remoteAddress == null ? "" : " from " + remoteAddress)); - } else { - throw new TTransportException( - TTransportException.CORRUPTED_DATA, - "Frame size (" + size + ") larger than protect max size (" + thriftMaxFrameSize + ")!"); - } + if (error == null) { + return; } - int high24 = size >>> 8; - if (high24 >= 0x160300 && high24 <= 0x160303 && (i32buf[3] & 0xFF) <= 0x02) { - // The typical TLS ClientHello requests start with 0x160300 ~ 0x160303 - // The 4th byte is typically in [0x00, 0x01, 0x02]. - SocketAddress remoteAddress = null; - if (underlying instanceof TSocket) { - remoteAddress = ((TSocket) underlying).getSocket().getRemoteSocketAddress(); - } - close(); - throw new TTransportException( - TTransportException.CORRUPTED_DATA, - String.format( - "Singular frame size (%d) detected, you may be sending TLS ClientHello requests" - + "%s to the Non-SSL Thrift-RPC" - + " port, please confirm that you are using the right configuration", - size, remoteAddress == null ? "" : " from " + remoteAddress)); + SocketAddress remoteAddress = null; + if (underlying instanceof TSocket) { + remoteAddress = ((TSocket) underlying).getSocket().getRemoteSocketAddress(); } + String remoteInfo = (remoteAddress == null) ? "" : " from " + remoteAddress; + close(); - readBuffer.fill(underlying, size); + error.throwException(size, remoteInfo, thriftMaxFrameSize); + } + + private enum FrameError { + HTTP_REQUEST( + "Singular frame size (%d) detected, you may be sending HTTP GET/POST%s " + + "requests to the Thrift-RPC port, please confirm that you are using the right port"), + TLS_REQUEST( + "Singular frame size (%d) detected, you may be sending TLS ClientHello " + + "requests%s to the Non-SSL Thrift-RPC port, please confirm that you are using " + + "the right configuration"), + NEGATIVE_FRAME_SIZE("Read a negative frame size (%d)%s!"), + FRAME_SIZE_EXCEEDED("Frame size (%d) larger than protect max size (%d)%s!"); + + private final String messageFormat; + + FrameError(String messageFormat) { + this.messageFormat = messageFormat; + } + + void throwException(int size, String remoteInfo, int maxSize) throws TTransportException { + String message = + (this == FRAME_SIZE_EXCEEDED) + ? String.format(messageFormat, size, maxSize, remoteInfo) + : String.format(messageFormat, size, remoteInfo); + throw new TTransportException(TTransportException.CORRUPTED_DATA, message); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index bb62d241cc995..36a148430cb21 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -807,7 +807,7 @@ public class IoTDBConfig { private float udfCollectorMemoryBudgetInMB = (float) (1.0 / 3 * udfMemoryBudgetInMB); /** Unit: byte */ - private int thriftMaxFrameSize = 536870912; + private int thriftMaxFrameSize = getDefaultThriftMaxFrameSize(); private int thriftDefaultBufferSize = RpcUtils.THRIFT_DEFAULT_BUF_CAPACITY; @@ -2572,10 +2572,16 @@ public int getThriftMaxFrameSize() { } public void setThriftMaxFrameSize(int thriftMaxFrameSize) { - this.thriftMaxFrameSize = thriftMaxFrameSize; + this.thriftMaxFrameSize = + thriftMaxFrameSize <= 0 ? getDefaultThriftMaxFrameSize() : thriftMaxFrameSize; BaseRpcTransportFactory.setThriftMaxFrameSize(this.thriftMaxFrameSize); } + private static int getDefaultThriftMaxFrameSize() { + return Math.min( + 64 * 1024 * 1024, (int) Math.min(Runtime.getRuntime().maxMemory() / 64, Integer.MAX_VALUE)); + } + public int getThriftDefaultBufferSize() { return thriftDefaultBufferSize; } 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 7e903948c1944..6ee056d068565 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 @@ -803,10 +803,6 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException properties.getProperty( "dn_thrift_max_frame_size", String.valueOf(conf.getThriftMaxFrameSize())))); - if (conf.getThriftMaxFrameSize() < IoTDBConstant.LEFT_SIZE_IN_REQUEST * 2) { - conf.setThriftMaxFrameSize(IoTDBConstant.LEFT_SIZE_IN_REQUEST * 2); - } - conf.setThriftDefaultBufferSize( Integer.parseInt( properties.getProperty( diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index e4d6b56895ac0..5a32d6a123140 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -533,10 +533,10 @@ dn_rpc_selector_thread_count=1 # Datatype: int dn_rpc_max_concurrent_client_num=1000 -# thrift max frame size, 512MB by default +# thrift max frame size in bytes. When set to 0, use min(64MB, datanode heap memory / 64) # effectiveMode: restart # Datatype: int -dn_thrift_max_frame_size=536870912 +dn_thrift_max_frame_size=0 # thrift init buffer size # effectiveMode: restart diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java index 8b4352cb95abd..adf72842797ee 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java @@ -285,7 +285,6 @@ private IoTDBConstant() {} public static final String MQTT_MAX_MESSAGE_SIZE = "mqtt_max_message_size"; // thrift - public static final int LEFT_SIZE_IN_REQUEST = 4 * 1024 * 1024; public static final int DEFAULT_FETCH_SIZE = 5000; public static final int DEFAULT_CONNECTION_TIMEOUT_MS = 0; From 6f47cca0efbdb381b03135f752a42aaa87e8fb0d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 10 Nov 2025 19:28:00 +0800 Subject: [PATCH 043/180] Changed the SQL of AlterEncodingCompressor statement & banned the "root" timeSeries & handled the empty intersection path & refactored the IT (#16725) (cherry picked from commit b4d5b4235e12763466df989492f2d54ec0ed5b0d) --- .../IoTDBAlterEncodingCompressorIT.java | 44 ++++++++++++++----- .../apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 | 2 +- .../org/apache/iotdb/db/qp/sql/SqlLexer.g4 | 4 ++ .../executor/ClusterConfigTaskExecutor.java | 5 +++ .../queryengine/plan/parser/ASTVisitor.java | 3 ++ 5 files changed, 46 insertions(+), 12 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAlterEncodingCompressorIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAlterEncodingCompressorIT.java index 42c0275ab35b2..7c9495d16ebd9 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAlterEncodingCompressorIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBAlterEncodingCompressorIT.java @@ -72,7 +72,15 @@ public void alterEncodingAndCompressorTest() throws Exception { statement.execute("create timeSeries root.vehicle.wind.a int32"); try { - statement.execute("alter timeSeries root.nonExist.** set encoding=PLAIN"); + statement.execute("alter timeSeries root set STORAGE_PROPERTIES encoding=PLAIN"); + fail(); + } catch (final SQLException e) { + Assert.assertEquals("701: The timeSeries shall not be root.", e.getMessage()); + } + + try { + statement.execute( + "alter timeSeries root.nonExist.** set STORAGE_PROPERTIES encoding=PLAIN"); fail(); } catch (final SQLException e) { Assert.assertEquals( @@ -81,41 +89,47 @@ public void alterEncodingAndCompressorTest() throws Exception { } try { - statement.execute("alter timeSeries if exists root.nonExist.** set encoding=PLAIN"); + statement.execute( + "alter timeSeries if exists root.nonExist.** set STORAGE_PROPERTIES encoding=PLAIN"); } catch (final SQLException e) { fail( "Alter encoding & compressor shall not fail when timeSeries not exists if set if exists"); } try { - statement.execute("alter timeSeries if exists root.vehicle.** set encoding=aaa"); + statement.execute( + "alter timeSeries if exists root.vehicle.** set STORAGE_PROPERTIES encoding=aaa"); fail(); } catch (final SQLException e) { Assert.assertEquals("701: Unsupported encoding: AAA", e.getMessage()); } try { - statement.execute("alter timeSeries if exists root.vehicle.** set compressor=aaa"); + statement.execute( + "alter timeSeries if exists root.vehicle.** set STORAGE_PROPERTIES compressor=aaa"); fail(); } catch (final SQLException e) { Assert.assertEquals("701: Unsupported compressor: AAA", e.getMessage()); } try { - statement.execute("alter timeSeries if exists root.vehicle.** set falseKey=aaa"); + statement.execute( + "alter timeSeries if exists root.vehicle.** set STORAGE_PROPERTIES falseKey=aaa"); fail(); } catch (final SQLException e) { Assert.assertEquals("701: property falsekey is unsupported yet.", e.getMessage()); } try { - statement.execute("alter timeSeries if exists root.vehicle.** set encoding=DICTIONARY"); + statement.execute( + "alter timeSeries if exists root.vehicle.** set STORAGE_PROPERTIES encoding=DICTIONARY"); fail(); } catch (final SQLException e) { Assert.assertTrue(e.getMessage().contains("encoding DICTIONARY does not support INT32")); } - statement.execute("alter timeSeries root.** set encoding=Plain, compressor=LZMA2"); + statement.execute( + "alter timeSeries root.** set STORAGE_PROPERTIES encoding=Plain, compressor=LZMA2"); try (final ResultSet resultSet = statement.executeQuery("SHOW TIMESERIES")) { while (resultSet.next()) { @@ -133,17 +147,18 @@ public void alterEncodingAndCompressorTest() throws Exception { EnvFactory.getEnv().getConnection("IoTDBUser", "!@#$!dfdfzvd343"); final Statement statement = connection.createStatement()) { try { - statement.execute("alter timeSeries root.vechile.** set encoding=PLAIN, compressor=LZMA2"); + statement.execute( + "alter timeSeries root.vehicle.** set STORAGE_PROPERTIES encoding=PLAIN, compressor=LZMA2"); fail(); } catch (final SQLException e) { Assert.assertEquals( - "803: No permissions for this operation, please add privilege WRITE_SCHEMA on [root.vechile.**]", + "803: No permissions for this operation, please add privilege WRITE_SCHEMA on [root.vehicle.**]", e.getMessage()); } try { statement.execute( - "alter timeSeries root.vechile.wind.a, root.__audit.** set encoding=PLAIN, compressor=LZMA2"); + "alter timeSeries root.vehicle.wind.a, root.__audit.** set STORAGE_PROPERTIES encoding=PLAIN, compressor=LZMA2"); fail(); } catch (final SQLException e) { Assert.assertEquals( @@ -153,10 +168,17 @@ public void alterEncodingAndCompressorTest() throws Exception { try { statement.execute( - "alter timeSeries if permitted root.vehicle.**, root.__audit.** set encoding=GORILLA, compressor=GZIP"); + "alter timeSeries if permitted root.vehicle.**, root.__audit.** set STORAGE_PROPERTIES encoding=GORILLA, compressor=GZIP"); } catch (final SQLException e) { fail("Alter encoding & compressor shall not fail when no privileges if set if permitted"); } + + try { + statement.execute( + "alter timeSeries if permitted root.nonExist.** set STORAGE_PROPERTIES encoding=GORILLA, compressor=GZIP"); + } catch (final SQLException e) { + fail("Alter encoding & compressor shall not fail if the intersected paths are empty"); + } } try (final Connection connection = EnvFactory.getEnv().getConnection(); diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 index 7fd2bd490339e..8a1a0f5468f64 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 @@ -178,7 +178,7 @@ alterClause ; alterEncodingCompressor - : ALTER TIMESERIES (IF EXISTS)? (IF PERMITTED)? prefixPath (COMMA prefixPath)* SET attributePair (COMMA attributePair)* + : ALTER TIMESERIES (IF EXISTS)? (IF PERMITTED)? prefixPath (COMMA prefixPath)* SET STORAGE_PROPERTIES attributePair (COMMA attributePair)* ; aliasClause diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 index 7a0c196939862..f63fd21114944 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 @@ -846,6 +846,10 @@ STOP : S T O P ; +STORAGE_PROPERTIES + : S T O R A G E '_' P R O P E R T I E S + ; + SUBSCRIPTION : S U B S C R I P T I O N ; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index 71b6f58f83e06..517e201f3d8cf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -2803,6 +2803,11 @@ public SettableFuture alterEncodingCompressor( final String queryId, final AlterEncodingCompressorStatement alterEncodingCompressorStatement) { final SettableFuture future = SettableFuture.create(); + // Will only occur if no permission + if (alterEncodingCompressorStatement.getPatternTree().isEmpty()) { + future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); + return future; + } final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java index 64d606ef72ee4..123f7dcae523c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java @@ -695,6 +695,9 @@ public Statement visitAlterEncodingCompressor( } } + if (tree.isEmpty()) { + throw new SemanticException("The timeSeries shall not be root."); + } return new AlterEncodingCompressorStatement( tree, encoding, From 26b5ae3bb188be2ea8b18820aaf0b7e0de14c620 Mon Sep 17 00:00:00 2001 From: CritasWang Date: Tue, 11 Nov 2025 10:44:50 +0800 Subject: [PATCH 044/180] chore!: Bind real config address for Thrift and Raft services (#16717) * chore: Bind real internal address for Thrift and Raft services * keep ExternalRpcAddress use config value * Update iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/ExternalRPCService.java Co-authored-by: Haonan --------- Co-authored-by: Haonan (cherry picked from commit 8dbf2003cfde4c2f68fa5fd4e3110c5e7aa334b5) --- .../iotdb/consensus/ratis/RatisConsensus.java | 1 + .../service/AbstractThriftServiceThread.java | 15 +++++++++++---- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java index 045fa77917441..265192858bf97 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java @@ -157,6 +157,7 @@ public RatisConsensus(ConsensusConfig config, IStateMachine.Registry registry) { this.storageDir = new File(config.getStorageDir()); RaftServerConfigKeys.setStorageDir(properties, Collections.singletonList(storageDir)); + GrpcConfigKeys.Server.setHost(properties, config.getThisNodeEndPoint().getIp()); GrpcConfigKeys.Server.setPort(properties, config.getThisNodeEndPoint().getPort()); this.parameters = Utils.initRatisConfig(properties, config.getRatisConfig()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/AbstractThriftServiceThread.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/AbstractThriftServiceThread.java index 39b11ba677a34..1bcf15dbc233a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/AbstractThriftServiceThread.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/AbstractThriftServiceThread.java @@ -320,14 +320,21 @@ private THsHaServer.Args initAsyncedHshaPoolArgs( @SuppressWarnings("java:S2259") private TServerTransport openTransport(String bindAddress, int port) throws TTransportException { - // bind any address - return new TServerSocket(new InetSocketAddress(port)); + if (bindAddress == null) { + // bind any address + return new TServerSocket(new InetSocketAddress(port)); + } + return new TServerSocket(new InetSocketAddress(bindAddress, port)); } private TServerTransport openNonblockingTransport( String bindAddress, int port, int connectionTimeoutInMS) throws TTransportException { - // bind any address - return new TNonblockingServerSocket(new InetSocketAddress(port), connectionTimeoutInMS); + if (bindAddress == null) { + // bind any address + return new TNonblockingServerSocket(new InetSocketAddress(port), connectionTimeoutInMS); + } + return new TNonblockingServerSocket( + new InetSocketAddress(bindAddress, port), connectionTimeoutInMS); } public void setThreadStopLatch(CountDownLatch threadStopLatch) { From a3807d284262fc1b0102f5a08dbd2b2fac2470c4 Mon Sep 17 00:00:00 2001 From: Weihao Li <60659567+Wei-hao-Li@users.noreply.github.com> Date: Tue, 11 Nov 2025 10:45:02 +0800 Subject: [PATCH 045/180] Make ErrMsg more readable when the predicate has too many conjunctions because of too many devices #16715 (cherry picked from commit 93b2d1fe99917ec02dcca3e8d4ad53ef7bc573f8) --- .../queryengine/plan/analyze/PredicateUtils.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/PredicateUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/PredicateUtils.java index 381e8cfb5a0d6..9b45d2c51fa55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/PredicateUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/PredicateUtils.java @@ -338,6 +338,12 @@ public static Expression combineConjuncts(List conjuncts) { if (conjuncts.size() == 1) { return conjuncts.get(0); } + + if (conjuncts.size() > 1000) { + throw new SemanticException( + "There are too many conjuncts (more than 1000) in predicate after rewriting, this may be caused by too many devices in query, try to use ALIGN BY DEVICE"); + } + return constructRightDeepTreeWithAnd(conjuncts); } @@ -346,14 +352,8 @@ private static Expression constructRightDeepTreeWithAnd(List conjunc if (conjuncts.size() == 2) { return new LogicAndExpression(conjuncts.get(0), conjuncts.get(1)); } else { - try { - return new LogicAndExpression( - conjuncts.get(0), - constructRightDeepTreeWithAnd(conjuncts.subList(1, conjuncts.size()))); - } catch (StackOverflowError e) { - throw new SemanticException( - "There are too many conjuncts in predicate after rewriting, this may be caused by too many devices, try to use ALIGN BY DEVICE"); - } + return new LogicAndExpression( + conjuncts.get(0), constructRightDeepTreeWithAnd(conjuncts.subList(1, conjuncts.size()))); } } From d24cddadbd2f80fb4b1e7896515e5ddc508aeb95 Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Tue, 11 Nov 2025 10:50:20 +0800 Subject: [PATCH 046/180] Pipe: Fix unhandled timeout exceptions in Thrift Client (#16727) * Pipe: Fix unhandled timeout exceptions in Thrift Client * update (cherry picked from commit 93c55577e6f051307fa97656b17150aa038cb5de) --- .../java/org/apache/iotdb/commons/client/ThriftClient.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ThriftClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ThriftClient.java index 6c1b9d3fb0a79..0b78e51a99a8f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ThriftClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ThriftClient.java @@ -115,7 +115,8 @@ static boolean isConnectionBroken(Throwable cause) { && (hasExpectedMessage(cause, "Connection reset by peer") || hasExpectedMessage(cause, "Broken pipe"))) || (cause instanceof ConnectException && hasExpectedMessage(cause, "Connection refused") - || (cause instanceof ClosedChannelException)); + || (cause instanceof ClosedChannelException)) + || (cause instanceof java.util.concurrent.TimeoutException); } static boolean hasExpectedMessage(Throwable cause, String expectedMessage) { From 15f39d4854302b7cef2cb44a88c5e3f0479fb9fa Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 11 Nov 2025 14:18:08 +0800 Subject: [PATCH 047/180] Fixed the procedure has_more_state bug & Fixed the IT for encoding & compressor (#16729) * IT-fix * fix (cherry picked from commit 468a91a4d2e7c014b14f5712d400bcc68905e563) --- .../treemodel/manual/IoTDBPipeInclusionIT.java | 16 ++++++++++++---- .../schema/AlterEncodingCompressorProcedure.java | 9 ++++++--- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java index 4778f1f9b9bfd..61eaf14bb4e62 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java @@ -81,8 +81,7 @@ public void testPureSchemaInclusion() throws Exception { // banned "create timeSeries root.ln.wf01.wt01.status with datatype=BOOLEAN,encoding=PLAIN", "ALTER timeSeries root.ln.wf01.wt01.status ADD TAGS tag3=v3", - "ALTER timeSeries root.ln.wf01.wt01.status ADD ATTRIBUTES attr4=v4", - "ALTER timeSeries root.** set compressor=ZSTD"), + "ALTER timeSeries root.ln.wf01.wt01.status ADD ATTRIBUTES attr4=v4"), null); TestUtils.assertDataEventuallyOnEnv( @@ -90,14 +89,23 @@ public void testPureSchemaInclusion() throws Exception { "show timeseries root.ln.**", "Timeseries,Alias,Database,DataType,Encoding,Compression,Tags,Attributes,Deadband,DeadbandParameters,ViewType,", Collections.singleton( - "root.ln.wf01.wt01.status,null,root.ln,BOOLEAN,PLAIN,ZSTD,{\"tag3\":\"v3\"},{\"attr4\":\"v4\"},null,null,BASE,")); + "root.ln.wf01.wt01.status,null,root.ln,BOOLEAN,PLAIN,LZ4,{\"tag3\":\"v3\"},{\"attr4\":\"v4\"},null,null,BASE,")); TestUtils.executeNonQueries( senderEnv, Arrays.asList( - "insert into root.ln.wf01.wt01(time, status) values(now(), false)", "flush"), + "ALTER timeSeries root.** set STORAGE_PROPERTIES compressor=ZSTD", + "insert into root.ln.wf01.wt01(time, status) values(now(), false)", + "flush"), null); + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "show timeseries root.ln.**", + "Timeseries,Alias,Database,DataType,Encoding,Compression,Tags,Attributes,Deadband,DeadbandParameters,ViewType,", + Collections.singleton( + "root.ln.wf01.wt01.status,null,root.ln,BOOLEAN,PLAIN,ZSTD,{\"tag3\":\"v3\"},{\"attr4\":\"v4\"},null,null,BASE,")); + TestUtils.assertDataAlwaysOnEnv( receiverEnv, "select * from root.ln.**", "Time,", Collections.emptySet()); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java index 627547ce72236..f92cc5d2dd8bd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java @@ -126,7 +126,9 @@ protected Flow executeFromState( SerializeUtils.deserializeCompressorNullable(compressor), requestMessage); } - alterEncodingCompressorInSchemaRegion(env); + if (!alterEncodingCompressorInSchemaRegion(env)) { + return Flow.NO_MORE_STATE; + } break; case CLEAR_CACHE: LOGGER.info("Invalidate cache of timeSeries {}", requestMessage); @@ -146,7 +148,7 @@ protected Flow executeFromState( } } - private void alterEncodingCompressorInSchemaRegion(final ConfigNodeProcedureEnv env) { + private boolean alterEncodingCompressorInSchemaRegion(final ConfigNodeProcedureEnv env) { final Map relatedSchemaRegionGroup = env.getConfigManager().getRelatedSchemaRegionGroup(patternTree, mayAlterAudit); @@ -160,7 +162,7 @@ private void alterEncodingCompressorInSchemaRegion(final ConfigNodeProcedureEnv .collect(Collectors.toList()), false))); } - return; + return false; } final DataNodeTSStatusTaskExecutor alterEncodingCompressorTask = @@ -223,6 +225,7 @@ protected void onAllReplicasetFailure( }; alterEncodingCompressorTask.execute(); setNextState(AlterEncodingCompressorState.CLEAR_CACHE); + return true; } private void collectPayload4Pipe(final ConfigNodeProcedureEnv env) { From 8a3dbcb7db47a98561ddb8620933587be5a20ccc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 11 Nov 2025 19:26:28 +0800 Subject: [PATCH 048/180] Optimized the lock for encoding & compressor's invalidate cache (#16733) * fix * jr (cherry picked from commit a6e8493bfc0a910f54d4e9ad7abbb37cfd94bc56) --- .../impl/schema/AlterEncodingCompressorProcedure.java | 2 +- .../impl/schema/DeleteTimeSeriesProcedure.java | 7 ++++--- .../thrift/impl/DataNodeInternalRPCServiceImpl.java | 11 ++++++++--- .../thrift-datanode/src/main/thrift/datanode.thrift | 1 + 4 files changed, 14 insertions(+), 7 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java index f92cc5d2dd8bd..6f6d4b162d4d1 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterEncodingCompressorProcedure.java @@ -132,7 +132,7 @@ protected Flow executeFromState( break; case CLEAR_CACHE: LOGGER.info("Invalidate cache of timeSeries {}", requestMessage); - invalidateCache(env, patternTreeBytes, requestMessage, this::setFailure); + invalidateCache(env, patternTreeBytes, requestMessage, this::setFailure, false); collectPayload4Pipe(env); return Flow.NO_MORE_STATE; default: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java index 594b6b035bbbb..c49f80948fc1d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java @@ -122,7 +122,7 @@ protected Flow executeFromState( } case CLEAN_DATANODE_SCHEMA_CACHE: LOGGER.info("Invalidate cache of timeSeries {}", requestMessage); - invalidateCache(env, patternTreeBytes, requestMessage, this::setFailure); + invalidateCache(env, patternTreeBytes, requestMessage, this::setFailure, true); setNextState(DeleteTimeSeriesState.DELETE_DATA); break; case DELETE_DATA: @@ -202,13 +202,14 @@ public static void invalidateCache( final ConfigNodeProcedureEnv env, final ByteBuffer patternTreeBytes, final String requestMessage, - final Consumer setFailure) { + final Consumer setFailure, + final boolean needLock) { final Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); final DataNodeAsyncRequestContext clientHandler = new DataNodeAsyncRequestContext<>( CnToDnAsyncRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, - new TInvalidateMatchedSchemaCacheReq(patternTreeBytes), + new TInvalidateMatchedSchemaCacheReq(patternTreeBytes).setNeedLock(needLock), dataNodeLocationMap); CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); final Map statusMap = clientHandler.getResponseMap(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 90d7f20fb251c..dcc62fcd3c8f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -708,7 +708,10 @@ public TSStatus rollbackSchemaBlackList(TRollbackSchemaBlackListReq req) { @Override public TSStatus invalidateMatchedSchemaCache(final TInvalidateMatchedSchemaCacheReq req) { final TreeDeviceSchemaCacheManager cache = TreeDeviceSchemaCacheManager.getInstance(); - DataNodeSchemaLockManager.getInstance().takeWriteLock(SchemaLockType.VALIDATE_VS_DELETION_TREE); + if (req.needLock || !req.isSetNeedLock()) { + DataNodeSchemaLockManager.getInstance() + .takeWriteLock(SchemaLockType.VALIDATE_VS_DELETION_TREE); + } try { cache.takeWriteLock(); try { @@ -717,8 +720,10 @@ public TSStatus invalidateMatchedSchemaCache(final TInvalidateMatchedSchemaCache cache.releaseWriteLock(); } } finally { - DataNodeSchemaLockManager.getInstance() - .releaseWriteLock(SchemaLockType.VALIDATE_VS_DELETION_TREE); + if (req.needLock || !req.isSetNeedLock()) { + DataNodeSchemaLockManager.getInstance() + .releaseWriteLock(SchemaLockType.VALIDATE_VS_DELETION_TREE); + } } return RpcUtils.SUCCESS_STATUS; } diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index 9416633a2c41b..caaf44c16a770 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -440,6 +440,7 @@ struct TRollbackSchemaBlackListReq { struct TInvalidateMatchedSchemaCacheReq { 1: required binary pathPatternTree + 2: optional bool needLock } struct TFetchSchemaBlackListReq { From 25d5ba551355dd032b1d210b96148d4da529f8ab Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 12 Nov 2025 10:34:59 +0800 Subject: [PATCH 049/180] Pipe: support path exclusion under tree model (#16632) * backup * refact * add comments * minor improve * add ITs * improve IT * basic impl for meta exclusion * impl getIntersection for ExclusionIoTDBTreePattern & add tests for PipeStatementTreePatternParseVisitorTest * pending at testCommitSetSchemaTemplate * add IT * fixup * apply review * apply review * add TreePattern.checkAndLogPatternCoverage * fixup * throws PipeException If the inclusion pattern is fully covered by the exclusion pattern (cherry picked from commit d48347cdc8d7b636130c10871ba8140cb857f3c0) --- .../auto/basic/IoTDBTreePatternFormatIT.java | 616 ++++++++++-------- .../manual/IoTDBPipeInclusionIT.java | 72 ++ .../protocol/IoTDBConfigNodeReceiver.java | 13 +- .../pipe/source/IoTDBConfigRegionSource.java | 6 +- ...igPhysicalPlanTreePatternParseVisitor.java | 46 +- ...ysicalPlanTreePatternParseVisitorTest.java | 7 +- .../task/connection/PipeEventCollector.java | 4 +- .../TsFileInsertionEventParserProvider.java | 6 +- .../thrift/IoTDBDataNodeReceiver.java | 11 +- .../PipeStatementTreePatternParseVisitor.java | 22 +- .../dataregion/IoTDBDataRegionSource.java | 6 +- .../PipePlanTreePatternParseVisitor.java | 41 +- .../pipe/pattern/TreePatternCoverageTest.java | 133 ++++ ...eStatementTreePatternParseVisitorTest.java | 202 ++++-- .../PipePlanTreePatternParseVisitorTest.java | 7 +- .../config/constant/PipeSourceConstant.java | 5 + .../pattern/IoTDBTreePattern.java | 68 +- .../pattern/IoTDBTreePatternOperations.java | 52 ++ .../pattern/PrefixTreePattern.java | 70 +- .../pattern/SingleTreePattern.java | 59 -- .../datastructure/pattern/TreePattern.java | 369 +++++++++-- .../pattern/UnionIoTDBTreePattern.java | 115 ++-- .../pattern/UnionTreePattern.java | 17 +- .../WithExclusionIoTDBTreePattern.java | 213 ++++++ .../pattern/WithExclusionTreePattern.java | 119 ++++ .../pipe/source/IoTDBNonDataRegionSource.java | 14 +- 26 files changed, 1690 insertions(+), 603 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/TreePatternCoverageTest.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/IoTDBTreePatternOperations.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/SingleTreePattern.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/WithExclusionIoTDBTreePattern.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/WithExclusionTreePattern.java diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/auto/basic/IoTDBTreePatternFormatIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/auto/basic/IoTDBTreePatternFormatIT.java index 4e2871067931b..d8992d6c23229 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/auto/basic/IoTDBTreePatternFormatIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/auto/basic/IoTDBTreePatternFormatIT.java @@ -38,6 +38,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -208,323 +209,398 @@ public void testIoTDBPatternWithLegacySyntax() throws Exception { } } - @Test - public void testMultiplePrefixPatternHistoricalData() throws Exception { - final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); - - final String receiverIp = receiverDataNode.getIp(); - final int receiverPort = receiverDataNode.getPort(); - - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); + //////////////////////////// Multiple & Exclusion //////////////////////////// - extractorAttributes.put("extractor.pattern", "root.db.d1.s, root.db2.d1.s"); - extractorAttributes.put("extractor.inclusion", "data.insert"); + private void testPipeWithMultiplePatterns( + final Map extractorAttributes, + final List insertQueries, + final boolean isHistorical, + final String validationSelectQuery, + final String validationSelectHeader, + final Set expectedResultSet) + throws Exception { - connectorAttributes.put("connector", "iotdb-thrift-connector"); - connectorAttributes.put("connector.batch.enable", "false"); - connectorAttributes.put("connector.ip", receiverIp); - connectorAttributes.put("connector.port", Integer.toString(receiverPort)); - - TestUtils.executeNonQueries( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s, s1) values (1, 1, 1)", - "insert into root.db.d2(time, s) values (2, 2)", - "insert into root.db2.d1(time, s) values (3, 3)"), - null); - awaitUntilFlush(senderEnv); - - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); - - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - - Assert.assertEquals( - TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p1").getCode()); - - final Set expectedResSet = new HashSet<>(); - expectedResSet.add("1,null,1.0,1.0,"); - expectedResSet.add("3,3.0,null,null,"); - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select * from root.db2.**,root.db.**", - "Time,root.db2.d1.s,root.db.d1.s,root.db.d1.s1,", - expectedResSet); - } - } - - @Test - public void testMultipleIoTDBPatternHistoricalData() throws Exception { + // 1. Get receiver connection details final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); - final String receiverIp = receiverDataNode.getIp(); final int receiverPort = receiverDataNode.getPort(); try (final SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - final Map extractorAttributes = new HashMap<>(); + + // 2. Define standard processor and connector attributes final Map processorAttributes = new HashMap<>(); final Map connectorAttributes = new HashMap<>(); - - extractorAttributes.put("extractor.path", "root.db.**, root.db2.d1.*"); - extractorAttributes.put("extractor.inclusion", "data.insert"); - connectorAttributes.put("connector", "iotdb-thrift-connector"); connectorAttributes.put("connector.batch.enable", "false"); connectorAttributes.put("connector.ip", receiverIp); connectorAttributes.put("connector.port", Integer.toString(receiverPort)); - TestUtils.executeNonQueries( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s, s1) values (1, 1, 1)", - "insert into root.db.d2(time, s) values (2, 2)", - "insert into root.db2.d1(time, s, t) values (3, 3, 3)", - "insert into root.db3.d1(time, s) values (4, 4)"), - null); - awaitUntilFlush(senderEnv); + // 3. Handle historical data insertion (if applicable) + if (isHistorical) { + TestUtils.executeNonQueries(senderEnv, insertQueries, null); + awaitUntilFlush(senderEnv); + } - final TSStatus status = + // 4. Create the pipe + final TSStatus createStatus = client.createPipe( new TCreatePipeReq("p1", connectorAttributes) .setExtractorAttributes(extractorAttributes) .setProcessorAttributes(processorAttributes)); + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), createStatus.getCode()); - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + // 5. Start the pipe + final TSStatus startStatus = client.startPipe("p1"); + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), startStatus.getCode()); - Assert.assertEquals( - TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p1").getCode()); + // 6. Handle realtime data insertion (if applicable) + if (!isHistorical) { + TestUtils.executeNonQueries(senderEnv, insertQueries, null); + awaitUntilFlush(senderEnv); + } - final Set expectedResSet = new HashSet<>(); - expectedResSet.add("1,null,null,1.0,1.0,null,"); - expectedResSet.add("2,null,null,null,null,2.0,"); - expectedResSet.add("3,3.0,3.0,null,null,null,"); + // 7. Validate data eventually arrives on the receiver TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select * from root.db2.**,root.db.**", - "Time,root.db2.d1.s,root.db2.d1.t,root.db.d1.s,root.db.d1.s1,root.db.d2.s,", - expectedResSet); + receiverEnv, validationSelectQuery, validationSelectHeader, expectedResultSet); } } @Test - public void testMultipleHybridPatternHistoricalData() throws Exception { - final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); - - final String receiverIp = receiverDataNode.getIp(); - final int receiverPort = receiverDataNode.getPort(); - - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); - - extractorAttributes.put("extractor.path", "root.db.d1.*"); - extractorAttributes.put("extractor.pattern", "root.db2.d1.s"); - extractorAttributes.put("extractor.inclusion", "data.insert"); - - connectorAttributes.put("connector", "iotdb-thrift-connector"); - connectorAttributes.put("connector.batch.enable", "false"); - connectorAttributes.put("connector.ip", receiverIp); - connectorAttributes.put("connector.port", Integer.toString(receiverPort)); - - TestUtils.executeNonQueries( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s, s1) values (1, 1, 1)", - "insert into root.db2.d1(time, s) values (2, 2)", - "insert into root.db3.d1(time, s) values (3, 3)"), - null); - awaitUntilFlush(senderEnv); - - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); - - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - - Assert.assertEquals( - TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p1").getCode()); - - final Set expectedResSet = new HashSet<>(); - expectedResSet.add("1,1.0,1.0,null,"); - expectedResSet.add("2,null,null,2.0,"); - - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select * from root.db.**,root.db2.**", - "Time,root.db.d1.s,root.db.d1.s1,root.db2.d1.s,", - expectedResSet); - } + public void testMultiplePrefixPatternHistoricalData() throws Exception { + // Define extractor attributes + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.pattern", "root.db.d1.s, root.db2.d1.s"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + // Define data to be inserted + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + "insert into root.db.d2(time, s) values (2, 2)", + "insert into root.db2.d1(time, s) values (3, 3)"); + + // Define expected results on receiver + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,null,1.0,1.0,"); + expectedResSet.add("3,3.0,null,null,"); + + // Execute the common test logic + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + true, // isHistorical = true + "select * from root.db2.**,root.db.**", + "Time,root.db2.d1.s,root.db.d1.s,root.db.d1.s1,", + expectedResSet); } @Test public void testMultiplePrefixPatternRealtimeData() throws Exception { - final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); - - final String receiverIp = receiverDataNode.getIp(); - final int receiverPort = receiverDataNode.getPort(); - - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); - - extractorAttributes.put("extractor.pattern", "root.db.d1.s, root.db2.d1.s"); - extractorAttributes.put("extractor.inclusion", "data.insert"); - - connectorAttributes.put("connector", "iotdb-thrift-connector"); - connectorAttributes.put("connector.batch.enable", "false"); - connectorAttributes.put("connector.ip", receiverIp); - connectorAttributes.put("connector.port", Integer.toString(receiverPort)); - - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); - - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - - Assert.assertEquals( - TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p1").getCode()); - - TestUtils.executeNonQueries( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s, s1) values (1, 1, 1)", - "insert into root.db.d2(time, s) values (2, 2)", - "insert into root.db2.d1(time, s) values (3, 3)"), - null); - awaitUntilFlush(senderEnv); + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.pattern", "root.db.d1.s, root.db2.d1.s"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + "insert into root.db.d2(time, s) values (2, 2)", + "insert into root.db2.d1(time, s) values (3, 3)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,null,1.0,1.0,"); + expectedResSet.add("3,3.0,null,null,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + false, // isHistorical = false + "select * from root.db2.**,root.db.**", + "Time,root.db2.d1.s,root.db.d1.s,root.db.d1.s1,", + expectedResSet); + } - final Set expectedResSet = new HashSet<>(); - expectedResSet.add("1,null,1.0,1.0,"); - expectedResSet.add("3,3.0,null,null,"); - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select * from root.db2.**,root.db.**", - "Time,root.db2.d1.s,root.db.d1.s,root.db.d1.s1,", - expectedResSet); - } + @Test + public void testMultipleIoTDBPatternHistoricalData() throws Exception { + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.path", "root.db.**, root.db2.d1.*"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + "insert into root.db.d2(time, s) values (2, 2)", + "insert into root.db2.d1(time, s, t) values (3, 3, 3)", + "insert into root.db3.d1(time, s) values (4, 4)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,null,null,1.0,1.0,null,"); + expectedResSet.add("2,null,null,null,null,2.0,"); + expectedResSet.add("3,3.0,3.0,null,null,null,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + true, // isHistorical = true + "select * from root.db2.**,root.db.**", + "Time,root.db2.d1.s,root.db2.d1.t,root.db.d1.s,root.db.d1.s1,root.db.d2.s,", + expectedResSet); } @Test public void testMultipleIoTDBPatternRealtimeData() throws Exception { - final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); - - final String receiverIp = receiverDataNode.getIp(); - final int receiverPort = receiverDataNode.getPort(); - - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); - - extractorAttributes.put("extractor.path", "root.db.**, root.db2.d1.*"); - extractorAttributes.put("extractor.inclusion", "data.insert"); - - connectorAttributes.put("connector", "iotdb-thrift-connector"); - connectorAttributes.put("connector.batch.enable", "false"); - connectorAttributes.put("connector.ip", receiverIp); - connectorAttributes.put("connector.port", Integer.toString(receiverPort)); - - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); - - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - - Assert.assertEquals( - TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p1").getCode()); - - TestUtils.executeNonQueries( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s, s1) values (1, 1, 1)", - "insert into root.db.d2(time, s) values (2, 2)", - "insert into root.db2.d1(time, s, t) values (3, 3, 3)", - "insert into root.db3.d1(time, s) values (4, 4)"), - null); - awaitUntilFlush(senderEnv); + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.path", "root.db.**, root.db2.d1.*"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + "insert into root.db.d2(time, s) values (2, 2)", + "insert into root.db2.d1(time, s, t) values (3, 3, 3)", + "insert into root.db3.d1(time, s) values (4, 4)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,null,null,1.0,1.0,null,"); + expectedResSet.add("2,null,null,null,null,2.0,"); + expectedResSet.add("3,3.0,3.0,null,null,null,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + false, // isHistorical = false + "select * from root.db2.**,root.db.**", + "Time,root.db2.d1.s,root.db2.d1.t,root.db.d1.s,root.db.d1.s1,root.db.d2.s,", + expectedResSet); + } - final Set expectedResSet = new HashSet<>(); - expectedResSet.add("1,null,null,1.0,1.0,null,"); - expectedResSet.add("2,null,null,null,null,2.0,"); - expectedResSet.add("3,3.0,3.0,null,null,null,"); - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select * from root.db2.**,root.db.**", - "Time,root.db2.d1.s,root.db2.d1.t,root.db.d1.s,root.db.d1.s1,root.db.d2.s,", - expectedResSet); - } + @Test + public void testMultipleHybridPatternHistoricalData() throws Exception { + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.path", "root.db.d1.*"); + extractorAttributes.put("extractor.pattern", "root.db2.d1.s"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + "insert into root.db2.d1(time, s) values (2, 2)", + "insert into root.db3.d1(time, s) values (3, 3)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,1.0,1.0,null,"); + expectedResSet.add("2,null,null,2.0,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + true, // isHistorical = true + "select * from root.db.**,root.db2.**", + "Time,root.db.d1.s,root.db.d1.s1,root.db2.d1.s,", + expectedResSet); } @Test public void testMultipleHybridPatternRealtimeData() throws Exception { - final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); - - final String receiverIp = receiverDataNode.getIp(); - final int receiverPort = receiverDataNode.getPort(); - - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); - - extractorAttributes.put("extractor.path", "root.db.d1.*"); - extractorAttributes.put("extractor.pattern", "root.db2.d1.s"); - extractorAttributes.put("extractor.inclusion", "data.insert"); - - connectorAttributes.put("connector", "iotdb-thrift-connector"); - connectorAttributes.put("connector.batch.enable", "false"); - connectorAttributes.put("connector.ip", receiverIp); - connectorAttributes.put("connector.port", Integer.toString(receiverPort)); + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.path", "root.db.d1.*"); + extractorAttributes.put("extractor.pattern", "root.db2.d1.s"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + "insert into root.db2.d1(time, s) values (2, 2)", + "insert into root.db3.d1(time, s) values (3, 3)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,1.0,1.0,null,"); + expectedResSet.add("2,null,null,2.0,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + false, // isHistorical = false + "select * from root.db.**,root.db2.**", + "Time,root.db.d1.s,root.db.d1.s1,root.db2.d1.s,", + expectedResSet); + } - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); + @Test + public void testPrefixPatternWithExclusionHistoricalData() throws Exception { + final Map extractorAttributes = new HashMap<>(); + // Inclusion: Match everything under root.db.d1 and root.db.d2 + extractorAttributes.put("extractor.pattern", "root.db.d1, root.db.d2"); + // Exclusion: Exclude anything with the prefix root.db.d1.s1 + extractorAttributes.put("extractor.pattern.exclusion", "root.db.d1.s1"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + // s matches, s1 is excluded + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + // s matches + "insert into root.db.d2(time, s) values (2, 2)", + "insert into root.db1.d1(time, s) values (3, 3)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,1.0,null,"); + expectedResSet.add("2,null,2.0,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + true, // isHistorical = true + "select * from root.db.**", + "Time,root.db.d1.s,root.db.d2.s,", + expectedResSet); + } - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + @Test + public void testPrefixPatternWithExclusionRealtimeData() throws Exception { + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.pattern", "root.db.d1, root.db.d2"); + extractorAttributes.put("extractor.pattern.exclusion", "root.db.d1.s1"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + "insert into root.db.d2(time, s) values (2, 2)", + "insert into root.db1.d1(time, s) values (3, 3)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,1.0,null,"); + expectedResSet.add("2,null,2.0,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + false, // isHistorical = false + "select * from root.db.**", + "Time,root.db.d1.s,root.db.d2.s,", + expectedResSet); + } - Assert.assertEquals( - TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p1").getCode()); + @Test + public void testIoTDBPatternWithExclusionHistoricalData() throws Exception { + final Map extractorAttributes = new HashMap<>(); + // Inclusion: Match everything under root.db + extractorAttributes.put("extractor.path", "root.db.**"); + // Exclusion: Exclude root.db.d1.s* and root.db.d3.* + extractorAttributes.put("extractor.path.exclusion", "root.db.d1.s*, root.db.d3.*"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + // s, s1 excluded, t matches + "insert into root.db.d1(time, s, s1, t) values (1, 1, 1, 1)", + // s matches + "insert into root.db.d2(time, s) values (2, 2)", + // s excluded + "insert into root.db.d3(time, s) values (3, 3)", + "insert into root.db1.d1(time, s) values (4, 4)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,1.0,null,"); + expectedResSet.add("2,null,2.0,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + true, // isHistorical = true + "select * from root.db.**", + "Time,root.db.d1.t,root.db.d2.s,", + expectedResSet); + } - TestUtils.executeNonQueries( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s, s1) values (1, 1, 1)", - "insert into root.db2.d1(time, s) values (2, 2)", - "insert into root.db3.d1(time, s) values (3, 3)"), - null); - awaitUntilFlush(senderEnv); + @Test + public void testIoTDBPatternWithExclusionRealtimeData() throws Exception { + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.path", "root.db.**"); + extractorAttributes.put("extractor.path.exclusion", "root.db.d1.s*, root.db.d3.*"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1, t) values (1, 1, 1, 1)", + "insert into root.db.d2(time, s) values (2, 2)", + "insert into root.db.d3(time, s) values (3, 3)", + "insert into root.db1.d1(time, s) values (4, 4)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("1,1.0,null,"); + expectedResSet.add("2,null,2.0,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + false, // isHistorical = false + "select * from root.db.**", + "Time,root.db.d1.t,root.db.d2.s,", + expectedResSet); + } - final Set expectedResSet = new HashSet<>(); - expectedResSet.add("1,1.0,1.0,null,"); - expectedResSet.add("2,null,null,2.0,"); + @Test + public void testHybridPatternWithHybridExclusionHistoricalData() throws Exception { + final Map extractorAttributes = new HashMap<>(); + // Inclusion: Match root.db.** (IoTDB) AND root.db2.d1 (Prefix) + extractorAttributes.put("extractor.path", "root.db.**"); + extractorAttributes.put("extractor.pattern", "root.db2.d1"); + // Exclusion: Exclude root.db.d1.* (IoTDB) AND root.db2.d1.s (Prefix) + extractorAttributes.put("extractor.path.exclusion", "root.db.d1.*"); + extractorAttributes.put("extractor.pattern.exclusion", "root.db2.d1.s"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + // s, s1 excluded by path.exclusion + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + // s matches + "insert into root.db.d2(time, s) values (2, 2)", + // s excluded by pattern.exclusion, t matches + "insert into root.db2.d1(time, s, t) values (3, 3, 3)", + "insert into root.db3.d1(time, s) values (4, 4)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("2,2.0,null,"); + expectedResSet.add("3,null,3.0,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + true, // isHistorical = true + "select * from root.db.**,root.db2.**", + "Time,root.db.d2.s,root.db2.d1.t,", + expectedResSet); + } - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select * from root.db.**,root.db2.**", - "Time,root.db.d1.s,root.db.d1.s1,root.db2.d1.s,", - expectedResSet); - } + @Test + public void testHybridPatternWithHybridExclusionRealtimeData() throws Exception { + final Map extractorAttributes = new HashMap<>(); + extractorAttributes.put("extractor.path", "root.db.**"); + extractorAttributes.put("extractor.pattern", "root.db2.d1"); + extractorAttributes.put("extractor.path.exclusion", "root.db.d1.*"); + extractorAttributes.put("extractor.pattern.exclusion", "root.db2.d1.s"); + extractorAttributes.put("extractor.inclusion", "data.insert"); + + final List insertQueries = + Arrays.asList( + "insert into root.db.d1(time, s, s1) values (1, 1, 1)", + "insert into root.db.d2(time, s) values (2, 2)", + "insert into root.db2.d1(time, s, t) values (3, 3, 3)", + "insert into root.db3.d1(time, s) values (4, 4)"); + + final Set expectedResSet = new HashSet<>(); + expectedResSet.add("2,2.0,null,"); + expectedResSet.add("3,null,3.0,"); + + testPipeWithMultiplePatterns( + extractorAttributes, + insertQueries, + false, // isHistorical = false + "select * from root.db.**,root.db2.**", + "Time,root.db.d2.s,root.db2.d1.t,", + expectedResSet); } } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java index 61eaf14bb4e62..54657f9c50d71 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/dual/treemodel/manual/IoTDBPipeInclusionIT.java @@ -182,6 +182,78 @@ public void testPureSchemaInclusionWithMultiplePattern() throws Exception { } } + @Test + public void testPureSchemaInclusionWithExclusionPattern() throws Exception { + final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); + + final String receiverIp = receiverDataNode.getIp(); + final int receiverPort = receiverDataNode.getPort(); + + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + final Map extractorAttributes = new HashMap<>(); + final Map processorAttributes = new HashMap<>(); + final Map connectorAttributes = new HashMap<>(); + + extractorAttributes.put("extractor.inclusion", "schema"); + // Include root.ln.** + extractorAttributes.put("path", "root.ln.**"); + // Exclude root.ln.wf02.* and root.ln.wf03.wt01.status + extractorAttributes.put("path.exclusion", "root.ln.wf02.**, root.ln.wf03.wt01.status"); + + connectorAttributes.put("connector", "iotdb-thrift-connector"); + connectorAttributes.put("connector.ip", receiverIp); + connectorAttributes.put("connector.port", Integer.toString(receiverPort)); + + final TSStatus status = + client.createPipe( + new TCreatePipeReq("testPipe", connectorAttributes) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes)); + + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("testPipe").getCode()); + + // Do not fail if the failure has nothing to do with pipe + // Because the failures will randomly generate due to resource limitation + TestUtils.executeNonQueries( + senderEnv, + Arrays.asList( + // Should be included + "create timeseries root.ln.wf01.wt01.status with datatype=BOOLEAN,encoding=PLAIN", + "ALTER timeseries root.ln.wf01.wt01.status ADD TAGS tag3=v3", + "ALTER timeseries root.ln.wf01.wt01.status ADD ATTRIBUTES attr4=v4", + // Should be excluded by root.ln.wf02.* + "create timeseries root.ln.wf02.wt01.status with datatype=BOOLEAN,encoding=PLAIN", + "ALTER timeseries root.ln.wf02.wt01.status ADD TAGS tag3=v3", + "ALTER timeseries root.ln.wf02.wt01.status ADD ATTRIBUTES attr4=v4", + // Should be excluded by root.ln.wf03.wt01.status + "create timeseries root.ln.wf03.wt01.status with datatype=BOOLEAN,encoding=PLAIN", + "ALTER timeseries root.ln.wf03.wt01.status ADD TAGS tag3=v3", + "ALTER timeseries root.ln.wf03.wt01.status ADD ATTRIBUTES attr4=v4"), + null); + + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "show timeseries root.ln.**", + "Timeseries,Alias,Database,DataType,Encoding,Compression,Tags,Attributes,Deadband,DeadbandParameters,ViewType,", + // Only wf01 should be synced + Collections.singleton( + "root.ln.wf01.wt01.status,null,root.ln,BOOLEAN,PLAIN,LZ4,{\"tag3\":\"v3\"},{\"attr4\":\"v4\"},null,null,BASE,")); + + TestUtils.executeNonQueries( + senderEnv, + Arrays.asList( + "insert into root.ln.wf01.wt01(time, status) values(now(), false)", "flush"), + null); + + TestUtils.assertDataAlwaysOnEnv( + receiverEnv, "select * from root.ln.**", "Time,", Collections.emptySet()); + } + } + @Test public void testAuthExclusion() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java index dc2e491c96cc2..daf91c51c4c48 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java @@ -28,9 +28,9 @@ import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.path.PathPatternTreeUtils; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.commons.pipe.receiver.IoTDBFileReceiver; import org.apache.iotdb.commons.pipe.receiver.PipeReceiverStatusHandler; import org.apache.iotdb.commons.pipe.sink.payload.airgap.AirGapPseudoTPipeTransferRequest; @@ -1127,12 +1127,11 @@ protected TSStatus loadFileV2( parameters.get(ColumnHeaderConstant.TYPE)); final boolean isTreeModelDataAllowedToBeCaptured = parameters.containsKey(PipeTransferFileSealReqV2.TREE); - final List treePatterns = - TreePattern.parseMultiplePatterns( + final TreePattern treePattern = + TreePattern.parsePatternFromString( parameters.get(ColumnHeaderConstant.PATH_PATTERN), + isTreeModelDataAllowedToBeCaptured, p -> new IoTDBTreePattern(isTreeModelDataAllowedToBeCaptured, p)); - final TreePattern treePattern = - TreePattern.buildUnionPattern(isTreeModelDataAllowedToBeCaptured, treePatterns); final TablePattern tablePattern = new TablePattern( parameters.containsKey(PipeTransferFileSealReqV2.TABLE), @@ -1141,13 +1140,13 @@ protected TSStatus loadFileV2( final List results = new ArrayList<>(); while (generator.hasNext()) { IoTDBConfigRegionSource.parseConfigPlan( - generator.next(), (UnionIoTDBTreePattern) treePattern, tablePattern) + generator.next(), (IoTDBTreePatternOperations) treePattern, tablePattern) .filter( configPhysicalPlan -> IoTDBConfigRegionSource.isTypeListened( configPhysicalPlan, executionTypes, - (UnionIoTDBTreePattern) treePattern, + (IoTDBTreePatternOperations) treePattern, tablePattern)) .ifPresent( configPhysicalPlan -> diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/IoTDBConfigRegionSource.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/IoTDBConfigRegionSource.java index 252f7331d0cd1..f3c6996fb59dc 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/IoTDBConfigRegionSource.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/IoTDBConfigRegionSource.java @@ -25,8 +25,8 @@ import org.apache.iotdb.commons.exception.auth.AccessDeniedException; import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.commons.pipe.datastructure.queue.listening.AbstractPipeListeningQueue; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.PipeSnapshotEvent; @@ -253,7 +253,7 @@ protected Optional trimRealtimeEventByPipePattern( public static Optional parseConfigPlan( final ConfigPhysicalPlan plan, - final UnionIoTDBTreePattern treePattern, + final IoTDBTreePatternOperations treePattern, final TablePattern tablePattern) { Optional result = Optional.of(plan); final Boolean isTableDatabasePlan = isTableDatabasePlan(plan); @@ -296,7 +296,7 @@ protected boolean isTypeListened(final PipeWritePlanEvent event) { public static boolean isTypeListened( final ConfigPhysicalPlan plan, final Set listenedTypeSet, - final UnionIoTDBTreePattern treePattern, + final IoTDBTreePatternOperations treePattern, final TablePattern tablePattern) { final Boolean isTableDatabasePlan = isTableDatabasePlan(plan); return listenedTypeSet.contains(plan.getType()) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitor.java index 8e9a3d2cc0fe8..0324598b0a43b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitor.java @@ -22,8 +22,8 @@ import org.apache.iotdb.commons.auth.entity.PrivilegeType; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanVisitor; import org.apache.iotdb.confignode.consensus.request.write.auth.AuthorTreePlan; @@ -60,12 +60,12 @@ /** * The {@link PipeConfigPhysicalPlanTreePatternParseVisitor} will transform the schema {@link - * ConfigPhysicalPlan}s using {@link UnionIoTDBTreePattern}. Rule: + * ConfigPhysicalPlan}s using {@link IoTDBTreePatternOperations}. Rule: * *

1. All patterns in the output {@link ConfigPhysicalPlan} will be the intersection of the - * original {@link ConfigPhysicalPlan}'s patterns and the given {@link UnionIoTDBTreePattern}. + * original {@link ConfigPhysicalPlan}'s patterns and the given {@link IoTDBTreePatternOperations}. * - *

2. If a pattern does not intersect with the {@link UnionIoTDBTreePattern}, it's dropped. + *

2. If a pattern does not intersect with the {@link IoTDBTreePatternOperations}, it's dropped. * *

3. If all the patterns in the {@link ConfigPhysicalPlan} is dropped, the {@link * ConfigPhysicalPlan} is dropped. @@ -74,13 +74,13 @@ * one is used in the {@link PipeConfigRegionWritePlanEvent} in {@link ConfigRegionListeningQueue}. */ public class PipeConfigPhysicalPlanTreePatternParseVisitor - extends ConfigPhysicalPlanVisitor, UnionIoTDBTreePattern> { + extends ConfigPhysicalPlanVisitor, IoTDBTreePatternOperations> { private static final Logger LOGGER = LoggerFactory.getLogger(PipeConfigPhysicalPlanTreePatternParseVisitor.class); @Override public Optional visitPlan( - final ConfigPhysicalPlan plan, final UnionIoTDBTreePattern pattern) { + final ConfigPhysicalPlan plan, final IoTDBTreePatternOperations pattern) { return Optional.of(plan); } @@ -92,7 +92,7 @@ public Optional visitPlan( // Other matches using "matchPrefixPath" are with the same principle. @Override public Optional visitCreateDatabase( - final DatabaseSchemaPlan createDatabasePlan, final UnionIoTDBTreePattern pattern) { + final DatabaseSchemaPlan createDatabasePlan, final IoTDBTreePatternOperations pattern) { return pattern.matchPrefixPath(createDatabasePlan.getSchema().getName()) ? Optional.of(createDatabasePlan) : Optional.empty(); @@ -100,7 +100,7 @@ public Optional visitCreateDatabase( @Override public Optional visitAlterDatabase( - final DatabaseSchemaPlan alterDatabasePlan, final UnionIoTDBTreePattern pattern) { + final DatabaseSchemaPlan alterDatabasePlan, final IoTDBTreePatternOperations pattern) { return pattern.matchPrefixPath(alterDatabasePlan.getSchema().getName()) ? Optional.of(alterDatabasePlan) : Optional.empty(); @@ -108,7 +108,7 @@ public Optional visitAlterDatabase( @Override public Optional visitDeleteDatabase( - final DeleteDatabasePlan deleteDatabasePlan, final UnionIoTDBTreePattern pattern) { + final DeleteDatabasePlan deleteDatabasePlan, final IoTDBTreePatternOperations pattern) { return pattern.matchPrefixPath(deleteDatabasePlan.getName()) ? Optional.of(deleteDatabasePlan) : Optional.empty(); @@ -117,7 +117,7 @@ public Optional visitDeleteDatabase( @Override public Optional visitCreateSchemaTemplate( final CreateSchemaTemplatePlan createSchemaTemplatePlan, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { // This is a deserialized template and can be arbitrarily altered final Template template = createSchemaTemplatePlan.getTemplate(); template.getSchemaMap().keySet().removeIf(measurement -> !pattern.matchTailNode(measurement)); @@ -129,7 +129,7 @@ public Optional visitCreateSchemaTemplate( @Override public Optional visitCommitSetSchemaTemplate( final CommitSetSchemaTemplatePlan commitSetSchemaTemplatePlan, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { return pattern.matchPrefixPath(commitSetSchemaTemplatePlan.getPath()) ? Optional.of(commitSetSchemaTemplatePlan) : Optional.empty(); @@ -138,7 +138,7 @@ public Optional visitCommitSetSchemaTemplate( @Override public Optional visitPipeUnsetSchemaTemplate( final PipeUnsetSchemaTemplatePlan pipeUnsetSchemaTemplatePlan, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { return pattern.matchPrefixPath(pipeUnsetSchemaTemplatePlan.getPath()) ? Optional.of(pipeUnsetSchemaTemplatePlan) : Optional.empty(); @@ -147,7 +147,7 @@ public Optional visitPipeUnsetSchemaTemplate( @Override public Optional visitExtendSchemaTemplate( final ExtendSchemaTemplatePlan extendSchemaTemplatePlan, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { final TemplateExtendInfo extendInfo = extendSchemaTemplatePlan.getTemplateExtendInfo(); final int[] filteredIndexes = IntStream.range(0, extendInfo.getMeasurements().size()) @@ -167,30 +167,30 @@ public Optional visitExtendSchemaTemplate( @Override public Optional visitGrantUser( - final AuthorTreePlan grantUserPlan, final UnionIoTDBTreePattern pattern) { + final AuthorTreePlan grantUserPlan, final IoTDBTreePatternOperations pattern) { return visitTreeAuthorPlan(grantUserPlan, pattern); } @Override public Optional visitRevokeUser( - final AuthorTreePlan revokeUserPlan, final UnionIoTDBTreePattern pattern) { + final AuthorTreePlan revokeUserPlan, final IoTDBTreePatternOperations pattern) { return visitTreeAuthorPlan(revokeUserPlan, pattern); } @Override public Optional visitGrantRole( - final AuthorTreePlan revokeUserPlan, final UnionIoTDBTreePattern pattern) { + final AuthorTreePlan revokeUserPlan, final IoTDBTreePatternOperations pattern) { return visitTreeAuthorPlan(revokeUserPlan, pattern); } @Override public Optional visitRevokeRole( - final AuthorTreePlan revokeUserPlan, final UnionIoTDBTreePattern pattern) { + final AuthorTreePlan revokeUserPlan, final IoTDBTreePatternOperations pattern) { return visitTreeAuthorPlan(revokeUserPlan, pattern); } private Optional visitTreeAuthorPlan( - final AuthorTreePlan pathRelatedAuthorTreePlan, final UnionIoTDBTreePattern pattern) { + final AuthorTreePlan pathRelatedAuthorTreePlan, final IoTDBTreePatternOperations pattern) { final List intersectedPaths = pathRelatedAuthorTreePlan.getNodeNameList().stream() .map(pattern::getIntersection) @@ -219,7 +219,7 @@ private Optional visitTreeAuthorPlan( @Override public Optional visitPipeDeleteTimeSeries( final PipeDeleteTimeSeriesPlan pipeDeleteTimeSeriesPlan, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { try { final PathPatternTree intersectedTree = pattern.getIntersection( @@ -238,7 +238,7 @@ public Optional visitPipeDeleteTimeSeries( @Override public Optional visitPipeDeleteLogicalView( final PipeDeleteLogicalViewPlan pipeDeleteLogicalViewPlan, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { try { final PathPatternTree intersectedTree = pattern.getIntersection( @@ -257,7 +257,7 @@ public Optional visitPipeDeleteLogicalView( @Override public Optional visitPipeAlterEncodingCompressor( final PipeAlterEncodingCompressorPlan pipeAlterEncodingCompressorPlan, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { try { final PathPatternTree intersectedTree = pattern.getIntersection( @@ -281,7 +281,7 @@ public Optional visitPipeAlterEncodingCompressor( @Override public Optional visitPipeDeactivateTemplate( final PipeDeactivateTemplatePlan pipeDeactivateTemplatePlan, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { final Map> newTemplateSetInfo = pipeDeactivateTemplatePlan.getTemplateSetInfo().entrySet().stream() .flatMap( @@ -304,7 +304,7 @@ public Optional visitPipeDeactivateTemplate( @Override public Optional visitTTL( - final SetTTLPlan setTTLPlan, final UnionIoTDBTreePattern pattern) { + final SetTTLPlan setTTLPlan, final IoTDBTreePatternOperations pattern) { final PartialPath partialPath = new PartialPath(setTTLPlan.getPathPattern()); final List intersectionList = pattern.matchPrefixPath(partialPath.getFullPath()) diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitorTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitorTest.java index 442351cc19d2b..155713539a761 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitorTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/source/PipeConfigPhysicalPlanTreePatternParseVisitorTest.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; import org.apache.iotdb.confignode.consensus.request.write.auth.AuthorTreePlan; @@ -55,11 +56,11 @@ public class PipeConfigPhysicalPlanTreePatternParseVisitorTest { - private final UnionIoTDBTreePattern prefixPathPattern = + private final IoTDBTreePatternOperations prefixPathPattern = new UnionIoTDBTreePattern(new IoTDBTreePattern("root.db.device.**")); - private final UnionIoTDBTreePattern fullPathPattern = + private final IoTDBTreePatternOperations fullPathPattern = new UnionIoTDBTreePattern(new IoTDBTreePattern("root.db.device.s1")); - private final UnionIoTDBTreePattern multiplePathPattern = + private final IoTDBTreePatternOperations multiplePathPattern = new UnionIoTDBTreePattern( Arrays.asList( new IoTDBTreePattern("root.db.device.s1"), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java index f979d4763b5da..f119070c891d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java @@ -22,7 +22,7 @@ import org.apache.iotdb.commons.audit.UserEntity; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; @@ -172,7 +172,7 @@ private void parseAndCollectEvent(final PipeDeleteDataNodeEvent deleteDataEvent) (deleteDataEvent.getDeleteDataNode() instanceof DeleteDataNode ? IoTDBSchemaRegionSource.TREE_PATTERN_PARSE_VISITOR.process( deleteDataEvent.getDeleteDataNode(), - (UnionIoTDBTreePattern) deleteDataEvent.getTreePattern()) + (IoTDBTreePatternOperations) deleteDataEvent.getTreePattern()) : IoTDBSchemaRegionSource.TABLE_PATTERN_PARSE_VISITOR .process(deleteDataEvent.getDeleteDataNode(), deleteDataEvent.getTablePattern()) .flatMap( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/TsFileInsertionEventParserProvider.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/TsFileInsertionEventParserProvider.java index 8afd162779099..5632ab8122607 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/TsFileInsertionEventParserProvider.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/TsFileInsertionEventParserProvider.java @@ -21,9 +21,9 @@ import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.parser.query.TsFileInsertionEventQueryParser; import org.apache.iotdb.db.pipe.event.common.tsfile.parser.scan.TsFileInsertionEventScanParser; @@ -114,8 +114,8 @@ public TsFileInsertionEventParser provide(final boolean isWithMod) throws IOExce isWithMod); } - if (treePattern instanceof UnionIoTDBTreePattern - && !((UnionIoTDBTreePattern) treePattern).mayMatchMultipleTimeSeriesInOneDevice()) { + if (treePattern instanceof IoTDBTreePatternOperations + && !((IoTDBTreePatternOperations) treePattern).mayMatchMultipleTimeSeriesInOneDevice()) { // If the pattern matches only one time series in one device, use query container here // because there is no timestamps merge overhead. // diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index 4096f5f931ba5..8a9ceb61c930c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -28,9 +28,9 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.commons.pipe.receiver.IoTDBFileReceiver; import org.apache.iotdb.commons.pipe.receiver.PipeReceiverStatusHandler; import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; @@ -609,12 +609,11 @@ private TSStatus loadSchemaSnapShot( parameters.get(ColumnHeaderConstant.TYPE)); final boolean isTreeModelDataAllowedToBeCaptured = parameters.containsKey(PipeTransferFileSealReqV2.TREE); - final List treePatterns = - TreePattern.parseMultiplePatterns( + final TreePattern treePattern = + TreePattern.parsePatternFromString( parameters.get(ColumnHeaderConstant.PATH_PATTERN), + isTreeModelDataAllowedToBeCaptured, p -> new IoTDBTreePattern(isTreeModelDataAllowedToBeCaptured, p)); - final TreePattern treePattern = - TreePattern.buildUnionPattern(isTreeModelDataAllowedToBeCaptured, treePatterns); final TablePattern tablePattern = new TablePattern( parameters.containsKey(PipeTransferFileSealReqV2.TABLE), @@ -636,7 +635,7 @@ private TSStatus loadSchemaSnapShot( // Here we apply the statements as many as possible // Even if there are failed statements STATEMENT_TREE_PATTERN_PARSE_VISITOR - .process(originalStatement, (UnionIoTDBTreePattern) treePattern) + .process(originalStatement, (IoTDBTreePatternOperations) treePattern) .flatMap(parsedStatement -> batchVisitor.process(parsedStatement, null)) .ifPresent(statement -> results.add(executeStatementAndClassifyExceptions(statement))); } else if (treeOrTableStatement diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTreePatternParseVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTreePatternParseVisitor.java index aa11fd69efa4f..64f11c0630ed1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTreePatternParseVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTreePatternParseVisitor.java @@ -19,8 +19,8 @@ package org.apache.iotdb.db.pipe.receiver.visitor; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.queryengine.plan.statement.StatementNode; import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; @@ -37,12 +37,12 @@ /** * The {@link PipeStatementTreePatternParseVisitor} will transform the schema {@link Statement}s - * using {@link UnionIoTDBTreePattern}. Rule: + * using {@link IoTDBTreePatternOperations}. Rule: * *

1. All patterns in the output {@link Statement} will be the intersection of the original - * {@link Statement}'s patterns and the given {@link UnionIoTDBTreePattern}. + * {@link Statement}'s patterns and the given {@link IoTDBTreePatternOperations}. * - *

2. If a pattern does not intersect with the {@link UnionIoTDBTreePattern}, it's dropped. + *

2. If a pattern does not intersect with the {@link IoTDBTreePatternOperations}, it's dropped. * *

3. If all the patterns in the {@link Statement} is dropped, the {@link Statement} is dropped. * @@ -50,16 +50,16 @@ * from the {@link SRStatementGenerator} and will no longer be used. */ public class PipeStatementTreePatternParseVisitor - extends StatementVisitor, UnionIoTDBTreePattern> { + extends StatementVisitor, IoTDBTreePatternOperations> { @Override public Optional visitNode( - final StatementNode statement, final UnionIoTDBTreePattern pattern) { + final StatementNode statement, final IoTDBTreePatternOperations pattern) { return Optional.of((Statement) statement); } @Override public Optional visitCreateTimeseries( - final CreateTimeSeriesStatement statement, final UnionIoTDBTreePattern pattern) { + final CreateTimeSeriesStatement statement, final IoTDBTreePatternOperations pattern) { return pattern.matchesMeasurement( statement.getPath().getIDeviceID(), statement.getPath().getMeasurement()) ? Optional.of(statement) @@ -68,7 +68,7 @@ public Optional visitCreateTimeseries( @Override public Optional visitCreateAlignedTimeseries( - final CreateAlignedTimeSeriesStatement statement, final UnionIoTDBTreePattern pattern) { + final CreateAlignedTimeSeriesStatement statement, final IoTDBTreePatternOperations pattern) { final int[] filteredIndexes = IntStream.range(0, statement.getMeasurements().size()) .filter( @@ -109,7 +109,7 @@ public Optional visitCreateAlignedTimeseries( @Override public Optional visitAlterTimeSeries( final AlterTimeSeriesStatement alterTimeSeriesStatement, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { return pattern.matchesMeasurement( alterTimeSeriesStatement.getPath().getIDeviceID(), alterTimeSeriesStatement.getPath().getMeasurement()) @@ -120,7 +120,7 @@ public Optional visitAlterTimeSeries( @Override public Optional visitActivateTemplate( final ActivateTemplateStatement activateTemplateStatement, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { return pattern.matchDevice(activateTemplateStatement.getPath().getFullPath()) ? Optional.of(activateTemplateStatement) : Optional.empty(); @@ -129,7 +129,7 @@ public Optional visitActivateTemplate( @Override public Optional visitCreateLogicalView( final CreateLogicalViewStatement createLogicalViewStatement, - final UnionIoTDBTreePattern pattern) { + final IoTDBTreePatternOperations pattern) { final int[] filteredIndexes = IntStream.range(0, createLogicalViewStatement.getTargetPathList().size()) .filter( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/IoTDBDataRegionSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/IoTDBDataRegionSource.java index 6931726157ef3..4e69b47cc8df2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/IoTDBDataRegionSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/IoTDBDataRegionSource.java @@ -23,8 +23,8 @@ import org.apache.iotdb.commons.pipe.agent.task.PipeTaskAgent; import org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.commons.pipe.source.IoTDBSource; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -300,8 +300,8 @@ private void validatePattern(final TreePattern treePattern) { } if (shouldExtractDeletion - && !(treePattern instanceof UnionIoTDBTreePattern - && (((UnionIoTDBTreePattern) treePattern).isPrefixOrFullPath()))) { + && !(treePattern instanceof IoTDBTreePatternOperations + && (((IoTDBTreePatternOperations) treePattern).isPrefixOrFullPath()))) { throw new IllegalArgumentException( String.format( "The path pattern %s is not valid for the source. Only prefix or full path is allowed.", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/schemaregion/PipePlanTreePatternParseVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/schemaregion/PipePlanTreePatternParseVisitor.java index b3f4aa3f88cb3..041c19b8330f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/schemaregion/PipePlanTreePatternParseVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/schemaregion/PipePlanTreePatternParseVisitor.java @@ -21,8 +21,8 @@ import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; @@ -55,12 +55,12 @@ /** * The {@link PipePlanTreePatternParseVisitor} will transform the schema {@link PlanNode}s using - * {@link UnionIoTDBTreePattern}. Rule: + * {@link IoTDBTreePatternOperations}. Rule: * *

1. All patterns in the output {@link PlanNode} will be the intersection of the original {@link - * PlanNode}'s patterns and the given {@link UnionIoTDBTreePattern}. + * PlanNode}'s patterns and the given {@link IoTDBTreePatternOperations}. * - *

2. If a pattern does not intersect with the {@link UnionIoTDBTreePattern}, it's dropped. + *

2. If a pattern does not intersect with the {@link IoTDBTreePatternOperations}, it's dropped. * *

3. If all the patterns in the {@link PlanNode} is dropped, the {@link PlanNode} is dropped. * @@ -68,15 +68,16 @@ * one is used in the {@link PipeSchemaRegionWritePlanEvent} in {@link SchemaRegionListeningQueue}. */ public class PipePlanTreePatternParseVisitor - extends PlanVisitor, UnionIoTDBTreePattern> { + extends PlanVisitor, IoTDBTreePatternOperations> { @Override - public Optional visitPlan(final PlanNode node, final UnionIoTDBTreePattern pattern) { + public Optional visitPlan( + final PlanNode node, final IoTDBTreePatternOperations pattern) { return Optional.of(node); } @Override public Optional visitCreateTimeSeries( - final CreateTimeSeriesNode node, final UnionIoTDBTreePattern pattern) { + final CreateTimeSeriesNode node, final IoTDBTreePatternOperations pattern) { return pattern.matchesMeasurement( node.getPath().getIDeviceID(), node.getPath().getMeasurement()) ? Optional.of(node) @@ -85,7 +86,7 @@ public Optional visitCreateTimeSeries( @Override public Optional visitCreateAlignedTimeSeries( - final CreateAlignedTimeSeriesNode node, final UnionIoTDBTreePattern pattern) { + final CreateAlignedTimeSeriesNode node, final IoTDBTreePatternOperations pattern) { final int[] filteredIndexes = IntStream.range(0, node.getMeasurements().size()) .filter( @@ -111,7 +112,7 @@ public Optional visitCreateAlignedTimeSeries( @Override public Optional visitCreateMultiTimeSeries( - final CreateMultiTimeSeriesNode node, final UnionIoTDBTreePattern pattern) { + final CreateMultiTimeSeriesNode node, final IoTDBTreePatternOperations pattern) { final Map filteredMeasurementGroupMap = node.getMeasurementGroupMap().entrySet().stream() .filter(entry -> pattern.matchPrefixPath(entry.getKey().getFullPath())) @@ -130,7 +131,9 @@ public Optional visitCreateMultiTimeSeries( } private static MeasurementGroup trimMeasurementGroup( - final IDeviceID device, final MeasurementGroup group, final UnionIoTDBTreePattern pattern) { + final IDeviceID device, + final MeasurementGroup group, + final IoTDBTreePatternOperations pattern) { final int[] filteredIndexes = IntStream.range(0, group.size()) .filter(index -> pattern.matchesMeasurement(device, group.getMeasurements().get(index))) @@ -165,7 +168,7 @@ private static MeasurementGroup trimMeasurementGroup( @Override public Optional visitAlterTimeSeries( - final AlterTimeSeriesNode node, final UnionIoTDBTreePattern pattern) { + final AlterTimeSeriesNode node, final IoTDBTreePatternOperations pattern) { return pattern.matchesMeasurement( node.getPath().getIDeviceID(), node.getPath().getMeasurement()) ? Optional.of(node) @@ -174,7 +177,7 @@ public Optional visitAlterTimeSeries( @Override public Optional visitInternalCreateTimeSeries( - final InternalCreateTimeSeriesNode node, final UnionIoTDBTreePattern pattern) { + final InternalCreateTimeSeriesNode node, final IoTDBTreePatternOperations pattern) { final MeasurementGroup group = pattern.matchPrefixPath(node.getDevicePath().getFullPath()) ? trimMeasurementGroup( @@ -191,7 +194,7 @@ public Optional visitInternalCreateTimeSeries( @Override public Optional visitActivateTemplate( - final ActivateTemplateNode node, final UnionIoTDBTreePattern pattern) { + final ActivateTemplateNode node, final IoTDBTreePatternOperations pattern) { return pattern.matchDevice(node.getActivatePath().getFullPath()) ? Optional.of(node) : Optional.empty(); @@ -199,7 +202,7 @@ public Optional visitActivateTemplate( @Override public Optional visitInternalBatchActivateTemplate( - final InternalBatchActivateTemplateNode node, final UnionIoTDBTreePattern pattern) { + final InternalBatchActivateTemplateNode node, final IoTDBTreePatternOperations pattern) { final Map> filteredTemplateActivationMap = node.getTemplateActivationMap().entrySet().stream() .filter(entry -> pattern.matchDevice(entry.getKey().getFullPath())) @@ -213,7 +216,7 @@ public Optional visitInternalBatchActivateTemplate( @Override public Optional visitInternalCreateMultiTimeSeries( - final InternalCreateMultiTimeSeriesNode node, final UnionIoTDBTreePattern pattern) { + final InternalCreateMultiTimeSeriesNode node, final IoTDBTreePatternOperations pattern) { final Map> filteredDeviceMap = node.getDeviceMap().entrySet().stream() .filter(entry -> pattern.matchPrefixPath(entry.getKey().getFullPath())) @@ -237,7 +240,7 @@ public Optional visitInternalCreateMultiTimeSeries( @Override public Optional visitBatchActivateTemplate( - final BatchActivateTemplateNode node, final UnionIoTDBTreePattern pattern) { + final BatchActivateTemplateNode node, final IoTDBTreePatternOperations pattern) { final Map> filteredTemplateActivationMap = node.getTemplateActivationMap().entrySet().stream() .filter(entry -> pattern.matchDevice(entry.getKey().getFullPath())) @@ -250,7 +253,7 @@ public Optional visitBatchActivateTemplate( @Override public Optional visitCreateLogicalView( - final CreateLogicalViewNode node, final UnionIoTDBTreePattern pattern) { + final CreateLogicalViewNode node, final IoTDBTreePatternOperations pattern) { final Map filteredViewPathToSourceMap = node.getViewPathToSourceExpressionMap().entrySet().stream() .filter( @@ -265,7 +268,7 @@ public Optional visitCreateLogicalView( @Override public Optional visitAlterLogicalView( - final AlterLogicalViewNode node, final UnionIoTDBTreePattern pattern) { + final AlterLogicalViewNode node, final IoTDBTreePatternOperations pattern) { final Map filteredViewPathToSourceMap = node.getViewPathToSourceMap().entrySet().stream() .filter( @@ -280,7 +283,7 @@ public Optional visitAlterLogicalView( @Override public Optional visitDeleteData( - final DeleteDataNode node, final UnionIoTDBTreePattern pattern) { + final DeleteDataNode node, final IoTDBTreePatternOperations pattern) { final List intersectedPaths = node.getPathList().stream() .map(pattern::getIntersection) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/TreePatternCoverageTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/TreePatternCoverageTest.java new file mode 100644 index 0000000000000..dd91a34e8f959 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/TreePatternCoverageTest.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.pattern; + +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.PrefixTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; +import org.apache.iotdb.pipe.api.exception.PipeException; + +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertThrows; + +/** + * Test class for {@link TreePattern#checkAndLogPatternCoverage(TreePattern, TreePattern)}. It + * verifies the returned counts for different coverage scenarios. + */ +public class TreePatternCoverageTest { + + // ======================================================================== + // Helper Methods + // ======================================================================== + + private IoTDBTreePattern iotdb(final String pattern) { + return new IoTDBTreePattern(true, pattern); + } + + private PrefixTreePattern prefix(final String pattern) { + return new PrefixTreePattern(true, pattern); + } + + private UnionIoTDBTreePattern union(final IoTDBTreePattern... patterns) { + return new UnionIoTDBTreePattern(true, Arrays.asList(patterns)); + } + + // ======================================================================== + // Test Cases + // ======================================================================== + + @Test + public void testFullCoverageIoTDB() { + final TreePattern inclusion = iotdb("root.a.b.c"); + final TreePattern exclusion = iotdb("root.a.**"); + + // Verify that a PipeException is thrown + final PipeException e = + assertThrows( + PipeException.class, + () -> TreePattern.checkAndLogPatternCoverage(inclusion, exclusion)); + } + + @Test + public void testPartialCoverageIoTDB() { + final TreePattern inclusion = + union( + iotdb("root.a.b.c"), // covered + iotdb("root.x.y.z") // not covered + ); + final TreePattern exclusion = iotdb("root.a.**"); // only covers the first one + + // 2 inclusion paths, 1 covered + final int[] counts = TreePattern.checkAndLogPatternCoverage(inclusion, exclusion); + assertArrayEquals(new int[] {1, 2}, counts); + } + + @Test + public void testNoCoverage() { + final TreePattern inclusion = iotdb("root.a.b.c"); + final TreePattern exclusion = iotdb("root.x.y.z"); + + // 1 inclusion path, 0 covered + final int[] counts = TreePattern.checkAndLogPatternCoverage(inclusion, exclusion); + assertArrayEquals(new int[] {0, 1}, counts); + } + + @Test + public void testFullCoveragePrefix() { + // Prefix "root.a.b" corresponds to 4 variants + // (root.a.b, root.a.b*, root.a.b.**, root.a.b*.**) + final TreePattern inclusion = prefix("root.a.b"); + // IoTDB "root.a.**" should cover all 4 variants + final TreePattern exclusion = iotdb("root.a.**"); + + // Verify that a PipeException is thrown + final PipeException e = + assertThrows( + PipeException.class, + () -> TreePattern.checkAndLogPatternCoverage(inclusion, exclusion)); + } + + @Test + public void testPartialCoveragePrefix() { + // Prefix "root.a.b" corresponds to 4 variants + final TreePattern inclusion = prefix("root.a.b"); + // IoTDB "root.a.b" only includes the exact "root.a.b" path + final TreePattern exclusion = iotdb("root.a.b"); + + // 4 inclusion paths, 1 covered + final int[] counts = TreePattern.checkAndLogPatternCoverage(inclusion, exclusion); + assertArrayEquals(new int[] {1, 4}, counts); + } + + @Test + public void testPrefixNoCoverage() { + final TreePattern inclusion = prefix("root.a.b"); + final TreePattern exclusion = iotdb("root.x.y.z"); + + // 4 inclusion paths, 0 covered + final int[] counts = TreePattern.checkAndLogPatternCoverage(inclusion, exclusion); + assertArrayEquals(new int[] {0, 4}, counts); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeStatementTreePatternParseVisitorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeStatementTreePatternParseVisitorTest.java index 85ae98ac26a85..2dc0a8b9332da 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeStatementTreePatternParseVisitorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeStatementTreePatternParseVisitorTest.java @@ -23,7 +23,9 @@ import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.WithExclusionIoTDBTreePattern; import org.apache.iotdb.commons.schema.view.viewExpression.leaf.TimeSeriesViewOperand; import org.apache.iotdb.db.pipe.receiver.visitor.PipeStatementTreePatternParseVisitor; import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterTimeSeriesStatement; @@ -44,27 +46,44 @@ public class PipeStatementTreePatternParseVisitorTest { - private final UnionIoTDBTreePattern prefixPathPattern = + private final IoTDBTreePatternOperations prefixPathPattern = new UnionIoTDBTreePattern(new IoTDBTreePattern("root.db.device.**")); - private final UnionIoTDBTreePattern fullPathPattern = + private final IoTDBTreePatternOperations fullPathPattern = new UnionIoTDBTreePattern(new IoTDBTreePattern("root.db.device.s1")); - private final UnionIoTDBTreePattern multiplePathPattern = + private final IoTDBTreePatternOperations multiplePathPattern = new UnionIoTDBTreePattern( Arrays.asList( new IoTDBTreePattern("root.db.device.s1"), new IoTDBTreePattern("root.db.device.s2"))); + private final IoTDBTreePatternOperations exclusionPattern = + new WithExclusionIoTDBTreePattern( + new UnionIoTDBTreePattern( + new IoTDBTreePattern("root.db.device.**")), // Inclusion: root.db.device.** + new UnionIoTDBTreePattern( + new IoTDBTreePattern("root.db.device.s2")) // Exclusion: root.db.device.s2 + ); @Test public void testCreateTimeSeries() throws IllegalPathException { - final CreateTimeSeriesStatement createTimeSeriesStatement = new CreateTimeSeriesStatement(); - createTimeSeriesStatement.setPath(new MeasurementPath("root.db.device.s1")); - createTimeSeriesStatement.setDataType(TSDataType.FLOAT); - createTimeSeriesStatement.setEncoding(TSEncoding.RLE); - createTimeSeriesStatement.setCompressor(CompressionType.SNAPPY); - createTimeSeriesStatement.setProps(Collections.emptyMap()); - createTimeSeriesStatement.setTags(Collections.emptyMap()); - createTimeSeriesStatement.setAttributes(Collections.emptyMap()); - createTimeSeriesStatement.setAlias("a1"); + final CreateTimeSeriesStatement createTimeSeriesStatementS1 = new CreateTimeSeriesStatement(); + createTimeSeriesStatementS1.setPath(new MeasurementPath("root.db.device.s1")); + createTimeSeriesStatementS1.setDataType(TSDataType.FLOAT); + createTimeSeriesStatementS1.setEncoding(TSEncoding.RLE); + createTimeSeriesStatementS1.setCompressor(CompressionType.SNAPPY); + createTimeSeriesStatementS1.setProps(Collections.emptyMap()); + createTimeSeriesStatementS1.setTags(Collections.emptyMap()); + createTimeSeriesStatementS1.setAttributes(Collections.emptyMap()); + createTimeSeriesStatementS1.setAlias("a1"); + + final CreateTimeSeriesStatement createTimeSeriesStatementS2 = new CreateTimeSeriesStatement(); + createTimeSeriesStatementS2.setPath(new MeasurementPath("root.db.device.s2")); + createTimeSeriesStatementS2.setDataType(TSDataType.INT32); + createTimeSeriesStatementS2.setEncoding(TSEncoding.PLAIN); + createTimeSeriesStatementS2.setCompressor(CompressionType.SNAPPY); + createTimeSeriesStatementS2.setProps(Collections.emptyMap()); + createTimeSeriesStatementS2.setTags(Collections.emptyMap()); + createTimeSeriesStatementS2.setAttributes(Collections.emptyMap()); + createTimeSeriesStatementS2.setAlias("a2"); final CreateTimeSeriesStatement createTimeSeriesStatementToFilter = new CreateTimeSeriesStatement(); @@ -75,90 +94,128 @@ public void testCreateTimeSeries() throws IllegalPathException { createTimeSeriesStatementToFilter.setProps(Collections.emptyMap()); createTimeSeriesStatementToFilter.setTags(Collections.emptyMap()); createTimeSeriesStatementToFilter.setAttributes(Collections.emptyMap()); - createTimeSeriesStatementToFilter.setAlias("a2"); + createTimeSeriesStatementToFilter.setAlias("a3"); Assert.assertEquals( - createTimeSeriesStatement, + createTimeSeriesStatementS1, new PipeStatementTreePatternParseVisitor() - .visitCreateTimeseries(createTimeSeriesStatement, prefixPathPattern) + .visitCreateTimeseries(createTimeSeriesStatementS1, prefixPathPattern) .orElseThrow(AssertionError::new)); Assert.assertFalse( new PipeStatementTreePatternParseVisitor() .visitCreateTimeseries(createTimeSeriesStatementToFilter, prefixPathPattern) .isPresent()); + Assert.assertEquals( - createTimeSeriesStatement, + createTimeSeriesStatementS1, new PipeStatementTreePatternParseVisitor() - .visitCreateTimeseries(createTimeSeriesStatement, multiplePathPattern) + .visitCreateTimeseries(createTimeSeriesStatementS1, multiplePathPattern) .orElseThrow(AssertionError::new)); Assert.assertFalse( new PipeStatementTreePatternParseVisitor() .visitCreateTimeseries(createTimeSeriesStatementToFilter, multiplePathPattern) .isPresent()); + + Assert.assertEquals( + createTimeSeriesStatementS1, + new PipeStatementTreePatternParseVisitor() + .visitCreateTimeseries(createTimeSeriesStatementS1, exclusionPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + new PipeStatementTreePatternParseVisitor() + .visitCreateTimeseries(createTimeSeriesStatementS2, exclusionPattern) + .isPresent()); + Assert.assertFalse( + new PipeStatementTreePatternParseVisitor() + .visitCreateTimeseries(createTimeSeriesStatementToFilter, exclusionPattern) + .isPresent()); } @Test public void testCreateAlignedTimeSeries() throws IllegalPathException { - final CreateAlignedTimeSeriesStatement expectedCreateAlignedTimeSeriesStatement = - new CreateAlignedTimeSeriesStatement(); - expectedCreateAlignedTimeSeriesStatement.setDevicePath(new PartialPath("root.db.device")); - expectedCreateAlignedTimeSeriesStatement.setMeasurements(Collections.singletonList("s1")); - expectedCreateAlignedTimeSeriesStatement.setDataTypes( - Collections.singletonList(TSDataType.FLOAT)); - expectedCreateAlignedTimeSeriesStatement.setEncodings( - Collections.singletonList(TSEncoding.RLE)); - expectedCreateAlignedTimeSeriesStatement.setCompressors( - Collections.singletonList(CompressionType.SNAPPY)); - expectedCreateAlignedTimeSeriesStatement.setTagsList( - Collections.singletonList(Collections.emptyMap())); - expectedCreateAlignedTimeSeriesStatement.setAttributesList( - Collections.singletonList(Collections.emptyMap())); - expectedCreateAlignedTimeSeriesStatement.setAliasList(Collections.singletonList("a1")); - - final CreateAlignedTimeSeriesStatement originalCreateAlignedTimeSeriesStatement = - new CreateAlignedTimeSeriesStatement(); - originalCreateAlignedTimeSeriesStatement.setDevicePath(new PartialPath("root.db.device")); - originalCreateAlignedTimeSeriesStatement.setMeasurements(Arrays.asList("s1", "s2")); - originalCreateAlignedTimeSeriesStatement.setDataTypes( - Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN)); - originalCreateAlignedTimeSeriesStatement.setEncodings( - Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN)); - originalCreateAlignedTimeSeriesStatement.setCompressors( - Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY)); - originalCreateAlignedTimeSeriesStatement.setTagsList( - Arrays.asList(Collections.emptyMap(), Collections.emptyMap())); - originalCreateAlignedTimeSeriesStatement.setAttributesList( - Arrays.asList(Collections.emptyMap(), Collections.emptyMap())); - originalCreateAlignedTimeSeriesStatement.setAliasList(Arrays.asList("a1", "a2")); + final CreateAlignedTimeSeriesStatement expectedS1Only = new CreateAlignedTimeSeriesStatement(); + expectedS1Only.setDevicePath(new PartialPath("root.db.device")); + expectedS1Only.setMeasurements(Collections.singletonList("s1")); + expectedS1Only.setDataTypes(Collections.singletonList(TSDataType.FLOAT)); + expectedS1Only.setEncodings(Collections.singletonList(TSEncoding.RLE)); + expectedS1Only.setCompressors(Collections.singletonList(CompressionType.SNAPPY)); + expectedS1Only.setTagsList(Collections.singletonList(Collections.emptyMap())); + expectedS1Only.setAttributesList(Collections.singletonList(Collections.emptyMap())); + expectedS1Only.setAliasList(Collections.singletonList("a1")); + + final CreateAlignedTimeSeriesStatement originalS1S2 = new CreateAlignedTimeSeriesStatement(); + originalS1S2.setDevicePath(new PartialPath("root.db.device")); + originalS1S2.setMeasurements(Arrays.asList("s1", "s2")); + originalS1S2.setDataTypes(Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN)); + originalS1S2.setEncodings(Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN)); + originalS1S2.setCompressors(Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY)); + originalS1S2.setTagsList(Arrays.asList(Collections.emptyMap(), Collections.emptyMap())); + originalS1S2.setAttributesList(Arrays.asList(Collections.emptyMap(), Collections.emptyMap())); + originalS1S2.setAliasList(Arrays.asList("a1", "a2")); + + final CreateAlignedTimeSeriesStatement originalS1S2S3 = new CreateAlignedTimeSeriesStatement(); + originalS1S2S3.setDevicePath(new PartialPath("root.db.device")); + originalS1S2S3.setMeasurements(Arrays.asList("s1", "s2", "s3")); + originalS1S2S3.setDataTypes( + Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN, TSDataType.INT32)); + originalS1S2S3.setEncodings(Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN, TSEncoding.RLE)); + originalS1S2S3.setCompressors( + Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY, CompressionType.SNAPPY)); + originalS1S2S3.setTagsList( + Arrays.asList(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap())); + originalS1S2S3.setAttributesList( + Arrays.asList(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap())); + originalS1S2S3.setAliasList(Arrays.asList("a1", "a2", "a3")); + + Assert.assertEquals( + expectedS1Only, + new PipeStatementTreePatternParseVisitor() + .visitCreateAlignedTimeseries(originalS1S2, fullPathPattern) + .orElseThrow(AssertionError::new)); Assert.assertEquals( - expectedCreateAlignedTimeSeriesStatement, + originalS1S2, new PipeStatementTreePatternParseVisitor() - .visitCreateAlignedTimeseries(originalCreateAlignedTimeSeriesStatement, fullPathPattern) + .visitCreateAlignedTimeseries(originalS1S2S3, multiplePathPattern) .orElseThrow(AssertionError::new)); + final CreateAlignedTimeSeriesStatement expectedS1S3 = new CreateAlignedTimeSeriesStatement(); + expectedS1S3.setDevicePath(new PartialPath("root.db.device")); + expectedS1S3.setMeasurements(Arrays.asList("s1", "s3")); + expectedS1S3.setDataTypes(Arrays.asList(TSDataType.FLOAT, TSDataType.INT32)); + expectedS1S3.setEncodings(Arrays.asList(TSEncoding.RLE, TSEncoding.RLE)); + expectedS1S3.setCompressors(Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY)); + expectedS1S3.setTagsList(Arrays.asList(Collections.emptyMap(), Collections.emptyMap())); + expectedS1S3.setAttributesList(Arrays.asList(Collections.emptyMap(), Collections.emptyMap())); + expectedS1S3.setAliasList(Arrays.asList("a1", "a3")); + Assert.assertEquals( - originalCreateAlignedTimeSeriesStatement, + expectedS1S3, new PipeStatementTreePatternParseVisitor() - .visitCreateAlignedTimeseries( - originalCreateAlignedTimeSeriesStatement, multiplePathPattern) + .visitCreateAlignedTimeseries(originalS1S2S3, exclusionPattern) .orElseThrow(AssertionError::new)); } @Test public void testAlterTimeSeries() throws IllegalPathException { - final AlterTimeSeriesStatement alterTimeSeriesStatement = new AlterTimeSeriesStatement(true); - final Map attributeMap = Collections.singletonMap("k1", "v1"); - alterTimeSeriesStatement.setPath(new MeasurementPath("root.db.device.s1")); - alterTimeSeriesStatement.setAlterMap(attributeMap); - alterTimeSeriesStatement.setTagsMap(Collections.emptyMap()); - alterTimeSeriesStatement.setAttributesMap(attributeMap); - alterTimeSeriesStatement.setAlias(""); + + final AlterTimeSeriesStatement alterTimeSeriesStatementS1 = new AlterTimeSeriesStatement(true); + alterTimeSeriesStatementS1.setPath(new MeasurementPath("root.db.device.s1")); + alterTimeSeriesStatementS1.setAlterMap(attributeMap); + alterTimeSeriesStatementS1.setTagsMap(Collections.emptyMap()); + alterTimeSeriesStatementS1.setAttributesMap(attributeMap); + alterTimeSeriesStatementS1.setAlias(""); + + final AlterTimeSeriesStatement alterTimeSeriesStatementS2 = new AlterTimeSeriesStatement(true); + alterTimeSeriesStatementS2.setPath(new MeasurementPath("root.db.device.s2")); // Different path + alterTimeSeriesStatementS2.setAlterMap(attributeMap); + alterTimeSeriesStatementS2.setTagsMap(Collections.emptyMap()); + alterTimeSeriesStatementS2.setAttributesMap(attributeMap); + alterTimeSeriesStatementS2.setAlias(""); final AlterTimeSeriesStatement alterTimeSeriesStatementToFilter = new AlterTimeSeriesStatement(true); - alterTimeSeriesStatementToFilter.setPath(new MeasurementPath("root.db1.device.s1")); alterTimeSeriesStatementToFilter.setAlterMap(attributeMap); alterTimeSeriesStatementToFilter.setTagsMap(Collections.emptyMap()); @@ -166,23 +223,38 @@ public void testAlterTimeSeries() throws IllegalPathException { alterTimeSeriesStatementToFilter.setAlias(""); Assert.assertEquals( - alterTimeSeriesStatement, + alterTimeSeriesStatementS1, new PipeStatementTreePatternParseVisitor() - .visitAlterTimeSeries(alterTimeSeriesStatement, fullPathPattern) + .visitAlterTimeSeries(alterTimeSeriesStatementS1, fullPathPattern) .orElseThrow(AssertionError::new)); Assert.assertFalse( new PipeStatementTreePatternParseVisitor() .visitAlterTimeSeries(alterTimeSeriesStatementToFilter, prefixPathPattern) .isPresent()); + Assert.assertEquals( - alterTimeSeriesStatement, + alterTimeSeriesStatementS1, new PipeStatementTreePatternParseVisitor() - .visitAlterTimeSeries(alterTimeSeriesStatement, multiplePathPattern) + .visitAlterTimeSeries(alterTimeSeriesStatementS1, multiplePathPattern) .orElseThrow(AssertionError::new)); Assert.assertFalse( new PipeStatementTreePatternParseVisitor() .visitAlterTimeSeries(alterTimeSeriesStatementToFilter, multiplePathPattern) .isPresent()); + + Assert.assertEquals( + alterTimeSeriesStatementS1, + new PipeStatementTreePatternParseVisitor() + .visitAlterTimeSeries(alterTimeSeriesStatementS1, exclusionPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + new PipeStatementTreePatternParseVisitor() + .visitAlterTimeSeries(alterTimeSeriesStatementS2, exclusionPattern) + .isPresent()); + Assert.assertFalse( + new PipeStatementTreePatternParseVisitor() + .visitAlterTimeSeries(alterTimeSeriesStatementToFilter, exclusionPattern) + .isPresent()); } @Test diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/PipePlanTreePatternParseVisitorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/PipePlanTreePatternParseVisitorTest.java index f4aa8f56a7efa..8aebe3ea0f78b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/PipePlanTreePatternParseVisitorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/PipePlanTreePatternParseVisitorTest.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; import org.apache.iotdb.commons.schema.view.viewExpression.leaf.TimeSeriesViewOperand; @@ -57,11 +58,11 @@ public class PipePlanTreePatternParseVisitorTest { - private final UnionIoTDBTreePattern prefixPathPattern = + private final IoTDBTreePatternOperations prefixPathPattern = new UnionIoTDBTreePattern(new IoTDBTreePattern("root.db.device.**")); - private final UnionIoTDBTreePattern fullPathPattern = + private final IoTDBTreePatternOperations fullPathPattern = new UnionIoTDBTreePattern(new IoTDBTreePattern("root.db.device.s1")); - private final UnionIoTDBTreePattern multiplePathPattern = + private final IoTDBTreePatternOperations multiplePathPattern = new UnionIoTDBTreePattern( Arrays.asList( new IoTDBTreePattern("root.db.device.s1"), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSourceConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSourceConstant.java index 95eb6cb895591..d13bdf7d046cc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSourceConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSourceConstant.java @@ -60,6 +60,11 @@ public class PipeSourceConstant { public static final String EXTRACTOR_PATTERN_FORMAT_IOTDB_VALUE = "iotdb"; public static final String EXTRACTOR_PATTERN_PREFIX_DEFAULT_VALUE = "root"; public static final String EXTRACTOR_PATTERN_IOTDB_DEFAULT_VALUE = "root.**"; + public static final String EXTRACTOR_PATTERN_EXCLUSION_KEY = "extractor.pattern.exclusion"; + public static final String SOURCE_PATTERN_EXCLUSION_KEY = "source.pattern.exclusion"; + public static final String EXTRACTOR_PATH_EXCLUSION_KEY = "extractor.path.exclusion"; + public static final String SOURCE_PATH_EXCLUSION_KEY = "source.path.exclusion"; + public static final String EXTRACTOR_DATABASE_NAME_KEY = "extractor.database-name"; public static final String SOURCE_DATABASE_NAME_KEY = "source.database-name"; public static final String EXTRACTOR_TABLE_NAME_KEY = "extractor.table-name"; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/IoTDBTreePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/IoTDBTreePattern.java index 80e7df9ee5da2..38d7aafccc7c0 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/IoTDBTreePattern.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/IoTDBTreePattern.java @@ -36,14 +36,17 @@ import java.util.List; import java.util.Objects; -public class IoTDBTreePattern extends SingleTreePattern { +public class IoTDBTreePattern extends IoTDBTreePatternOperations { + private final String pattern; private final PartialPath patternPartialPath; + private static volatile DevicePathGetter devicePathGetter = PartialPath::new; private static volatile MeasurementPathGetter measurementPathGetter = MeasurementPath::new; public IoTDBTreePattern(final boolean isTreeModelDataAllowedToBeCaptured, final String pattern) { - super(isTreeModelDataAllowedToBeCaptured, pattern); + super(isTreeModelDataAllowedToBeCaptured); + this.pattern = pattern != null ? pattern : getDefaultPattern(); try { patternPartialPath = new PartialPath(getPattern()); @@ -56,11 +59,22 @@ public IoTDBTreePattern(final String pattern) { this(true, pattern); } - @Override - public String getDefaultPattern() { + private String getDefaultPattern() { return PipeSourceConstant.EXTRACTOR_PATTERN_IOTDB_DEFAULT_VALUE; } + //////////////////////////// Tree Pattern Operations //////////////////////////// + + @Override + public String getPattern() { + return pattern; + } + + @Override + public boolean isRoot() { + return Objects.isNull(pattern) || this.pattern.equals(this.getDefaultPattern()); + } + @Override public boolean isLegal() { if (!pattern.startsWith("root")) { @@ -129,12 +143,20 @@ public boolean matchesMeasurement(final IDeviceID device, final String measureme } } + @Override + public List getBaseInclusionPaths() { + return Collections.singletonList(patternPartialPath); + } + + //////////////////////////// IoTDB Tree Pattern Operations //////////////////////////// + /** * Check if the {@link TreePattern} matches the given prefix path. In schema transmission, this * can be used to detect whether the given path can act as a parent path of the {@link * TreePattern}, and to transmit possibly used schemas like database creation and template * setting. */ + @Override public boolean matchPrefixPath(final String path) { try { return patternPartialPath.matchPrefixPath(new PartialPath(path)); @@ -146,6 +168,7 @@ public boolean matchPrefixPath(final String path) { /** * This is the precise form of the device overlap and is used only be device template transfer. */ + @Override public boolean matchDevice(final String devicePath) { try { return patternPartialPath.overlapWith(new MeasurementPath(devicePath, "*")); @@ -158,6 +181,7 @@ public boolean matchDevice(final String devicePath) { * Return if the given tail node matches the pattern's tail node. Caller shall ensure that it is a * prefix or full path pattern. */ + @Override public boolean matchTailNode(final String tailNode) { return !isFullPath() || patternPartialPath.getTailNode().equals(tailNode); } @@ -166,6 +190,7 @@ public boolean matchTailNode(final String tailNode) { * Get the intersection of the given {@link PartialPath} and the {@link TreePattern}, Only used by * schema transmission. Caller shall ensure that it is a prefix or full path pattern. */ + @Override public List getIntersection(final PartialPath partialPath) { if (isFullPath()) { return partialPath.matchFullPath(patternPartialPath) @@ -179,6 +204,7 @@ public List getIntersection(final PartialPath partialPath) { * Get the intersection of the given {@link PathPatternTree} and the {@link TreePattern}. Only * used by schema transmission. Caller shall ensure that it is a prefix or full path pattern. */ + @Override public PathPatternTree getIntersection(final PathPatternTree patternTree) { final PathPatternTree thisPatternTree = new PathPatternTree(); thisPatternTree.appendPathPattern(patternPartialPath); @@ -186,7 +212,17 @@ public PathPatternTree getIntersection(final PathPatternTree patternTree) { return patternTree.intersectWithFullPathPrefixTree(thisPatternTree); } - public boolean isPrefix() { + @Override + public boolean isPrefixOrFullPath() { + return isPrefix() || isFullPath(); + } + + @Override + public boolean mayMatchMultipleTimeSeriesInOneDevice() { + return PathPatternUtil.hasWildcard(patternPartialPath.getTailNode()); + } + + private boolean isPrefix() { return PathPatternUtil.isMultiLevelMatchWildcard(patternPartialPath.getTailNode()) && !new PartialPath( Arrays.copyOfRange( @@ -194,13 +230,11 @@ public boolean isPrefix() { .hasWildcard(); } - public boolean isFullPath() { + private boolean isFullPath() { return !patternPartialPath.hasWildcard(); } - public boolean mayMatchMultipleTimeSeriesInOneDevice() { - return PathPatternUtil.hasWildcard(patternPartialPath.getTailNode()); - } + //////////////////////////// Getter //////////////////////////// public static void setDevicePathGetter(final DevicePathGetter devicePathGetter) { IoTDBTreePattern.devicePathGetter = devicePathGetter; @@ -210,11 +244,6 @@ public static void setMeasurementPathGetter(final MeasurementPathGetter measurem IoTDBTreePattern.measurementPathGetter = measurementPathGetter; } - @Override - public String toString() { - return "IoTDBPipePattern" + super.toString(); - } - public interface DevicePathGetter { PartialPath apply(final IDeviceID deviceId) throws IllegalPathException; } @@ -223,4 +252,15 @@ public interface MeasurementPathGetter { MeasurementPath apply(final IDeviceID deviceId, final String measurement) throws IllegalPathException; } + + //////////////////////////// Object //////////////////////////// + + @Override + public String toString() { + return "IoTDBTreePattern{pattern='" + + pattern + + "', isTreeModelDataAllowedToBeCaptured=" + + isTreeModelDataAllowedToBeCaptured + + '}'; + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/IoTDBTreePatternOperations.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/IoTDBTreePatternOperations.java new file mode 100644 index 0000000000000..1d99c0d90a225 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/IoTDBTreePatternOperations.java @@ -0,0 +1,52 @@ +/* + * 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.commons.pipe.datastructure.pattern; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; + +import java.util.List; + +/** + * An interface for TreePattern classes that support IoTDB-specific path matching operations, such + * as those used by schema-aware visitors. + */ +public abstract class IoTDBTreePatternOperations extends TreePattern { + + protected IoTDBTreePatternOperations(final boolean isTreeModelDataAllowedToBeCaptured) { + super(isTreeModelDataAllowedToBeCaptured); + } + + //////////////////////////// IoTDB Pattern Operations //////////////////////////// + + public abstract boolean matchPrefixPath(final String path); + + public abstract boolean matchDevice(final String devicePath); + + public abstract boolean matchTailNode(final String tailNode); + + public abstract List getIntersection(final PartialPath partialPath); + + public abstract PathPatternTree getIntersection(final PathPatternTree patternTree); + + public abstract boolean isPrefixOrFullPath(); + + public abstract boolean mayMatchMultipleTimeSeriesInOneDevice(); +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/PrefixTreePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/PrefixTreePattern.java index 5a3763afaaed3..73d50393fb696 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/PrefixTreePattern.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/PrefixTreePattern.java @@ -19,7 +19,9 @@ package org.apache.iotdb.commons.pipe.datastructure.pattern; +import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant; import org.apache.iotdb.commons.utils.PathUtils; @@ -27,23 +29,39 @@ import org.apache.tsfile.external.commons.lang3.StringUtils; import org.apache.tsfile.file.metadata.IDeviceID; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; -public class PrefixTreePattern extends SingleTreePattern { +public class PrefixTreePattern extends TreePattern { + + private final String pattern; public PrefixTreePattern(final boolean isTreeModelDataAllowedToBeCaptured, final String pattern) { - super(isTreeModelDataAllowedToBeCaptured, pattern); + super(isTreeModelDataAllowedToBeCaptured); + this.pattern = pattern != null ? pattern : getDefaultPattern(); } public PrefixTreePattern(final String pattern) { this(true, pattern); } - @Override - public String getDefaultPattern() { + private String getDefaultPattern() { return PipeSourceConstant.EXTRACTOR_PATTERN_PREFIX_DEFAULT_VALUE; } + @Override + public String getPattern() { + return pattern; + } + + @Override + public boolean isRoot() { + return Objects.isNull(pattern) || this.pattern.equals(this.getDefaultPattern()); + } + @Override public boolean isLegal() { if (!pattern.startsWith("root")) { @@ -131,8 +149,50 @@ public boolean matchesMeasurement(final IDeviceID device, String measurement) { && dotAndMeasurement.startsWith(pattern.substring(deviceStr.length())); } + @Override + public List getBaseInclusionPaths() { + if (isRoot()) { + return Collections.singletonList(new PartialPath(new String[] {"root", "**"})); + } + + final List paths = new ArrayList<>(); + try { + // 1. "root.d1" + paths.add(new PartialPath(pattern)); + } catch (final IllegalPathException ignored) { + } + try { + // 2. "root.d1*" + paths.add(new PartialPath(pattern + "*")); + } catch (final IllegalPathException ignored) { + } + try { + // 3. "root.d1.**" + paths.add( + new PartialPath( + pattern + TsFileConstant.PATH_SEPARATOR + IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD)); + } catch (final IllegalPathException ignored) { + } + try { + // 4. "root.d1*.**" + paths.add( + new PartialPath( + pattern + + "*" + + TsFileConstant.PATH_SEPARATOR + + IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD)); + } catch (final IllegalPathException ignored) { + } + + return paths; + } + @Override public String toString() { - return "PrefixPipePattern" + super.toString(); + return "PrefixTreePattern{pattern='" + + pattern + + "', isTreeModelDataAllowedToBeCaptured=" + + isTreeModelDataAllowedToBeCaptured + + '}'; } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/SingleTreePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/SingleTreePattern.java deleted file mode 100644 index b88e760d76a9d..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/SingleTreePattern.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.pipe.datastructure.pattern; - -import java.util.Objects; - -public abstract class SingleTreePattern extends TreePattern { - - protected final String pattern; - - protected SingleTreePattern( - final boolean isTreeModelDataAllowedToBeCaptured, final String pattern) { - super(isTreeModelDataAllowedToBeCaptured); - this.pattern = pattern != null ? pattern : getDefaultPattern(); - } - - @Override - public boolean isSingle() { - return true; - } - - @Override - public String getPattern() { - return pattern; - } - - @Override - public boolean isRoot() { - return Objects.isNull(pattern) || this.pattern.equals(this.getDefaultPattern()); - } - - public abstract String getDefaultPattern(); - - @Override - public String toString() { - return "{pattern='" - + pattern - + "', isTreeModelDataAllowedToBeCaptured=" - + isTreeModelDataAllowedToBeCaptured - + '}'; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/TreePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/TreePattern.java index cc5997582d413..e9b0fbe2180e5 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/TreePattern.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/TreePattern.java @@ -19,9 +19,11 @@ package org.apache.iotdb.commons.pipe.datastructure.pattern; +import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.tsfile.file.metadata.IDeviceID; import org.slf4j.Logger; @@ -35,12 +37,16 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.EXTRACTOR_PATH_EXCLUSION_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.EXTRACTOR_PATH_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.EXTRACTOR_PATTERN_EXCLUSION_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.EXTRACTOR_PATTERN_FORMAT_IOTDB_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.EXTRACTOR_PATTERN_FORMAT_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.EXTRACTOR_PATTERN_FORMAT_PREFIX_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.EXTRACTOR_PATTERN_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.SOURCE_PATH_EXCLUSION_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.SOURCE_PATH_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.SOURCE_PATTERN_EXCLUSION_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.SOURCE_PATTERN_FORMAT_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant.SOURCE_PATTERN_KEY; @@ -58,6 +64,64 @@ public boolean isTreeModelDataAllowedToBeCaptured() { return isTreeModelDataAllowedToBeCaptured; } + //////////////////////////// Interface //////////////////////////// + + public abstract String getPattern(); + + public abstract boolean isRoot(); + + /** Check if this pattern is legal. Different pattern type may have different rules. */ + public abstract boolean isLegal(); + + /** Check if this pattern matches all time-series under a database. */ + public abstract boolean coversDb(final String db); + + /** Check if a device's all measurements are covered by this pattern. */ + public abstract boolean coversDevice(final IDeviceID device); + + /** + * Check if a database may have some measurements matched by the pattern. + * + * @return {@code true} if the pattern may overlap with the database, {@code false} otherwise. + */ + public abstract boolean mayOverlapWithDb(final String db); + + /** + * Check if a device may have some measurements matched by the pattern. + * + *

NOTE1: this is only called when {@link TreePattern#coversDevice} is {@code false}. + * + *

NOTE2: this is just a loose check and may have false positives. To further check if a + * measurement matches the pattern, please use {@link TreePattern#matchesMeasurement} after this. + */ + public abstract boolean mayOverlapWithDevice(final IDeviceID device); + + /** + * Check if a full path with device and measurement can be matched by pattern. + * + *

NOTE: this is only called when {@link TreePattern#mayOverlapWithDevice} is {@code true}. + */ + public abstract boolean matchesMeasurement(final IDeviceID device, final String measurement); + + /** + * Get all 'base' PartialPath patterns that this pattern represents, for the purpose of checking + * pattern coverage. + * + *

For IoTDB patterns, it's their direct PartialPath. + * + *

For Prefix patterns (e.g., "root.d1"), it's approximated as a union of PartialPaths to model + * its string-based matching: "root.d1", "root.d1*", "root.d1.**", and "root.d1*.**". + * + *

For Union patterns, it's a list from all sub-patterns. + * + *

For Exclusion patterns, it's the *effective* set of paths. + * + * @return A list of PartialPaths representing the inclusion paths. + */ + public abstract List getBaseInclusionPaths(); + + //////////////////////////// Utilities //////////////////////////// + public static List applyIndexesOnList( final int[] filteredIndexes, final List originalList) { return Objects.nonNull(originalList) @@ -66,7 +130,8 @@ public static List applyIndexesOnList( } /** - * Interpret from source parameters and get a {@link TreePattern}. + * Interpret from source parameters and get a {@link TreePattern}. This method parses both + * inclusion and exclusion patterns. * * @return The interpreted {@link TreePattern} which is not {@code null}. */ @@ -75,9 +140,165 @@ public static TreePattern parsePipePatternFromSourceParameters( final boolean isTreeModelDataAllowedToBeCaptured = isTreeModelDataAllowToBeCaptured(sourceParameters); - final String path = sourceParameters.getStringByKeys(EXTRACTOR_PATH_KEY, SOURCE_PATH_KEY); - final String pattern = - sourceParameters.getStringByKeys(EXTRACTOR_PATTERN_KEY, SOURCE_PATTERN_KEY); + // 1. Define the default inclusion pattern (matches all, "root.**") + // This is used if no inclusion patterns are specified. + final TreePattern defaultInclusionPattern = + buildUnionPattern( + isTreeModelDataAllowedToBeCaptured, + Collections.singletonList( + new IoTDBTreePattern(isTreeModelDataAllowedToBeCaptured, null))); + + // 2. Parse INCLUSION patterns using the helper + final TreePattern inclusionPattern = + parsePatternUnion( + sourceParameters, + isTreeModelDataAllowedToBeCaptured, + // 'path' keys (IoTDB wildcard) + EXTRACTOR_PATH_KEY, + SOURCE_PATH_KEY, + // 'pattern' keys (Prefix or IoTDB via format) + EXTRACTOR_PATTERN_KEY, + SOURCE_PATTERN_KEY, + // Default pattern if no keys are found + defaultInclusionPattern); + + // 3. Parse EXCLUSION patterns using the helper + final TreePattern exclusionPattern = + parsePatternUnion( + sourceParameters, + isTreeModelDataAllowedToBeCaptured, + // 'path.exclusion' keys (IoTDB wildcard) + EXTRACTOR_PATH_EXCLUSION_KEY, + SOURCE_PATH_EXCLUSION_KEY, + // 'pattern.exclusion' keys (Prefix) + EXTRACTOR_PATTERN_EXCLUSION_KEY, + SOURCE_PATTERN_EXCLUSION_KEY, + // Default for exclusion is "match nothing" (null) + null); + + // 4. Combine inclusion and exclusion + if (exclusionPattern == null) { + // No exclusion defined, return the inclusion pattern directly + return inclusionPattern; + } else { + // If both inclusion and exclusion patterns support IoTDB operations, + // use the specialized ExclusionIoTDBTreePattern + if (inclusionPattern instanceof IoTDBTreePatternOperations + && exclusionPattern instanceof IoTDBTreePatternOperations) { + return new WithExclusionIoTDBTreePattern( + isTreeModelDataAllowedToBeCaptured, + (IoTDBTreePatternOperations) inclusionPattern, + (IoTDBTreePatternOperations) exclusionPattern); + } + // Both are defined, wrap them in an ExclusionTreePattern + return new WithExclusionTreePattern( + isTreeModelDataAllowedToBeCaptured, inclusionPattern, exclusionPattern); + } + } + + /** + * The main entry point for parsing a pattern string. This method can handle simple patterns + * ("root.a"), union patterns ("root.a,root.b"), and exclusion patterns ("INCLUSION(root.a), + * EXCLUSION(root.b)"). + */ + public static TreePattern parsePatternFromString( + final String patternString, + final boolean isTreeModelDataAllowedToBeCaptured, + final Function basePatternSupplier) { + final String trimmedPattern = (patternString == null) ? "" : patternString.trim(); + if (trimmedPattern.isEmpty()) { + return basePatternSupplier.apply(""); + } + + // 1. Check if it's an Exclusion pattern + if (trimmedPattern.startsWith("INCLUSION(") && trimmedPattern.endsWith(")")) { + // Find the closing parenthesis for "INCLUSION(...)" + final int inclusionEndIndex = + findMatchingParenthesis(trimmedPattern, "INCLUSION(".length() - 1); + if (inclusionEndIndex == -1) { + // Malformed, treat as a normal pattern + return buildUnionPattern( + isTreeModelDataAllowedToBeCaptured, + parseMultiplePatterns(trimmedPattern, basePatternSupplier)); + } + + // Look for the ", EXCLUSION(" part + final String remaining = trimmedPattern.substring(inclusionEndIndex + 1).trim(); + if (!remaining.startsWith(", EXCLUSION(")) { + // Malformed, treat as a normal pattern + return buildUnionPattern( + isTreeModelDataAllowedToBeCaptured, + parseMultiplePatterns(trimmedPattern, basePatternSupplier)); + } + + try { + // Extract the string inside INCLUSION(...) + final String inclusionSubstring = + trimmedPattern.substring("INCLUSION(".length(), inclusionEndIndex); + + // Extract the string inside EXCLUSION(...) + final String exclusionSubstring = + trimmedPattern.substring( + inclusionEndIndex + ", EXCLUSION(".length() + 1, trimmedPattern.length() - 1); + + // 2. Parse recursively + final TreePattern inclusionPattern = + parsePatternFromString( + inclusionSubstring, isTreeModelDataAllowedToBeCaptured, basePatternSupplier); + final TreePattern exclusionPattern = + parsePatternFromString( + exclusionSubstring, isTreeModelDataAllowedToBeCaptured, basePatternSupplier); + + // 3. Build ExclusionTreePattern + if (inclusionPattern instanceof IoTDBTreePatternOperations + && exclusionPattern instanceof IoTDBTreePatternOperations) { + return new WithExclusionIoTDBTreePattern( + isTreeModelDataAllowedToBeCaptured, + (IoTDBTreePatternOperations) inclusionPattern, + (IoTDBTreePatternOperations) exclusionPattern); + } + return new WithExclusionTreePattern( + isTreeModelDataAllowedToBeCaptured, inclusionPattern, exclusionPattern); + } catch (final Exception e) { + // Error during parsing (e.g., index out of bounds), treat as a normal pattern + return buildUnionPattern( + isTreeModelDataAllowedToBeCaptured, + parseMultiplePatterns(trimmedPattern, basePatternSupplier)); + } + } + + // 4. Not an Exclusion pattern, treat as a normal pattern + return buildUnionPattern( + isTreeModelDataAllowedToBeCaptured, + parseMultiplePatterns(trimmedPattern, basePatternSupplier)); + } + + /** + * A private helper method to parse a set of 'path' and 'pattern' keys into a single union + * TreePattern. This contains the original logic of parsePipePatternFromSourceParameters. + * + * @param sourceParameters The source parameters. + * @param isTreeModelDataAllowedToBeCaptured Flag for TreePattern constructor. + * @param extractorPathKey Key for extractor path (e.g., "extractor.path"). + * @param sourcePathKey Key for source path (e.g., "source.path"). + * @param extractorPatternKey Key for extractor pattern (e.g., "extractor.pattern"). + * @param sourcePatternKey Key for source pattern (e.g., "source.pattern"). + * @param defaultPattern The pattern to return if both path and pattern are null. If this + * parameter is null, this method returns null. + * @return The parsed TreePattern, or defaultPattern, or null if defaultPattern is null and no + * patterns are specified. + */ + private static TreePattern parsePatternUnion( + final PipeParameters sourceParameters, + final boolean isTreeModelDataAllowedToBeCaptured, + final String extractorPathKey, + final String sourcePathKey, + final String extractorPatternKey, + final String sourcePatternKey, + final TreePattern defaultPattern) { + + final String path = sourceParameters.getStringByKeys(extractorPathKey, sourcePathKey); + final String pattern = sourceParameters.getStringByKeys(extractorPatternKey, sourcePatternKey); // 1. If both "source.path" and "source.pattern" are specified, their union will be used. if (path != null && pattern != null) { @@ -110,10 +331,8 @@ public static TreePattern parsePipePatternFromSourceParameters( } // 4. If neither "source.path" nor "source.pattern" is specified, - // this pipe source will match all data. - return buildUnionPattern( - isTreeModelDataAllowedToBeCaptured, - Collections.singletonList(new IoTDBTreePattern(isTreeModelDataAllowedToBeCaptured, null))); + // return the provided default pattern (which may be null). + return defaultPattern; } /** @@ -153,7 +372,7 @@ private static List parsePatternsFromPatternParameter( } } - public static List parseMultiplePatterns( + private static List parseMultiplePatterns( final String pattern, final Function patternSupplier) { if (pattern.isEmpty()) { return Collections.singletonList(patternSupplier.apply(pattern)); @@ -191,7 +410,7 @@ public static List parseMultiplePatterns( * are IoTDBTreePattern, it returns an IoTDBUnionTreePattern. Otherwise, it returns a general * UnionTreePattern. */ - public static TreePattern buildUnionPattern( + private static TreePattern buildUnionPattern( final boolean isTreeModelDataAllowedToBeCaptured, final List patterns) { // Check if all instances in the list are of type IoTDBTreePattern boolean allIoTDB = true; @@ -214,6 +433,28 @@ public static TreePattern buildUnionPattern( } } + /** Helper method to find the matching closing parenthesis, respecting backticks. */ + private static int findMatchingParenthesis(final String text, final int openParenIndex) { + int depth = 1; + boolean inBackticks = false; + + for (int i = openParenIndex + 1; i < text.length(); i++) { + final char c = text.charAt(i); + + if (c == '`') { + inBackticks = !inBackticks; + } else if (c == '(' && !inBackticks) { + depth++; + } else if (c == ')' && !inBackticks) { + depth--; + if (depth == 0) { + return i; // Found the matching closing parenthesis + } + } + } + return -1; // Not found + } + public static boolean isTreeModelDataAllowToBeCaptured(final PipeParameters sourceParameters) { return sourceParameters.getBooleanOrDefault( Arrays.asList( @@ -230,42 +471,86 @@ public static boolean isTreeModelDataAllowToBeCaptured(final PipeParameters sour .equals(SystemConstant.SQL_DIALECT_TREE_VALUE)); } - public abstract boolean isSingle(); - - public abstract String getPattern(); - - public abstract boolean isRoot(); - - /** Check if this pattern is legal. Different pattern type may have different rules. */ - public abstract boolean isLegal(); - - /** Check if this pattern matches all time-series under a database. */ - public abstract boolean coversDb(final String db); - - /** Check if a device's all measurements are covered by this pattern. */ - public abstract boolean coversDevice(final IDeviceID device); - /** - * Check if a database may have some measurements matched by the pattern. + * Checks if the exclusion pattern fully or partially covers the inclusion pattern and logs a + * warning or throws an exception. * - * @return {@code true} if the pattern may overlap with the database, {@code false} otherwise. - */ - public abstract boolean mayOverlapWithDb(final String db); - - /** - * Check if a device may have some measurements matched by the pattern. + *

This check uses the 'base' inclusion paths from both patterns. It is intended to catch + * configuration errors where an exclusion rule negates all or part of an inclusion rule. * - *

NOTE1: this is only called when {@link TreePattern#coversDevice} is {@code false}. + *

If the exclusion pattern *fully* covers the inclusion pattern (and the inclusion pattern is + * not empty), this method will throw a {@link PipeException} to prevent the creation of a pipe + * that matches nothing. * - *

NOTE2: this is just a loose check and may have false positives. To further check if a - * measurement matches the pattern, please use {@link TreePattern#matchesMeasurement} after this. - */ - public abstract boolean mayOverlapWithDevice(final IDeviceID device); - - /** - * Check if a full path with device and measurement can be matched by pattern. + *

If the exclusion pattern *partially* covers the inclusion pattern, a WARN log will be + * generated. * - *

NOTE: this is only called when {@link TreePattern#mayOverlapWithDevice} is {@code true}. + * @param inclusion The inclusion pattern. + * @param exclusion The exclusion pattern. + * @return An int array `[coveredCount, totalInclusionPaths]` for testing non-failing scenarios. + * @throws PipeException If the inclusion pattern is fully covered by the exclusion pattern. */ - public abstract boolean matchesMeasurement(final IDeviceID device, final String measurement); + public static int[] checkAndLogPatternCoverage( + final TreePattern inclusion, final TreePattern exclusion) throws PipeException { + if (inclusion == null || exclusion == null) { + return new int[] {0, 0}; + } + + final List inclusionPaths; + final List exclusionPaths; + int coveredCount = 0; + + try { + // Get the list of individual paths from both patterns + inclusionPaths = inclusion.getBaseInclusionPaths(); + exclusionPaths = exclusion.getBaseInclusionPaths(); + + if (inclusionPaths.isEmpty() || exclusionPaths.isEmpty()) { + // Nothing to check + return new int[] {0, inclusionPaths.size()}; + } + + for (final PartialPath incPath : inclusionPaths) { + // Check if *any* exclusion path includes this inclusion path + final boolean isCovered = + exclusionPaths.stream().anyMatch(excPath -> excPath.include(incPath)); + if (isCovered) { + coveredCount++; + } + } + } catch (final Exception e) { + // This check is best-effort. Do not fail construction. + LOGGER.warn( + "Pipe: Failed to perform pattern coverage check for inclusion [{}] and exclusion [{}].", + inclusion.getPattern(), + exclusion.getPattern(), + e); + // Return -1 to indicate failure in tests + return new int[] {-1, -1}; + } + + if (coveredCount == inclusionPaths.size() && !inclusionPaths.isEmpty()) { + // All inclusion paths are covered by the exclusion + final String msg = + String.format( + "Pipe: The provided exclusion pattern fully covers the inclusion pattern. " + + "This pipe pattern will match nothing. " + + "Inclusion: [%s], Exclusion: [%s]", + inclusion.getPattern(), exclusion.getPattern()); + LOGGER.warn(msg); + throw new PipeException(msg); + } else if (coveredCount > 0) { + // Some inclusion paths are covered + LOGGER.warn( + "Pipe: The provided exclusion pattern covers {} out of {} inclusion paths. " + + "These paths will be excluded. " + + "Inclusion: [{}], Exclusion: [{}]", + coveredCount, + inclusionPaths.size(), + inclusion.getPattern(), + exclusion.getPattern()); + } + + return new int[] {coveredCount, inclusionPaths.size()}; + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/UnionIoTDBTreePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/UnionIoTDBTreePattern.java index a44f4c9d6479b..1d047ed89b8e1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/UnionIoTDBTreePattern.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/UnionIoTDBTreePattern.java @@ -35,7 +35,7 @@ * Represents a union of multiple {@link IoTDBTreePattern}s. This specialized class ensures type * safety and provides access to methods specific to IoTDBTreePattern, such as getIntersection. */ -public class UnionIoTDBTreePattern extends TreePattern { +public class UnionIoTDBTreePattern extends IoTDBTreePatternOperations { private final List patterns; @@ -55,100 +55,109 @@ public UnionIoTDBTreePattern(final IoTDBTreePattern pattern) { this.patterns = Collections.singletonList(pattern); } - // ********************************************************************** - // IoTDBTreePattern-specific aggregated methods - // ********************************************************************** + //////////////////////////// Tree Pattern Operations //////////////////////////// - public boolean matchPrefixPath(final String path) { - return patterns.stream().anyMatch(p -> p.matchPrefixPath(path)); + @Override + public String getPattern() { + return patterns.stream().map(TreePattern::getPattern).collect(Collectors.joining(",")); } - public boolean matchDevice(final String devicePath) { - return patterns.stream().anyMatch(p -> p.matchDevice(devicePath)); + @Override + public boolean isRoot() { + return patterns.stream().anyMatch(TreePattern::isRoot); } - public boolean matchTailNode(final String tailNode) { - return patterns.stream().anyMatch(p -> p.matchTailNode(tailNode)); + @Override + public boolean isLegal() { + return patterns.stream().allMatch(TreePattern::isLegal); } - public List getIntersection(final PartialPath partialPath) { - final Set uniqueIntersections = new LinkedHashSet<>(); - for (final IoTDBTreePattern pattern : patterns) { - uniqueIntersections.addAll(pattern.getIntersection(partialPath)); - } - return new ArrayList<>(uniqueIntersections); + @Override + public boolean coversDb(final String db) { + return patterns.stream().anyMatch(p -> p.coversDb(db)); } - public PathPatternTree getIntersection(final PathPatternTree patternTree) { - final PathPatternTree resultTree = new PathPatternTree(); - for (final IoTDBTreePattern pattern : patterns) { - final PathPatternTree intersection = pattern.getIntersection(patternTree); - if (intersection.isEmpty()) { - continue; - } - intersection.getAllPathPatterns().forEach(resultTree::appendPathPattern); - } - resultTree.constructTree(); - return resultTree; + @Override + public boolean coversDevice(final IDeviceID device) { + return patterns.stream().anyMatch(p -> p.coversDevice(device)); } - public boolean isPrefixOrFullPath() { - return patterns.stream().allMatch(p -> p.isPrefix() || p.isFullPath()); + @Override + public boolean mayOverlapWithDb(final String db) { + return patterns.stream().anyMatch(p -> p.mayOverlapWithDb(db)); } - public boolean mayMatchMultipleTimeSeriesInOneDevice() { - return patterns.stream().anyMatch(IoTDBTreePattern::mayMatchMultipleTimeSeriesInOneDevice); + @Override + public boolean mayOverlapWithDevice(final IDeviceID device) { + return patterns.stream().anyMatch(p -> p.mayOverlapWithDevice(device)); } - // ********************************************************************** - // Implementation of abstract methods from TreePattern - // ********************************************************************** - @Override - public boolean isSingle() { - return patterns.size() == 1; + public boolean matchesMeasurement(final IDeviceID device, final String measurement) { + return patterns.stream().anyMatch(p -> p.matchesMeasurement(device, measurement)); } @Override - public String getPattern() { - return patterns.stream().map(TreePattern::getPattern).collect(Collectors.joining(",")); + public List getBaseInclusionPaths() { + final List paths = new ArrayList<>(); + for (final TreePattern p : patterns) { + paths.addAll(p.getBaseInclusionPaths()); + } + return paths; } + //////////////////////////// IoTDB Tree Pattern Operations //////////////////////////// + @Override - public boolean isRoot() { - return patterns.stream().anyMatch(TreePattern::isRoot); + public boolean matchPrefixPath(final String path) { + return patterns.stream().anyMatch(p -> p.matchPrefixPath(path)); } @Override - public boolean isLegal() { - return patterns.stream().allMatch(TreePattern::isLegal); + public boolean matchDevice(final String devicePath) { + return patterns.stream().anyMatch(p -> p.matchDevice(devicePath)); } @Override - public boolean coversDb(final String db) { - return patterns.stream().anyMatch(p -> p.coversDb(db)); + public boolean matchTailNode(final String tailNode) { + return patterns.stream().anyMatch(p -> p.matchTailNode(tailNode)); } @Override - public boolean coversDevice(final IDeviceID device) { - return patterns.stream().anyMatch(p -> p.coversDevice(device)); + public List getIntersection(final PartialPath partialPath) { + final Set uniqueIntersections = new LinkedHashSet<>(); + for (final IoTDBTreePattern pattern : patterns) { + uniqueIntersections.addAll(pattern.getIntersection(partialPath)); + } + return new ArrayList<>(uniqueIntersections); } @Override - public boolean mayOverlapWithDb(final String db) { - return patterns.stream().anyMatch(p -> p.mayOverlapWithDb(db)); + public PathPatternTree getIntersection(final PathPatternTree patternTree) { + final PathPatternTree resultTree = new PathPatternTree(); + for (final IoTDBTreePattern pattern : patterns) { + final PathPatternTree intersection = pattern.getIntersection(patternTree); + if (intersection.isEmpty()) { + continue; + } + intersection.getAllPathPatterns().forEach(resultTree::appendPathPattern); + } + resultTree.constructTree(); + return resultTree; } @Override - public boolean mayOverlapWithDevice(final IDeviceID device) { - return patterns.stream().anyMatch(p -> p.mayOverlapWithDevice(device)); + public boolean isPrefixOrFullPath() { + return patterns.stream().allMatch(IoTDBTreePattern::isPrefixOrFullPath); } @Override - public boolean matchesMeasurement(final IDeviceID device, final String measurement) { - return patterns.stream().anyMatch(p -> p.matchesMeasurement(device, measurement)); + public boolean mayMatchMultipleTimeSeriesInOneDevice() { + return patterns.stream().anyMatch(IoTDBTreePattern::mayMatchMultipleTimeSeriesInOneDevice); } + //////////////////////////// Object //////////////////////////// + @Override public String toString() { return "UnionIoTDBTreePattern{" diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/UnionTreePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/UnionTreePattern.java index e43a18ad73855..161dfc46f3326 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/UnionTreePattern.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/UnionTreePattern.java @@ -19,8 +19,11 @@ package org.apache.iotdb.commons.pipe.datastructure.pattern; +import org.apache.iotdb.commons.path.PartialPath; + import org.apache.tsfile.file.metadata.IDeviceID; +import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; @@ -38,11 +41,6 @@ public UnionTreePattern( this.patterns = patterns; } - @Override - public boolean isSingle() { - return patterns.size() == 1; - } - @Override public String getPattern() { return patterns.stream().map(TreePattern::getPattern).collect(Collectors.joining(",")); @@ -83,6 +81,15 @@ public boolean matchesMeasurement(final IDeviceID device, final String measureme return patterns.stream().anyMatch(p -> p.matchesMeasurement(device, measurement)); } + @Override + public List getBaseInclusionPaths() { + final List paths = new ArrayList<>(); + for (final TreePattern p : patterns) { + paths.addAll(p.getBaseInclusionPaths()); + } + return paths; + } + @Override public String toString() { return "UnionTreePattern{" diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/WithExclusionIoTDBTreePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/WithExclusionIoTDBTreePattern.java new file mode 100644 index 0000000000000..f2e580e9cdfe0 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/WithExclusionIoTDBTreePattern.java @@ -0,0 +1,213 @@ +/* + * 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.commons.pipe.datastructure.pattern; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; + +import org.apache.tsfile.file.metadata.IDeviceID; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Represents an exclusion pattern specifically for IoTDB-operation-aware patterns. It holds an + * inclusion and exclusion pattern, both implementing {@link IoTDBTreePatternOperations}. + * + *

The logic is: "Matches inclusion AND NOT exclusion" for all methods. + */ +public class WithExclusionIoTDBTreePattern extends IoTDBTreePatternOperations { + + private final IoTDBTreePatternOperations inclusionPattern; + private final IoTDBTreePatternOperations exclusionPattern; + + public WithExclusionIoTDBTreePattern( + final boolean isTreeModelDataAllowedToBeCaptured, + final IoTDBTreePatternOperations inclusionPattern, + final IoTDBTreePatternOperations exclusionPattern) { + super(isTreeModelDataAllowedToBeCaptured); + this.inclusionPattern = inclusionPattern; + this.exclusionPattern = exclusionPattern; + + TreePattern.checkAndLogPatternCoverage(inclusionPattern, exclusionPattern); + } + + public WithExclusionIoTDBTreePattern( + final IoTDBTreePatternOperations inclusionPattern, + final IoTDBTreePatternOperations exclusionPattern) { + this(true, inclusionPattern, exclusionPattern); + } + + //////////////////////////// Tree Pattern Operations //////////////////////////// + + @Override + public String getPattern() { + return "INCLUSION(" + + inclusionPattern.getPattern() + + "), EXCLUSION(" + + exclusionPattern.getPattern() + + ")"; + } + + @Override + public boolean isRoot() { + // Since the exclusion is not empty, the whole pattern is always not root because it may more or + // less filter some data. + return false; + } + + @Override + public boolean isLegal() { + return inclusionPattern.isLegal() && exclusionPattern.isLegal(); + } + + @Override + public boolean coversDb(final String db) { + return inclusionPattern.coversDb(db) && !exclusionPattern.mayOverlapWithDb(db); + } + + @Override + public boolean coversDevice(final IDeviceID device) { + return inclusionPattern.coversDevice(device) && !exclusionPattern.mayOverlapWithDevice(device); + } + + @Override + public boolean mayOverlapWithDb(final String db) { + return inclusionPattern.mayOverlapWithDb(db) && !exclusionPattern.coversDb(db); + } + + @Override + public boolean mayOverlapWithDevice(final IDeviceID device) { + return inclusionPattern.mayOverlapWithDevice(device) && !exclusionPattern.coversDevice(device); + } + + @Override + public boolean matchesMeasurement(final IDeviceID device, final String measurement) { + return inclusionPattern.matchesMeasurement(device, measurement) + && !exclusionPattern.matchesMeasurement(device, measurement); + } + + @Override + public List getBaseInclusionPaths() { + throw new UnsupportedOperationException(); + } + + //////////////////////////// IoTDB Tree Pattern Operations //////////////////////////// + + @Override + public boolean matchPrefixPath(final String path) { + return inclusionPattern.matchPrefixPath(path) && !exclusionPattern.matchPrefixPath(path); + } + + @Override + public boolean matchDevice(final String devicePath) { + return inclusionPattern.matchDevice(devicePath) && !exclusionPattern.matchDevice(devicePath); + } + + @Override + public boolean matchTailNode(final String tailNode) { + return inclusionPattern.matchTailNode(tailNode) && !exclusionPattern.matchTailNode(tailNode); + } + + @Override + public List getIntersection(final PartialPath partialPath) { + // 1. Calculate Intersection(Input, Inclusion) + final List inclusionIntersections = inclusionPattern.getIntersection(partialPath); + if (inclusionIntersections.isEmpty()) { + return Collections.emptyList(); + } + + // 2. Calculate Intersection(Input, Exclusion) + final List exclusionIntersections = exclusionPattern.getIntersection(partialPath); + if (exclusionIntersections.isEmpty()) { + // Optimization: No exclusion intersection, return inclusion intersection directly + return inclusionIntersections; + } + + // 3. Perform the "Subtraction" + // Filter out paths from inclusionIntersections that are fully covered by any path + // in exclusionIntersections. + return inclusionIntersections.stream() + .filter( + incPath -> + exclusionIntersections.stream().noneMatch(excPath -> excPath.include(incPath))) + .collect(Collectors.toList()); + } + + @Override + public PathPatternTree getIntersection(final PathPatternTree patternTree) { + // 1. Calculate Intersection(Input, Inclusion) + final PathPatternTree inclusionIntersectionTree = inclusionPattern.getIntersection(patternTree); + if (inclusionIntersectionTree.isEmpty()) { + return inclusionIntersectionTree; // Return empty tree + } + + // 2. Calculate Intersection(Input, Exclusion) + final PathPatternTree exclusionIntersectionTree = exclusionPattern.getIntersection(patternTree); + if (exclusionIntersectionTree.isEmpty()) { + // Optimization: No exclusion intersection, return inclusion intersection directly + return inclusionIntersectionTree; + } + + // 3. Perform the "Subtraction" + final List inclusionPaths = inclusionIntersectionTree.getAllPathPatterns(); + final List exclusionPaths = exclusionIntersectionTree.getAllPathPatterns(); + + final PathPatternTree finalResultTree = new PathPatternTree(); + for (final PartialPath incPath : inclusionPaths) { + // Check if the current inclusion path is covered by *any* exclusion path pattern + boolean excluded = exclusionPaths.stream().anyMatch(excPath -> excPath.include(incPath)); + + if (!excluded) { + finalResultTree.appendPathPattern(incPath); // Add non-excluded path to the result tree + } + } + + finalResultTree.constructTree(); + return finalResultTree; + } + + @Override + public boolean isPrefixOrFullPath() { + // Both must be prefix/full-path for the exclusion logic to be sound + return inclusionPattern.isPrefixOrFullPath() && exclusionPattern.isPrefixOrFullPath(); + } + + @Override + public boolean mayMatchMultipleTimeSeriesInOneDevice() { + // If inclusion might, the result might (even if exclusion trims it) + return inclusionPattern.mayMatchMultipleTimeSeriesInOneDevice(); + } + + //////////////////////////// Object //////////////////////////// + + @Override + public String toString() { + return "ExclusionIoTDBTreePattern{" + + "inclusionPattern=" + + inclusionPattern + + ", exclusionPattern=" + + exclusionPattern + + ", isTreeModelDataAllowedToBeCaptured=" + + isTreeModelDataAllowedToBeCaptured + + '}'; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/WithExclusionTreePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/WithExclusionTreePattern.java new file mode 100644 index 0000000000000..0cf1d0d1179aa --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/pattern/WithExclusionTreePattern.java @@ -0,0 +1,119 @@ +/* + * 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.commons.pipe.datastructure.pattern; + +import org.apache.iotdb.commons.path.PartialPath; + +import org.apache.tsfile.file.metadata.IDeviceID; + +import java.util.List; + +/** + * Represents a pattern that includes data matched by an inclusion pattern, except for data matched + * by an exclusion pattern. + * + *

The logic implemented in the methods is: "Matches the inclusion pattern AND NOT the exclusion + * pattern." + */ +public class WithExclusionTreePattern extends TreePattern { + + private final TreePattern inclusionPattern; + private final TreePattern exclusionPattern; + + public WithExclusionTreePattern( + final boolean isTreeModelDataAllowedToBeCaptured, + final TreePattern inclusionPattern, + final TreePattern exclusionPattern) { + super(isTreeModelDataAllowedToBeCaptured); + this.inclusionPattern = inclusionPattern; + this.exclusionPattern = exclusionPattern; + + TreePattern.checkAndLogPatternCoverage(inclusionPattern, exclusionPattern); + } + + @Override + public String getPattern() { + return "INCLUSION(" + + inclusionPattern.getPattern() + + "), EXCLUSION(" + + exclusionPattern.getPattern() + + ")"; + } + + @Override + public boolean isRoot() { + // Since the exclusion is not empty, the whole pattern is always not root because it may more or + // less filter some data. + return false; + } + + @Override + public boolean isLegal() { + return inclusionPattern.isLegal() && exclusionPattern.isLegal(); + } + + @Override + public boolean coversDb(final String db) { + // Covers DB if inclusion covers it AND exclusion doesn't overlap at all. + return inclusionPattern.coversDb(db) && !exclusionPattern.mayOverlapWithDb(db); + } + + @Override + public boolean coversDevice(final IDeviceID device) { + // Covers device if inclusion covers it AND exclusion doesn't overlap at all. + return inclusionPattern.coversDevice(device) && !exclusionPattern.mayOverlapWithDevice(device); + } + + @Override + public boolean mayOverlapWithDb(final String db) { + // May overlap if inclusion overlaps AND exclusion doesn't fully cover it. + return inclusionPattern.mayOverlapWithDb(db) && !exclusionPattern.coversDb(db); + } + + @Override + public boolean mayOverlapWithDevice(final IDeviceID device) { + // May overlap if inclusion overlaps AND exclusion doesn't fully cover it. + return inclusionPattern.mayOverlapWithDevice(device) && !exclusionPattern.coversDevice(device); + } + + @Override + public boolean matchesMeasurement(final IDeviceID device, final String measurement) { + // The core logic: Must match inclusion AND NOT match exclusion. + return inclusionPattern.matchesMeasurement(device, measurement) + && !exclusionPattern.matchesMeasurement(device, measurement); + } + + @Override + public List getBaseInclusionPaths() { + throw new UnsupportedOperationException(); + } + + @Override + public String toString() { + return "ExclusionTreePattern{" + + "inclusionPattern=" + + inclusionPattern + + ", exclusionPattern=" + + exclusionPattern + + ", isTreeModelDataAllowedToBeCaptured=" + + isTreeModelDataAllowedToBeCaptured + + '}'; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/source/IoTDBNonDataRegionSource.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/source/IoTDBNonDataRegionSource.java index acb0ae825ac44..80e2445193622 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/source/IoTDBNonDataRegionSource.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/source/IoTDBNonDataRegionSource.java @@ -24,9 +24,9 @@ import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.exception.auth.AccessDeniedException; +import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePatternOperations; import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.UnionIoTDBTreePattern; import org.apache.iotdb.commons.pipe.datastructure.queue.ConcurrentIterableLinkedQueue; import org.apache.iotdb.commons.pipe.datastructure.queue.listening.AbstractPipeListeningQueue; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; @@ -53,7 +53,7 @@ @TableModel public abstract class IoTDBNonDataRegionSource extends IoTDBSource { - protected UnionIoTDBTreePattern treePattern; + protected IoTDBTreePatternOperations treePattern; protected TablePattern tablePattern; private List historicalEvents = new LinkedList<>(); @@ -78,14 +78,14 @@ public void customize( final TreePattern pattern = TreePattern.parsePipePatternFromSourceParameters(parameters); - if (!(pattern instanceof UnionIoTDBTreePattern - && (((UnionIoTDBTreePattern) pattern).isPrefixOrFullPath()))) { + if (!(pattern instanceof IoTDBTreePatternOperations + && (((IoTDBTreePatternOperations) pattern).isPrefixOrFullPath()))) { throw new IllegalArgumentException( String.format( "The path pattern %s is not valid for the source. Only prefix or full path is allowed.", pattern.getPattern())); } - treePattern = (UnionIoTDBTreePattern) pattern; + treePattern = (IoTDBTreePatternOperations) pattern; tablePattern = TablePattern.parsePipePatternFromSourceParameters(parameters); } @@ -178,7 +178,7 @@ public EnrichedEvent supply() throws Exception { pipeName, creationTime, pipeTaskMeta, - treePattern, + (TreePattern) treePattern, tablePattern, userId, userName, @@ -241,7 +241,7 @@ public EnrichedEvent supply() throws Exception { pipeName, creationTime, pipeTaskMeta, - treePattern, + (TreePattern) treePattern, tablePattern, userId, userName, From f19824d287be7f66ecdb08882e79a91d31278abb Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Wed, 12 Nov 2025 11:07:58 +0800 Subject: [PATCH 050/180] Change the privilege of showVersionStatement #16736 (cherry picked from commit 43b89b9a4802174864bd61699b6bfa5c2c34e723) --- .../apache/iotdb/db/it/auth/IoTDBSystemPermissionIT.java | 3 +-- .../db/it/auth/IoTDBSystemPermissionRelationalIT.java | 3 +-- .../relational/it/query/recent/IoTDBMaintainAuthIT.java | 8 ++------ .../plan/execution/config/TableConfigTaskVisitor.java | 1 - .../plan/relational/security/TreeAccessCheckVisitor.java | 2 +- 5 files changed, 5 insertions(+), 12 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBSystemPermissionIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBSystemPermissionIT.java index e5a8ee7ed836d..2ebdd37758fe4 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBSystemPermissionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBSystemPermissionIT.java @@ -193,6 +193,7 @@ public void manageCQTest() { @Test public void maintainOperationsTest() { executeQuery("show queries", "test6", "test123123456"); + executeQuery("show version", "test6", "test123123456"); assertNonQueryTestFail( "kill query '20250918_015728_00003_1'", "714: No such query", "test6", "test123123456"); assertNonQueriesTestFail( @@ -205,7 +206,6 @@ public void maintainOperationsTest() { "set configuration 'enable_seq_space_compaction'='true'", "start repair data", "stop repair data", - "show version" }, "803: No permissions for this operation, please add privilege SYSTEM", "test6", @@ -220,7 +220,6 @@ public void maintainOperationsTest() { executeNonQuery("start repair data", "test6", "test123123456"); executeNonQuery("stop repair data", "test6", "test123123456"); executeQuery("show queries", "test6", "test123123456"); - executeNonQuery("show version", "test6", "test123123456"); } @Test diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBSystemPermissionRelationalIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBSystemPermissionRelationalIT.java index 98326a61ca967..aab39581ee5d4 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBSystemPermissionRelationalIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBSystemPermissionRelationalIT.java @@ -65,6 +65,7 @@ public static void tearDown() throws Exception { @Test public void maintainOperationsTest() { executeTableQuery("show queries", "test6", "test123123456"); + executeTableNonQuery("show version", "test6", "test123123456"); assertTableNonQueryTestFail( "kill query '20250918_015728_00003_1'", "714: No such query", "test6", "test123123456"); assertTableNonQueriesTestFail( @@ -77,7 +78,6 @@ public void maintainOperationsTest() { "set configuration enable_seq_space_compaction='true'", "start repair data", "stop repair data", - "show version" }, "803: Access Denied: No permissions for this operation, please add privilege SYSTEM", "test6", @@ -92,7 +92,6 @@ public void maintainOperationsTest() { executeTableNonQuery("start repair data", "test6", "test123123456"); executeTableNonQuery("stop repair data", "test6", "test123123456"); executeTableQuery("show queries", "test6", "test123123456"); - executeTableNonQuery("show version", "test6", "test123123456"); } @Test diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBMaintainAuthIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBMaintainAuthIT.java index e3acb18488326..56425b4d7fff1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBMaintainAuthIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBMaintainAuthIT.java @@ -112,12 +112,8 @@ public void maintainAuthTest() { tableQueryNoVerifyResultTest("SHOW CURRENT_USER", expectedHeader, USER_2, PASSWORD); // case 5: show version - tableAssertTestFail( - "SHOW VERSION", - TSStatusCode.NO_PERMISSION.getStatusCode() - + ": Access Denied: No permissions for this operation, please add privilege SYSTEM", - USER_2, - PASSWORD); + expectedHeader = new String[] {"Version", "BuildInfo"}; + tableQueryNoVerifyResultTest("SHOW VERSION", expectedHeader, USER_2, PASSWORD); // case 6: show current_timestamp expectedHeader = new String[] {"CurrentTimestamp"}; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java index 0475a27c26dab..32b964ec584f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java @@ -1358,7 +1358,6 @@ protected IConfigTask visitShowCurrentDatabase( @Override protected IConfigTask visitShowVersion(ShowVersion node, MPPQueryContext context) { context.setQueryType(QueryType.READ); - accessControl.checkUserGlobalSysPrivilege(context); return new ShowVersionTask(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java index d24ec3ef0d3d5..aa8c4d0ed8041 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java @@ -1730,7 +1730,7 @@ public TSStatus visitShowVariables( @Override public TSStatus visitShowVersion(ShowVersionStatement statement, TreeAccessCheckContext context) { - return checkGlobalAuth(context, PrivilegeType.MAINTAIN, () -> ""); + return SUCCEED; } @Override From ff6f7a26ccf36b6bdf6bac53dcd5f1f5edcf9ee4 Mon Sep 17 00:00:00 2001 From: Yongzao Date: Wed, 12 Nov 2025 16:41:41 +0800 Subject: [PATCH 051/180] [AINode] Package AINode via PyInstaller (#16707) (cherry picked from commit 49c625bb83ec28f1c11303504766416f59381a45) --- .github/workflows/cluster-it-1c1d1a.yml | 2 +- integration-test/src/assembly/mpp-test.xml | 2 +- .../it/env/cluster/node/AINodeWrapper.java | 25 +- .../it/AINodeConcurrentInferenceIT.java | 62 +- iotdb-core/ainode/.gitignore | 6 +- iotdb-core/ainode/ainode.spec | 199 ++++++ iotdb-core/ainode/ainode.xml | 17 +- iotdb-core/ainode/build_binary.py | 596 ++++++++++++++++++ iotdb-core/ainode/iotdb/ainode/core/config.py | 28 +- .../ainode/iotdb/ainode/core/constant.py | 18 +- .../ainode/core/inference/pool_controller.py | 47 +- .../pool_scheduler/basic_pool_scheduler.py | 7 +- iotdb-core/ainode/iotdb/ainode/core/log.py | 4 +- iotdb-core/ainode/iotdb/ainode/core/script.py | 80 +-- iotdb-core/ainode/poetry.lock | 111 +++- iotdb-core/ainode/pom.xml | 92 +-- iotdb-core/ainode/pyproject.toml | 29 +- .../ainode/resources/syncPythonVersion.groovy | 33 - scripts/conf/ainode-env.sh | 138 ---- scripts/conf/windows/ainode-env.bat | 129 ---- scripts/sbin/start-ainode.sh | 41 +- scripts/sbin/windows/start-ainode.bat | 48 +- 22 files changed, 1028 insertions(+), 686 deletions(-) create mode 100644 iotdb-core/ainode/ainode.spec create mode 100644 iotdb-core/ainode/build_binary.py delete mode 100644 scripts/conf/ainode-env.sh delete mode 100644 scripts/conf/windows/ainode-env.bat diff --git a/.github/workflows/cluster-it-1c1d1a.yml b/.github/workflows/cluster-it-1c1d1a.yml index f1b583f8efe51..d4c40fa7ad889 100644 --- a/.github/workflows/cluster-it-1c1d1a.yml +++ b/.github/workflows/cluster-it-1c1d1a.yml @@ -59,5 +59,5 @@ jobs: uses: actions/upload-artifact@v4 with: name: cluster-log-ainode-${{ matrix.os }} - path: integration-test/target/ainode-logs + path: integration-test/target/*-logs retention-days: 30 diff --git a/integration-test/src/assembly/mpp-test.xml b/integration-test/src/assembly/mpp-test.xml index 4bcd32c7ee0e2..3915c4593a946 100644 --- a/integration-test/src/assembly/mpp-test.xml +++ b/integration-test/src/assembly/mpp-test.xml @@ -63,7 +63,7 @@ lib - ${project.basedir}/../iotdb-core/ainode/dist/ + ${project.basedir}/../iotdb-core/ainode/dist/ainode/ 0755 diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/node/AINodeWrapper.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/node/AINodeWrapper.java index 20c31f4614666..e118d6c3a98ff 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/node/AINodeWrapper.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/node/AINodeWrapper.java @@ -25,9 +25,9 @@ import org.apache.tsfile.external.commons.io.file.PathUtils; import org.slf4j.Logger; -import java.io.BufferedWriter; import java.io.File; -import java.io.FileWriter; +import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.IOException; import java.nio.file.Files; import java.nio.file.LinkOption; @@ -37,6 +37,7 @@ import java.nio.file.StandardCopyOption; import java.util.ArrayList; import java.util.List; +import java.util.Properties; import java.util.stream.Stream; import static org.apache.iotdb.it.env.cluster.ClusterConstant.AI_NODE_NAME; @@ -62,15 +63,19 @@ public class AINodeWrapper extends AbstractNodeWrapper { public static final String CACHE_BUILT_IN_MODEL_PATH = "/data/ainode/models/weights"; private void replaceAttribute(String[] keys, String[] values, String filePath) { - try (BufferedWriter writer = new BufferedWriter(new FileWriter(filePath, true))) { - for (int i = 0; i < keys.length; i++) { - String line = keys[i] + "=" + values[i]; - writer.newLine(); - writer.write(line); - } + Properties props = new Properties(); + try (FileInputStream in = new FileInputStream(filePath)) { + props.load(in); + } catch (IOException e) { + logger.warn("Failed to load existing AINode properties from {}, because: ", filePath, e); + } + for (int i = 0; i < keys.length; i++) { + props.setProperty(keys[i], values[i]); + } + try (FileOutputStream out = new FileOutputStream(filePath)) { + props.store(out, "Updated by AINode integration-test env"); } catch (IOException e) { - logger.error( - "Failed to set attribute for AINode in file: {} because {}", filePath, e.getMessage()); + logger.error("Failed to save properties to {}, because:", filePath, e); } } diff --git a/integration-test/src/test/java/org/apache/iotdb/ainode/it/AINodeConcurrentInferenceIT.java b/integration-test/src/test/java/org/apache/iotdb/ainode/it/AINodeConcurrentInferenceIT.java index 42fcf1d30d6c6..a08990d472fe6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/ainode/it/AINodeConcurrentInferenceIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/ainode/it/AINodeConcurrentInferenceIT.java @@ -90,33 +90,6 @@ private static void prepareDataForTableModel() throws SQLException { } } - // @Test - public void concurrentCPUCallInferenceTest() throws SQLException, InterruptedException { - concurrentCPUCallInferenceTest("timer_xl"); - concurrentCPUCallInferenceTest("sundial"); - } - - private void concurrentCPUCallInferenceTest(String modelId) - throws SQLException, InterruptedException { - try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TREE_SQL_DIALECT); - Statement statement = connection.createStatement()) { - final int threadCnt = 4; - final int loop = 10; - final int predictLength = 96; - statement.execute(String.format("LOAD MODEL %s TO DEVICES 'cpu'", modelId)); - checkModelOnSpecifiedDevice(statement, modelId, "cpu"); - concurrentInference( - statement, - String.format( - "CALL INFERENCE(%s, 'SELECT s FROM root.AI', predict_length=%d)", - modelId, predictLength), - threadCnt, - loop, - predictLength); - statement.execute(String.format("UNLOAD MODEL %s FROM DEVICES 'cpu'", modelId)); - } - } - // @Test public void concurrentGPUCallInferenceTest() throws SQLException, InterruptedException { concurrentGPUCallInferenceTest("timer_xl"); @@ -150,39 +123,6 @@ private void concurrentGPUCallInferenceTest(String modelId) String forecastUDTFSql = "SELECT forecast(s, 'MODEL_ID'='%s', 'PREDICT_LENGTH'='%d') FROM root.AI"; - @Test - public void concurrentCPUForecastTest() throws SQLException, InterruptedException { - concurrentCPUForecastTest("timer_xl", forecastUDTFSql); - concurrentCPUForecastTest("sundial", forecastUDTFSql); - concurrentCPUForecastTest("timer_xl", forecastTableFunctionSql); - concurrentCPUForecastTest("sundial", forecastTableFunctionSql); - } - - private void concurrentCPUForecastTest(String modelId, String selectSQL) - throws SQLException, InterruptedException { - try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); - Statement statement = connection.createStatement()) { - final int threadCnt = 4; - final int loop = 10; - final int predictLength = 96; - statement.execute(String.format("LOAD MODEL %s TO DEVICES 'cpu'", modelId)); - checkModelOnSpecifiedDevice(statement, modelId, "cpu"); - long startTime = System.currentTimeMillis(); - concurrentInference( - statement, - String.format(selectSQL, modelId, predictLength), - threadCnt, - loop, - predictLength); - long endTime = System.currentTimeMillis(); - LOGGER.info( - String.format( - "Model %s concurrent inference %d reqs (%d threads, %d loops) in CPU takes time: %dms", - modelId, threadCnt * loop, threadCnt, loop, endTime - startTime)); - statement.execute(String.format("UNLOAD MODEL %s FROM DEVICES 'cpu'", modelId)); - } - } - @Test public void concurrentGPUForecastTest() throws SQLException, InterruptedException { concurrentGPUForecastTest("timer_xl", forecastUDTFSql); @@ -221,7 +161,7 @@ private void checkModelOnSpecifiedDevice(Statement statement, String modelId, St throws SQLException, InterruptedException { Set targetDevices = ImmutableSet.copyOf(device.split(",")); LOGGER.info("Checking model: {} on target devices: {}", modelId, targetDevices); - for (int retry = 0; retry < 20; retry++) { + for (int retry = 0; retry < 200; retry++) { Set foundDevices = new HashSet<>(); try (final ResultSet resultSet = statement.executeQuery(String.format("SHOW LOADED MODELS '%s'", device))) { diff --git a/iotdb-core/ainode/.gitignore b/iotdb-core/ainode/.gitignore index 8cc2098c3fd85..bdb2698ec782c 100644 --- a/iotdb-core/ainode/.gitignore +++ b/iotdb-core/ainode/.gitignore @@ -14,8 +14,6 @@ # generated by maven /iotdb/ainode/conf/ -# .whl of ainode, generated by Poetry +# generated by pyinstaller /dist/ - -# the config to build ainode, it will be generated automatically -pyproject.toml +/build/ diff --git a/iotdb-core/ainode/ainode.spec b/iotdb-core/ainode/ainode.spec new file mode 100644 index 0000000000000..a131b2bcff217 --- /dev/null +++ b/iotdb-core/ainode/ainode.spec @@ -0,0 +1,199 @@ +# -*- mode: python ; coding: utf-8 -*- +# +# 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 this 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. +# + +from pathlib import Path + +# Get project root directory +project_root = Path(SPECPATH).parent + +block_cipher = None + +# Auto-collect all submodules of large dependency libraries +# Using collect_all automatically includes all dependencies and avoids manual maintenance of hiddenimports +from PyInstaller.utils.hooks import collect_all, collect_submodules, collect_data_files + +# Collect only essential data files and binaries for large libraries +# Using collect_all for all submodules slows down startup significantly. +# However, for certain libraries with many dynamic imports (e.g., torch, transformers, safetensors), +# collect_all is necessary to ensure all required modules are included. +# For other libraries, we use lighter-weight collection methods to improve startup time. +all_datas = [] +all_binaries = [] +all_hiddenimports = [] + +# Only collect essential data files and binaries for critical libraries +# This reduces startup time by avoiding unnecessary module imports +essential_libraries = { + 'torch': True, # Keep collect_all for torch as it has many dynamic imports + 'transformers': True, # Keep collect_all for transformers + 'safetensors': True, # Keep collect_all for safetensors +} + +# For other libraries, use selective collection to speed up startup +other_libraries = ['sktime', 'scipy', 'pandas', 'sklearn', 'statsmodels', 'optuna'] + +for lib in essential_libraries: + try: + lib_datas, lib_binaries, lib_hiddenimports = collect_all(lib) + all_datas.extend(lib_datas) + all_binaries.extend(lib_binaries) + all_hiddenimports.extend(lib_hiddenimports) + except Exception: + pass + +# For other libraries, only collect submodules (lighter weight) +# This relies on PyInstaller's dependency analysis to include what's actually used +for lib in other_libraries: + try: + submodules = collect_submodules(lib) + all_hiddenimports.extend(submodules) + # Only collect essential data files and binaries, not all submodules + # This significantly reduces startup time + try: + lib_datas, lib_binaries, _ = collect_all(lib) + all_datas.extend(lib_datas) + all_binaries.extend(lib_binaries) + except Exception: + # If collect_all fails, try collect_data_files for essential data only + try: + lib_datas = collect_data_files(lib) + all_datas.extend(lib_datas) + except Exception: + pass + except Exception: + pass + +# Project-specific packages that need their submodules collected +# Only list top-level packages - collect_submodules will recursively collect all submodules +TOP_LEVEL_PACKAGES = [ + 'iotdb.ainode.core', # This will include all sub-packages: manager, model, inference, etc. + 'iotdb.thrift', # This will include all thrift sub-packages +] + +# Collect all submodules for project packages automatically +# Using top-level packages avoids duplicate collection +for package in TOP_LEVEL_PACKAGES: + try: + submodules = collect_submodules(package) + all_hiddenimports.extend(submodules) + except Exception: + # If package doesn't exist or collection fails, add the package itself + all_hiddenimports.append(package) + +# Add parent packages to ensure they are included +all_hiddenimports.extend(['iotdb', 'iotdb.ainode']) + +# Multiprocessing support for PyInstaller +# When using multiprocessing with PyInstaller, we need to ensure proper handling +multiprocessing_modules = [ + 'multiprocessing', + 'multiprocessing.spawn', + 'multiprocessing.popen_spawn_posix', + 'multiprocessing.popen_spawn_win32', + 'multiprocessing.popen_fork', + 'multiprocessing.popen_forkserver', + 'multiprocessing.context', + 'multiprocessing.reduction', + 'multiprocessing.util', + 'torch.multiprocessing', + 'torch.multiprocessing.spawn', +] + +# Additional dependencies that may need explicit import +# These are external libraries that might use dynamic imports +external_dependencies = [ + 'huggingface_hub', + 'tokenizers', + 'hf_xet', + 'einops', + 'dynaconf', + 'tzlocal', + 'thrift', + 'psutil', + 'requests', +] + +all_hiddenimports.extend(multiprocessing_modules) +all_hiddenimports.extend(external_dependencies) + +# Analyze main entry file +# Note: Do NOT add virtual environment site-packages to pathex manually. +# When PyInstaller is run from the virtual environment's Python, it automatically +# detects and uses the virtual environment's site-packages. +a = Analysis( + ['iotdb/ainode/core/script.py'], + pathex=[str(project_root)], + binaries=all_binaries, + datas=all_datas, + hiddenimports=all_hiddenimports, + hookspath=[], + hooksconfig={}, + runtime_hooks=[], + excludes=[ + # Exclude unnecessary modules to reduce size and improve startup time + # Note: Do not exclude unittest, as torch and other libraries require it + # Only exclude modules that are definitely not used and not required by dependencies + 'matplotlib', + 'IPython', + 'jupyter', + 'notebook', + 'pytest', + 'test', + 'tests' + ], + win_no_prefer_redirects=False, + win_private_assemblies=False, + cipher=block_cipher, + noarchive=True, # Set to True to speed up startup - files are not archived into PYZ +) + +# Package all PYZ files +pyz = PYZ(a.pure, a.zipped_data, cipher=block_cipher) + +# Create executable (onedir mode for faster startup) +exe = EXE( + pyz, + a.scripts, + [], + exclude_binaries=True, + name='ainode', + debug=False, + bootloader_ignore_signals=False, + strip=False, + upx=True, + console=True, + disable_windowed_traceback=False, + argv_emulation=False, + target_arch=None, + codesign_identity=None, + entitlements_file=None, +) + +# Collect all files into a directory (onedir mode) +coll = COLLECT( + exe, + a.binaries, + a.zipfiles, + a.datas, + strip=False, + upx=True, + upx_exclude=[], + name='ainode', +) \ No newline at end of file diff --git a/iotdb-core/ainode/ainode.xml b/iotdb-core/ainode/ainode.xml index beab4b69c01dd..6de635c133854 100644 --- a/iotdb-core/ainode/ainode.xml +++ b/iotdb-core/ainode/ainode.xml @@ -42,6 +42,10 @@ + + iotdb/ainode/conf + conf + resources/conf conf @@ -52,19 +56,8 @@ 0755 - dist + dist/ainode lib - - *.whl - - - - ${project.basedir}/../../scripts/conf - conf - - ainode-env.* - **/ainode-env.* - 0755 diff --git a/iotdb-core/ainode/build_binary.py b/iotdb-core/ainode/build_binary.py new file mode 100644 index 0000000000000..1b5f99da21493 --- /dev/null +++ b/iotdb-core/ainode/build_binary.py @@ -0,0 +1,596 @@ +# 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. +# + + +""" +PyInstaller build script (Python version) +""" + +import os +import shutil +import subprocess +import sys +from pathlib import Path + + +def get_venv_base_dir(): + """ + Get the base directory for virtual environments outside the project. + + Returns: + Path: Base directory path + - Linux/macOS: ~/.cache/iotdb-ainode-build/ + - Windows: %LOCALAPPDATA%\\iotdb-ainode-build\\ + """ + if sys.platform == "win32": + localappdata = os.environ.get("LOCALAPPDATA") or os.environ.get( + "APPDATA", os.path.expanduser("~") + ) + base_dir = Path(localappdata) / "iotdb-ainode-build" + else: + base_dir = Path.home() / ".cache" / "iotdb-ainode-build" + + return base_dir + + +def setup_venv(): + """ + Create virtual environment outside the project directory. + + The virtual environment is created in a platform-specific location: + - Linux/macOS: ~/.cache/iotdb-ainode-build// + - Windows: %LOCALAPPDATA%\\iotdb-ainode-build\\\\ + + The same venv is reused across multiple builds of the same project. + + Returns: + Path: Path to the virtual environment directory + """ + script_dir = Path(__file__).parent + venv_base_dir = get_venv_base_dir() + venv_dir = venv_base_dir / script_dir.name + + if venv_dir.exists(): + print(f"Virtual environment already exists at: {venv_dir}") + return venv_dir + + venv_base_dir.mkdir(parents=True, exist_ok=True) + print(f"Creating virtual environment at: {venv_dir}") + subprocess.run([sys.executable, "-m", "venv", str(venv_dir)], check=True) + print("Virtual environment created successfully") + return venv_dir + + +def get_venv_python(venv_dir): + """Get Python executable path in virtual environment""" + if sys.platform == "win32": + return venv_dir / "Scripts" / "python.exe" + else: + return venv_dir / "bin" / "python" + + +def update_pip(venv_python): + """Update pip in the virtual environment to the latest version.""" + print("Updating pip...") + subprocess.run( + [str(venv_python), "-m", "pip", "install", "--upgrade", "pip"], check=True + ) + print("pip updated successfully") + + +def install_poetry(venv_python): + """Install poetry 2.2.1 in the virtual environment.""" + print("Installing poetry 2.2.1...") + subprocess.run( + [ + str(venv_python), + "-m", + "pip", + "install", + "poetry==2.2.1", + ], + check=True, + ) + # Get installed version + version_result = subprocess.run( + [str(venv_python), "-m", "poetry", "--version"], + capture_output=True, + text=True, + check=True, + ) + print(f"Poetry installed: {version_result.stdout.strip()}") + + +def get_venv_env(venv_dir): + """ + Get environment variables configured for the virtual environment. + + Sets VIRTUAL_ENV and prepends the venv's bin/Scripts directory to PATH + so that tools installed in the venv take precedence. + Also sets POETRY_VIRTUALENVS_PATH to force poetry to use our venv. + + Returns: + dict: Environment variables dictionary + """ + env = os.environ.copy() + env["VIRTUAL_ENV"] = str(venv_dir.absolute()) + + venv_bin = str(venv_dir / ("Scripts" if sys.platform == "win32" else "bin")) + env["PATH"] = f"{venv_bin}{os.pathsep}{env.get('PATH', '')}" + + # Force poetry to use our virtual environment by setting POETRY_VIRTUALENVS_PATH + # This tells poetry where to look for/create virtual environments + env["POETRY_VIRTUALENVS_PATH"] = str(venv_dir.parent.absolute()) + + return env + + +def get_poetry_executable(venv_dir): + """Get poetry executable path in the virtual environment.""" + if sys.platform == "win32": + return venv_dir / "Scripts" / "poetry.exe" + else: + return venv_dir / "bin" / "poetry" + + +def install_dependencies(venv_python, venv_dir, script_dir): + """ + Install project dependencies using poetry. + + Configures poetry to use the external virtual environment and installs + all dependencies from pyproject.toml. + """ + print("Installing dependencies with poetry...") + venv_env = get_venv_env(venv_dir) + poetry_exe = get_poetry_executable(venv_dir) + + # Configure poetry settings + print("Configuring poetry settings...") + try: + # Set poetry to not create venvs in project directory + subprocess.run( + [str(poetry_exe), "config", "virtualenvs.in-project", "false"], + cwd=str(script_dir), + env=venv_env, + check=True, + capture_output=True, + text=True, + ) + # Set poetry virtualenvs path to our venv directory's parent + # This forces poetry to look for/create venvs in the same location as our venv + subprocess.run( + [ + str(poetry_exe), + "config", + "virtualenvs.path", + str(venv_dir.parent.absolute()), + ], + cwd=str(script_dir), + env=venv_env, + check=True, + capture_output=True, + text=True, + ) + # Ensure poetry can use virtual environments + subprocess.run( + [str(poetry_exe), "config", "virtualenvs.create", "true"], + cwd=str(script_dir), + env=venv_env, + check=True, + capture_output=True, + text=True, + ) + except Exception as e: + print(f"Warning: Failed to configure poetry settings: {e}") + # Continue anyway, as these may not be critical + + # Remove any existing poetry virtual environments for this project + # This ensures poetry will use our specified virtual environment + print("Removing any existing poetry virtual environments...") + remove_result = subprocess.run( + [str(poetry_exe), "env", "remove", "--all"], + cwd=str(script_dir), + env=venv_env, + check=False, # Don't fail if no venv exists + capture_output=True, + text=True, + ) + if remove_result.stdout: + print(remove_result.stdout.strip()) + if remove_result.stderr: + stderr = remove_result.stderr.strip() + # Ignore "No virtualenv has been activated" error + if "no virtualenv" not in stderr.lower(): + print(remove_result.stderr.strip()) + + # Verify the virtual environment Python is valid before configuring poetry + print(f"Verifying virtual environment Python at: {venv_python}") + if not venv_python.exists(): + print(f"ERROR: Virtual environment Python not found at: {venv_python}") + sys.exit(1) + + python_version_result = subprocess.run( + [str(venv_python), "--version"], + capture_output=True, + text=True, + check=False, + ) + if python_version_result.returncode != 0: + print(f"ERROR: Virtual environment Python is not executable: {venv_python}") + sys.exit(1) + print(f" Python version: {python_version_result.stdout.strip()}") + + # Instead of using poetry env use (which creates new venvs), we'll use a different approach: + # 1. Create a symlink from poetry's expected venv location to our venv + # 2. Or, directly use poetry install with VIRTUAL_ENV set (poetry should detect it) + # + # The issue is that poetry env use creates venvs with hash-based names in its cache. + # We need to work around this by either: + # - Creating a symlink from poetry's expected location to our venv + # - Or bypassing poetry env use entirely and using poetry install directly + + # Strategy: Create a symlink from poetry's expected venv location to our venv + # Poetry creates venvs with names like: --py + # We need to find out what poetry would name our venv, then create a symlink + + print(f"Configuring poetry to use virtual environment at: {venv_dir}") + + # Get poetry's expected venv name by checking what it would create + # First, let's try poetry env use, but catch if it tries to create a new venv + result = subprocess.run( + [str(poetry_exe), "env", "use", str(venv_python)], + cwd=str(script_dir), + env=venv_env, + check=False, + capture_output=True, + text=True, + ) + + output_text = (result.stdout or "") + (result.stderr or "") + + # If poetry is creating a new venv, we need to stop it and use a different approach + if ( + "Creating virtualenv" in output_text + or "Creating virtual environment" in output_text + or "Using virtualenv:" in output_text + ): + print("Poetry is attempting to create/use a new virtual environment.") + print( + "Stopping this and using alternative approach: creating symlink to our venv..." + ) + + # Extract the venv path poetry is trying to create/use + # Look for patterns like "Using virtualenv: /path/to/venv" or "Creating virtualenv name in /path" + import re + + poetry_venv_path = None + + # Try to extract from "Using virtualenv: /path/to/venv" + using_match = re.search(r"Using virtualenv:\s*([^\s\n]+)", output_text) + if using_match: + poetry_venv_path = Path(using_match.group(1)) + + # If not found, try to extract from "Creating virtualenv name in /path" + if not poetry_venv_path: + creating_match = re.search( + r"Creating virtualenv[^\n]*in\s+([^\s\n]+)", output_text + ) + if creating_match: + venv_dir_path = Path(creating_match.group(1)) + # Extract venv name from the output + name_match = re.search(r"Creating virtualenv\s+([^\s]+)", output_text) + if name_match: + venv_name = name_match.group(1) + poetry_venv_path = venv_dir_path / venv_name + + # If still not found, try to find any path in pypoetry/virtualenvs + if not poetry_venv_path: + pypoetry_match = re.search( + r"([^\s]+pypoetry[^\s]*virtualenvs[^\s]+)", output_text + ) + if pypoetry_match: + poetry_venv_path = Path(pypoetry_match.group(1)) + + if poetry_venv_path: + print(f"Poetry wants to create/use venv at: {poetry_venv_path}") + + # Remove the venv poetry just created (if it exists) + if poetry_venv_path.exists() and poetry_venv_path.is_dir(): + print(f"Removing poetry's newly created venv: {poetry_venv_path}") + shutil.rmtree(poetry_venv_path, ignore_errors=True) + + # Create a symlink from poetry's expected location to our venv + print(f"Creating symlink from {poetry_venv_path} to {venv_dir}") + try: + if poetry_venv_path.exists() or poetry_venv_path.is_symlink(): + if poetry_venv_path.is_symlink(): + poetry_venv_path.unlink() + elif poetry_venv_path.is_dir(): + shutil.rmtree(poetry_venv_path, ignore_errors=True) + poetry_venv_path.parent.mkdir(parents=True, exist_ok=True) + poetry_venv_path.symlink_to(venv_dir) + print(f"✓ Symlink created successfully") + except Exception as e: + print(f"WARNING: Failed to create symlink: {e}") + print("Will try to use poetry install directly with VIRTUAL_ENV set") + else: + print("Could not determine poetry's venv path from output") + print(f"Output was: {output_text}") + else: + if result.stdout: + print(result.stdout.strip()) + if result.stderr: + stderr = result.stderr.strip() + if stderr: + print(f"Poetry output: {stderr}") + + # Verify poetry is using the correct virtual environment BEFORE running lock/install + # This is critical - if poetry uses the wrong venv, dependencies won't be installed correctly + print("Verifying poetry virtual environment...") + + # Wait a moment for symlink to be recognized (if we created one) + import time + + time.sleep(0.5) + + verify_result = subprocess.run( + [str(poetry_exe), "env", "info", "--path"], + cwd=str(script_dir), + env=venv_env, + check=False, # Don't fail if poetry hasn't activated a venv yet + capture_output=True, + text=True, + ) + + expected_venv_path_resolved = str(Path(venv_dir.absolute()).resolve()) + + # If poetry env info fails, it might mean poetry hasn't activated the venv yet + if verify_result.returncode != 0: + print( + "Warning: poetry env info failed, poetry may not have activated the virtual environment yet" + ) + print( + "This may be okay if we created a symlink - poetry should use it when running commands" + ) + poetry_venv_path_resolved = None + else: + poetry_venv_path = verify_result.stdout.strip() + + # Normalize paths for comparison (resolve symlinks, etc.) + poetry_venv_path_resolved = str(Path(poetry_venv_path).resolve()) + + # Only verify path if we successfully got poetry's venv path + if poetry_venv_path_resolved is not None: + if poetry_venv_path_resolved != expected_venv_path_resolved: + print( + f"ERROR: Poetry is using {poetry_venv_path}, but expected {expected_venv_path_resolved}" + ) + print( + "Poetry must use the virtual environment we created for the build to work correctly." + ) + print("The symlink approach may not have worked. Please check the symlink.") + sys.exit(1) + else: + print( + f"✓ Poetry is correctly using virtual environment: {poetry_venv_path}" + ) + else: + print("Warning: Could not verify poetry virtual environment path") + print( + "Continuing anyway - poetry should use the venv via symlink or VIRTUAL_ENV" + ) + + # Update lock file and install dependencies + # Re-verify environment before each command to ensure poetry doesn't switch venvs + def verify_poetry_env(): + verify_result = subprocess.run( + [str(poetry_exe), "env", "info", "--path"], + cwd=str(script_dir), + env=venv_env, + check=False, # Don't fail if poetry env info is not available + capture_output=True, + text=True, + ) + if verify_result.returncode == 0: + current_path = str(Path(verify_result.stdout.strip()).resolve()) + expected_path = str(Path(venv_dir.absolute()).resolve()) + if current_path != expected_path: + print( + f"ERROR: Poetry switched to different virtual environment: {current_path}" + ) + print(f"Expected: {expected_path}") + sys.exit(1) + # If poetry env info fails, we can't verify, but continue anyway + # Poetry should still use the Python we specified via env use + return True + + print("Running poetry lock...") + verify_poetry_env() # Verify before lock + result = subprocess.run( + [str(poetry_exe), "lock"], + cwd=str(script_dir), + env=venv_env, + check=True, + capture_output=True, + text=True, + ) + if result.stdout: + print(result.stdout) + if result.stderr: + print(result.stderr) + verify_poetry_env() # Verify after lock + + print("Running poetry install...") + verify_poetry_env() # Verify before install + result = subprocess.run( + [str(poetry_exe), "install"], + cwd=str(script_dir), + env=venv_env, + check=True, + capture_output=True, + text=True, + ) + if result.stdout: + print(result.stdout) + if result.stderr: + print(result.stderr) + verify_poetry_env() # Verify after install + + # Verify installation by checking if key packages are installed + # This is critical - if packages aren't installed, PyInstaller won't find them + print("Verifying package installation...") + test_packages = ["torch", "transformers", "tokenizers"] + missing_packages = [] + for package in test_packages: + test_result = subprocess.run( + [str(venv_python), "-c", f"import {package}; print({package}.__version__)"], + capture_output=True, + text=True, + check=False, + ) + if test_result.returncode == 0: + version = test_result.stdout.strip() + print(f" ✓ {package} {version} installed") + else: + error_msg = ( + test_result.stderr.strip() if test_result.stderr else "Unknown error" + ) + print(f" ✗ {package} NOT found in virtual environment: {error_msg}") + missing_packages.append(package) + + if missing_packages: + print( + f"\nERROR: Required packages are missing from virtual environment: {', '.join(missing_packages)}" + ) + print("This indicates that poetry did not install dependencies correctly.") + print("Please check the poetry install output above for errors.") + sys.exit(1) + + print("Dependencies installed successfully") + + +def check_pyinstaller(venv_python): + """ + Check if PyInstaller is installed. + + PyInstaller should be installed via poetry install from pyproject.toml. + If it's missing, it means poetry install failed or didn't complete. + """ + try: + result = subprocess.run( + [ + str(venv_python), + "-c", + "import PyInstaller; print(PyInstaller.__version__)", + ], + capture_output=True, + text=True, + check=True, + ) + version = result.stdout.strip() + print(f"PyInstaller version: {version}") + return True + except (subprocess.CalledProcessError, FileNotFoundError): + print("ERROR: PyInstaller is not installed in the virtual environment") + print("PyInstaller should be installed via poetry install from pyproject.toml") + print( + "This indicates that poetry install may have failed or didn't complete correctly." + ) + return False + + +def build(): + """ + Execute the complete build process. + + Steps: + 1. Setup virtual environment (outside project directory) + 2. Update pip and install 2.2.1 poetry + 3. Install project dependencies (including PyInstaller from pyproject.toml) + 4. Build executable using PyInstaller + """ + script_dir = Path(__file__).parent + + venv_dir = setup_venv() + venv_python = get_venv_python(venv_dir) + + update_pip(venv_python) + install_poetry(venv_python) + install_dependencies(venv_python, venv_dir, script_dir) + + if not check_pyinstaller(venv_python): + sys.exit(1) + + print("=" * 50) + print("IoTDB AINode PyInstaller Build Script") + print("=" * 50) + print() + + print("Starting build...") + print() + + spec_file = script_dir / "ainode.spec" + if not spec_file.exists(): + print(f"Error: Spec file not found: {spec_file}") + sys.exit(1) + + # Set up environment for PyInstaller + # When using venv_python, PyInstaller should automatically detect the virtual environment + # and use its site-packages. We should NOT manually add site-packages to pathex. + pyinstaller_env = get_venv_env(venv_dir) + + # Verify we're using the correct Python + python_prefix_result = subprocess.run( + [str(venv_python), "-c", "import sys; print(sys.prefix)"], + capture_output=True, + text=True, + check=True, + ) + python_prefix = python_prefix_result.stdout.strip() + print(f"Using Python from: {python_prefix}") + + # Ensure PyInstaller runs from the virtual environment + # The venv_python should automatically set up the correct environment + cmd = [ + str(venv_python), + "-m", + "PyInstaller", + "--noconfirm", + str(spec_file), + ] + + try: + subprocess.run(cmd, check=True, env=pyinstaller_env) + except subprocess.CalledProcessError as e: + print(f"\nError: Build failed: {e}") + sys.exit(1) + + print() + print("=" * 50) + print("Build completed!") + print("=" * 50) + print() + print("Executable location: dist/ainode/ainode") + print() + print("Usage:") + print(" ./dist/ainode/ainode start # Start AINode") + print() + + +if __name__ == "__main__": + build() diff --git a/iotdb-core/ainode/iotdb/ainode/core/config.py b/iotdb-core/ainode/iotdb/ainode/core/config.py index 328f4a5faa610..afcf0683d7d04 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/config.py +++ b/iotdb-core/ainode/iotdb/ainode/core/config.py @@ -16,6 +16,7 @@ # under the License. # import os +import re from iotdb.ainode.core.constant import ( AINODE_BUILD_INFO, @@ -37,8 +38,6 @@ AINODE_INFERENCE_MODEL_MEM_USAGE_MAP, AINODE_LOG_DIR, AINODE_MODELS_DIR, - AINODE_ROOT_CONF_DIRECTORY_NAME, - AINODE_ROOT_DIR, AINODE_RPC_ADDRESS, AINODE_RPC_PORT, AINODE_SYSTEM_DIR, @@ -315,9 +314,7 @@ def _load_config_from_file(self) -> None: if "ainode_id" in system_configs: self._config.set_ainode_id(int(system_configs["ainode_id"])) - git_file = os.path.join( - AINODE_ROOT_DIR, AINODE_ROOT_CONF_DIRECTORY_NAME, AINODE_CONF_GIT_FILE_NAME - ) + git_file = os.path.join(AINODE_CONF_DIRECTORY_NAME, AINODE_CONF_GIT_FILE_NAME) if os.path.exists(git_file): git_configs = load_properties(git_file) if "git.commit.id.abbrev" in git_configs: @@ -327,9 +324,7 @@ def _load_config_from_file(self) -> None: build_info += "-dev" self._config.set_build_info(build_info) - pom_file = os.path.join( - AINODE_ROOT_DIR, AINODE_ROOT_CONF_DIRECTORY_NAME, AINODE_CONF_POM_FILE_NAME - ) + pom_file = os.path.join(AINODE_CONF_DIRECTORY_NAME, AINODE_CONF_POM_FILE_NAME) if os.path.exists(pom_file): pom_configs = load_properties(pom_file) if "version" in pom_configs: @@ -453,18 +448,29 @@ def get_config(self) -> AINodeConfig: return self._config +def unescape_java_properties(value: str) -> str: + """Undo Java Properties escaping rules""" + value = value.replace("\\t", "\t") + value = value.replace("\\n", "\n") + value = value.replace("\\r", "\r") + value = value.replace("\\\\", "\\") + value = re.sub(r"\\([:=\s])", r"\1", value) + return value + + def load_properties(filepath, sep="=", comment_char="#"): """ Read the file passed as parameter as a properties file. """ props = {} - with open(filepath, "rt") as f: + with open(filepath, "rt", encoding="utf-8") as f: for line in f: l = line.strip() if l and not l.startswith(comment_char): - key_value = l.split(sep) + key_value = l.split(sep, 1) key = key_value[0].strip() - value = sep.join(key_value[1:]).strip().strip('"') + value = key_value[1].strip().strip('"') + value = unescape_java_properties(value) props[key] = value return props diff --git a/iotdb-core/ainode/iotdb/ainode/core/constant.py b/iotdb-core/ainode/iotdb/ainode/core/constant.py index f4547e99803e0..b9923d3e3ee7e 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/constant.py +++ b/iotdb-core/ainode/iotdb/ainode/core/constant.py @@ -15,7 +15,6 @@ # specific language governing permissions and limitations # under the License. # -import inspect import logging import os from enum import Enum @@ -24,10 +23,10 @@ from iotdb.ainode.core.model.model_enums import BuiltInModelType from iotdb.thrift.common.ttypes import TEndPoint +IOTDB_AINODE_HOME = os.getenv("IOTDB_AINODE_HOME", "") AINODE_VERSION_INFO = "UNKNOWN" AINODE_BUILD_INFO = "UNKNOWN" -AINODE_CONF_DIRECTORY_NAME = "conf" -AINODE_ROOT_CONF_DIRECTORY_NAME = "conf" +AINODE_CONF_DIRECTORY_NAME = os.path.join(IOTDB_AINODE_HOME, "conf") AINODE_CONF_FILE_NAME = "iotdb-ainode.properties" AINODE_CONF_GIT_FILE_NAME = "git.properties" AINODE_CONF_POM_FILE_NAME = "pom.properties" @@ -62,13 +61,12 @@ ) # AINode folder structure -AINODE_ROOT_DIR = os.path.dirname( - os.path.dirname(os.path.abspath(inspect.getfile(inspect.currentframe()))) -) -AINODE_MODELS_DIR = "data/ainode/models" -AINODE_BUILTIN_MODELS_DIR = "data/ainode/models/weights" # For built-in models, we only need to store their weights and config. -AINODE_SYSTEM_DIR = "data/ainode/system" -AINODE_LOG_DIR = "logs" +AINODE_MODELS_DIR = os.path.join(IOTDB_AINODE_HOME, "data/ainode/models") +AINODE_BUILTIN_MODELS_DIR = os.path.join( + IOTDB_AINODE_HOME, "data/ainode/models/weights" +) # For built-in models, we only need to store their weights and config. +AINODE_SYSTEM_DIR = os.path.join(IOTDB_AINODE_HOME, "data/ainode/system") +AINODE_LOG_DIR = os.path.join(IOTDB_AINODE_HOME, "logs") # AINode log LOG_FILE_TYPE = ["all", "info", "warn", "error"] diff --git a/iotdb-core/ainode/iotdb/ainode/core/inference/pool_controller.py b/iotdb-core/ainode/iotdb/ainode/core/inference/pool_controller.py index 069a6b9ced6d3..00bb3b5568b0f 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/inference/pool_controller.py +++ b/iotdb-core/ainode/iotdb/ainode/core/inference/pool_controller.py @@ -50,7 +50,6 @@ from iotdb.ainode.core.util.thread_name import ThreadName logger = Logger() -MODEL_MANAGER = ModelManager() class PoolController: @@ -59,6 +58,7 @@ class PoolController: """ def __init__(self, result_queue: mp.Queue): + self._model_manager = ModelManager() # structure: {model_id: {device_id: PoolGroup}} self._request_pool_map: Dict[str, Dict[str, PoolGroup]] = {} self._new_pool_id = AtomicInt() @@ -82,24 +82,25 @@ def first_req_init(self, model_id: str): """ Initialize the pools when the first request for the given model_id arrives. """ - if not self.has_request_pools(model_id, device.index): - # TODO: choose a device based on some strategy - device = self.DEFAULT_DEVICE - actions = self._pool_scheduler.schedule(model_id, device) - for action in actions: - if action.action == ScaleActionType.SCALE_UP: - # initialize the first pool - self._first_pool_init(action.model_id, str(device)) - # start a background thread to expand pools - expand_thread = threading.Thread( - target=self._expand_pools_on_device, - args=(action.model_id, str(device), action.amount - 1), - daemon=True, - ) - expand_thread.start() - elif action.action == ScaleActionType.SCALE_DOWN: - # TODO: implement scale down logic - pass + pass + # if not self.has_request_pools(model_id, device.index): + # # TODO: choose a device based on some strategy + # device = self.DEFAULT_DEVICE + # actions = self._pool_scheduler.schedule(model_id, device) + # for action in actions: + # if action.action == ScaleActionType.SCALE_UP: + # # initialize the first pool + # self._first_pool_init(action.model_id, str(device)) + # # start a background thread to expand pools + # expand_thread = threading.Thread( + # target=self._expand_pools_on_device, + # args=(action.model_id, str(device), action.amount - 1), + # daemon=True, + # ) + # expand_thread.start() + # elif action.action == ScaleActionType.SCALE_DOWN: + # # TODO: implement scale down logic + # pass def _first_pool_init(self, model_id: str, device_str: str): """ @@ -194,7 +195,7 @@ def _load_model_task(self, model_id: str, device_id_list: list[str]): def _load_model_on_device_task(device_id: str): if not self.has_request_pools(model_id, device_id): actions = self._pool_scheduler.schedule_load_model_to_device( - MODEL_MANAGER.get_model_info(model_id), device_id + self._model_manager.get_model_info(model_id), device_id ) for action in actions: if action.action == ScaleActionType.SCALE_UP: @@ -221,7 +222,7 @@ def _unload_model_task(self, model_id: str, device_id_list: list[str]): def _unload_model_on_device_task(device_id: str): if self.has_request_pools(model_id, device_id): actions = self._pool_scheduler.schedule_unload_model_from_device( - MODEL_MANAGER.get_model_info(model_id), device_id + self._model_manager.get_model_info(model_id), device_id ) for action in actions: if action.action == ScaleActionType.SCALE_DOWN: @@ -256,7 +257,7 @@ def _expand_pools_on_device(self, model_id: str, device_id: str, count: int): def _expand_pool_on_device(*_): result_queue = mp.Queue() pool_id = self._new_pool_id.get_and_increment() - model_info = MODEL_MANAGER.get_model_info(model_id) + model_info = self._model_manager.get_model_info(model_id) model_type = model_info.model_type if model_type == BuiltInModelType.SUNDIAL.value: config = SundialConfig() @@ -277,7 +278,7 @@ def _expand_pool_on_device(*_): ) pool.start() self._register_pool(model_id, device_id, pool_id, pool, result_queue) - if not pool.ready_event.wait(timeout=30): + if not pool.ready_event.wait(timeout=300): logger.error( f"[Inference][Device-{device_id}][Pool-{pool_id}] Pool failed to be ready in time" ) diff --git a/iotdb-core/ainode/iotdb/ainode/core/inference/pool_scheduler/basic_pool_scheduler.py b/iotdb-core/ainode/iotdb/ainode/core/inference/pool_scheduler/basic_pool_scheduler.py index 5ee1b4f0c9a29..6a2bd2b619aa7 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/inference/pool_scheduler/basic_pool_scheduler.py +++ b/iotdb-core/ainode/iotdb/ainode/core/inference/pool_scheduler/basic_pool_scheduler.py @@ -41,8 +41,6 @@ logger = Logger() -MODEL_MANAGER = ModelManager() - def _estimate_shared_pool_size_by_total_mem( device: torch.device, @@ -106,6 +104,7 @@ class BasicPoolScheduler(AbstractPoolScheduler): def __init__(self, request_pool_map: Dict[str, Dict[str, PoolGroup]]): super().__init__(request_pool_map) + self._model_manager = ModelManager() def schedule(self, model_id: str) -> List[ScaleAction]: """ @@ -123,7 +122,7 @@ def schedule_load_model_to_device( self, model_info: ModelInfo, device_id: str ) -> List[ScaleAction]: existing_model_infos = [ - MODEL_MANAGER.get_model_info(existing_model_id) + self._model_manager.get_model_info(existing_model_id) for existing_model_id, pool_group_map in self._request_pool_map.items() if existing_model_id != model_info.model_id and device_id in pool_group_map ] @@ -140,7 +139,7 @@ def schedule_unload_model_from_device( self, model_info: ModelInfo, device_id: str ) -> List[ScaleAction]: existing_model_infos = [ - MODEL_MANAGER.get_model_info(existing_model_id) + self._model_manager.get_model_info(existing_model_id) for existing_model_id, pool_group_map in self._request_pool_map.items() if existing_model_id != model_info.model_id and device_id in pool_group_map ] diff --git a/iotdb-core/ainode/iotdb/ainode/core/log.py b/iotdb-core/ainode/iotdb/ainode/core/log.py index a7a05d0ea9298..fd121b26349d7 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/log.py +++ b/iotdb-core/ainode/iotdb/ainode/core/log.py @@ -116,7 +116,9 @@ def gzip_rotator(src: str, dst: str): file_handler.setFormatter(self.logger_format) self.logger.addHandler(file_handler) - self.info(f"Logger init successfully.") + self.info( + f"Logger init successfully, log file prefix name {log_file_name_prefix}." + ) # interfaces def debug(self, *msg): diff --git a/iotdb-core/ainode/iotdb/ainode/core/script.py b/iotdb-core/ainode/iotdb/ainode/core/script.py index 82ed32bc5a245..38653d7ceab9f 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/script.py +++ b/iotdb-core/ainode/iotdb/ainode/core/script.py @@ -15,68 +15,27 @@ # specific language governing permissions and limitations # under the License. # -import os -import shutil +import multiprocessing import sys import torch.multiprocessing as mp from iotdb.ainode.core.ai_node import AINode -from iotdb.ainode.core.config import AINodeDescriptor -from iotdb.ainode.core.constant import TSStatusCode -from iotdb.ainode.core.exception import MissingConfigError from iotdb.ainode.core.log import Logger -from iotdb.ainode.core.rpc.client import ClientManager -from iotdb.thrift.common.ttypes import TAINodeLocation, TEndPoint logger = Logger() -def remove_ainode(arguments): - # Delete the current node - if len(arguments) == 2: - target_ainode_id = AINodeDescriptor().get_config().get_ainode_id() - target_rpc_address = AINodeDescriptor().get_config().get_ain_rpc_address() - target_rpc_port = AINodeDescriptor().get_config().get_ain_rpc_port() - - # Delete the node with a given id - elif len(arguments) == 3: - target_ainode_id = int(arguments[2]) - ainode_configuration_map = ( - ClientManager() - .borrow_config_node_client() - .get_ainode_configuration(target_ainode_id) - ) - - end_point = ainode_configuration_map[target_ainode_id].location.internalEndPoint - target_rpc_address = end_point.ip - target_rpc_port = end_point.port - - if not end_point: - raise MissingConfigError( - "NodeId: {} not found in cluster ".format(target_ainode_id) - ) - - logger.info("Got target AINode id: {}".format(target_ainode_id)) - - else: - raise MissingConfigError("Invalid command") - - location = TAINodeLocation( - target_ainode_id, TEndPoint(target_rpc_address, target_rpc_port) - ) - status = ClientManager().borrow_config_node_client().node_remove(location) - - if status.code == TSStatusCode.SUCCESS_STATUS.get_status_code(): - logger.info("IoTDB-AINode has successfully removed.") - if os.path.exists(AINodeDescriptor().get_config().get_ain_models_dir()): - shutil.rmtree(AINodeDescriptor().get_config().get_ain_models_dir()) - - def main(): + # Handle PyInstaller: filter out Python arguments that might be passed to subprocesses + # These arguments are not needed in frozen executables and cause warnings + # Note: This filtering should happen AFTER freeze_support() has handled child processes + if getattr(sys, "frozen", False): + python_args_to_filter = ["-I", "-B", "-S", "-E", "-O", "-OO"] + sys.argv = [arg for arg in sys.argv if arg not in python_args_to_filter] + + logger.info(f"Starting IoTDB-AINode process with sys argv {sys.argv}.") arguments = sys.argv - # load config - AINodeDescriptor() if len(arguments) == 1: logger.info("Command line argument must be specified.") return @@ -89,19 +48,22 @@ def main(): ai_node = AINode() ai_node.start() except Exception as e: - logger.error("Start AINode failed, because of: {}".format(e)) - sys.exit(1) - # TODO: remove the following function, and add a destroy script - elif command == "remove": - try: - logger.info("Removing AINode...") - remove_ainode(arguments) - except Exception as e: - logger.error("Remove AINode failed, because of: {}".format(e)) + logger.warning("Start AINode failed, because of: {}".format(e)) sys.exit(1) else: logger.warning("Unknown argument: {}.".format(command)) if __name__ == "__main__": + # PyInstaller multiprocessing support + # freeze_support() is essential for PyInstaller frozen executables on all platforms + # It detects if the current process is a multiprocessing child process + # If it is, it executes the child process target function and exits + # If it's not, it returns immediately and continues with main() execution + # This prevents child processes from executing the main application logic + if getattr(sys, "frozen", False): + # Call freeze_support() for both standard multiprocessing and torch.multiprocessing + multiprocessing.freeze_support() + mp.freeze_support() + main() diff --git a/iotdb-core/ainode/poetry.lock b/iotdb-core/ainode/poetry.lock index 7a22506d06fe5..a2d8ce581f16e 100644 --- a/iotdb-core/ainode/poetry.lock +++ b/iotdb-core/ainode/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 2.1.4 and should not be changed by hand. +# This file is automatically @generated by Poetry 2.2.1 and should not be changed by hand. [[package]] name = "alembic" @@ -20,6 +20,18 @@ typing-extensions = ">=4.12" [package.extras] tz = ["tzdata"] +[[package]] +name = "altgraph" +version = "0.17.4" +description = "Python graph (network) package" +optional = false +python-versions = "*" +groups = ["main"] +files = [ + {file = "altgraph-0.17.4-py2.py3-none-any.whl", hash = "sha256:642743b4750de17e655e6711601b077bc6598dbfa3ba5fa2b2a35ce12b508dff"}, + {file = "altgraph-0.17.4.tar.gz", hash = "sha256:1b5afbb98f6c4dcadb2e2ae6ab9fa994bbb8c1d75f4fa96d340f9437ae454406"}, +] + [[package]] name = "black" version = "25.1.0" @@ -510,6 +522,22 @@ cli = ["jsonargparse[signatures] (>=4.38.0)", "tomlkit"] docs = ["requests (>=2.0.0)"] typing = ["mypy (>=1.0.0)", "types-setuptools"] +[[package]] +name = "macholib" +version = "1.16.3" +description = "Mach-O header analysis and editing" +optional = false +python-versions = "*" +groups = ["main"] +markers = "sys_platform == \"darwin\"" +files = [ + {file = "macholib-1.16.3-py2.py3-none-any.whl", hash = "sha256:0e315d7583d38b8c77e815b1ecbdbf504a8258d8b3e17b61165c6feb60d18f2c"}, + {file = "macholib-1.16.3.tar.gz", hash = "sha256:07ae9e15e8e4cd9a788013d81f5908b3609aa76f9b1421bae9c4d7606ec86a30"}, +] + +[package.dependencies] +altgraph = ">=0.17" + [[package]] name = "mako" version = "1.3.10" @@ -1141,6 +1169,19 @@ numpy = ">=1.4" [package.extras] test = ["pytest", "pytest-cov", "scipy"] +[[package]] +name = "pefile" +version = "2023.2.7" +description = "Python PE parsing module" +optional = false +python-versions = ">=3.6.0" +groups = ["main"] +markers = "sys_platform == \"win32\"" +files = [ + {file = "pefile-2023.2.7-py3-none-any.whl", hash = "sha256:da185cd2af68c08a6cd4481f7325ed600a88f6a813bad9dea07ab3ef73d8d8d6"}, + {file = "pefile-2023.2.7.tar.gz", hash = "sha256:82e6114004b3d6911c77c3953e3838654b04511b8b66e8583db70c65998017dc"}, +] + [[package]] name = "platformdirs" version = "4.4.0" @@ -1181,6 +1222,57 @@ files = [ dev = ["abi3audit", "black", "check-manifest", "coverage", "packaging", "pylint", "pyperf", "pypinfo", "pyreadline ; os_name == \"nt\"", "pytest", "pytest-cov", "pytest-instafail", "pytest-subtests", "pytest-xdist", "pywin32 ; os_name == \"nt\" and platform_python_implementation != \"PyPy\"", "requests", "rstcheck", "ruff", "setuptools", "sphinx", "sphinx_rtd_theme", "toml-sort", "twine", "virtualenv", "vulture", "wheel", "wheel ; os_name == \"nt\" and platform_python_implementation != \"PyPy\"", "wmi ; os_name == \"nt\" and platform_python_implementation != \"PyPy\""] test = ["pytest", "pytest-instafail", "pytest-subtests", "pytest-xdist", "pywin32 ; os_name == \"nt\" and platform_python_implementation != \"PyPy\"", "setuptools", "wheel ; os_name == \"nt\" and platform_python_implementation != \"PyPy\"", "wmi ; os_name == \"nt\" and platform_python_implementation != \"PyPy\""] +[[package]] +name = "pyinstaller" +version = "6.16.0" +description = "PyInstaller bundles a Python application and all its dependencies into a single package." +optional = false +python-versions = "<3.15,>=3.8" +groups = ["main"] +files = [ + {file = "pyinstaller-6.16.0-py3-none-macosx_10_13_universal2.whl", hash = "sha256:7fd1c785219a87ca747c21fa92f561b0d2926a7edc06d0a0fe37f3736e00bd7a"}, + {file = "pyinstaller-6.16.0-py3-none-manylinux2014_aarch64.whl", hash = "sha256:b756ddb9007b8141c5476b553351f9d97559b8af5d07f9460869bfae02be26b0"}, + {file = "pyinstaller-6.16.0-py3-none-manylinux2014_i686.whl", hash = "sha256:0a48f55b85ff60f83169e10050f2759019cf1d06773ad1c4da3a411cd8751058"}, + {file = "pyinstaller-6.16.0-py3-none-manylinux2014_ppc64le.whl", hash = "sha256:73ba72e04fcece92e32518bbb1e1fb5ac2892677943dfdff38e01a06e8742851"}, + {file = "pyinstaller-6.16.0-py3-none-manylinux2014_s390x.whl", hash = "sha256:b1752488248f7899281b17ca3238eefb5410521291371a686a4f5830f29f52b3"}, + {file = "pyinstaller-6.16.0-py3-none-manylinux2014_x86_64.whl", hash = "sha256:ba618a61627ee674d6d68e5de084ba17c707b59a4f2a856084b3999bdffbd3f0"}, + {file = "pyinstaller-6.16.0-py3-none-musllinux_1_1_aarch64.whl", hash = "sha256:c8b7ef536711617e12fef4673806198872033fa06fa92326ad7fd1d84a9fa454"}, + {file = "pyinstaller-6.16.0-py3-none-musllinux_1_1_x86_64.whl", hash = "sha256:d1ebf84d02c51fed19b82a8abb4df536923abd55bb684d694e1356e4ae2a0ce5"}, + {file = "pyinstaller-6.16.0-py3-none-win32.whl", hash = "sha256:6d5f8617f3650ff9ef893e2ab4ddbf3c0d23d0c602ef74b5df8fbef4607840c8"}, + {file = "pyinstaller-6.16.0-py3-none-win_amd64.whl", hash = "sha256:bc10eb1a787f99fea613509f55b902fbd2d8b73ff5f51ff245ea29a481d97d41"}, + {file = "pyinstaller-6.16.0-py3-none-win_arm64.whl", hash = "sha256:d0af8a401de792c233c32c44b16d065ca9ab8262ee0c906835c12bdebc992a64"}, + {file = "pyinstaller-6.16.0.tar.gz", hash = "sha256:53559fe1e041a234f2b4dcc3288ea8bdd57f7cad8a6644e422c27bb407f3edef"}, +] + +[package.dependencies] +altgraph = "*" +macholib = {version = ">=1.8", markers = "sys_platform == \"darwin\""} +packaging = ">=22.0" +pefile = {version = ">=2022.5.30,<2024.8.26 || >2024.8.26", markers = "sys_platform == \"win32\""} +pyinstaller-hooks-contrib = ">=2025.8" +pywin32-ctypes = {version = ">=0.2.1", markers = "sys_platform == \"win32\""} +setuptools = ">=42.0.0" + +[package.extras] +completion = ["argcomplete"] +hook-testing = ["execnet (>=1.5.0)", "psutil", "pytest (>=2.7.3)"] + +[[package]] +name = "pyinstaller-hooks-contrib" +version = "2025.9" +description = "Community maintained hooks for PyInstaller" +optional = false +python-versions = ">=3.8" +groups = ["main"] +files = [ + {file = "pyinstaller_hooks_contrib-2025.9-py3-none-any.whl", hash = "sha256:ccbfaa49399ef6b18486a165810155e5a8d4c59b41f20dc5da81af7482aaf038"}, + {file = "pyinstaller_hooks_contrib-2025.9.tar.gz", hash = "sha256:56e972bdaad4e9af767ed47d132362d162112260cbe488c9da7fee01f228a5a6"}, +] + +[package.dependencies] +packaging = ">=22.0" +setuptools = ">=42.0.0" + [[package]] name = "python-dateutil" version = "2.9.0.post0" @@ -1208,6 +1300,19 @@ files = [ {file = "pytz-2025.2.tar.gz", hash = "sha256:360b9e3dbb49a209c21ad61809c7fb453643e048b38924c765813546746e81c3"}, ] +[[package]] +name = "pywin32-ctypes" +version = "0.2.3" +description = "A (partial) reimplementation of pywin32 using ctypes/cffi" +optional = false +python-versions = ">=3.6" +groups = ["main"] +markers = "sys_platform == \"win32\"" +files = [ + {file = "pywin32-ctypes-0.2.3.tar.gz", hash = "sha256:d162dc04946d704503b2edc4d55f3dba5c1d539ead017afa00142c38b9885755"}, + {file = "pywin32_ctypes-0.2.3-py3-none-any.whl", hash = "sha256:8a1513379d709975552d202d942d9837758905c8d01eb82b8bcc30918929e7b8"}, +] + [[package]] name = "pyyaml" version = "6.0.3" @@ -2222,5 +2327,5 @@ zstd = ["zstandard (>=0.18.0)"] [metadata] lock-version = "2.1" -python-versions = ">=3.11,<=3.13.5" -content-hash = "08746b5f45ee9836e2d4368f8e394d55d27f556789fb1254e7584b9eedd49d5b" +python-versions = ">=3.11.0,<3.14.0" +content-hash = "37441b66c5cf440280823bb97248fa10b0ca20d7fef4d20aec616e3860a31736" diff --git a/iotdb-core/ainode/pom.xml b/iotdb-core/ainode/pom.xml index 63c8f2e3f554c..1a3be76b63369 100644 --- a/iotdb-core/ainode/pom.xml +++ b/iotdb-core/ainode/pom.xml @@ -87,9 +87,6 @@ target - - venv - @@ -142,6 +139,9 @@ ${project.build.sourceEncoding} + copy-python-dependencies generate-sources @@ -212,55 +212,6 @@ org.codehaus.mojo exec-maven-plugin - - - python-venv - initialize - - exec - - - ${python.exe.bin} - - -m - venv - ./venv - - - - - - python-upgrade-pip - initialize - - exec - - - ${python.venv.bin}${python.exe.bin} - - -m - pip - install - --upgrade - pip - - - - - - python-install-poetry - initialize - - exec - - - ${python.venv.bin}pip3 - - install - poetry - - - python-compile compile @@ -268,44 +219,13 @@ exec - ${python.venv.bin}poetry + ${python.exe.bin} + ${project.basedir} - build + build_binary.py - - diff --git a/iotdb-core/ainode/pyproject.toml b/iotdb-core/ainode/pyproject.toml index 634f3e09a7445..331cb8ab32a34 100644 --- a/iotdb-core/ainode/pyproject.toml +++ b/iotdb-core/ainode/pyproject.toml @@ -76,24 +76,7 @@ exclude = [ ] [tool.poetry.dependencies] -python = ">=3.11,<=3.13.5" - -# Core scientific stack -numpy = [ - { version = "^2.3.2", python = ">=3.10" }, - { version = "^1.26.4", python = ">=3.9,<3.10" } -] -scipy = [ - { version = "^1.12.0", python = ">=3.10" }, - { version = "^1.11.4", python = ">=3.9,<3.10" } -] -pandas = "^2.3.2" -scikit-learn = [ - { version = "^1.7.1", python = ">=3.10" }, - { version = "^1.5.2", python = ">=3.9,<3.10" } -] -statsmodels = "^0.14.5" -sktime = "0.38.5" +python = ">=3.11.0,<3.14.0" # ---- DL / HF stack ---- torch = ">=2.7.0" @@ -104,6 +87,14 @@ huggingface_hub = "^0.34.4" safetensors = "^0.6.2" einops = "^0.8.1" +# ---- Core scientific stack ---- +numpy = "^2.3.2" +scipy = "^1.12.0" +pandas = "^2.3.2" +scikit-learn = "^1.7.1" +statsmodels = "^0.14.5" +sktime = "0.38.5" + # ---- Optimizers / utils ---- optuna = "^4.4.0" psutil = "^7.0.0" @@ -116,6 +107,7 @@ tzlocal = "^5.3.1" hf_xet = ">=1.1.9" # ---- Tooling ---- +pyinstaller = "6.16.0" black = "25.1.0" isort = "6.0.1" setuptools = ">=75.3.0" @@ -127,3 +119,4 @@ ainode = "iotdb.ainode.core.script:main" [tool.isort] profile = "black" +skip = ["build", "dist", "target"] \ No newline at end of file diff --git a/iotdb-core/ainode/resources/syncPythonVersion.groovy b/iotdb-core/ainode/resources/syncPythonVersion.groovy index 0061930bfca2f..ecd0f2bdf21a2 100644 --- a/iotdb-core/ainode/resources/syncPythonVersion.groovy +++ b/iotdb-core/ainode/resources/syncPythonVersion.groovy @@ -120,41 +120,8 @@ def checkPython() { } } - -// On Ubuntu it seems that venv is generally available, but the 'ensurepip' command fails. -// In this case we need to install the python3-venv package. Unfortunately checking the -// venv is successful in this case, so we need this slightly odd test. -def checkPythonVenv() { - print "Detecting venv: " - try { - def python = project.properties['python.exe.bin'] - def cmdArray = [python, "-Im", "ensurepip"] - def process = cmdArray.execute() - def stdOut = new StringBuilder() - def stdErr = new StringBuilder() - process.waitForProcessOutput(stdOut, stdErr) - if (stdErr.contains("No module named")) { - println "missing" - println "--- output of version `python -Im \"ensurepip\"` command ---" - println output - println "------------------------------------------------------------" - allConditionsMet = false - } else { - println " OK" - } - } catch (Exception e) { - println "missing" - println "--- failed with exception ---" - println e - e.printStackTrace() - println "----------------------------------------------------" - allConditionsMet = false - } -} - // Check the python environment is setup correctly. checkPython() -checkPythonVenv() if (!allConditionsMet) { throw new RuntimeException("Not all conditions met, see log for details.") diff --git a/scripts/conf/ainode-env.sh b/scripts/conf/ainode-env.sh deleted file mode 100644 index 1ec434ad2d983..0000000000000 --- a/scripts/conf/ainode-env.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/bin/bash -# -# 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. -# - -# The defaulte venv environment is used if ain_interpreter_dir is not set. Please use absolute path without quotation mark -# ain_interpreter_dir= - -# Set ain_force_reinstall to 1 to force reinstall AINode -ain_force_reinstall=0 - -# don't install dependencies online -ain_install_offline=0 - -SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" >/dev/null 2>&1 && pwd)" - -# fetch parameters with names -while getopts "i:t:rnm:" opt; do - case $opt in - i) - p_ain_interpreter_dir="$OPTARG" - ;; - r) - p_ain_force_reinstall=1 - ;; - t) ;; - n) - p_ain_no_dependencies="--no-dependencies" - ;; - m) - p_pypi_mirror="$OPTARG" - ;; - \?) - echo "Invalid option -$OPTARG" >&2 - exit 1 - ;; - esac -done - -if [ -z "$p_ain_interpreter_dir" ]; then - echo "No interpreter_dir is set, use default value." -else - ain_interpreter_dir="$p_ain_interpreter_dir" -fi - -if [ -z "$p_ain_force_reinstall" ]; then - echo "No check_version is set, use default value." -else - ain_force_reinstall="$p_ain_force_reinstall" -fi -echo Script got inputs: "ain_interpreter_dir: $ain_interpreter_dir", "ain_force_reinstall: $ain_force_reinstall" - -if [ -z $ain_interpreter_dir ]; then - $(dirname "$0")/../venv/bin/python3 -c "import sys; print(sys.executable)" && - echo "Activate default venv environment" || ( - echo "Creating default venv environment" && python3 -m venv "$(dirname "$0")/../venv" - ) - ain_interpreter_dir="$SCRIPT_DIR/../venv/bin/python3" -fi -echo "Calling venv to check: $ain_interpreter_dir" - -# Change the working directory to the parent directory -cd "$SCRIPT_DIR/.." - -echo "Confirming AINode..." -$ain_interpreter_dir -m pip config set global.disable-pip-version-check true -$ain_interpreter_dir -m pip list | grep "apache-iotdb-ainode" >/dev/null -if [ $? -eq 0 ]; then - if [ $ain_force_reinstall -eq 0 ]; then - echo "AINode is already installed" - exit 0 - fi -fi - -ain_only_ainode=1 - -# if $ain_install_offline is 1 then do not install dependencies -if [ $ain_install_offline -eq 1 ]; then - # if offline and not -n, then install dependencies - if [ -z "$p_ain_no_dependencies" ]; then - ain_only_ainode=0 - else - ain_only_ainode=1 - fi - p_ain_no_dependencies="--no-dependencies" - echo "Installing AINode offline----without dependencies..." -fi - -if [ $ain_force_reinstall -eq 1 ]; then - p_ain_force_reinstall="--force-reinstall" -else - p_ain_force_reinstall="" -fi - -echo "Installing AINode..." -cd "$SCRIPT_DIR/../lib/" -shopt -s nullglob -for i in *.whl; do - if [[ $i =~ "ainode" ]]; then - echo Installing AINode body: $i - if [ -z "$p_pypi_mirror" ]; then - $ain_interpreter_dir -m pip install "$i" $p_ain_force_reinstall --no-warn-script-location $p_ain_no_dependencies --find-links https://download.pytorch.org/whl/cpu/torch_stable.html - else - $ain_interpreter_dir -m pip install "$i" $p_ain_force_reinstall -i $p_pypi_mirror --no-warn-script-location $p_ain_no_dependencies --find-links https://download.pytorch.org/whl/cpu/torch_stable.html - fi - else - # if ain_only_ainode is 0 then install dependencies - if [ $ain_only_ainode -eq 0 ]; then - echo Installing dependencies $i - if [ -z "$p_pypi_mirror" ]; then - $ain_interpreter_dir -m pip install "$i" $p_ain_force_reinstall --no-warn-script-location $p_ain_no_dependencies - else - $ain_interpreter_dir -m pip install "$i" $p_ain_force_reinstall -i $p_pypi_mirror --no-warn-script-location $p_ain_no_dependencies - fi - fi - fi - if [ $? -eq 1 ]; then - echo "Failed to install AINode" - exit 1 - fi -done -echo "AINode is installed successfully" -exit 0 diff --git a/scripts/conf/windows/ainode-env.bat b/scripts/conf/windows/ainode-env.bat deleted file mode 100644 index 2c01d411a2cdf..0000000000000 --- a/scripts/conf/windows/ainode-env.bat +++ /dev/null @@ -1,129 +0,0 @@ -@REM -@REM Licensed to the Apache Software Foundation (ASF) under one -@REM or more contributor license agreements. See the NOTICE file -@REM distributed with this work for additional information -@REM regarding copyright ownership. The ASF licenses this file -@REM to you under the Apache License, Version 2.0 (the -@REM "License"); you may not use this file except in compliance -@REM with the License. You may obtain a copy of the License at -@REM -@REM http://www.apache.org/licenses/LICENSE-2.0 -@REM -@REM Unless required by applicable law or agreed to in writing, -@REM software distributed under the License is distributed on an -@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -@REM KIND, either express or implied. See the License for the -@REM specific language governing permissions and limitations -@REM under the License. -@REM - -@echo off - -@REM The defaulte venv environment is used if ain_interpreter_dir is not set. Please use absolute path without quotation mark -@REM set ain_interpreter_dir= - -@REM Set ain_force_reinstall to 1 to force reinstall ainode -set ain_force_reinstall=0 - -@REM don't install dependencies online -set ain_install_offline=0 - -pushd %~dp0..\.. -if NOT DEFINED IOTDB_AINODE_HOME set IOTDB_AINODE_HOME=%cd% - -:initial -if "%1"=="" goto done -set aux=%1 -if "%aux:~0,2%"=="-r" ( - set ain_force_reinstall=1 - shift - goto initial -) -if "%aux:~0,2%"=="-n" ( - set ain_no_dependencies=--no-dependencies - shift - goto initial -) -if "%aux:~0,1%"=="-" ( - set nome=%aux:~1,250% -) else ( - set "%nome%=%1" - set nome= -) -shift -goto initial - -:done -@REM check if the parameters are set -if "%i%"=="" ( - echo No interpreter_dir is set, use default value. -) else ( - set ain_interpreter_dir=%i% -) - -echo Script got inputs: ain_interpreter_dir: %ain_interpreter_dir% , ain_force_reinstall: %ain_force_reinstall% -if "%ain_interpreter_dir%"=="" ( - %IOTDB_AINODE_HOME%//venv//Scripts//python.exe -c "import sys; print(sys.executable)" && ( - echo Activate default venv environment - ) || ( - echo Creating default venv environment - python -m venv "%IOTDB_AINODE_HOME%//venv" - ) - set ain_interpreter_dir="%IOTDB_AINODE_HOME%//venv//Scripts//python.exe" -) - -@REM Switch the working directory to the directory one level above the script -cd %IOTDB_AINODE_HOME% - -echo Confirming ainode -%ain_interpreter_dir% -m pip config set global.disable-pip-version-check true -%ain_interpreter_dir% -m pip list | findstr /C:"apache-iotdb-ainode" >nul -if %errorlevel% == 0 ( - if %ain_force_reinstall% == 0 ( - echo ainode is already installed - exit /b 0 - ) -) - -set ain_only_ainode=1 -@REM if $ain_install_offline is 1 then do not install dependencies -if %ain_install_offline% == 1 ( - @REM if offline and not -n, then install dependencies - if "%ain_no_dependencies%"=="" ( - set ain_only_ainode=0 - ) else ( - set ain_only_ainode=1 - ) - set ain_no_dependencies=--no-dependencies - echo Installing ainode offline----without dependencies... -) - -if %ain_force_reinstall% == 1 ( - set ain_force_reinstall=--force-reinstall -) else ( - set ain_force_reinstall= -) - -echo Installing ainode... -@REM Print current work dir -cd lib -for %%i in (*.whl *.tar.gz) do ( - echo %%i | findstr "ainode" >nul && ( - echo Installing ainode body: %%i - %ain_interpreter_dir% -m pip install %%i %ain_force_reinstall% --no-warn-script-location %ain_no_dependencies% --find-links https://download.pytorch.org/whl/cpu/torch_stable.html - ) || ( - @REM if ain_only_ainode is 0 then install dependencies - if %ain_only_ainode% == 0 ( - echo Installing dependencies: %%i - set ain_force_reinstall=--force-reinstall - %ain_interpreter_dir% -m pip install %%i %ain_force_reinstall% --no-warn-script-location %ain_no_dependencies% --find-links https://download.pytorch.org/whl/cpu/torch_stable.html - ) - ) - if %errorlevel% == 1 ( - echo Failed to install ainode - exit /b 1 - ) -) -echo ainode is installed successfully -cd .. -exit /b 0 diff --git a/scripts/sbin/start-ainode.sh b/scripts/sbin/start-ainode.sh index 4ab202a209ed3..671de54ba1a86 100644 --- a/scripts/sbin/start-ainode.sh +++ b/scripts/sbin/start-ainode.sh @@ -23,24 +23,12 @@ echo Starting IoTDB AINode echo --------------------------- IOTDB_AINODE_HOME="$(cd "`dirname "$0"`"/..; pwd)" - +export IOTDB_AINODE_HOME echo "IOTDB_AINODE_HOME: $IOTDB_AINODE_HOME" -chmod u+x $IOTDB_AINODE_HOME/conf/ainode-env.sh -ain_interpreter_dir=$(sed -n 's/^ain_interpreter_dir=\(.*\)$/\1/p' $IOTDB_AINODE_HOME/conf/ainode-env.sh) -bash $IOTDB_AINODE_HOME/conf/ainode-env.sh $* -if [ $? -eq 1 ]; then - echo "Environment check failed. Exiting..." - exit 1 -fi - # fetch parameters with names while getopts "i:rn" opt; do case $opt in - i) p_ain_interpreter_dir="$OPTARG" - ;; - r) p_ain_force_reinstall="$OPTARG" - ;; n) ;; \?) echo "Invalid option -$OPTARG" >&2 @@ -49,31 +37,10 @@ while getopts "i:rn" opt; do esac done -# If ain_interpreter_dir in parameters is empty: -if [ -z "$p_ain_interpreter_dir" ]; then - # If ain_interpreter_dir in ../conf/ainode-env.sh is empty, set default value to ../venv/bin/python3 - if [ -z "$ain_interpreter_dir" ]; then - ain_interpreter_dir="$IOTDB_AINODE_HOME/venv/bin/python3" - fi -else - # If ain_interpreter_dir in parameters is not empty, set ain_interpreter_dir to the value in parameters - ain_interpreter_dir="$p_ain_interpreter_dir" -fi - -# check if ain_interpreter_dir is an absolute path -if [[ "$ain_interpreter_dir" != /* ]]; then - ain_interpreter_dir="$IOTDB_AINODE_HOME/$ain_interpreter_dir" -fi - -echo Script got parameter: ain_interpreter_dir: $ain_interpreter_dir - -# Change the working directory to the parent directory -cd "$IOTDB_AINODE_HOME" - -ain_ainode_dir=$(dirname "$ain_interpreter_dir")/ainode +ain_ainode_executable="$IOTDB_AINODE_HOME/lib/ainode" -echo Script got ainode dir: ain_ainode_dir: $ain_ainode_dir +echo Script got ainode executable: "$ain_ainode_executable" echo Starting AINode... -$ain_ainode_dir start +$ain_ainode_executable start diff --git a/scripts/sbin/windows/start-ainode.bat b/scripts/sbin/windows/start-ainode.bat index 0a83865fd23c0..1d9a4306bafb7 100644 --- a/scripts/sbin/windows/start-ainode.bat +++ b/scripts/sbin/windows/start-ainode.bat @@ -26,54 +26,12 @@ echo ``````````````````````````` pushd %~dp0..\.. if NOT DEFINED IOTDB_AINODE_HOME set IOTDB_AINODE_HOME=%cd% -call %IOTDB_AINODE_HOME%\\conf\\windows\\ainode-env.bat %* -if %errorlevel% neq 0 ( - echo Environment check failed. Exiting... - exit /b 1 -) +set ain_ainode_executable=%IOTDB_AINODE_HOME%\lib\ainode -for /f "tokens=2 delims==" %%a in ('findstr /i /c:"^ain_interpreter_dir" "%IOTDB_AINODE_HOME%\\conf\\windows\\ainode-env.bat"') do ( - set _ain_interpreter_dir=%%a - goto :done -) - -:initial -if "%1"=="" goto done -set aux=%1 -if "%aux:~0,1%"=="-" ( - set nome=%aux:~1,250% -) else ( - set "%nome%=%1" - set nome= -) -shift -goto initial - -:done -if "%i%"=="" ( - if "%_ain_interpreter_dir%"=="" ( - set _ain_interpreter_dir=%IOTDB_AINODE_HOME%\\venv\\Scripts\\python.exe - ) -) else ( - set _ain_interpreter_dir=%i% -) - -echo Script got parameter: ain_interpreter_dir: %_ain_interpreter_dir% - -cd %IOTDB_AINODE_HOME% - -for %%i in ("%_ain_interpreter_dir%") do set "parent=%%~dpi" - -set ain_ainode_dir=%parent%\ainode.exe - -set ain_ainode_dir_new=%parent%\Scripts\\ainode.exe +echo Script got ainode executable: %ain_ainode_executable% echo Starting AINode... -%ain_ainode_dir% start -if %errorlevel% neq 0 ( - echo ain_ainode_dir_new is %ain_ainode_dir_new% - %ain_ainode_dir_new% start -) +%$ain_ainode_executable% start pause \ No newline at end of file From 4e1df942a492317af3a28fb473c99f6e732ccfc9 Mon Sep 17 00:00:00 2001 From: Le Yang <562593859@qq.com> Date: Wed, 12 Nov 2025 20:16:33 +0800 Subject: [PATCH 052/180] Support the JDBC client to prevent SQL injection (#16734) (cherry picked from commit 96e8681dd51afdce5027a60d01f5ed555bab8ac7) --- .../iotdb/jdbc/IoTDBPreparedStatement.java | 17 +- .../jdbc/IoTDBPreparedStatementTest.java | 154 +++++++++++++++++- 2 files changed, 156 insertions(+), 15 deletions(-) diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java index bb6a7641ab554..c92b6549bf9d0 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java @@ -909,19 +909,18 @@ public void setShort(int parameterIndex, short x) throws SQLException { @Override public void setString(int parameterIndex, String x) { - // if the sql is an insert statement and the value is not a string literal, add single quotes - // The table model only supports single quotes, the tree model sql both single and double quotes - if ("table".equalsIgnoreCase(getSqlDialect()) - || ((sql.trim().toUpperCase().startsWith("INSERT") - && !((x.startsWith("'") && x.endsWith("'")) - || ((x.startsWith("\"") && x.endsWith("\"")) - && "tree".equals(getSqlDialect())))))) { - this.parameters.put(parameterIndex, "'" + x + "'"); + if (x == null) { + this.parameters.put(parameterIndex, null); } else { - this.parameters.put(parameterIndex, x); + this.parameters.put(parameterIndex, "'" + escapeSingleQuotes(x) + "'"); } } + private String escapeSingleQuotes(String value) { + // Escape single quotes with double single quotes + return value.replace("'", "''"); + } + @Override public void setTime(int parameterIndex, Time x) throws SQLException { try { diff --git a/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java b/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java index 1a523d9459f4b..2ae65dfed2aea 100644 --- a/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java +++ b/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java @@ -200,7 +200,7 @@ public void oneStringArgument1() throws Exception { ArgumentCaptor.forClass(TSExecuteStatementReq.class); verify(client).executeStatementV2(argument.capture()); assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 'abcde' and time > 2017-11-1 0:13:00", + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < '''abcde''' and time > 2017-11-1 0:13:00", argument.getValue().getStatement()); } @@ -217,7 +217,7 @@ public void oneStringArgument2() throws Exception { ArgumentCaptor.forClass(TSExecuteStatementReq.class); verify(client).executeStatementV2(argument.capture()); assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < \"abcde\" and time > 2017-11-1 0:13:00", + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < '\"abcde\"' and time > 2017-11-1 0:13:00", argument.getValue().getStatement()); } @@ -233,7 +233,7 @@ public void oneStringArgument3() throws Exception { ArgumentCaptor.forClass(TSExecuteStatementReq.class); verify(client).executeStatementV2(argument.capture()); assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01", argument.getValue().getStatement()); + "SELECT status, 'temperature' FROM root.ln.wf01.wt01", argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -325,7 +325,7 @@ public void testInsertStatement1() throws Exception { ArgumentCaptor.forClass(TSExecuteStatementReq.class); verify(client).executeStatementV2(argument.capture()); assertEquals( - "INSERT INTO root.ln.wf01.wt01(time,a,b,c,d,e,f) VALUES(12324,false,123,123234345,123.423,-1323.0,'abc')", + "INSERT INTO root.ln.wf01.wt01(time,a,b,c,d,e,f) VALUES(12324,false,123,123234345,123.423,-1323.0,'''abc''')", argument.getValue().getStatement()); } @@ -351,7 +351,7 @@ public void testInsertStatement2() throws Exception { ArgumentCaptor.forClass(TSExecuteStatementReq.class); verify(client).executeStatementV2(argument.capture()); assertEquals( - "INSERT INTO root.ln.wf01.wt01(time,a,b,c,d,e,f,g,h) VALUES(2017-11-01T00:13:00,false,123,123234345,123.423,-1323.0,\"abc\",'abc','abc')", + "INSERT INTO root.ln.wf01.wt01(time,a,b,c,d,e,f,g,h) VALUES(2017-11-01T00:13:00,false,123,123234345,123.423,-1323.0,'\"abc\"','abc','''abc''')", argument.getValue().getStatement()); } @@ -374,7 +374,7 @@ public void testInsertStatement3() throws Exception { ArgumentCaptor.forClass(TSExecuteStatementReq.class); verify(client).executeStatementV2(argument.capture()); assertEquals( - "INSERT INTO root.ln.wf01.wt02(time,a,b,c,d,e,f) VALUES(2020-01-01T10:10:10,false,123,123234345,123.423,-1323.0,\"abc\")", + "INSERT INTO root.ln.wf01.wt02(time,a,b,c,d,e,f) VALUES(2020-01-01T10:10:10,false,123,123234345,123.423,-1323.0,'\"abc\"')", argument.getValue().getStatement()); } @@ -400,4 +400,146 @@ public void testInsertStatement4() throws Exception { "INSERT INTO root.ln.wf01.wt02(time,a,b,c,d,e,f) VALUES(2020-01-01T10:10:10,false,123,123234345,123.423,-1323.0,'abc')", argument.getValue().getStatement()); } + + // ========== Table Model SQL Injection Prevention Tests ========== + + @SuppressWarnings("resource") + @Test + public void testTableModelLoginInjectionWithComment() throws Exception { + // Login interface SQL injection attack 1: Using -- comments to bypass password checks + when(connection.getSqlDialect()).thenReturn("table"); + String sql = "SELECT * FROM users WHERE username = ? AND password = ?"; + IoTDBPreparedStatement ps = + new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "admin' --"); + ps.setString(2, "password"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT * FROM users WHERE username = 'admin'' --' AND password = 'password'", + argument.getValue().getStatement()); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelLoginInjectionWithORCondition() throws Exception { + // Login interface SQL injection attack 2: Bypassing authentication by using 'OR '1'='1 + when(connection.getSqlDialect()).thenReturn("table"); + String sql = "SELECT * FROM users WHERE username = ? AND password = ?"; + IoTDBPreparedStatement ps = + new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "admin"); + ps.setString(2, "' OR '1'='1"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT * FROM users WHERE username = 'admin' AND password = ''' OR ''1''=''1'", + argument.getValue().getStatement()); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelQueryWithMultipleInjectionVectors() throws Exception { + when(connection.getSqlDialect()).thenReturn("table"); + String sql = "SELECT * FROM users WHERE email = ?"; + IoTDBPreparedStatement ps = + new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "'; DROP TABLE users;"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT * FROM users WHERE email = '''; DROP TABLE users;'", + argument.getValue().getStatement()); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelString1() throws Exception { + when(connection.getSqlDialect()).thenReturn("table"); + String sql = "SELECT * FROM users WHERE password = ?"; + IoTDBPreparedStatement ps = + new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "a'b"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals("SELECT * FROM users WHERE password = 'a''b'", argument.getValue().getStatement()); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelString2() throws Exception { + when(connection.getSqlDialect()).thenReturn("table"); + String sql = "SELECT * FROM users WHERE password = ?"; + IoTDBPreparedStatement ps = + new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "a\'b"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals("SELECT * FROM users WHERE password = 'a''b'", argument.getValue().getStatement()); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelString3() throws Exception { + when(connection.getSqlDialect()).thenReturn("table"); + String sql = "SELECT * FROM users WHERE password = ?"; + IoTDBPreparedStatement ps = + new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "a\\'b"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT * FROM users WHERE password = 'a\\''b'", argument.getValue().getStatement()); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelString4() throws Exception { + when(connection.getSqlDialect()).thenReturn("table"); + String sql = "SELECT * FROM users WHERE password = ?"; + IoTDBPreparedStatement ps = + new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "a\\\'b"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT * FROM users WHERE password = 'a\\''b'", argument.getValue().getStatement()); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelStringWithNull() throws Exception { + when(connection.getSqlDialect()).thenReturn("table"); + String sql = "SELECT * FROM users WHERE email = ?"; + IoTDBPreparedStatement ps = + new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, null); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals("SELECT * FROM users WHERE email = null", argument.getValue().getStatement()); + } } From a1fe4104c9d855a4b7968de824dd1686335d43d0 Mon Sep 17 00:00:00 2001 From: Jiang Tian Date: Wed, 12 Nov 2025 21:39:50 +0800 Subject: [PATCH 053/180] Fix that a deletion test may fail due to inconsistency caused by leader redistribution (#16744) (cherry picked from commit b7f9f8d49afcf32ed7739e600e74561b72c65d1b) --- .../apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java index 997a80a76f1bd..d08e26be4f0c4 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBDeletionTableIT.java @@ -117,6 +117,8 @@ public static void setUpClass() { .setMemtableSizeThreshold(10000); // Adjust MemTable threshold size to make it flush automatically EnvFactory.getEnv().getConfig().getDataNodeConfig().setCompactionScheduleInterval(5000); + // avoid inconsistency caused by leader migration + EnvFactory.getEnv().getConfig().getConfigNodeConfig().setLeaderDistributionPolicy("HASH"); EnvFactory.getEnv().initClusterEnvironment(); } From d712ffc4d5fdb60af0cd97c9b494209a143b1bef Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 12 Nov 2025 21:44:35 +0800 Subject: [PATCH 054/180] Fixed the NPE caused by concurrent "check for createTimeSeries" and deleteTimeSeries (#16742) * befix * shi (cherry picked from commit 7515568c8e4649fdbb793a5cd539f299be734281) --- .../iotdb/db/schemaengine/schemaregion/ISchemaRegion.java | 5 ++++- .../schemaregion/impl/SchemaRegionMemoryImpl.java | 6 +++++- .../schemaregion/impl/SchemaRegionPBTreeImpl.java | 6 +++++- 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/ISchemaRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/ISchemaRegion.java index 916e52b4acabc..c087d638d4a2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/ISchemaRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/ISchemaRegion.java @@ -146,7 +146,10 @@ void createTimeSeries(final ICreateTimeSeriesPlan plan, final long offset) * @param aliasList a list of alias that you want to check * @return returns a map contains index of the measurements or alias that threw the exception, and * exception details. The exceptions describe whether the measurement or alias exists. For - * example, a MeasurementAlreadyExistException means this measurement exists. + * example, a MeasurementAlreadyExistException means this measurement exists. If there are + * exceptions during check, this may return an empty map, then all the measurements will be + * re-checked under consensus layer, which guarantees safety(Yet may cause unnecessary replay + * of raft log) */ Map checkMeasurementExistence( final PartialPath devicePath, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java index 3a80abba2b8c7..39be89f093808 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java @@ -874,7 +874,11 @@ public void createAlignedTimeSeries(final ICreateAlignedTimeSeriesPlan plan) @Override public Map checkMeasurementExistence( PartialPath devicePath, List measurementList, List aliasList) { - return mTree.checkMeasurementExistence(devicePath, measurementList, aliasList); + try { + return mTree.checkMeasurementExistence(devicePath, measurementList, aliasList); + } catch (final Exception e) { + return Collections.emptyMap(); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java index 58e8fd786a5ac..944ffe0ea5357 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java @@ -855,7 +855,11 @@ public void createAlignedTimeSeries(final ICreateAlignedTimeSeriesPlan plan) @Override public Map checkMeasurementExistence( PartialPath devicePath, List measurementList, List aliasList) { - return mtree.checkMeasurementExistence(devicePath, measurementList, aliasList); + try { + return mtree.checkMeasurementExistence(devicePath, measurementList, aliasList); + } catch (final Exception e) { + return Collections.emptyMap(); + } } @Override From 92878f4ad9387339ed2c510646cae2da05e6dc7f Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 12 Nov 2025 21:46:37 +0800 Subject: [PATCH 055/180] Pipe: Optimized the default behavior in meta sync for retries to wait for other regions (#16740) * partial * cfg * core * fix * fix (cherry picked from commit 6e8748e02561df3bc90aaab650ea245e95e4645a) --- .../org/apache/iotdb/rpc/TSStatusCode.java | 1 + .../agent/task/PipeDataNodeTaskAgent.java | 4 +- .../visitor/PipeStatementTSStatusVisitor.java | 37 ++++++++++-- .../realtime/assigner/DisruptorQueue.java | 4 +- .../schemaregion/utils/MetaUtils.java | 6 +- .../iotdb/commons/conf/CommonConfig.java | 60 ++++++++++++------- .../iotdb/commons/pipe/config/PipeConfig.java | 33 ++++++---- .../commons/pipe/config/PipeDescriptor.java | 15 +++-- .../receiver/PipeReceiverStatusHandler.java | 18 ++++-- 9 files changed, 119 insertions(+), 59 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index 84534adfadbed..4fd90ca4b9492 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -295,6 +295,7 @@ public enum TSStatusCode { PIPE_TRANSFER_SLICE_OUT_OF_ORDER(1812), PIPE_PUSH_META_TIMEOUT(1813), PIPE_PUSH_META_NOT_ENOUGH_MEMORY(1814), + PIPE_RECEIVER_PARALLEL_OR_USER_CONFLICT_EXCEPTION(1815), // Subscription SUBSCRIPTION_VERSION_ERROR(1900), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 87e744985f8ab..438dfa3f2334e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -892,8 +892,8 @@ private long calculateAssignerMemory(final PipeParameters sourceParameters) { .getLeft()) { return 0; } - return PipeConfig.getInstance().getPipeExtractorAssignerDisruptorRingBufferSize() - * PipeConfig.getInstance().getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes() + return PipeConfig.getInstance().getPipeSourceAssignerDisruptorRingBufferSize() + * PipeConfig.getInstance().getPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes() * Math.min(StorageEngine.getInstance().getDataRegionNumber(), 10); } catch (final IllegalPathException e) { return 0; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTSStatusVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTSStatusVisitor.java index 882485a6c72f1..31eecc7b3a955 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTSStatusVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTSStatusVisitor.java @@ -106,11 +106,22 @@ public TSStatus visitInsertBase( } else if (context.getCode() == TSStatusCode.OUT_OF_TTL.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); - } else if (context.getCode() == TSStatusCode.METADATA_ERROR.getStatusCode() - && (context.getMessage().contains(DataTypeMismatchException.REGISTERED_TYPE_STRING) - && config.isEnablePartialInsert())) { - return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) + } else if (context.getCode() == TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()) { + return new TSStatus( + TSStatusCode.PIPE_RECEIVER_PARALLEL_OR_USER_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); + } else if (context.getCode() == TSStatusCode.METADATA_ERROR.getStatusCode()) { + if (context.getMessage().contains(DataTypeMismatchException.REGISTERED_TYPE_STRING) + && config.isEnablePartialInsert()) { + return new TSStatus( + TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(context.getMessage()); + } + if (context.getMessage().contains("does not exist")) { + return new TSStatus( + TSStatusCode.PIPE_RECEIVER_PARALLEL_OR_USER_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(context.getMessage()); + } } return visitStatement(insertBaseStatement, context); } @@ -226,14 +237,24 @@ public TSStatus visitActivateTemplate( @Override public TSStatus visitBatchActivateTemplate( final BatchActivateTemplateStatement batchActivateTemplateStatement, final TSStatus context) { + boolean userConflict = false; if (context.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { for (final TSStatus status : context.getSubStatus()) { if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() && status.getCode() != TSStatusCode.TEMPLATE_IS_IN_USE.getStatusCode()) { return visitStatement(batchActivateTemplateStatement, context); } + if (context.getCode() == TSStatusCode.METADATA_ERROR.getStatusCode() + && context.isSetMessage() + && context.getMessage().contains("has not been set any template")) { + userConflict = true; + } } - return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) + return (userConflict + ? new TSStatus( + TSStatusCode.PIPE_RECEIVER_PARALLEL_OR_USER_CONFLICT_EXCEPTION.getStatusCode()) + : new TSStatus( + TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode())) .setMessage(context.getMessage()); } return visitGeneralActivateTemplate(batchActivateTemplateStatement, context); @@ -245,6 +266,12 @@ private TSStatus visitGeneralActivateTemplate( return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); } + if (context.getCode() == TSStatusCode.METADATA_ERROR.getStatusCode() + && context.isSetMessage() + && context.getMessage().contains("has not been set any template")) { + return new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(context.getMessage()); + } return visitStatement(activateTemplateStatement, context); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java index cb4aba1e15adb..fc30f14be5b70 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java @@ -49,9 +49,9 @@ public DisruptorQueue( final EventHandler eventHandler, final Consumer onAssignedHook) { final PipeConfig config = PipeConfig.getInstance(); - final int ringBufferSize = config.getPipeExtractorAssignerDisruptorRingBufferSize(); + final int ringBufferSize = config.getPipeSourceAssignerDisruptorRingBufferSize(); final long ringBufferEntrySizeInBytes = - config.getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes(); + config.getPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes(); allocatedMemoryBlock = PipeDataNodeResourceManager.memory() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/MetaUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/MetaUtils.java index fae3a21fd68de..11b41b238a340 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/MetaUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/MetaUtils.java @@ -70,9 +70,9 @@ public static PartialPath getDatabasePathByLevel(PartialPath path, int level) throw new IllegalPathException( path.getFullPath(), "it does not start with " + IoTDBConstant.PATH_ROOT); } - String[] storageGroupNodes = new String[level + 1]; - System.arraycopy(nodeNames, 0, storageGroupNodes, 0, level + 1); - return new PartialPath(storageGroupNodes); + String[] databaseNodes = new String[level + 1]; + System.arraycopy(nodeNames, 0, databaseNodes, 0, level + 1); + return new PartialPath(databaseNodes); } /** 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 43a637d5c2c15..9102a117eca4b 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 @@ -214,6 +214,8 @@ public class CommonConfig { private int pipeSubtaskExecutorMaxThreadNum = Math.max(5, Runtime.getRuntime().availableProcessors() / 2); + private boolean pipeRetryLocallyForParallelOrUserConflict = true; + private int pipeDataStructureTabletRowSize = 2048; private int pipeDataStructureTabletSizeInBytes = 2097152; private double pipeDataStructureTabletMemoryBlockAllocationRejectThreshold = 0.3; @@ -256,8 +258,8 @@ public class CommonConfig { private long pipeMaxWaitFinishTime = 10 * 1000; - private int pipeExtractorAssignerDisruptorRingBufferSize = 128; - private long pipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes = 72 * KB; + private int pipeSourceAssignerDisruptorRingBufferSize = 128; + private long pipeSourceAssignerDisruptorRingBufferEntrySizeInBytes = 72 * KB; private long pipeSourceMatcherCacheSize = 1024; private int pipeConnectorHandshakeTimeoutMs = 10 * 1000; // 10 seconds @@ -996,38 +998,37 @@ public void setPipeTotalFloatingMemoryProportion(double pipeTotalFloatingMemoryP "pipeTotalFloatingMemoryProportion is set to {}.", pipeTotalFloatingMemoryProportion); } - public int getPipeExtractorAssignerDisruptorRingBufferSize() { - return pipeExtractorAssignerDisruptorRingBufferSize; + public int getPipeSourceAssignerDisruptorRingBufferSize() { + return pipeSourceAssignerDisruptorRingBufferSize; } - public void setPipeExtractorAssignerDisruptorRingBufferSize( - int pipeExtractorAssignerDisruptorRingBufferSize) { - if (this.pipeExtractorAssignerDisruptorRingBufferSize - == pipeExtractorAssignerDisruptorRingBufferSize) { + public void setPipeSourceAssignerDisruptorRingBufferSize( + int pipeSourceAssignerDisruptorRingBufferSize) { + if (this.pipeSourceAssignerDisruptorRingBufferSize + == pipeSourceAssignerDisruptorRingBufferSize) { return; } - this.pipeExtractorAssignerDisruptorRingBufferSize = - pipeExtractorAssignerDisruptorRingBufferSize; + this.pipeSourceAssignerDisruptorRingBufferSize = pipeSourceAssignerDisruptorRingBufferSize; logger.info( - "pipeExtractorAssignerDisruptorRingBufferSize is set to {}.", - pipeExtractorAssignerDisruptorRingBufferSize); + "pipeSourceAssignerDisruptorRingBufferSize is set to {}.", + pipeSourceAssignerDisruptorRingBufferSize); } - public long getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes() { - return pipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes; + public long getPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes() { + return pipeSourceAssignerDisruptorRingBufferEntrySizeInBytes; } - public void setPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes( - long pipeExtractorAssignerDisruptorRingBufferEntrySize) { - if (pipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes - == pipeExtractorAssignerDisruptorRingBufferEntrySize) { + public void setPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes( + long pipeSourceAssignerDisruptorRingBufferEntrySize) { + if (pipeSourceAssignerDisruptorRingBufferEntrySizeInBytes + == pipeSourceAssignerDisruptorRingBufferEntrySize) { return; } - this.pipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes = - pipeExtractorAssignerDisruptorRingBufferEntrySize; + this.pipeSourceAssignerDisruptorRingBufferEntrySizeInBytes = + pipeSourceAssignerDisruptorRingBufferEntrySize; logger.info( - "pipeExtractorAssignerDisruptorRingBufferEntrySize is set to {}.", - pipeExtractorAssignerDisruptorRingBufferEntrySize); + "pipeSourceAssignerDisruptorRingBufferEntrySize is set to {}.", + pipeSourceAssignerDisruptorRingBufferEntrySize); } public long getPipeSourceMatcherCacheSize() { @@ -1400,6 +1401,21 @@ public void setPipeSubtaskExecutorMaxThreadNum(int pipeSubtaskExecutorMaxThreadN logger.info("pipeSubtaskExecutorMaxThreadNum is set to {}.", pipeSubtaskExecutorMaxThreadNum); } + public boolean isPipeRetryLocallyForParallelOrUserConflict() { + return pipeRetryLocallyForParallelOrUserConflict; + } + + public void setPipeRetryLocallyForParallelOrUserConflict( + boolean pipeRetryLocallyForParallelOrUserConflict) { + if (this.pipeRetryLocallyForParallelOrUserConflict + == pipeRetryLocallyForParallelOrUserConflict) { + return; + } + this.pipeRetryLocallyForParallelOrUserConflict = pipeRetryLocallyForParallelOrUserConflict; + logger.info( + "pipeRetryLocallyForParallelOrUserConflict is set to {}.", pipeSubtaskExecutorMaxThreadNum); + } + public long getPipeSubtaskExecutorPendingQueueMaxBlockingTimeMs() { return pipeSubtaskExecutorPendingQueueMaxBlockingTimeMs; } 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 4e840d51c5505..637628149b6e7 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 @@ -143,14 +143,14 @@ public long getPipeMaxWaitFinishTime() { return COMMON_CONFIG.getPipeMaxWaitFinishTime(); } - /////////////////////////////// Extractor /////////////////////////////// + /////////////////////////////// Source /////////////////////////////// - public int getPipeExtractorAssignerDisruptorRingBufferSize() { - return COMMON_CONFIG.getPipeExtractorAssignerDisruptorRingBufferSize(); + public int getPipeSourceAssignerDisruptorRingBufferSize() { + return COMMON_CONFIG.getPipeSourceAssignerDisruptorRingBufferSize(); } - public long getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes() { - return COMMON_CONFIG.getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes(); + public long getPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes() { + return COMMON_CONFIG.getPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes(); } public long getPipeSourceMatcherCacheSize() { @@ -337,8 +337,8 @@ public long getPipePeriodicalLogMinIntervalSeconds() { return COMMON_CONFIG.getPipePeriodicalLogMinIntervalSeconds(); } - public long getPipeLoggerCacheMaxSizeInBytes() { - return COMMON_CONFIG.getPipeLoggerCacheMaxSizeInBytes(); + public boolean isPipeRetryLocallyForParallelOrUserConflict() { + return COMMON_CONFIG.isPipeRetryLocallyForParallelOrUserConflict(); } /////////////////////////////// Logger /////////////////////////////// @@ -359,6 +359,10 @@ public int getPipeTsFilePinMaxLogIntervalRounds() { return COMMON_CONFIG.getPipeTsFilePinMaxLogIntervalRounds(); } + public long getPipeLoggerCacheMaxSizeInBytes() { + return COMMON_CONFIG.getPipeLoggerCacheMaxSizeInBytes(); + } + /////////////////////////////// Memory /////////////////////////////// public boolean getPipeMemoryManagementEnabled() { @@ -482,12 +486,12 @@ public void printAllConfigs() { LOGGER.info("PipeMaxWaitFinishTime: {}", getPipeMaxWaitFinishTime()); LOGGER.info( - "PipeExtractorAssignerDisruptorRingBufferSize: {}", - getPipeExtractorAssignerDisruptorRingBufferSize()); + "PipeSourceAssignerDisruptorRingBufferSize: {}", + getPipeSourceAssignerDisruptorRingBufferSize()); LOGGER.info( - "PipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes: {}", - getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes()); - LOGGER.info("PipeExtractorMatcherCacheSize: {}", getPipeSourceMatcherCacheSize()); + "PipeSourceAssignerDisruptorRingBufferEntrySizeInBytes: {}", + getPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes()); + LOGGER.info("PipeSourceMatcherCacheSize: {}", getPipeSourceMatcherCacheSize()); LOGGER.info("PipeConnectorHandshakeTimeoutMs: {}", getPipeConnectorHandshakeTimeoutMs()); LOGGER.info("PipeConnectorTransferTimeoutMs: {}", getPipeConnectorTransferTimeoutMs()); @@ -592,12 +596,15 @@ public void printAllConfigs() { LOGGER.info("PipeReceiverLoadConversionEnabled: {}", isPipeReceiverLoadConversionEnabled()); LOGGER.info( "PipePeriodicalLogMinIntervalSeconds: {}", getPipePeriodicalLogMinIntervalSeconds()); - LOGGER.info("PipeLoggerCacheMaxSizeInBytes: {}", getPipeLoggerCacheMaxSizeInBytes()); + LOGGER.info( + "PipeRetryLocallyForParallelOrUserConflict: {}", + isPipeRetryLocallyForParallelOrUserConflict()); LOGGER.info("PipeMetaReportMaxLogNumPerRound: {}", getPipeMetaReportMaxLogNumPerRound()); LOGGER.info("PipeMetaReportMaxLogIntervalRounds: {}", getPipeMetaReportMaxLogIntervalRounds()); LOGGER.info("PipeTsFilePinMaxLogNumPerRound: {}", getPipeTsFilePinMaxLogNumPerRound()); LOGGER.info("PipeTsFilePinMaxLogIntervalRounds: {}", getPipeTsFilePinMaxLogIntervalRounds()); + LOGGER.info("PipeLoggerCacheMaxSizeInBytes: {}", getPipeLoggerCacheMaxSizeInBytes()); LOGGER.info("PipeMemoryManagementEnabled: {}", getPipeMemoryManagementEnabled()); LOGGER.info("PipeMemoryAllocateMaxRetries: {}", getPipeMemoryAllocateMaxRetries()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index 760a7b5e51fc3..77aae8a3252e7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -290,17 +290,21 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr properties.getProperty( "pipe_subtask_executor_pending_queue_max_blocking_time_ms", String.valueOf(config.getPipeSubtaskExecutorPendingQueueMaxBlockingTimeMs())))); + config.setPipeRetryLocallyForParallelOrUserConflict( + Boolean.parseBoolean( + properties.getProperty( + "pipe_retry_locally_for_user_conflict", + String.valueOf(config.isPipeRetryLocallyForParallelOrUserConflict())))); - config.setPipeExtractorAssignerDisruptorRingBufferSize( + config.setPipeSourceAssignerDisruptorRingBufferSize( Integer.parseInt( Optional.ofNullable( properties.getProperty("pipe_source_assigner_disruptor_ring_buffer_size")) .orElse( properties.getProperty( "pipe_extractor_assigner_disruptor_ring_buffer_size", - String.valueOf( - config.getPipeExtractorAssignerDisruptorRingBufferSize()))))); - config.setPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes( // 1MB + String.valueOf(config.getPipeSourceAssignerDisruptorRingBufferSize()))))); + config.setPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes( // 1MB Integer.parseInt( Optional.ofNullable( properties.getProperty( @@ -309,8 +313,7 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr properties.getProperty( "pipe_extractor_assigner_disruptor_ring_buffer_entry_size_in_bytes", String.valueOf( - config - .getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes()))))); + config.getPipeSourceAssignerDisruptorRingBufferEntrySizeInBytes()))))); config.setPipeSourceMatcherCacheSize( Integer.parseInt( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java index 3bb9e14d18167..350746d7b8e52 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.exception.pipe.PipeNonReportException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeSinkRetryTimesConfigurableException; import org.apache.iotdb.commons.pipe.agent.task.subtask.PipeSubtask; +import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; import org.apache.iotdb.commons.utils.RetryUtils; import org.apache.iotdb.pipe.api.event.Event; @@ -131,6 +132,7 @@ public void handle( } case 1810: // PIPE_RECEIVER_USER_CONFLICT_EXCEPTION + case 1815: // PIPE_RECEIVER_PARALLEL_OR_USER_CONFLICT_EXCEPTION if (!isRetryAllowedWhenConflictOccurs) { LOGGER.warn( "User conflict exception: will be ignored because retry is not allowed. event: {}. status: {}", @@ -165,12 +167,16 @@ public void handle( + " seconds", status); exceptionEventHasBeenRetried.set(true); - throw new PipeRuntimeSinkRetryTimesConfigurableException( - exceptionMessage, - (int) - Math.max( - PipeSubtask.MAX_RETRY_TIMES, - Math.min(CONFLICT_RETRY_MAX_TIMES, retryMaxMillisWhenConflictOccurs * 1.1))); + throw status.getCode() == 1815 + && PipeConfig.getInstance().isPipeRetryLocallyForParallelOrUserConflict() + ? new PipeNonReportException(exceptionMessage) + : new PipeRuntimeSinkRetryTimesConfigurableException( + exceptionMessage, + (int) + Math.max( + PipeSubtask.MAX_RETRY_TIMES, + Math.min( + CONFLICT_RETRY_MAX_TIMES, retryMaxMillisWhenConflictOccurs * 1.1))); } case 803: // NO_PERMISSION From 197c4f18af9c4eac2472a6f728412d9bfe47ee3d Mon Sep 17 00:00:00 2001 From: libo Date: Thu, 13 Nov 2025 11:17:23 +0800 Subject: [PATCH 056/180] Adjust the exception message cause by the WHERE clause, which uses a range comparison on the same field, specifically when the left value of the range is greater than the right value of the range. (#16741) (cherry picked from commit f608a2d2e6a2ddf8b80ae0e36dbc3424a7ca6127) --- .../apache/iotdb/db/it/IoTDBDeletionIT.java | 21 +++++++++++++++++++ .../queryengine/plan/parser/ASTVisitor.java | 12 ++++++++--- 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBDeletionIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBDeletionIT.java index 576b9c75f2183..17ceb9b696255 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBDeletionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBDeletionIT.java @@ -19,10 +19,12 @@ package org.apache.iotdb.db.it; +import org.apache.iotdb.db.queryengine.plan.parser.ASTVisitor; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.ClusterIT; import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.apache.iotdb.jdbc.IoTDBSQLException; import org.apache.iotdb.rpc.TSStatusCode; import org.junit.AfterClass; @@ -467,6 +469,25 @@ public void testDelAfterUpdate() throws SQLException { } } + @Test + public void testDeleteByRangeComparison() throws SQLException { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE root.test"); + statement.execute("CREATE ALIGNED TIMESERIES root.test.g_0.d2(s_0 int32)"); + statement.execute("INSERT INTO root.test.g_0.d_2(time,s_0) VALUES(1, 1)"); + + try { + statement.execute("DELETE FROM root.test.g_0.d_2.s_0 WHERE time > 4 AND time < 0"); + } catch (IoTDBSQLException e) { + Assert.assertEquals( + e.getErrorCode() + ": " + ASTVisitor.DELETE_RANGE_COMPARISON_ERROR_MSG, e.getMessage()); + } + + statement.execute("DROP DATABASE root.test"); + } + } + private static void prepareSeries() { try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java index 123f7dcae523c..be30b4e1b37f8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java @@ -344,6 +344,9 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor { private static final String LIMIT_CONFIGURATION_ENABLED_ERROR_MSG = "Limit configuration is not enabled, please enable it first."; + public static final String DELETE_RANGE_COMPARISON_ERROR_MSG = + "For delete statement, where clause use a range comparison on the same field, the left value of the range cannot be greater than the right value of the range, it must be written like this : time > 5 and time < 10"; + private static final String NODE_NAME_IN_INTO_PATH_MATCHER = "([a-zA-Z0-9_${}\\u2E80-\\u9FFF]+)"; private static final Pattern NODE_NAME_IN_INTO_PATH_PATTERN = Pattern.compile(NODE_NAME_IN_INTO_PATH_MATCHER); @@ -2993,9 +2996,12 @@ private TimeRange parseDeleteTimeRange(Expression predicate) { parseDeleteTimeRange(((LogicAndExpression) predicate).getLeftExpression()); TimeRange rightTimeRange = parseDeleteTimeRange(((LogicAndExpression) predicate).getRightExpression()); - return new TimeRange( - Math.max(leftTimeRange.getMin(), rightTimeRange.getMin()), - Math.min(leftTimeRange.getMax(), rightTimeRange.getMax())); + long min = Math.max(leftTimeRange.getMin(), rightTimeRange.getMin()); + long max = Math.min(leftTimeRange.getMax(), rightTimeRange.getMax()); + if (min > max) { + throw new SemanticException(DELETE_RANGE_COMPARISON_ERROR_MSG); + } + return new TimeRange(min, max); } else if (predicate instanceof CompareBinaryExpression) { if (((CompareBinaryExpression) predicate).getLeftExpression() instanceof TimestampOperand) { return parseTimeRangeForDeleteTimeRange( From 401c5e9208e88252fd8b68bf98be2e4d781ff06e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 13 Nov 2025 14:23:43 +0800 Subject: [PATCH 057/180] Pipe: Trimmed some of the pipe logs (#16747) * fix * fix (cherry picked from commit e53d27c0ad096a0788ac9da2664b35ddaf1bde02) --- .../assembly/resources/conf/logback-datanode.xml | 15 --------------- .../visitor/PipeStatementTSStatusVisitor.java | 5 +++-- 2 files changed, 3 insertions(+), 17 deletions(-) diff --git a/iotdb-core/datanode/src/assembly/resources/conf/logback-datanode.xml b/iotdb-core/datanode/src/assembly/resources/conf/logback-datanode.xml index 557fd8b160f51..e827a7cbef208 100644 --- a/iotdb-core/datanode/src/assembly/resources/conf/logback-datanode.xml +++ b/iotdb-core/datanode/src/assembly/resources/conf/logback-datanode.xml @@ -207,21 +207,6 @@ INFO - - ${IOTDB_HOME}/logs/log_datanode_pipe.log - - ${IOTDB_HOME}/logs/log-datanode-pipe-%d{yyyyMMdd}.log.gz - 30 - - true - - %d [%t] %-5p %C{25}:%L - %m %n - utf-8 - - - INFO - - ${IOTDB_HOME}/logs/log_explain_analyze.log diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTSStatusVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTSStatusVisitor.java index 31eecc7b3a955..198e8edb67713 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTSStatusVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementTSStatusVisitor.java @@ -98,8 +98,9 @@ public TSStatus visitInsertMultiTablets( @Override public TSStatus visitInsertBase( final InsertBaseStatement insertBaseStatement, final TSStatus context) { - if (context.getCode() == TSStatusCode.SYSTEM_READ_ONLY.getStatusCode() - || context.getCode() == TSStatusCode.WRITE_PROCESS_REJECT.getStatusCode()) { + // If the system is read-only, we shall not classify it into temporary unavailable exception to + // avoid to many logs + if (context.getCode() == TSStatusCode.WRITE_PROCESS_REJECT.getStatusCode()) { return new TSStatus( TSStatusCode.PIPE_RECEIVER_TEMPORARY_UNAVAILABLE_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); From 43c0320204d2f26c15d20ec767e200b4bc44eca3 Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Fri, 14 Nov 2025 14:33:52 +0800 Subject: [PATCH 058/180] Active Load: Fixes errors caused by Load copying resource files. (#16753) (cherry picked from commit db7c801d8465a8d0dc9fa5e844d14102045b59a6) --- .../iotdb/db/storageengine/load/active/ActiveLoadUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java index 5bfa9b71105ea..669013ec45054 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java @@ -91,11 +91,11 @@ private static boolean loadTsFilesToActiveDir( targetDir = targetFilePath; } - loadTsFileAsyncToTargetDir(targetDir, file, isDeleteAfterLoad); loadTsFileAsyncToTargetDir( targetDir, new File(file.getAbsolutePath() + ".resource"), isDeleteAfterLoad); loadTsFileAsyncToTargetDir( targetDir, new File(file.getAbsolutePath() + ".mods"), isDeleteAfterLoad); + loadTsFileAsyncToTargetDir(targetDir, file, isDeleteAfterLoad); return true; } From 8fe93f2d6228eb02a171e36e37c5c851e0a2309a Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Fri, 14 Nov 2025 15:06:44 +0800 Subject: [PATCH 059/180] Pipe: Modify the Pipe configuration item log name to be consistent with the Properties name. (#16732) * Pipe: Modify the Pipe configuration item log name to be consistent with the Properties name. * update (cherry picked from commit a6191d916b2ceca7f3b22e728ae4117b99290261) --- .../async/IoTDBDataRegionAsyncSink.java | 18 ++--- .../iotdb/commons/conf/CommonConfig.java | 67 +++++++++---------- .../iotdb/commons/pipe/config/PipeConfig.java | 18 ++--- .../commons/pipe/config/PipeDescriptor.java | 16 ++--- 4 files changed, 54 insertions(+), 65 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java index 567715e7292f2..e6e368a528009 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java @@ -539,14 +539,11 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { if ((retryEventQueue.isEmpty() && retryTsFileQueue.isEmpty()) || (!forced && retryEventQueueEventCounter.getTabletInsertionEventCount() - < PipeConfig.getInstance() - .getPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold() + < PipeConfig.getInstance().getPipeAsyncSinkForcedRetryTabletEventQueueSize() && retryEventQueueEventCounter.getTsFileInsertionEventCount() - < PipeConfig.getInstance() - .getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold() + < PipeConfig.getInstance().getPipeAsyncSinkForcedRetryTsFileEventQueueSize() && retryEventQueue.size() + retryTsFileQueue.size() - < PipeConfig.getInstance() - .getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold())) { + < PipeConfig.getInstance().getPipeAsyncSinkForcedRetryTotalEventQueueSize())) { return; } @@ -604,14 +601,11 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { if (System.currentTimeMillis() - retryStartTime > PipeConfig.getInstance().getPipeAsyncConnectorMaxRetryExecutionTimeMsPerCall()) { if (retryEventQueueEventCounter.getTabletInsertionEventCount() - < PipeConfig.getInstance() - .getPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold() + < PipeConfig.getInstance().getPipeAsyncSinkForcedRetryTabletEventQueueSize() && retryEventQueueEventCounter.getTsFileInsertionEventCount() - < PipeConfig.getInstance() - .getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold() + < PipeConfig.getInstance().getPipeAsyncSinkForcedRetryTsFileEventQueueSize() && retryEventQueue.size() + retryTsFileQueue.size() - < PipeConfig.getInstance() - .getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold()) { + < PipeConfig.getInstance().getPipeAsyncSinkForcedRetryTotalEventQueueSize()) { return; } 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 9102a117eca4b..d83e96eb100fb 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 @@ -269,9 +269,9 @@ public class CommonConfig { private long pipeConnectorRetryIntervalMs = 1000L; private boolean pipeConnectorRPCThriftCompressionEnabled = false; - private int pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold = 5; - private int pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold = 20; - private int pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold = 30; + private int pipeAsyncSinkForcedRetryTsFileEventQueueSize = 5; + private int pipeAsyncSinkForcedRetryTabletEventQueueSize = 20; + private int pipeAsyncSinkForcedRetryTotalEventQueueSize = 30; private long pipeAsyncConnectorMaxRetryExecutionTimeMsPerCall = 500; private int pipeAsyncConnectorSelectorNumber = Math.max(4, Runtime.getRuntime().availableProcessors() / 2); @@ -1058,7 +1058,7 @@ public void setPipeConnectorHandshakeTimeoutMs(long pipeConnectorHandshakeTimeou } finally { if (fPipeConnectorHandshakeTimeoutMs != this.pipeConnectorHandshakeTimeoutMs) { logger.info( - "pipeConnectorHandshakeTimeoutMs is set to {}.", fPipeConnectorHandshakeTimeoutMs); + "pipeConnectorHandshakeTimeoutMs is set to {}.", this.pipeConnectorHandshakeTimeoutMs); } } } @@ -1127,55 +1127,54 @@ public boolean isPipeConnectorRPCThriftCompressionEnabled() { return pipeConnectorRPCThriftCompressionEnabled; } - public void setPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold( - int pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold) { - if (this.pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold - == pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold) { + public void setPipeAsyncSinkForcedRetryTsFileEventQueueSize( + int pipeAsyncSinkForcedRetryTsFileEventQueueSize) { + if (this.pipeAsyncSinkForcedRetryTsFileEventQueueSize + == pipeAsyncSinkForcedRetryTsFileEventQueueSize) { return; } - this.pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold = - pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold; + this.pipeAsyncSinkForcedRetryTsFileEventQueueSize = + pipeAsyncSinkForcedRetryTsFileEventQueueSize; logger.info( - "pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold is set to {}.", - pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold); + "pipeAsyncSinkForcedRetryTsFileEventQueueSize is set to {}.", + pipeAsyncSinkForcedRetryTsFileEventQueueSize); } - public int getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold() { - return pipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold; + public int getPipeAsyncSinkForcedRetryTsFileEventQueueSize() { + return pipeAsyncSinkForcedRetryTsFileEventQueueSize; } - public void setPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold( - int pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold) { - if (this.pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold - == pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold) { + public void setPipeAsyncSinkForcedRetryTabletEventQueueSize( + int pipeAsyncSinkForcedRetryTabletEventQueueSize) { + if (this.pipeAsyncSinkForcedRetryTabletEventQueueSize + == pipeAsyncSinkForcedRetryTabletEventQueueSize) { return; } - this.pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold = - pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold; + this.pipeAsyncSinkForcedRetryTabletEventQueueSize = + pipeAsyncSinkForcedRetryTabletEventQueueSize; logger.info( - "pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold is set to {}.", - pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold); + "pipeAsyncSinkForcedRetryTabletEventQueueSize is set to {}.", + pipeAsyncSinkForcedRetryTabletEventQueueSize); } - public int getPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold() { - return pipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold; + public int getPipeAsyncSinkForcedRetryTabletEventQueueSize() { + return pipeAsyncSinkForcedRetryTabletEventQueueSize; } - public void setPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold( - int pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold) { - if (this.pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold - == pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold) { + public void setPipeAsyncSinkForcedRetryTotalEventQueueSize( + int pipeAsyncSinkForcedRetryTotalEventQueueSize) { + if (this.pipeAsyncSinkForcedRetryTotalEventQueueSize + == pipeAsyncSinkForcedRetryTotalEventQueueSize) { return; } - this.pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold = - pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold; + this.pipeAsyncSinkForcedRetryTotalEventQueueSize = pipeAsyncSinkForcedRetryTotalEventQueueSize; logger.info( - "pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold is set to {}.", - pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold); + "pipeAsyncSinkForcedRetryTotalEventQueueSize is set to {}.", + pipeAsyncSinkForcedRetryTotalEventQueueSize); } - public int getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold() { - return pipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold; + public int getPipeAsyncSinkForcedRetryTotalEventQueueSize() { + return pipeAsyncSinkForcedRetryTotalEventQueueSize; } public void setPipeAsyncConnectorMaxRetryExecutionTimeMsPerCall( 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 637628149b6e7..ce09fb1f2919a 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 @@ -183,16 +183,16 @@ public boolean isPipeConnectorRPCThriftCompressionEnabled() { return COMMON_CONFIG.isPipeConnectorRPCThriftCompressionEnabled(); } - public int getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold() { - return COMMON_CONFIG.getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold(); + public int getPipeAsyncSinkForcedRetryTsFileEventQueueSize() { + return COMMON_CONFIG.getPipeAsyncSinkForcedRetryTsFileEventQueueSize(); } - public int getPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold() { - return COMMON_CONFIG.getPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold(); + public int getPipeAsyncSinkForcedRetryTabletEventQueueSize() { + return COMMON_CONFIG.getPipeAsyncSinkForcedRetryTabletEventQueueSize(); } - public int getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold() { - return COMMON_CONFIG.getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold(); + public int getPipeAsyncSinkForcedRetryTotalEventQueueSize() { + return COMMON_CONFIG.getPipeAsyncSinkForcedRetryTotalEventQueueSize(); } public long getPipeAsyncConnectorMaxRetryExecutionTimeMsPerCall() { @@ -541,13 +541,13 @@ public void printAllConfigs() { LOGGER.info( "PipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold: {}", - getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold()); + getPipeAsyncSinkForcedRetryTsFileEventQueueSize()); LOGGER.info( "PipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold: {}", - getPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold()); + getPipeAsyncSinkForcedRetryTabletEventQueueSize()); LOGGER.info( "PipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold: {}", - getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold()); + getPipeAsyncSinkForcedRetryTotalEventQueueSize()); LOGGER.info( "PipeAsyncConnectorMaxRetryExecutionTimeMsPerCall: {}", getPipeAsyncConnectorMaxRetryExecutionTimeMsPerCall()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index 77aae8a3252e7..928ff5f25a598 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -368,7 +368,7 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr "pipe_async_connector_max_retry_execution_time_ms_per_call", String.valueOf( config.getPipeAsyncConnectorMaxRetryExecutionTimeMsPerCall()))))); - config.setPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold( + config.setPipeAsyncSinkForcedRetryTsFileEventQueueSize( Integer.parseInt( Optional.ofNullable( properties.getProperty("pipe_async_sink_forced_retry_tsfile_event_queue_size")) @@ -376,9 +376,8 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr properties.getProperty( "pipe_async_connector_forced_retry_tsfile_event_queue_size", String.valueOf( - config - .getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold()))))); - config.setPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold( + config.getPipeAsyncSinkForcedRetryTsFileEventQueueSize()))))); + config.setPipeAsyncSinkForcedRetryTabletEventQueueSize( Integer.parseInt( Optional.ofNullable( properties.getProperty("pipe_async_sink_forced_retry_tablet_event_queue_size")) @@ -386,18 +385,15 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr properties.getProperty( "pipe_async_connector_forced_retry_tablet_event_queue_size", String.valueOf( - config - .getPipeAsyncConnectorForcedRetryTabletEventQueueSizeThreshold()))))); - config.setPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold( + config.getPipeAsyncSinkForcedRetryTabletEventQueueSize()))))); + config.setPipeAsyncSinkForcedRetryTotalEventQueueSize( Integer.parseInt( Optional.ofNullable( properties.getProperty("pipe_async_sink_forced_retry_total_event_queue_size")) .orElse( properties.getProperty( "pipe_async_connector_forced_retry_total_event_queue_size", - String.valueOf( - config - .getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold()))))); + String.valueOf(config.getPipeAsyncSinkForcedRetryTotalEventQueueSize()))))); config.setRateLimiterHotReloadCheckIntervalMs( Integer.parseInt( properties.getProperty( From b396766f4f26e3bd59bc30d5e3fd79d790c5a3ec Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Fri, 14 Nov 2025 15:14:37 +0800 Subject: [PATCH 060/180] feat: encode load attributes in active load directories (#16722) * feat: encode load attributes in active load directories * spotless * fix * fix * fix * fix * fix * update it * update it * fix * update * update (cherry picked from commit 91e48f07f63a1161b46487a281858eb1b4b63479) --- .../iotdb/db/it/IoTDBLoadTsFileWithModIT.java | 60 +++- .../thrift/IoTDBDataNodeReceiver.java | 12 +- .../plan/analyze/load/LoadTsFileAnalyzer.java | 13 +- .../statement/crud/LoadTsFileStatement.java | 16 +- .../load/active/ActiveLoadDirScanner.java | 23 +- .../load/active/ActiveLoadPathHelper.java | 282 ++++++++++++++++++ .../load/active/ActiveLoadPendingQueue.java | 17 +- .../load/active/ActiveLoadTsFileLoader.java | 36 ++- .../load/active/ActiveLoadUtil.java | 37 ++- .../load/config/LoadTsFileConfigurator.java | 18 ++ 10 files changed, 460 insertions(+), 54 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPathHelper.java diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileWithModIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileWithModIT.java index 9d0a54f16728c..5578023e4f5c4 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileWithModIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileWithModIT.java @@ -85,7 +85,7 @@ private void generateFileWithNewModFile() // write mods file resource .getExclusiveModFile() - .write(new TreeDeletionEntry(new MeasurementPath("root.test.d1.s1"), 1, 2)); + .write(new TreeDeletionEntry(new MeasurementPath("root.test.d1.de.s1"), 1, 2)); resource.getExclusiveModFile().close(); } @@ -93,7 +93,7 @@ private void generateFileWithOldModFile() throws IOException, DataRegionException, WriteProcessException, IllegalPathException { TsFileResource resource = generateFile(); ModificationFileV1 oldModFile = ModificationFileV1.getNormalMods(resource); - oldModFile.write(new Deletion(new MeasurementPath("root.test.d1.s1"), Long.MAX_VALUE, 1, 2)); + oldModFile.write(new Deletion(new MeasurementPath("root.test.d1.de.s1"), Long.MAX_VALUE, 1, 2)); oldModFile.close(); } @@ -102,11 +102,11 @@ private TsFileResource generateFile() File tsfile = new File(tmpDir, "1-1-0-0.tsfile"); try (TsFileWriter writer = new TsFileWriter(tsfile)) { writer.registerAlignedTimeseries( - "root.test.d1", + "root.test.d1.de", Collections.singletonList(new MeasurementSchema("s1", TSDataType.BOOLEAN))); Tablet tablet = new Tablet( - "root.test.d1", + "root.test.d1.de", Collections.singletonList(new MeasurementSchema("s1", TSDataType.BOOLEAN))); for (int i = 0; i < 5; i++) { tablet.addTimestamp(i, i); @@ -138,13 +138,61 @@ public void testWithNewModFile() statement.execute(String.format("load \'%s\'", tmpDir.getAbsolutePath())); try (final ResultSet resultSet = - statement.executeQuery("select count(s1) as c from root.test.d1")) { + statement.executeQuery("select count(s1) as c from root.test.d1.de")) { Assert.assertTrue(resultSet.next()); Assert.assertEquals(3, resultSet.getLong("c")); } } } + @Test + public void testWithNewModFileAndLoadAttributes() + throws SQLException, + IOException, + DataRegionException, + WriteProcessException, + IllegalPathException { + generateFileWithNewModFile(); + final String databaseName = "root.test.d1"; + + try (final Connection connection = EnvFactory.getEnv().getConnection(); + final Statement statement = connection.createStatement()) { + + statement.execute( + String.format( + "load \'%s\' with (" + + "'database-name'='%s'," + + "'database-level'='2'," + + "'verify'='true'," + + "'on-success'='none'," + + "'async'='true')", + tmpDir.getAbsolutePath(), databaseName)); + + boolean databaseFound = false; + out: + for (int i = 0; i < 10; i++) { + try (final ResultSet resultSet = statement.executeQuery("show databases")) { + while (resultSet.next()) { + final String currentDatabase = resultSet.getString(1); + if (databaseName.equalsIgnoreCase(currentDatabase)) { + databaseFound = true; + break out; + } + } + + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + break; + } + } + } + Assert.assertTrue( + "The `database-level` parameter is not working; the generated database does not contain 'root.test.d1'.", + databaseFound); + } + } + @Test public void testWithOldModFile() throws SQLException, @@ -159,7 +207,7 @@ public void testWithOldModFile() statement.execute(String.format("load \'%s\'", tmpDir.getAbsolutePath())); try (final ResultSet resultSet = - statement.executeQuery("select count(s1) as c from root.test.d1")) { + statement.executeQuery("select count(s1) as c from root.test.d1.de")) { Assert.assertTrue(resultSet.next()); Assert.assertEquals(3, resultSet.getLong("c")); Assert.assertTrue( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index 8a9ceb61c930c..652530d11b481 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -100,6 +100,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.crud.LoadTsFileStatement; import org.apache.iotdb.db.queryengine.plan.statement.pipe.PipeEnrichedStatement; import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; +import org.apache.iotdb.db.storageengine.load.active.ActiveLoadPathHelper; import org.apache.iotdb.db.storageengine.load.active.ActiveLoadUtil; import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager; import org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType; @@ -570,7 +571,16 @@ protected TSStatus loadFileV2( private TSStatus loadTsFileAsync(final String dataBaseName, final List absolutePaths) throws IOException { - if (!ActiveLoadUtil.loadFilesToActiveDir(dataBaseName, absolutePaths, true)) { + final Map loadAttributes = + ActiveLoadPathHelper.buildAttributes( + dataBaseName, + null, + shouldConvertDataTypeOnTypeMismatch, + validateTsFile.get(), + null, + shouldMarkAsPipeRequest.get()); + + if (!ActiveLoadUtil.loadFilesToActiveDir(loadAttributes, absolutePaths, true)) { throw new PipeException("Load active listening pipe dir is not set."); } return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java index 4fe119566d7fb..076b529ab918a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java @@ -42,6 +42,7 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileResourceUtils; +import org.apache.iotdb.db.storageengine.load.active.ActiveLoadPathHelper; import org.apache.iotdb.db.storageengine.load.active.ActiveLoadUtil; import org.apache.iotdb.db.storageengine.load.converter.LoadTsFileDataTypeConverter; import org.apache.iotdb.db.storageengine.load.metrics.LoadTsFileCostMetricsSet; @@ -281,7 +282,17 @@ private boolean doAsyncLoad(final IAnalysis analysis) { } else { databaseName = null; } - if (ActiveLoadUtil.loadTsFileAsyncToActiveDir(tsFiles, databaseName, isDeleteAfterLoad)) { + final Map activeLoadAttributes = + ActiveLoadPathHelper.buildAttributes( + databaseName, + databaseLevel, + isConvertOnTypeMismatch, + isVerifySchema, + tabletConversionThresholdBytes, + isGeneratedByPipe); + + if (ActiveLoadUtil.loadTsFileAsyncToActiveDir( + tsFiles, activeLoadAttributes, isDeleteAfterLoad)) { analysis.setFinishQueryAfterAnalyze(true); setRealStatement(analysis); return true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java index dbc4cb7b64d64..2d74925971c31 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java @@ -48,6 +48,7 @@ import static org.apache.iotdb.db.storageengine.load.config.LoadTsFileConfigurator.ON_SUCCESS_DELETE_VALUE; import static org.apache.iotdb.db.storageengine.load.config.LoadTsFileConfigurator.ON_SUCCESS_KEY; import static org.apache.iotdb.db.storageengine.load.config.LoadTsFileConfigurator.ON_SUCCESS_NONE_VALUE; +import static org.apache.iotdb.db.storageengine.load.config.LoadTsFileConfigurator.PIPE_GENERATED_KEY; import static org.apache.iotdb.db.storageengine.load.config.LoadTsFileConfigurator.TABLET_CONVERSION_THRESHOLD_KEY; public class LoadTsFileStatement extends Statement { @@ -63,8 +64,6 @@ public class LoadTsFileStatement extends Statement { private boolean isGeneratedByPipe = false; private boolean isAsyncLoad = false; - private Map loadAttributes; - private List tsFiles; private List isTableModel; private List resources; @@ -245,15 +244,14 @@ public long getWritePointCount(int resourceIndex) { } public void setLoadAttributes(final Map loadAttributes) { - this.loadAttributes = loadAttributes; - initAttributes(); + initAttributes(loadAttributes); } public boolean isAsyncLoad() { return isAsyncLoad; } - private void initAttributes() { + private void initAttributes(final Map loadAttributes) { this.databaseLevel = LoadTsFileConfigurator.parseOrGetDefaultDatabaseLevel(loadAttributes); this.database = LoadTsFileConfigurator.parseDatabaseName(loadAttributes); this.deleteAfterLoad = LoadTsFileConfigurator.parseOrGetDefaultOnSuccess(loadAttributes); @@ -263,6 +261,9 @@ private void initAttributes() { LoadTsFileConfigurator.parseOrGetDefaultTabletConversionThresholdBytes(loadAttributes); this.verifySchema = LoadTsFileConfigurator.parseOrGetDefaultVerify(loadAttributes); this.isAsyncLoad = LoadTsFileConfigurator.parseOrGetDefaultAsyncLoad(loadAttributes); + if (LoadTsFileConfigurator.parseOrGetDefaultPipeGenerated(loadAttributes)) { + markIsGeneratedByPipe(); + } } public boolean reconstructStatementIfMiniFileConverted(final List isMiniTsFile) { @@ -314,7 +315,7 @@ public List getPaths() { @Override public org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement toRelationalStatement( MPPQueryContext context) { - loadAttributes = new HashMap<>(); + final Map loadAttributes = new HashMap<>(); loadAttributes.put(DATABASE_LEVEL_KEY, String.valueOf(databaseLevel)); if (database != null) { @@ -326,6 +327,9 @@ public org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement toRelat loadAttributes.put( TABLET_CONVERSION_THRESHOLD_KEY, String.valueOf(tabletConversionThresholdBytes)); loadAttributes.put(ASYNC_LOAD_KEY, String.valueOf(isAsyncLoad)); + if (isGeneratedByPipe) { + loadAttributes.put(PIPE_GENERATED_KEY, String.valueOf(true)); + } return new LoadTsFile(null, file.getAbsolutePath(), loadAttributes); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java index 5ac33dfbdd4bd..470cf702b226a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java @@ -39,6 +39,7 @@ import java.nio.file.SimpleFileVisitor; import java.nio.file.attribute.BasicFileAttributes; import java.util.Arrays; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; @@ -117,13 +118,23 @@ private void scan() throws IOException { .filter(this::isTsFileCompleted) .limit(currentAllowedPendingSize) .forEach( - file -> { - final File parentFile = new File(file).getParentFile(); + filePath -> { + final File tsFile = new File(filePath); + final Map attributes = + ActiveLoadPathHelper.parseAttributes(tsFile, listeningDirFile); + + final File parentFile = tsFile.getParentFile(); + final boolean isTableModel = + ActiveLoadPathHelper.containsDatabaseName(attributes) + || (parentFile != null + && !Objects.equals( + parentFile.getAbsoluteFile(), + listeningDirFile.getAbsoluteFile())); + activeLoadTsFileLoader.tryTriggerTsFileLoad( - file, - parentFile != null - && !Objects.equals( - parentFile.getAbsoluteFile(), listeningDirFile.getAbsoluteFile()), + tsFile.getAbsolutePath(), + listeningDirFile.getAbsolutePath(), + isTableModel, isGeneratedByPipe); }); } catch (UncheckedIOException e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPathHelper.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPathHelper.java new file mode 100644 index 0000000000000..7c83f20f36903 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPathHelper.java @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.load.active; + +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.sql.SemanticException; +import org.apache.iotdb.db.queryengine.plan.statement.crud.LoadTsFileStatement; +import org.apache.iotdb.db.storageengine.load.config.LoadTsFileConfigurator; + +import java.io.File; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +/** Utility methods for encoding and decoding load attributes into directory structures. */ +public final class ActiveLoadPathHelper { + + private static final String SEGMENT_SEPARATOR = "-"; + + private static final List KEY_ORDER = + Collections.unmodifiableList( + Arrays.asList( + LoadTsFileConfigurator.DATABASE_NAME_KEY, + LoadTsFileConfigurator.DATABASE_LEVEL_KEY, + LoadTsFileConfigurator.CONVERT_ON_TYPE_MISMATCH_KEY, + LoadTsFileConfigurator.TABLET_CONVERSION_THRESHOLD_KEY, + LoadTsFileConfigurator.VERIFY_KEY, + LoadTsFileConfigurator.DATABASE_KEY, + LoadTsFileConfigurator.PIPE_GENERATED_KEY)); + + private ActiveLoadPathHelper() { + throw new IllegalStateException("Utility class"); + } + + public static Map buildAttributes( + final String databaseName, + final Integer databaseLevel, + final Boolean convertOnTypeMismatch, + final Boolean verify, + final Long tabletConversionThresholdBytes, + final Boolean pipeGenerated) { + final Map attributes = new LinkedHashMap<>(); + if (Objects.nonNull(databaseName) && !databaseName.isEmpty()) { + attributes.put(LoadTsFileConfigurator.DATABASE_NAME_KEY, databaseName); + } + + if (Objects.nonNull(databaseLevel)) { + attributes.put(LoadTsFileConfigurator.DATABASE_LEVEL_KEY, databaseLevel.toString()); + } + + if (Objects.nonNull(convertOnTypeMismatch)) { + attributes.put( + LoadTsFileConfigurator.CONVERT_ON_TYPE_MISMATCH_KEY, + Boolean.toString(convertOnTypeMismatch)); + } + + if (Objects.nonNull(tabletConversionThresholdBytes)) { + attributes.put( + LoadTsFileConfigurator.TABLET_CONVERSION_THRESHOLD_KEY, + tabletConversionThresholdBytes.toString()); + } + + if (Objects.nonNull(verify)) { + attributes.put(LoadTsFileConfigurator.VERIFY_KEY, Boolean.toString(verify)); + } + + if (Objects.nonNull(pipeGenerated) && pipeGenerated) { + attributes.put(LoadTsFileConfigurator.PIPE_GENERATED_KEY, Boolean.TRUE.toString()); + } + return attributes; + } + + public static File resolveTargetDir(final File baseDir, final Map attributes) { + File current = baseDir; + for (final String key : KEY_ORDER) { + final String value = attributes.get(key); + if (value == null) { + continue; + } + current = new File(current, formatSegment(key, value)); + } + return current; + } + + public static Map parseAttributes(final File file, final File pendingDir) { + if (file == null) { + return Collections.emptyMap(); + } + + final Map attributes = new HashMap<>(); + File current = file.getParentFile(); + while (current != null) { + final String dirName = current.getName(); + if (pendingDir != null && current.equals(pendingDir)) { + break; + } + for (final String key : KEY_ORDER) { + final String prefix = key + SEGMENT_SEPARATOR; + if (dirName.startsWith(prefix)) { + extractAndValidateAttributeValue(key, dirName, prefix.length()) + .ifPresent(value -> attributes.putIfAbsent(key, value)); + break; + } + } + current = current.getParentFile(); + } + return attributes; + } + + public static File findPendingDirectory(final File file) { + if (file == null) { + return null; + } + String[] dirs = IoTDBDescriptor.getInstance().getConfig().getLoadActiveListeningDirs(); + File current = file; + while (current != null) { + for (final String dir : dirs) { + if (current.isDirectory() && current.getAbsolutePath().equals(dir)) { + return current; + } + } + current = current.getParentFile(); + } + return null; + } + + public static void applyAttributesToStatement( + final Map attributes, + final LoadTsFileStatement statement, + final boolean defaultVerify) { + + Optional.ofNullable(attributes.get(LoadTsFileConfigurator.DATABASE_NAME_KEY)) + .filter(name -> !name.isEmpty()) + .ifPresent(statement::setDatabase); + + if (statement.getDatabase() == null || statement.getDatabase().isEmpty()) { + Optional.ofNullable(attributes.get(LoadTsFileConfigurator.DATABASE_KEY)) + .filter(name -> !name.isEmpty()) + .ifPresent(statement::setDatabase); + } + + Optional.ofNullable(attributes.get(LoadTsFileConfigurator.DATABASE_LEVEL_KEY)) + .ifPresent( + level -> { + try { + statement.setDatabaseLevel(Integer.parseInt(level)); + } catch (final NumberFormatException ignored) { + // keep the default when parsing fails + } + }); + + Optional.ofNullable(attributes.get(LoadTsFileConfigurator.CONVERT_ON_TYPE_MISMATCH_KEY)) + .ifPresent(value -> statement.setConvertOnTypeMismatch(Boolean.parseBoolean(value))); + + Optional.ofNullable(attributes.get(LoadTsFileConfigurator.TABLET_CONVERSION_THRESHOLD_KEY)) + .ifPresent( + threshold -> { + try { + statement.setTabletConversionThresholdBytes(Long.parseLong(threshold)); + } catch (final NumberFormatException ignored) { + // keep the default when parsing fails + } + }); + + if (attributes.containsKey(LoadTsFileConfigurator.VERIFY_KEY)) { + statement.setVerifySchema( + Boolean.parseBoolean(attributes.get(LoadTsFileConfigurator.VERIFY_KEY))); + } else { + statement.setVerifySchema(defaultVerify); + } + + if (attributes.containsKey(LoadTsFileConfigurator.PIPE_GENERATED_KEY) + && Boolean.parseBoolean(attributes.get(LoadTsFileConfigurator.PIPE_GENERATED_KEY))) { + statement.markIsGeneratedByPipe(); + } + } + + public static boolean containsDatabaseName(final Map attributes) { + return attributes.containsKey(LoadTsFileConfigurator.DATABASE_NAME_KEY) + || attributes.containsKey(LoadTsFileConfigurator.DATABASE_KEY); + } + + private static String formatSegment(final String key, final String value) { + return key + SEGMENT_SEPARATOR + encodeValue(value); + } + + private static String encodeValue(final String value) { + try { + return URLEncoder.encode(value, StandardCharsets.UTF_8.toString()); + } catch (final UnsupportedEncodingException e) { + // UTF-8 should always be supported; fallback to raw value when unexpected + return value; + } + } + + private static Optional extractAndValidateAttributeValue( + final String key, final String dirName, final int prefixLength) { + if (dirName.length() <= prefixLength) { + return Optional.empty(); + } + + final String encodedValue = dirName.substring(prefixLength); + final String decodedValue = decodeValue(encodedValue); + try { + validateAttributeValue(key, decodedValue); + return Optional.of(decodedValue); + } catch (final SemanticException e) { + return Optional.empty(); + } + } + + private static void validateAttributeValue(final String key, final String value) { + switch (key) { + case LoadTsFileConfigurator.DATABASE_NAME_KEY: + if (value == null || value.isEmpty()) { + throw new SemanticException("Database name must not be empty."); + } + break; + case LoadTsFileConfigurator.DATABASE_LEVEL_KEY: + LoadTsFileConfigurator.validateDatabaseLevelParam(value); + break; + case LoadTsFileConfigurator.CONVERT_ON_TYPE_MISMATCH_KEY: + LoadTsFileConfigurator.validateConvertOnTypeMismatchParam(value); + break; + case LoadTsFileConfigurator.TABLET_CONVERSION_THRESHOLD_KEY: + validateTabletConversionThreshold(value); + break; + case LoadTsFileConfigurator.VERIFY_KEY: + LoadTsFileConfigurator.validateVerifyParam(value); + break; + default: + LoadTsFileConfigurator.validateParameters(key, value); + } + } + + private static void validateTabletConversionThreshold(final String value) { + try { + final long threshold = Long.parseLong(value); + if (threshold < 0) { + throw new SemanticException( + "Tablet conversion threshold must be a non-negative long value."); + } + } catch (final NumberFormatException e) { + throw new SemanticException( + String.format("Tablet conversion threshold '%s' is not a valid long value.", value)); + } + } + + private static String decodeValue(final String value) { + try { + return URLDecoder.decode(value, StandardCharsets.UTF_8.toString()); + } catch (final UnsupportedEncodingException e) { + return value; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPendingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPendingQueue.java index 3ec283cd2e0df..7b5f7166197d2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPendingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPendingQueue.java @@ -34,9 +34,13 @@ public class ActiveLoadPendingQueue { private final Set loadingFileSet = new HashSet<>(); public synchronized boolean enqueue( - final String file, final boolean isGeneratedByPipe, final boolean isTableModel) { + final String file, + final String pendingDir, + final boolean isGeneratedByPipe, + final boolean isTableModel) { if (!loadingFileSet.contains(file) && pendingFileSet.add(file)) { - pendingFileQueue.offer(new ActiveLoadEntry(file, isGeneratedByPipe, isTableModel)); + pendingFileQueue.offer( + new ActiveLoadEntry(file, pendingDir, isGeneratedByPipe, isTableModel)); ActiveLoadingFilesNumberMetricsSet.getInstance().increaseQueuingFileCounter(1); return true; @@ -76,11 +80,14 @@ public boolean isEmpty() { public static class ActiveLoadEntry { private final String file; + private final String pendingDir; private final boolean isGeneratedByPipe; private final boolean isTableModel; - public ActiveLoadEntry(String file, boolean isGeneratedByPipe, boolean isTableModel) { + public ActiveLoadEntry( + String file, String pendingDir, boolean isGeneratedByPipe, boolean isTableModel) { this.file = file; + this.pendingDir = pendingDir; this.isGeneratedByPipe = isGeneratedByPipe; this.isTableModel = isTableModel; } @@ -89,6 +96,10 @@ public String getFile() { return file; } + public String getPendingDir() { + return pendingDir; + } + public boolean isGeneratedByPipe() { return isGeneratedByPipe; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java index 62feb562be720..20817c9414634 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java @@ -56,6 +56,7 @@ import java.nio.file.attribute.BasicFileAttributes; import java.time.ZoneId; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.concurrent.LinkedBlockingQueue; @@ -84,12 +85,12 @@ public int getCurrentAllowedPendingSize() { } public void tryTriggerTsFileLoad( - String absolutePath, boolean isTabletMode, boolean isGeneratedByPipe) { + String absolutePath, String pendingDir, boolean isTabletMode, boolean isGeneratedByPipe) { if (CommonDescriptor.getInstance().getConfig().isReadOnly()) { return; } - if (pendingQueue.enqueue(absolutePath, isGeneratedByPipe, isTabletMode)) { + if (pendingQueue.enqueue(absolutePath, pendingDir, isGeneratedByPipe, isTabletMode)) { initFailDirIfNecessary(); adjustExecutorIfNecessary(); } @@ -216,24 +217,29 @@ private Optional tryGetNextPendingFile() private TSStatus loadTsFile( final ActiveLoadPendingQueue.ActiveLoadEntry entry, final IClientSession session) throws FileNotFoundException { - final LoadTsFileStatement statement = new LoadTsFileStatement(entry.getFile()); + final File tsFile = new File(entry.getFile()); + final LoadTsFileStatement statement = new LoadTsFileStatement(tsFile.getAbsolutePath()); final List files = statement.getTsFiles(); - // It should be noted here that the instructions in this code block do not need to use the - // DataBase, so the DataBase is assigned a value of null. If the DataBase is used later, an - // exception will be thrown. - final File parentFile; - statement.setDatabase( - files.isEmpty() - || !entry.isTableModel() - || (parentFile = files.get(0).getParentFile()) == null - ? null - : parentFile.getName()); statement.setDeleteAfterLoad(true); - statement.setConvertOnTypeMismatch(true); - statement.setVerifySchema(isVerify); statement.setAutoCreateDatabase( IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled()); + + final File pendingDir = + entry.getPendingDir() == null + ? ActiveLoadPathHelper.findPendingDirectory(tsFile) + : new File(entry.getPendingDir()); + final Map attributes = ActiveLoadPathHelper.parseAttributes(tsFile, pendingDir); + ActiveLoadPathHelper.applyAttributesToStatement(attributes, statement, isVerify); + + final File parentFile; + if (statement.getDatabase() == null && entry.isTableModel()) { + statement.setDatabase( + files.isEmpty() || (parentFile = files.get(0).getParentFile()) == null + ? null + : parentFile.getName()); + } + return executeStatement( entry.isGeneratedByPipe() ? new PipeEnrichedStatement(statement) : statement, session); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java index 669013ec45054..e3dbe43507d88 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java @@ -32,7 +32,9 @@ import java.io.File; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import static org.apache.iotdb.commons.utils.FileUtils.copyFileWithMD5Check; @@ -45,14 +47,16 @@ public class ActiveLoadUtil { private static volatile ILoadDiskSelector loadDiskSelector = updateLoadDiskSelector(); public static boolean loadTsFileAsyncToActiveDir( - final List tsFiles, final String dataBaseName, final boolean isDeleteAfterLoad) { + final List tsFiles, + final Map loadAttributes, + final boolean isDeleteAfterLoad) { if (tsFiles == null || tsFiles.isEmpty()) { return true; } try { for (File file : tsFiles) { - if (!loadTsFilesToActiveDir(dataBaseName, file, isDeleteAfterLoad)) { + if (!loadTsFilesToActiveDir(loadAttributes, file, isDeleteAfterLoad)) { return false; } } @@ -65,7 +69,7 @@ public static boolean loadTsFileAsyncToActiveDir( } private static boolean loadTsFilesToActiveDir( - final String dataBaseName, final File file, final boolean isDeleteAfterLoad) + final Map loadAttributes, final File file, final boolean isDeleteAfterLoad) throws IOException { if (file == null) { return true; @@ -84,12 +88,9 @@ private static boolean loadTsFilesToActiveDir( LOGGER.warn("Load active listening dir is not set."); return false; } - final File targetDir; - if (Objects.nonNull(dataBaseName)) { - targetDir = new File(targetFilePath, dataBaseName); - } else { - targetDir = targetFilePath; - } + final Map attributes = + Objects.nonNull(loadAttributes) ? loadAttributes : Collections.emptyMap(); + final File targetDir = ActiveLoadPathHelper.resolveTargetDir(targetFilePath, attributes); loadTsFileAsyncToTargetDir( targetDir, new File(file.getAbsolutePath() + ".resource"), isDeleteAfterLoad); @@ -100,7 +101,9 @@ private static boolean loadTsFilesToActiveDir( } public static boolean loadFilesToActiveDir( - final String dataBaseName, final List files, final boolean isDeleteAfterLoad) + final Map loadAttributes, + final List files, + final boolean isDeleteAfterLoad) throws IOException { if (files == null || files.isEmpty()) { return true; @@ -120,12 +123,9 @@ public static boolean loadFilesToActiveDir( LOGGER.warn("Load active listening dir is not set."); return false; } - final File targetDir; - if (Objects.nonNull(dataBaseName)) { - targetDir = new File(targetFilePath, dataBaseName); - } else { - targetDir = targetFilePath; - } + final Map attributes = + Objects.nonNull(loadAttributes) ? loadAttributes : Collections.emptyMap(); + final File targetDir = ActiveLoadPathHelper.resolveTargetDir(targetFilePath, attributes); for (final String file : files) { loadTsFileAsyncToTargetDir(targetDir, new File(file), isDeleteAfterLoad); @@ -138,6 +138,11 @@ private static void loadTsFileAsyncToTargetDir( if (!file.exists()) { return; } + if (!targetDir.exists() && !targetDir.mkdirs()) { + if (!targetDir.exists()) { + throw new IOException("Failed to create target directory " + targetDir.getAbsolutePath()); + } + } RetryUtils.retryOnException( () -> { if (isDeleteAfterLoad) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/config/LoadTsFileConfigurator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/config/LoadTsFileConfigurator.java index da94de532d13c..8478486781be5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/config/LoadTsFileConfigurator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/config/LoadTsFileConfigurator.java @@ -54,6 +54,9 @@ public static void validateParameters(final String key, final String value) { case VERIFY_KEY: validateVerifyParam(value); break; + case PIPE_GENERATED_KEY: + validatePipeGeneratedParam(value); + break; case ASYNC_LOAD_KEY: validateAsyncLoadParam(value); break; @@ -183,6 +186,21 @@ public static boolean parseOrGetDefaultVerify(final Map loadAttr loadAttributes.getOrDefault(VERIFY_KEY, String.valueOf(VERIFY_DEFAULT_VALUE))); } + public static final String PIPE_GENERATED_KEY = "pipe-generated"; + + public static void validatePipeGeneratedParam(final String pipeGenerated) { + if (!"true".equalsIgnoreCase(pipeGenerated) && !"false".equalsIgnoreCase(pipeGenerated)) { + throw new SemanticException( + String.format( + "Given %s value '%s' is not supported, please input a valid boolean value.", + PIPE_GENERATED_KEY, pipeGenerated)); + } + } + + public static boolean parseOrGetDefaultPipeGenerated(final Map loadAttributes) { + return Boolean.parseBoolean(loadAttributes.getOrDefault(PIPE_GENERATED_KEY, "false")); + } + public static final String ASYNC_LOAD_KEY = "async"; private static final boolean ASYNC_LOAD_DEFAULT_VALUE = false; From 3a6c3bbce547d5319f6146c9d0a9adee01c9c39a Mon Sep 17 00:00:00 2001 From: Yongzao Date: Sat, 15 Nov 2025 16:56:32 +0800 Subject: [PATCH 061/180] [AINode] Enable AINode start as background (-d) (#16762) (cherry picked from commit 1ebb9515767eee492a908cfad38a478f1ec7bba0) --- .../ainode/iotdb/ainode/core/ai_node.py | 10 ++++++--- .../ainode/core/inference/pool_controller.py | 2 +- .../ainode/core/manager/inference_manager.py | 4 ++-- .../ainode/iotdb/ainode/core/rpc/handler.py | 2 +- .../ainode/iotdb/ainode/core/rpc/service.py | 1 + scripts/sbin/start-ainode.sh | 17 ++++++++++---- scripts/sbin/windows/start-ainode.bat | 22 ++++++++++++++----- 7 files changed, 42 insertions(+), 16 deletions(-) diff --git a/iotdb-core/ainode/iotdb/ainode/core/ai_node.py b/iotdb-core/ainode/iotdb/ainode/core/ai_node.py index 587ae4930dede..d8f619e1b8d38 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/ai_node.py +++ b/iotdb-core/ainode/iotdb/ainode/core/ai_node.py @@ -163,7 +163,11 @@ def stop(self): self._rpc_handler.stop() if self._rpc_service: self._rpc_service.stop() - self._rpc_service.join(1) - if self._rpc_service.is_alive(): - logger.warning("RPC service thread failed to stop in time.") + for retry in range(30): + self._rpc_service.join(2) + if not self._rpc_service.is_alive(): + logger.warning( + "RPC service thread failed to stop in time, retrying..." + ) + break logger.info("IoTDB-AINode has successfully stopped.") diff --git a/iotdb-core/ainode/iotdb/ainode/core/inference/pool_controller.py b/iotdb-core/ainode/iotdb/ainode/core/inference/pool_controller.py index 00bb3b5568b0f..54580402ec293 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/inference/pool_controller.py +++ b/iotdb-core/ainode/iotdb/ainode/core/inference/pool_controller.py @@ -504,7 +504,7 @@ def get_load(self, model_id, device_id, pool_id) -> int: return pool_group.get_load(pool_id) return -1 - def shutdown(self): + def stop(self): self._stop_event.set() # shutdown pool controller diff --git a/iotdb-core/ainode/iotdb/ainode/core/manager/inference_manager.py b/iotdb-core/ainode/iotdb/ainode/core/manager/inference_manager.py index 841159d9b4c92..a67d576b0ec8c 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/manager/inference_manager.py +++ b/iotdb-core/ainode/iotdb/ainode/core/manager/inference_manager.py @@ -366,9 +366,9 @@ def inference(self, req: TInferenceReq): single_output=False, ) - def shutdown(self): + def stop(self): self._stop_event.set() - self._pool_controller.shutdown() + self._pool_controller.stop() while not self._result_queue.empty(): self._result_queue.get_nowait() self._result_queue.close() diff --git a/iotdb-core/ainode/iotdb/ainode/core/rpc/handler.py b/iotdb-core/ainode/iotdb/ainode/core/rpc/handler.py index 04ac139e7d481..f01e1594f0698 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/rpc/handler.py +++ b/iotdb-core/ainode/iotdb/ainode/core/rpc/handler.py @@ -69,7 +69,7 @@ def __init__(self, ainode): def stop(self) -> None: logger.info("Stopping the RPC service handler of IoTDB-AINode...") - self._inference_manager.shutdown() + self._inference_manager.stop() def stopAINode(self) -> TSStatus: self._ainode.stop() diff --git a/iotdb-core/ainode/iotdb/ainode/core/rpc/service.py b/iotdb-core/ainode/iotdb/ainode/core/rpc/service.py index c556ffd4ba0d5..fc93a0cce7d5b 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/rpc/service.py +++ b/iotdb-core/ainode/iotdb/ainode/core/rpc/service.py @@ -125,3 +125,4 @@ def stop(self) -> None: logger.info("Stopping the RPC service of IoTDB-AINode...") self._stop_event.set() self.__pool_server.stop() + self._handler.stop() diff --git a/scripts/sbin/start-ainode.sh b/scripts/sbin/start-ainode.sh index 671de54ba1a86..454a0389c3866 100644 --- a/scripts/sbin/start-ainode.sh +++ b/scripts/sbin/start-ainode.sh @@ -27,10 +27,14 @@ export IOTDB_AINODE_HOME echo "IOTDB_AINODE_HOME: $IOTDB_AINODE_HOME" # fetch parameters with names -while getopts "i:rn" opt; do +daemon_mode=false +while getopts "i:rnd" opt; do case $opt in n) ;; + d) + daemon_mode=true + ;; \?) echo "Invalid option -$OPTARG" >&2 exit 1 ;; @@ -41,6 +45,11 @@ ain_ainode_executable="$IOTDB_AINODE_HOME/lib/ainode" echo Script got ainode executable: "$ain_ainode_executable" -echo Starting AINode... - -$ain_ainode_executable start +if [ "$daemon_mode" = true ]; then + echo Starting AINode in daemon mode... + nohup $ain_ainode_executable start > /dev/null 2>&1 & + echo AINode started in background +else + echo Starting AINode... + $ain_ainode_executable start +fi diff --git a/scripts/sbin/windows/start-ainode.bat b/scripts/sbin/windows/start-ainode.bat index 1d9a4306bafb7..09ce914f22672 100644 --- a/scripts/sbin/windows/start-ainode.bat +++ b/scripts/sbin/windows/start-ainode.bat @@ -30,8 +30,20 @@ set ain_ainode_executable=%IOTDB_AINODE_HOME%\lib\ainode echo Script got ainode executable: %ain_ainode_executable% -echo Starting AINode... - -%$ain_ainode_executable% start - -pause \ No newline at end of file +set daemon_mode=false +:parse_args +if "%~1"=="" goto end_parse +if /i "%~1"=="-d" set daemon_mode=true +shift +goto parse_args +:end_parse + +if "%daemon_mode%"=="true" ( + echo Starting AINode in daemon mode... + start /B "" %ain_ainode_executable% start + echo AINode started in background +) else ( + echo Starting AINode... + %ain_ainode_executable% start + pause +) \ No newline at end of file From 71976f5ba5eebb3e6cccf055fbedcad3501c9db9 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 17 Nov 2025 11:20:16 +0800 Subject: [PATCH 062/180] fix (#16702) (cherry picked from commit 3a6322028cd6026df7ebc8a97020e45fc5d8bf00) --- .../receiver/PipeReceiverStatusHandler.java | 134 +++++++++--------- 1 file changed, 64 insertions(+), 70 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java index 350746d7b8e52..9fe87b22fd737 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java @@ -45,6 +45,9 @@ public class PipeReceiverStatusHandler { private static final Logger LOGGER = LoggerFactory.getLogger(PipeReceiverStatusHandler.class); + private static final String NO_PERMISSION = "No permission"; + private static final String UNCLASSIFIED_EXCEPTION = "Unclassified exception"; + private static final String NO_PERMISSION_STR = "No permissions for this operation"; private static final int CONFLICT_RETRY_MAX_TIMES = 100; @@ -183,83 +186,74 @@ public void handle( if (skipIfNoPrivileges) { return; } - - synchronized (this) { - recordExceptionStatusIfNecessary(recordMessage); - - if (exceptionEventHasBeenRetried.get() - && System.currentTimeMillis() - exceptionFirstEncounteredTime.get() - > retryMaxMillisWhenOtherExceptionsOccur) { - LOGGER.warn( - "No permission: retry timeout. will be ignored. event: {}. status: {}", - shouldRecordIgnoredDataWhenOtherExceptionsOccur ? recordMessage : "not recorded", - status); - resetExceptionStatus(); + handleOtherExceptions(status, exceptionMessage, recordMessage, true); + break; + case 305: + handleOtherExceptions(status, exceptionMessage, recordMessage, false); + break; + default: + // Some auth error may be wrapped in other codes + if (exceptionMessage.contains(NO_PERMISSION_STR)) { + if (skipIfNoPrivileges) { return; } - - // Reduce the log if retry forever - if (retryMaxMillisWhenOtherExceptionsOccur == Long.MAX_VALUE) { - PipeLogger.log(LOGGER::warn, "No permission: will retry forever. status: %s", status); - } else { - LOGGER.warn( - "No permission: will retry for at least {} seconds. status: {}", - (retryMaxMillisWhenOtherExceptionsOccur - + exceptionFirstEncounteredTime.get() - - System.currentTimeMillis()) - / 1000.0, - status); - } - - exceptionEventHasBeenRetried.set(true); - throw new PipeRuntimeSinkRetryTimesConfigurableException( - exceptionMessage, - (int) - Math.max( - PipeSubtask.MAX_RETRY_TIMES, - Math.min( - CONFLICT_RETRY_MAX_TIMES, retryMaxMillisWhenOtherExceptionsOccur * 1.1))); + handleOtherExceptions(status, exceptionMessage, recordMessage, true); + break; } + // Other exceptions + handleOtherExceptions(status, exceptionMessage, recordMessage, false); + break; + } + } - default: // Other exceptions - synchronized (this) { - recordExceptionStatusIfNecessary(recordMessage); + private synchronized void handleOtherExceptions( + final TSStatus status, + final String exceptionMessage, + final String recordMessage, + final boolean noPermission) { + recordExceptionStatusIfNecessary(recordMessage); + + if (exceptionEventHasBeenRetried.get() + && System.currentTimeMillis() - exceptionFirstEncounteredTime.get() + > retryMaxMillisWhenOtherExceptionsOccur) { + LOGGER.warn( + "{}: retry timeout. will be ignored. event: {}. status: {}", + getNoPermission(noPermission), + shouldRecordIgnoredDataWhenOtherExceptionsOccur ? recordMessage : "not recorded", + status); + resetExceptionStatus(); + return; + } - if (exceptionEventHasBeenRetried.get() - && System.currentTimeMillis() - exceptionFirstEncounteredTime.get() - > retryMaxMillisWhenOtherExceptionsOccur) { - LOGGER.warn( - "Unclassified exception: retry timeout. will be ignored. event: {}. status: {}", - shouldRecordIgnoredDataWhenOtherExceptionsOccur ? recordMessage : "not recorded", - status); - resetExceptionStatus(); - return; - } + // Reduce the log if retry forever + if (retryMaxMillisWhenOtherExceptionsOccur == Long.MAX_VALUE) { + PipeLogger.log( + LOGGER::warn, + "%s: will retry forever. status: %s", + getNoPermission(noPermission), + status); + } else { + LOGGER.warn( + "{}: will retry for at least {} seconds. status: {}", + getNoPermission(noPermission), + (retryMaxMillisWhenOtherExceptionsOccur + + exceptionFirstEncounteredTime.get() + - System.currentTimeMillis()) + / 1000.0, + status); + } - // Reduce the log if retry forever - if (retryMaxMillisWhenOtherExceptionsOccur == Long.MAX_VALUE) { - PipeLogger.log( - LOGGER::warn, "Unclassified exception: will retry forever. status: %s", status); - } else { - LOGGER.warn( - "Unclassified exception: will retry for at least {} seconds. status: {}", - (retryMaxMillisWhenOtherExceptionsOccur - + exceptionFirstEncounteredTime.get() - - System.currentTimeMillis()) - / 1000.0, - status); - } + exceptionEventHasBeenRetried.set(true); + throw new PipeRuntimeSinkRetryTimesConfigurableException( + exceptionMessage, + (int) + Math.max( + PipeSubtask.MAX_RETRY_TIMES, + Math.min(CONFLICT_RETRY_MAX_TIMES, retryMaxMillisWhenOtherExceptionsOccur * 1.1))); + } - exceptionEventHasBeenRetried.set(true); - throw new PipeRuntimeSinkRetryTimesConfigurableException( - exceptionMessage, - (int) - Math.max( - PipeSubtask.MAX_RETRY_TIMES, - Math.min( - CONFLICT_RETRY_MAX_TIMES, retryMaxMillisWhenOtherExceptionsOccur * 1.1))); - } - } + private static String getNoPermission(final boolean noPermission) { + return noPermission ? NO_PERMISSION : UNCLASSIFIED_EXCEPTION; } private void recordExceptionStatusIfNecessary(final String message) { From 104bcf1eb4fb77ce703e57f13397ce5183f4475c Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Mon, 17 Nov 2025 11:42:43 +0800 Subject: [PATCH 063/180] reduce error log when compaction service is stopped & fix bugs (#16756) * reduce error log when compaction service is stopped * fix writer for compaction tests * fix npe (cherry picked from commit a7f1527d04cd29846987e5cc6df557e2050bd5cc) --- .../execute/task/AbstractCompactionTask.java | 3 ++- .../execute/task/InnerSpaceCompactionTask.java | 14 +++++++------- .../compaction/utils/CompactionTestFileWriter.java | 6 +++++- 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/AbstractCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/AbstractCompactionTask.java index cf59a19552bb8..dd37f626d70de 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/AbstractCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/AbstractCompactionTask.java @@ -164,7 +164,8 @@ protected void handleException(Logger logger, Exception e) { } else if (e instanceof InterruptedException || Thread.interrupted() || e instanceof StopReadTsFileByInterruptException - || !tsFileManager.isAllowCompaction()) { + || !tsFileManager.isAllowCompaction() + || CompactionTaskManager.getInstance().isStopAllCompactionWorker()) { logger.warn( "{}-{} [Compaction] {} task interrupted", storageGroupName, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java index 9c7fcc5424fdd..9bc91bf6bf1ad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java @@ -117,14 +117,14 @@ public InnerSpaceCompactionTask( } protected static class InnerCompactionTaskFilesView { - protected List sortedAllSourceFilesInTask; - protected List sourceFilesInCompactionPerformer; - protected List skippedSourceFiles; + protected List sortedAllSourceFilesInTask = Collections.emptyList(); + protected List sourceFilesInCompactionPerformer = Collections.emptyList(); + protected List skippedSourceFiles = Collections.emptyList(); protected boolean sequence; - protected List sourceFilesInLog; - protected List targetFilesInLog; - protected List targetFilesInPerformer; - protected List renamedTargetFiles; + protected List sourceFilesInLog = Collections.emptyList(); + protected List targetFilesInLog = Collections.emptyList(); + protected List targetFilesInPerformer = Collections.emptyList(); + protected List renamedTargetFiles = Collections.emptyList(); protected long selectedFileSize; protected int sumOfCompactionCount; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionTestFileWriter.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionTestFileWriter.java index fe88adfac681a..42254428ef8d1 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionTestFileWriter.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionTestFileWriter.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.utils; +import org.apache.iotdb.db.storageengine.dataregion.compaction.io.CompactionTsFileWriter; +import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.constant.CompactionType; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.enums.TSDataType; @@ -49,7 +51,9 @@ public class CompactionTestFileWriter implements Closeable { public CompactionTestFileWriter(TsFileResource emptyFile) throws IOException { this.resource = emptyFile; - fileWriter = new TsFileIOWriter(emptyFile.getTsFile()); + fileWriter = + new CompactionTsFileWriter( + emptyFile.getTsFile(), 1024 * 1024 * 1024, CompactionType.INNER_SEQ_COMPACTION); } public IDeviceID startChunkGroup(String deviceNameWithoutParentPath) throws IOException { From dfda7a0b221afcdadb1d2038d65fcacd21a54fa7 Mon Sep 17 00:00:00 2001 From: Zeyu Zhang <111627783+Alchuang22-dev@users.noreply.github.com> Date: Tue, 18 Nov 2025 00:02:24 +0800 Subject: [PATCH 064/180] [AINode] Update AINodeClient for DataNode to borrow (#16647) (cherry picked from commit d49d7dd77115aaf951b2f7896ea4347cb1a77bfb) --- .../async/AsyncAINodeHeartbeatClientPool.java | 7 +- .../request/read/model/ShowModelPlan.java | 4 +- .../confignode/manager/ConfigManager.java | 26 +-- .../iotdb/confignode/manager/IManager.java | 18 +- .../confignode/manager/ModelManager.java | 37 ++-- .../impl/model/CreateModelProcedure.java | 4 +- .../impl/model/DropModelProcedure.java | 7 +- .../impl/node/RemoveAINodeProcedure.java | 4 +- .../thrift/ConfigNodeRPCServiceProcessor.java | 69 ++++--- .../protocol/client/AINodeClientFactory.java | 133 ++++++++++++++ .../db/protocol/client/ConfigNodeClient.java | 51 +----- .../protocol}/client/ainode/AINodeClient.java | 168 +++++++++++------- .../client/ainode/AINodeClientManager.java | 75 ++++++++ .../ainode/AsyncAINodeServiceClient.java | 2 +- .../process/ai/InferenceOperator.java | 4 +- ...formationSchemaContentSupplierFactory.java | 17 +- .../executor/ClusterConfigTaskExecutor.java | 109 ++++++------ .../config/metadata/ai/ShowAIDevicesTask.java | 2 +- .../metadata/ai/ShowLoadedModelsTask.java | 4 +- .../config/metadata/ai/ShowModelsTask.java | 4 +- .../function/tvf/ForecastTableFunction.java | 8 +- .../db/queryengine/plan/udf/UDTFForecast.java | 8 +- iotdb-core/node-commons/pom.xml | 6 - .../commons/client/ClientPoolFactory.java | 52 ------ .../client/ainode/AINodeClientManager.java | 40 ----- .../src/main/thrift/confignode.thrift | 83 ++------- 26 files changed, 487 insertions(+), 455 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/AINodeClientFactory.java rename iotdb-core/{node-commons/src/main/java/org/apache/iotdb/commons => datanode/src/main/java/org/apache/iotdb/db/protocol}/client/ainode/AINodeClient.java (74%) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AINodeClientManager.java rename iotdb-core/{node-commons/src/main/java/org/apache/iotdb/commons => datanode/src/main/java/org/apache/iotdb/db/protocol}/client/ainode/AsyncAINodeServiceClient.java (98%) delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AINodeClientManager.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncAINodeHeartbeatClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncAINodeHeartbeatClientPool.java index e09ccc79becbf..2721fedafb1e6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncAINodeHeartbeatClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncAINodeHeartbeatClientPool.java @@ -21,10 +21,10 @@ import org.apache.iotdb.ainode.rpc.thrift.TAIHeartbeatReq; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.commons.client.ClientPoolFactory; import org.apache.iotdb.commons.client.IClientManager; -import org.apache.iotdb.commons.client.ainode.AsyncAINodeServiceClient; import org.apache.iotdb.confignode.client.async.handlers.heartbeat.AINodeHeartbeatHandler; +import org.apache.iotdb.db.protocol.client.AINodeClientFactory; +import org.apache.iotdb.db.protocol.client.ainode.AsyncAINodeServiceClient; public class AsyncAINodeHeartbeatClientPool { @@ -33,8 +33,7 @@ public class AsyncAINodeHeartbeatClientPool { private AsyncAINodeHeartbeatClientPool() { clientManager = new IClientManager.Factory() - .createClientManager( - new ClientPoolFactory.AsyncAINodeHeartbeatServiceClientPoolFactory()); + .createClientManager(new AINodeClientFactory.AINodeHeartbeatClientPoolFactory()); } public void getAINodeHeartBeat( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/ShowModelPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/ShowModelPlan.java index 16bc16bc8725d..eca00e8827d96 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/ShowModelPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/model/ShowModelPlan.java @@ -19,9 +19,9 @@ package org.apache.iotdb.confignode.consensus.request.read.model; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsReq; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; import org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq; import java.util.Objects; @@ -33,7 +33,7 @@ public ShowModelPlan() { super(ConfigPhysicalPlanType.ShowModel); } - public ShowModelPlan(final TShowModelReq showModelReq) { + public ShowModelPlan(final TShowModelsReq showModelReq) { super(ConfigPhysicalPlanType.ShowModel); if (showModelReq.isSetModelId()) { this.modelName = showModelReq.getModelId(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index 502937713c2e5..5d4b09adfc710 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -20,7 +20,14 @@ package org.apache.iotdb.confignode.manager; import org.apache.iotdb.ainode.rpc.thrift.IDataSchema; +import org.apache.iotdb.ainode.rpc.thrift.TLoadModelReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowAIDevicesResp; +import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsResp; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsResp; import org.apache.iotdb.ainode.rpc.thrift.TTrainingReq; +import org.apache.iotdb.ainode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.common.rpc.thrift.TAINodeConfiguration; import org.apache.iotdb.common.rpc.thrift.TAINodeLocation; import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; @@ -42,8 +49,6 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.commons.auth.AuthException; import org.apache.iotdb.commons.auth.entity.PrivilegeUnion; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AINodeClientManager; import org.apache.iotdb.commons.cluster.NodeStatus; import org.apache.iotdb.commons.cluster.NodeType; import org.apache.iotdb.commons.conf.CommonConfig; @@ -213,7 +218,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TGetTriggerTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUDFTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUdfTableReq; -import org.apache.iotdb.confignode.rpc.thrift.TLoadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TMigrateRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TNodeVersionInfo; import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp; @@ -226,7 +230,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionTableResp; import org.apache.iotdb.confignode.rpc.thrift.TSetDataNodeStatusReq; import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowAIDevicesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowAINodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; @@ -235,10 +238,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodes4InformationSchemaResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDatabaseResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp; import org.apache.iotdb.confignode.rpc.thrift.TShowPipePluginReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp; @@ -256,12 +255,13 @@ import org.apache.iotdb.confignode.rpc.thrift.TSubscribeReq; import org.apache.iotdb.confignode.rpc.thrift.TThrottleQuotaResp; import org.apache.iotdb.confignode.rpc.thrift.TTimeSlotList; -import org.apache.iotdb.confignode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; import org.apache.iotdb.confignode.rpc.thrift.TUpdateModelInfoReq; import org.apache.iotdb.consensus.common.DataSet; import org.apache.iotdb.consensus.exception.ConsensusException; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.db.schemaengine.template.Template; import org.apache.iotdb.db.schemaengine.template.TemplateAlterOperationType; import org.apache.iotdb.db.schemaengine.template.alter.TemplateAlterOperationUtil; @@ -2863,19 +2863,19 @@ public TSStatus unloadModel(TUnloadModelReq req) { } @Override - public TShowModelResp showModel(TShowModelReq req) { + public TShowModelsResp showModel(TShowModelsReq req) { TSStatus status = confirmLeader(); return status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() ? modelManager.showModel(req) - : new TShowModelResp(status); + : new TShowModelsResp(status); } @Override - public TShowLoadedModelResp showLoadedModel(TShowLoadedModelReq req) { + public TShowLoadedModelsResp showLoadedModel(TShowLoadedModelsReq req) { TSStatus status = confirmLeader(); return status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() ? modelManager.showLoadedModel(req) - : new TShowLoadedModelResp(status, Collections.emptyMap()); + : new TShowLoadedModelsResp(status, Collections.emptyMap()); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java index e15c33e04b7ec..33e77db24907d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java @@ -19,6 +19,13 @@ package org.apache.iotdb.confignode.manager; +import org.apache.iotdb.ainode.rpc.thrift.TLoadModelReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowAIDevicesResp; +import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsResp; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsResp; +import org.apache.iotdb.ainode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; @@ -128,7 +135,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TGetTriggerTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUDFTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUdfTableReq; -import org.apache.iotdb.confignode.rpc.thrift.TLoadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TMigrateRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp; import org.apache.iotdb.confignode.rpc.thrift.TPipeConfigTransferReq; @@ -140,7 +146,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionTableResp; import org.apache.iotdb.confignode.rpc.thrift.TSetDataNodeStatusReq; import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowAIDevicesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowAINodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; @@ -149,10 +154,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodes4InformationSchemaResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDatabaseResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp; import org.apache.iotdb.confignode.rpc.thrift.TShowPipePluginReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp; @@ -166,7 +167,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TStartPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TStopPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TSubscribeReq; -import org.apache.iotdb.confignode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; import org.apache.iotdb.consensus.common.DataSet; @@ -893,10 +893,10 @@ TDataPartitionTableResp getOrCreateDataPartition( TSStatus unloadModel(TUnloadModelReq req); /** Return the model table. */ - TShowModelResp showModel(TShowModelReq req); + TShowModelsResp showModel(TShowModelsReq req); /** Return the loaded model instances. */ - TShowLoadedModelResp showLoadedModel(TShowLoadedModelReq req); + TShowLoadedModelsResp showLoadedModel(TShowLoadedModelsReq req); /** Return all available AI devices. */ TShowAIDevicesResp showAIDevices(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ModelManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ModelManager.java index 4c1f94eab9e01..3efdbc222b6d2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ModelManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ModelManager.java @@ -19,14 +19,15 @@ package org.apache.iotdb.confignode.manager; +import org.apache.iotdb.ainode.rpc.thrift.TLoadModelReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowAIDevicesResp; import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsReq; import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsResp; import org.apache.iotdb.ainode.rpc.thrift.TShowModelsReq; import org.apache.iotdb.ainode.rpc.thrift.TShowModelsResp; +import org.apache.iotdb.ainode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AINodeClientManager; import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.model.ModelInformation; import org.apache.iotdb.commons.model.ModelStatus; @@ -40,15 +41,10 @@ import org.apache.iotdb.confignode.rpc.thrift.TDropModelReq; import org.apache.iotdb.confignode.rpc.thrift.TGetModelInfoReq; import org.apache.iotdb.confignode.rpc.thrift.TGetModelInfoResp; -import org.apache.iotdb.confignode.rpc.thrift.TLoadModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowAIDevicesResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp; -import org.apache.iotdb.confignode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TUpdateModelInfoReq; import org.apache.iotdb.consensus.exception.ConsensusException; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.rpc.TSStatusCode; import org.slf4j.Logger; @@ -124,15 +120,16 @@ public TSStatus unloadModel(TUnloadModelReq req) { } } - public TShowModelResp showModel(final TShowModelReq req) { + public TShowModelsResp showModel(final TShowModelsReq req) { try (AINodeClient client = getAINodeClient()) { TShowModelsReq showModelsReq = new TShowModelsReq(); if (req.isSetModelId()) { showModelsReq.setModelId(req.getModelId()); } TShowModelsResp resp = client.showModels(showModelsReq); - TShowModelResp res = - new TShowModelResp().setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + TShowModelsResp res = + new TShowModelsResp() + .setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); res.setModelIdList(resp.getModelIdList()); res.setModelTypeMap(resp.getModelTypeMap()); res.setCategoryMap(resp.getCategoryMap()); @@ -140,26 +137,26 @@ public TShowModelResp showModel(final TShowModelReq req) { return res; } catch (Exception e) { LOGGER.warn("Failed to show models due to", e); - return new TShowModelResp() + return new TShowModelsResp() .setStatus( new TSStatus(TSStatusCode.AI_NODE_INTERNAL_ERROR.getStatusCode()) .setMessage(e.getMessage())); } } - public TShowLoadedModelResp showLoadedModel(final TShowLoadedModelReq req) { + public TShowLoadedModelsResp showLoadedModel(final TShowLoadedModelsReq req) { try (AINodeClient client = getAINodeClient()) { TShowLoadedModelsReq showModelsReq = new TShowLoadedModelsReq().setDeviceIdList(req.getDeviceIdList()); TShowLoadedModelsResp resp = client.showLoadedModels(showModelsReq); - TShowLoadedModelResp res = - new TShowLoadedModelResp() + TShowLoadedModelsResp res = + new TShowLoadedModelsResp() .setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); res.setDeviceLoadedModelsMap(resp.getDeviceLoadedModelsMap()); return res; } catch (Exception e) { LOGGER.warn("Failed to show loaded models due to", e); - return new TShowLoadedModelResp() + return new TShowLoadedModelsResp() .setStatus( new TSStatus(TSStatusCode.AI_NODE_INTERNAL_ERROR.getStatusCode()) .setMessage(e.getMessage())); @@ -235,7 +232,11 @@ private AINodeClient getAINodeClient() throws NoAvailableAINodeException, Client } TEndPoint targetAINodeEndPoint = new TEndPoint(aiNodeInfo.get(0).getInternalAddress(), aiNodeInfo.get(0).getInternalPort()); - return AINodeClientManager.getInstance().borrowClient(targetAINodeEndPoint); + try { + return AINodeClientManager.getInstance().borrowClient(targetAINodeEndPoint); + } catch (Exception e) { + throw new RuntimeException(e); + } } public List getModelDistributions(String modelName) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/model/CreateModelProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/model/CreateModelProcedure.java index 8282608466d62..989061610213d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/model/CreateModelProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/model/CreateModelProcedure.java @@ -21,8 +21,6 @@ import org.apache.iotdb.common.rpc.thrift.TAINodeConfiguration; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AINodeClientManager; import org.apache.iotdb.commons.exception.ainode.LoadModelException; import org.apache.iotdb.commons.model.ModelInformation; import org.apache.iotdb.commons.model.ModelStatus; @@ -36,6 +34,8 @@ import org.apache.iotdb.confignode.procedure.state.model.CreateModelState; import org.apache.iotdb.confignode.procedure.store.ProcedureType; import org.apache.iotdb.consensus.exception.ConsensusException; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.utils.ReadWriteIOUtils; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/model/DropModelProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/model/DropModelProcedure.java index 23e02ea2e1d8a..daa029e04ddfd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/model/DropModelProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/model/DropModelProcedure.java @@ -19,10 +19,9 @@ package org.apache.iotdb.confignode.procedure.impl.model; +import org.apache.iotdb.ainode.rpc.thrift.TDeleteModelReq; import org.apache.iotdb.common.rpc.thrift.TAINodeConfiguration; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AINodeClientManager; import org.apache.iotdb.commons.model.exception.ModelManagementException; import org.apache.iotdb.confignode.consensus.request.write.model.DropModelPlan; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; @@ -30,6 +29,8 @@ import org.apache.iotdb.confignode.procedure.impl.node.AbstractNodeProcedure; import org.apache.iotdb.confignode.procedure.state.model.DropModelState; import org.apache.iotdb.confignode.procedure.store.ProcedureType; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.thrift.TException; @@ -115,7 +116,7 @@ private void dropModelOnAINode(ConfigNodeProcedureEnv env) { .getRegisteredAINode(nodeId) .getLocation() .getInternalEndPoint())) { - TSStatus status = client.deleteModel(modelName); + TSStatus status = client.deleteModel(new TDeleteModelReq(modelName)); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { LOGGER.warn( "Failed to drop model [{}] on AINode [{}], status: {}", diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/RemoveAINodeProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/RemoveAINodeProcedure.java index 5f98930d07476..2cab08c28244e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/RemoveAINodeProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/RemoveAINodeProcedure.java @@ -21,8 +21,6 @@ import org.apache.iotdb.common.rpc.thrift.TAINodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AINodeClientManager; import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils; import org.apache.iotdb.confignode.consensus.request.write.ainode.RemoveAINodePlan; import org.apache.iotdb.confignode.consensus.request.write.model.DropModelInNodePlan; @@ -30,6 +28,8 @@ import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.state.RemoveAINodeState; import org.apache.iotdb.confignode.procedure.store.ProcedureType; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.rpc.TSStatusCode; import org.slf4j.Logger; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index bcc9f5068a5b7..59ce7352312f0 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -19,6 +19,8 @@ package org.apache.iotdb.confignode.service.thrift; +import org.apache.iotdb.common.rpc.thrift.TAINodeConfiguration; +import org.apache.iotdb.common.rpc.thrift.TAINodeLocation; import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TFlushReq; @@ -119,7 +121,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TCreateSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTableViewReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTopicReq; -import org.apache.iotdb.confignode.rpc.thrift.TCreateTrainingReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq; import org.apache.iotdb.confignode.rpc.thrift.TDataNodeConfigurationResp; import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq; @@ -151,6 +152,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq; import org.apache.iotdb.confignode.rpc.thrift.TExtendRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TFetchTableResp; +import org.apache.iotdb.confignode.rpc.thrift.TGetAINodeLocationResp; import org.apache.iotdb.confignode.rpc.thrift.TGetAllPipeInfoResp; import org.apache.iotdb.confignode.rpc.thrift.TGetAllSubscriptionInfoResp; import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp; @@ -176,7 +178,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TGetTriggerTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUDFTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUdfTableReq; -import org.apache.iotdb.confignode.rpc.thrift.TLoadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TLoginReq; import org.apache.iotdb.confignode.rpc.thrift.TMigrateRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp; @@ -194,7 +195,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaReplicationFactorReq; import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowAIDevicesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowAINodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; @@ -203,10 +203,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodes4InformationSchemaResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDatabaseResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp; import org.apache.iotdb.confignode.rpc.thrift.TShowPipePluginReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp; @@ -228,7 +224,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TSystemConfigurationResp; import org.apache.iotdb.confignode.rpc.thrift.TTestOperation; import org.apache.iotdb.confignode.rpc.thrift.TThrottleQuotaResp; -import org.apache.iotdb.confignode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; import org.apache.iotdb.confignode.rpc.thrift.TUpdateModelInfoReq; @@ -649,6 +644,34 @@ public TSStatus operatePermission(final TAuthorizerReq req) { req.getNewUsername())); } + @Override + public TGetAINodeLocationResp getAINodeLocation() throws TException { + final TGetAINodeLocationResp resp = new TGetAINodeLocationResp(); + final TSStatus status = new TSStatus(); + try { + final List registeredAINodes = + configManager.getNodeManager().getRegisteredAINodes(); + + if (registeredAINodes == null || registeredAINodes.isEmpty()) { + status.setCode(TSStatusCode.NO_REGISTERED_AI_NODE_ERROR.getStatusCode()); + status.setMessage("No registered AINode found"); + resp.setStatus(status); + return resp; + } + + final TAINodeLocation loc = registeredAINodes.get(0).getLocation(); + resp.setAiNodeLocation(loc); + status.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + status.setMessage("AINode location resolved"); + + } catch (Exception e) { + status.setCode(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + status.setMessage("getAINodeLocation failed: " + e.getMessage()); + } + resp.setStatus(status); + return resp; + } + @Override public TAuthorizerResp queryPermission(final TAuthorizerReq req) { final PermissionInfoResp dataSet = @@ -1349,31 +1372,6 @@ public TSStatus dropModel(TDropModelReq req) { return configManager.dropModel(req); } - @Override - public TSStatus loadModel(TLoadModelReq req) { - return configManager.loadModel(req); - } - - @Override - public TSStatus unloadModel(TUnloadModelReq req) { - return configManager.unloadModel(req); - } - - @Override - public TShowModelResp showModel(TShowModelReq req) { - return configManager.showModel(req); - } - - @Override - public TShowLoadedModelResp showLoadedModel(TShowLoadedModelReq req) { - return configManager.showLoadedModel(req); - } - - @Override - public TShowAIDevicesResp showAIDevices() { - return configManager.showAIDevices(); - } - @Override public TGetModelInfoResp getModelInfo(TGetModelInfoReq req) { return configManager.getModelInfo(req); @@ -1384,11 +1382,6 @@ public TSStatus updateModelInfo(TUpdateModelInfoReq req) throws TException { return configManager.updateModelInfo(req); } - @Override - public TSStatus createTraining(TCreateTrainingReq req) throws TException { - return configManager.createTraining(req); - } - @Override public TSStatus setSpaceQuota(final TSetSpaceQuotaReq req) throws TException { return configManager.setSpaceQuota(req); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/AINodeClientFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/AINodeClientFactory.java new file mode 100644 index 0000000000000..0d784617c0905 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/AINodeClientFactory.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.protocol.client; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.ClientManager; +import org.apache.iotdb.commons.client.ClientManagerMetrics; +import org.apache.iotdb.commons.client.IClientPoolFactory; +import org.apache.iotdb.commons.client.factory.ThriftClientFactory; +import org.apache.iotdb.commons.client.property.ClientPoolProperty; +import org.apache.iotdb.commons.client.property.ThriftClientProperty; +import org.apache.iotdb.commons.concurrent.ThreadName; +import org.apache.iotdb.commons.conf.CommonConfig; +import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AsyncAINodeServiceClient; + +import org.apache.commons.pool2.PooledObject; +import org.apache.commons.pool2.impl.DefaultPooledObject; +import org.apache.commons.pool2.impl.GenericKeyedObjectPool; + +import java.util.Optional; + +/** Dedicated factory for AINodeClient + AINodeClientPoolFactory. */ +public class AINodeClientFactory extends ThriftClientFactory { + + private static final int connectionTimeout = + CommonDescriptor.getInstance().getConfig().getDnConnectionTimeoutInMS(); + + public AINodeClientFactory( + ClientManager manager, ThriftClientProperty thriftProperty) { + super(manager, thriftProperty); + } + + @Override + public PooledObject makeObject(TEndPoint endPoint) throws Exception { + return new DefaultPooledObject<>( + new AINodeClient(thriftClientProperty, endPoint, clientManager)); + } + + @Override + public void destroyObject(TEndPoint key, PooledObject pooled) throws Exception { + pooled.getObject().invalidate(); + } + + @Override + public boolean validateObject(TEndPoint key, PooledObject pooledObject) { + return Optional.ofNullable(pooledObject.getObject().getTransport()) + .map(org.apache.thrift.transport.TTransport::isOpen) + .orElse(false); + } + + /** The PoolFactory originally inside ClientPoolFactory — now moved here. */ + public static class AINodeClientPoolFactory + implements IClientPoolFactory { + + @Override + public GenericKeyedObjectPool createClientPool( + ClientManager manager) { + + // Build thrift client properties + ThriftClientProperty thriftProperty = + new ThriftClientProperty.Builder() + .setConnectionTimeoutMs(connectionTimeout) + .setRpcThriftCompressionEnabled( + CommonDescriptor.getInstance().getConfig().isRpcThriftCompressionEnabled()) + .build(); + + GenericKeyedObjectPool pool = + new GenericKeyedObjectPool<>( + new AINodeClientFactory(manager, thriftProperty), + new ClientPoolProperty.Builder() + .setMaxClientNumForEachNode( + CommonDescriptor.getInstance().getConfig().getMaxClientNumForEachNode()) + .build() + .getConfig()); + + ClientManagerMetrics.getInstance() + .registerClientManager(this.getClass().getSimpleName(), pool); + + return pool; + } + } + + public static class AINodeHeartbeatClientPoolFactory + implements IClientPoolFactory { + + @Override + public GenericKeyedObjectPool createClientPool( + ClientManager manager) { + + final CommonConfig conf = CommonDescriptor.getInstance().getConfig(); + + GenericKeyedObjectPool clientPool = + new GenericKeyedObjectPool<>( + new AsyncAINodeServiceClient.Factory( + manager, + new ThriftClientProperty.Builder() + .setConnectionTimeoutMs(conf.getCnConnectionTimeoutInMS()) + .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) + .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .setPrintLogWhenEncounterException(false) + .build(), + ThreadName.ASYNC_DATANODE_HEARTBEAT_CLIENT_POOL.getName()), + new ClientPoolProperty.Builder() + .setMaxClientNumForEachNode(conf.getMaxClientNumForEachNode()) + .build() + .getConfig()); + + ClientManagerMetrics.getInstance() + .registerClientManager(this.getClass().getSimpleName(), clientPool); + + return clientPool; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java index 6854a3191fad3..2c037cf0f3e58 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java @@ -79,7 +79,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TCreateSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTableViewReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTopicReq; -import org.apache.iotdb.confignode.rpc.thrift.TCreateTrainingReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq; import org.apache.iotdb.confignode.rpc.thrift.TDataNodeConfigurationResp; import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq; @@ -111,6 +110,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq; import org.apache.iotdb.confignode.rpc.thrift.TExtendRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TFetchTableResp; +import org.apache.iotdb.confignode.rpc.thrift.TGetAINodeLocationResp; import org.apache.iotdb.confignode.rpc.thrift.TGetAllPipeInfoResp; import org.apache.iotdb.confignode.rpc.thrift.TGetAllSubscriptionInfoResp; import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp; @@ -136,7 +136,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TGetTriggerTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUDFTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUdfTableReq; -import org.apache.iotdb.confignode.rpc.thrift.TLoadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TLoginReq; import org.apache.iotdb.confignode.rpc.thrift.TMigrateRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp; @@ -154,7 +153,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaReplicationFactorReq; import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowAIDevicesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowAINodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; @@ -163,10 +161,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodes4InformationSchemaResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDatabaseResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp; import org.apache.iotdb.confignode.rpc.thrift.TShowPipePluginReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp; @@ -188,7 +182,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TSystemConfigurationResp; import org.apache.iotdb.confignode.rpc.thrift.TTestOperation; import org.apache.iotdb.confignode.rpc.thrift.TThrottleQuotaResp; -import org.apache.iotdb.confignode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; import org.apache.iotdb.confignode.rpc.thrift.TUpdateModelInfoReq; @@ -530,6 +523,11 @@ public TAINodeRestartResp restartAINode(TAINodeRestartReq req) throws TException throw new UnsupportedOperationException(UNSUPPORTED_INVOCATION); } + @Override + public TGetAINodeLocationResp getAINodeLocation() throws TException { + return client.getAINodeLocation(); + } + @Override public TSStatus removeAINode(TAINodeRemoveReq req) throws TException { return executeRemoteCallWithRetry( @@ -1353,53 +1351,16 @@ public TSStatus dropModel(TDropModelReq req) throws TException { () -> client.dropModel(req), status -> !updateConfigNodeLeader(status)); } - @Override - public TShowModelResp showModel(TShowModelReq req) throws TException { - return executeRemoteCallWithRetry( - () -> client.showModel(req), resp -> !updateConfigNodeLeader(resp.status)); - } - - @Override - public TShowLoadedModelResp showLoadedModel(TShowLoadedModelReq req) throws TException { - return executeRemoteCallWithRetry( - () -> client.showLoadedModel(req), resp -> !updateConfigNodeLeader(resp.status)); - } - - @Override - public TShowAIDevicesResp showAIDevices() throws TException { - return executeRemoteCallWithRetry( - () -> client.showAIDevices(), resp -> !updateConfigNodeLeader(resp.status)); - } - - @Override - public TSStatus loadModel(TLoadModelReq req) throws TException { - return executeRemoteCallWithRetry( - () -> client.loadModel(req), status -> !updateConfigNodeLeader(status)); - } - - public TSStatus unloadModel(TUnloadModelReq req) throws TException { - return executeRemoteCallWithRetry( - () -> client.unloadModel(req), status -> !updateConfigNodeLeader(status)); - } - - @Override public TGetModelInfoResp getModelInfo(TGetModelInfoReq req) throws TException { return executeRemoteCallWithRetry( () -> client.getModelInfo(req), resp -> !updateConfigNodeLeader(resp.getStatus())); } - @Override public TSStatus updateModelInfo(TUpdateModelInfoReq req) throws TException { return executeRemoteCallWithRetry( () -> client.updateModelInfo(req), status -> !updateConfigNodeLeader(status)); } - @Override - public TSStatus createTraining(TCreateTrainingReq req) throws TException { - return executeRemoteCallWithRetry( - () -> client.createTraining(req), status -> !updateConfigNodeLeader(status)); - } - @Override public TSStatus setSpaceQuota(TSetSpaceQuotaReq req) throws TException { return executeRemoteCallWithRetry( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AINodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AINodeClient.java similarity index 74% rename from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AINodeClient.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AINodeClient.java index 0058bc7a2fc64..54150b8f3007b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AINodeClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AINodeClient.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.commons.client.ainode; +package org.apache.iotdb.db.protocol.client.ainode; import org.apache.iotdb.ainode.rpc.thrift.IAINodeRPCService; import org.apache.iotdb.ainode.rpc.thrift.TConfigs; @@ -37,16 +37,23 @@ import org.apache.iotdb.ainode.rpc.thrift.TTrainingReq; import org.apache.iotdb.ainode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.ainode.rpc.thrift.TWindowParams; +import org.apache.iotdb.common.rpc.thrift.TAINodeLocation; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.ClientManager; +import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.factory.ThriftClientFactory; import org.apache.iotdb.commons.client.property.ThriftClientProperty; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.consensus.ConfigRegionId; import org.apache.iotdb.commons.exception.ainode.LoadModelException; import org.apache.iotdb.commons.model.ModelInformation; +import org.apache.iotdb.confignode.rpc.thrift.TGetAINodeLocationResp; +import org.apache.iotdb.db.protocol.client.ConfigNodeClient; +import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; +import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.rpc.TConfigurationConst; import org.apache.iotdb.rpc.TSStatusCode; @@ -67,6 +74,7 @@ import java.io.IOException; import java.util.Map; import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.iotdb.rpc.TSStatusCode.CAN_NOT_CONNECT_AINODE; import static org.apache.iotdb.rpc.TSStatusCode.INTERNAL_SERVER_ERROR; @@ -77,7 +85,7 @@ public class AINodeClient implements AutoCloseable, ThriftClient { private static final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig(); - private final TEndPoint endPoint; + private TEndPoint endPoint; private TTransport transport; @@ -86,11 +94,90 @@ public class AINodeClient implements AutoCloseable, ThriftClient { public static final String MSG_CONNECTION_FAIL = "Fail to connect to AINode. Please check status of AINode"; + private static final int MAX_RETRY = 3; + + @FunctionalInterface + private interface RemoteCall { + R apply(IAINodeRPCService.Client c) throws TException; + } private final TsBlockSerde tsBlockSerde = new TsBlockSerde(); ClientManager clientManager; + private static final IClientManager CONFIG_NODE_CLIENT_MANAGER = + ConfigNodeClientManager.getInstance(); + + private static final AtomicReference CURRENT_LOCATION = new AtomicReference<>(); + + public static TEndPoint getCurrentEndpoint() { + TAINodeLocation loc = CURRENT_LOCATION.get(); + if (loc == null) { + loc = refreshFromConfigNode(); + } + return (loc == null) ? null : pickEndpointFrom(loc); + } + + public static void updateGlobalAINodeLocation(final TAINodeLocation loc) { + if (loc != null) { + CURRENT_LOCATION.set(loc); + } + } + + private R executeRemoteCallWithRetry(RemoteCall call) throws TException { + TException last = null; + for (int attempt = 1; attempt <= MAX_RETRY; attempt++) { + try { + if (transport == null || !transport.isOpen()) { + final TEndPoint ep = getCurrentEndpoint(); + if (ep == null) { + throw new TException("AINode endpoint unavailable"); + } + this.endPoint = ep; + init(); + } + return call.apply(client); + } catch (TException e) { + last = e; + invalidate(); + final TAINodeLocation loc = refreshFromConfigNode(); + if (loc != null) { + this.endPoint = pickEndpointFrom(loc); + } + try { + Thread.sleep(1000L * attempt); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } + } + } + throw (last != null ? last : new TException(MSG_CONNECTION_FAIL)); + } + + private static TAINodeLocation refreshFromConfigNode() { + try (final ConfigNodeClient cn = + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + final TGetAINodeLocationResp resp = cn.getAINodeLocation(); + if (resp != null && resp.isSetAiNodeLocation()) { + final TAINodeLocation loc = resp.getAiNodeLocation(); + CURRENT_LOCATION.set(loc); + return loc; + } + } catch (Exception e) { + LoggerFactory.getLogger(AINodeClient.class) + .debug("[AINodeClient] refreshFromConfigNode failed: {}", e.toString()); + } + return null; + } + + private static TEndPoint pickEndpointFrom(final TAINodeLocation loc) { + if (loc == null) return null; + if (loc.isSetInternalEndPoint() && loc.getInternalEndPoint() != null) { + return loc.getInternalEndPoint(); + } + return null; + } + public AINodeClient( ThriftClientProperty property, TEndPoint endPoint, @@ -98,6 +185,7 @@ public AINodeClient( throws TException { this.property = property; this.clientManager = clientManager; + // Instance default endpoint (pool key). Global location can override it on retries. this.endPoint = endPoint; init(); } @@ -188,76 +276,28 @@ private ModelInformation parseModelInformation( modelName, inputShape, outputShape, inputType, outputType, attributes); } - public TSStatus deleteModel(String modelId) throws TException { - try { - return client.deleteModel(new TDeleteModelReq(modelId)); - } catch (TException e) { - logger.warn( - "Failed to connect to AINode from ConfigNode when executing {}: {}", - Thread.currentThread().getStackTrace()[1].getMethodName(), - e.getMessage()); - throw new TException(MSG_CONNECTION_FAIL); - } + public TSStatus deleteModel(TDeleteModelReq req) throws TException { + return executeRemoteCallWithRetry(c -> c.deleteModel(req)); } public TSStatus loadModel(TLoadModelReq req) throws TException { - try { - return client.loadModel(req); - } catch (TException e) { - logger.warn( - "Failed to connect to AINode from ConfigNode when executing {}: {}", - Thread.currentThread().getStackTrace()[1].getMethodName(), - e.getMessage()); - throw new TException(MSG_CONNECTION_FAIL); - } + return executeRemoteCallWithRetry(c -> c.loadModel(req)); } public TSStatus unloadModel(TUnloadModelReq req) throws TException { - try { - return client.unloadModel(req); - } catch (TException e) { - logger.warn( - "Failed to connect to AINode from ConfigNode when executing {}: {}", - Thread.currentThread().getStackTrace()[1].getMethodName(), - e.getMessage()); - throw new TException(MSG_CONNECTION_FAIL); - } + return executeRemoteCallWithRetry(c -> c.unloadModel(req)); } public TShowModelsResp showModels(TShowModelsReq req) throws TException { - try { - return client.showModels(req); - } catch (TException e) { - logger.warn( - "Failed to connect to AINode from ConfigNode when executing {}: {}", - Thread.currentThread().getStackTrace()[1].getMethodName(), - e.getMessage()); - throw new TException(MSG_CONNECTION_FAIL); - } + return executeRemoteCallWithRetry(c -> c.showModels(req)); } public TShowLoadedModelsResp showLoadedModels(TShowLoadedModelsReq req) throws TException { - try { - return client.showLoadedModels(req); - } catch (TException e) { - logger.warn( - "Failed to connect to AINode from ConfigNode when executing {}: {}", - Thread.currentThread().getStackTrace()[1].getMethodName(), - e.getMessage()); - throw new TException(MSG_CONNECTION_FAIL); - } + return executeRemoteCallWithRetry(c -> c.showLoadedModels(req)); } public TShowAIDevicesResp showAIDevices() throws TException { - try { - return client.showAIDevices(); - } catch (TException e) { - logger.warn( - "Failed to connect to AINode from ConfigNode when executing {}: {}", - Thread.currentThread().getStackTrace()[1].getMethodName(), - e.getMessage()); - throw new TException(MSG_CONNECTION_FAIL); - } + return executeRemoteCallWithRetry(IAINodeRPCService.Client::showAIDevices); } public TInferenceResp inference( @@ -274,7 +314,7 @@ public TInferenceResp inference( if (inferenceAttributes != null) { inferenceReq.setInferenceAttributes(inferenceAttributes); } - return client.inference(inferenceReq); + return executeRemoteCallWithRetry(c -> c.inference(inferenceReq)); } catch (IOException e) { throw new TException("An exception occurred while serializing input data", e); } catch (TException e) { @@ -292,7 +332,7 @@ public TForecastResp forecast( TForecastReq forecastReq = new TForecastReq(modelId, tsBlockSerde.serialize(inputTsBlock), outputLength); forecastReq.setOptions(options); - return client.forecast(forecastReq); + return executeRemoteCallWithRetry(c -> c.forecast(forecastReq)); } catch (IOException e) { TSStatus tsStatus = new TSStatus(INTERNAL_SERVER_ERROR.getStatusCode()); tsStatus.setMessage(String.format("Failed to serialize input tsblock %s", e.getMessage())); @@ -308,15 +348,7 @@ public TForecastResp forecast( } public TSStatus createTrainingTask(TTrainingReq req) throws TException { - try { - return client.createTrainingTask(req); - } catch (TException e) { - logger.warn( - "Failed to connect to AINode when executing {}: {}", - Thread.currentThread().getStackTrace()[1].getMethodName(), - e.getMessage()); - throw new TException(MSG_CONNECTION_FAIL); - } + return executeRemoteCallWithRetry(c -> c.createTrainingTask(req)); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AINodeClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AINodeClientManager.java new file mode 100644 index 0000000000000..faef1c1ae7b60 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AINodeClientManager.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.protocol.client.ainode; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.db.protocol.client.AINodeClientFactory; + +public class AINodeClientManager { + + public static final int DEFAULT_AINODE_ID = 0; + + private static final AINodeClientManager INSTANCE = new AINodeClientManager(); + + private final IClientManager clientManager; + + private volatile TEndPoint defaultAINodeEndPoint; + + private AINodeClientManager() { + this.clientManager = + new IClientManager.Factory() + .createClientManager(new AINodeClientFactory.AINodeClientPoolFactory()); + } + + public static AINodeClientManager getInstance() { + return INSTANCE; + } + + public void updateDefaultAINodeLocation(TEndPoint endPoint) { + this.defaultAINodeEndPoint = endPoint; + } + + public AINodeClient borrowClient(TEndPoint endPoint) throws Exception { + return clientManager.borrowClient(endPoint); + } + + public AINodeClient borrowClient(int aiNodeId) throws Exception { + if (aiNodeId != DEFAULT_AINODE_ID) { + throw new IllegalArgumentException("Unsupported AINodeId: " + aiNodeId); + } + if (defaultAINodeEndPoint == null) { + defaultAINodeEndPoint = AINodeClient.getCurrentEndpoint(); + } + return clientManager.borrowClient(defaultAINodeEndPoint); + } + + public void clear(TEndPoint endPoint) { + clientManager.clear(endPoint); + } + + public void clearAll() { + clientManager.close(); + } + + public IClientManager getRawClientManager() { + return clientManager; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AsyncAINodeServiceClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AsyncAINodeServiceClient.java similarity index 98% rename from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AsyncAINodeServiceClient.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AsyncAINodeServiceClient.java index ba0b1d11e70ce..26130287697c4 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AsyncAINodeServiceClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ainode/AsyncAINodeServiceClient.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.commons.client.ainode; +package org.apache.iotdb.db.protocol.client.ainode; import org.apache.iotdb.ainode.rpc.thrift.IAINodeRPCService; import org.apache.iotdb.common.rpc.thrift.TEndPoint; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/ai/InferenceOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/ai/InferenceOperator.java index ccdef60aaf0b9..7126af78b8b51 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/ai/InferenceOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/ai/InferenceOperator.java @@ -21,9 +21,9 @@ import org.apache.iotdb.ainode.rpc.thrift.TInferenceResp; import org.apache.iotdb.ainode.rpc.thrift.TWindowParams; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AINodeClientManager; import org.apache.iotdb.db.exception.runtime.ModelInferenceProcessException; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.db.queryengine.execution.operator.Operator; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java index db36048fcd054..51c8c72e89467 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java @@ -19,11 +19,14 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsResp; import org.apache.iotdb.common.rpc.thrift.Model; import org.apache.iotdb.common.rpc.thrift.TAINodeLocation; import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.audit.UserEntity; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; @@ -51,8 +54,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TNodeVersionInfo; import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeInfo; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq; @@ -66,6 +67,8 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.db.protocol.session.IClientSession; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.IQueryExecution; @@ -789,18 +792,18 @@ private static class ModelsSupplier extends TsBlockSupplier { private ModelsSupplier(final List dataTypes) throws Exception { super(dataTypes); - try (final ConfigNodeClient client = - ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { - iterator = new ModelIterator(client.showModel(new TShowModelReq())); + final TEndPoint ep = AINodeClient.getCurrentEndpoint(); + try (final AINodeClient ai = AINodeClientManager.getInstance().borrowClient(ep)) { + iterator = new ModelIterator(ai.showModels(new TShowModelsReq())); } } private static class ModelIterator implements Iterator { private int index = 0; - private final TShowModelResp resp; + private final TShowModelsResp resp; - private ModelIterator(TShowModelResp resp) { + private ModelIterator(TShowModelsResp resp) { this.resp = resp; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index 517e201f3d8cf..beba9896f0bb3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -19,6 +19,14 @@ package org.apache.iotdb.db.queryengine.plan.execution.config.executor; +import org.apache.iotdb.ainode.rpc.thrift.TLoadModelReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowAIDevicesResp; +import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsResp; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsReq; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsResp; +import org.apache.iotdb.ainode.rpc.thrift.TTrainingReq; +import org.apache.iotdb.ainode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.common.rpc.thrift.FunctionType; import org.apache.iotdb.common.rpc.thrift.Model; import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; @@ -93,12 +101,9 @@ import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTableViewReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTopicReq; -import org.apache.iotdb.confignode.rpc.thrift.TCreateTrainingReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq; import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveReq; import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveResp; -import org.apache.iotdb.confignode.rpc.thrift.TDataSchemaForTable; -import org.apache.iotdb.confignode.rpc.thrift.TDataSchemaForTree; import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; import org.apache.iotdb.confignode.rpc.thrift.TDeactivateSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TDeleteDatabasesReq; @@ -129,24 +134,18 @@ import org.apache.iotdb.confignode.rpc.thrift.TGetTriggerTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUDFTableResp; import org.apache.iotdb.confignode.rpc.thrift.TGetUdfTableReq; -import org.apache.iotdb.confignode.rpc.thrift.TLoadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TMigrateRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TPipeConfigTransferReq; import org.apache.iotdb.confignode.rpc.thrift.TPipeConfigTransferResp; import org.apache.iotdb.confignode.rpc.thrift.TReconstructRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo; import org.apache.iotdb.confignode.rpc.thrift.TRemoveRegionReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowAIDevicesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowAINodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; import org.apache.iotdb.confignode.rpc.thrift.TShowConfigNodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowDatabaseResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelResp; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeInfo; import org.apache.iotdb.confignode.rpc.thrift.TShowPipePluginReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; @@ -164,7 +163,6 @@ import org.apache.iotdb.confignode.rpc.thrift.TStartPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TStopPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TThrottleQuotaResp; -import org.apache.iotdb.confignode.rpc.thrift.TUnloadModelReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.BatchProcessException; @@ -177,6 +175,8 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.protocol.client.DataNodeClientPoolFactory; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.db.protocol.session.IClientSession; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; @@ -3627,20 +3627,19 @@ public SettableFuture dropModel(final String modelId) { @Override public SettableFuture showModels(final String modelId) { final SettableFuture future = SettableFuture.create(); - try (final ConfigNodeClient client = - CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { - final TShowModelReq req = new TShowModelReq(); + try (final AINodeClient ai = + AINodeClientManager.getInstance().borrowClient(AINodeClientManager.DEFAULT_AINODE_ID)) { + final TShowModelsReq req = new TShowModelsReq(); if (modelId != null) { req.setModelId(modelId); } - final TShowModelResp showModelResp = client.showModel(req); - if (showModelResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - future.setException(new IoTDBException(showModelResp.getStatus())); + final TShowModelsResp resp = ai.showModels(req); + if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + future.setException(new IoTDBException(resp.getStatus())); return future; } - // convert model info list and buildTsBlock - ShowModelsTask.buildTsBlock(showModelResp, future); - } catch (final ClientManagerException | TException e) { + ShowModelsTask.buildTsBlock(resp, future); + } catch (final Exception e) { future.setException(e); } return future; @@ -3649,21 +3648,17 @@ public SettableFuture showModels(final String modelId) { @Override public SettableFuture showLoadedModels(List deviceIdList) { final SettableFuture future = SettableFuture.create(); - try (final ConfigNodeClient client = - CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { - final TShowLoadedModelReq req = new TShowLoadedModelReq(); - if (deviceIdList != null) { - req.setDeviceIdList(deviceIdList); - } else { - req.setDeviceIdList(new ArrayList<>()); - } - final TShowLoadedModelResp resp = client.showLoadedModel(req); + try (final AINodeClient ai = + AINodeClientManager.getInstance().borrowClient(AINodeClientManager.DEFAULT_AINODE_ID)) { + final TShowLoadedModelsReq req = new TShowLoadedModelsReq(); + req.setDeviceIdList(deviceIdList != null ? deviceIdList : new ArrayList<>()); + final TShowLoadedModelsResp resp = ai.showLoadedModels(req); if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { future.setException(new IoTDBException(resp.getStatus())); return future; } ShowLoadedModelsTask.buildTsBlock(resp, future); - } catch (final ClientManagerException | TException e) { + } catch (final Exception e) { future.setException(e); } return future; @@ -3672,15 +3667,15 @@ public SettableFuture showLoadedModels(List deviceIdLi @Override public SettableFuture showAIDevices() { final SettableFuture future = SettableFuture.create(); - try (final ConfigNodeClient client = - CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { - final TShowAIDevicesResp resp = client.showAIDevices(); + try (final AINodeClient ai = + AINodeClientManager.getInstance().borrowClient(AINodeClientManager.DEFAULT_AINODE_ID)) { + final TShowAIDevicesResp resp = ai.showAIDevices(); if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { future.setException(new IoTDBException(resp.getStatus())); return future; } ShowAIDevicesTask.buildTsBlock(resp, future); - } catch (final ClientManagerException | TException e) { + } catch (final Exception e) { future.setException(e); } return future; @@ -3690,16 +3685,16 @@ public SettableFuture showAIDevices() { public SettableFuture loadModel( String existingModelId, List deviceIdList) { final SettableFuture future = SettableFuture.create(); - try (final ConfigNodeClient client = - CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + try (final AINodeClient ai = + AINodeClientManager.getInstance().borrowClient(AINodeClientManager.DEFAULT_AINODE_ID)) { final TLoadModelReq req = new TLoadModelReq(existingModelId, deviceIdList); - final TSStatus result = client.loadModel(req); + final TSStatus result = ai.loadModel(req); if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != result.getCode()) { future.setException(new IoTDBException(result)); } else { future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } - } catch (final ClientManagerException | TException e) { + } catch (final Exception e) { future.setException(e); } return future; @@ -3709,16 +3704,16 @@ public SettableFuture loadModel( public SettableFuture unloadModel( String existingModelId, List deviceIdList) { final SettableFuture future = SettableFuture.create(); - try (final ConfigNodeClient client = - CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + try (final AINodeClient ai = + AINodeClientManager.getInstance().borrowClient(AINodeClientManager.DEFAULT_AINODE_ID)) { final TUnloadModelReq req = new TUnloadModelReq(existingModelId, deviceIdList); - final TSStatus result = client.unloadModel(req); + final TSStatus result = ai.unloadModel(req); if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != result.getCode()) { future.setException(new IoTDBException(result)); } else { future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } - } catch (final ClientManagerException | TException e) { + } catch (final Exception e) { future.setException(e); } return future; @@ -3734,28 +3729,24 @@ public SettableFuture createTraining( @Nullable String targetSql, @Nullable List pathList) { final SettableFuture future = SettableFuture.create(); - try (final ConfigNodeClient client = - CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { - final TCreateTrainingReq req = new TCreateTrainingReq(modelId, isTableModel, existingModelId); - - if (isTableModel) { - TDataSchemaForTable dataSchemaForTable = new TDataSchemaForTable(); - dataSchemaForTable.setTargetSql(targetSql); - req.setDataSchemaForTable(dataSchemaForTable); - } else { - TDataSchemaForTree dataSchemaForTree = new TDataSchemaForTree(); - dataSchemaForTree.setPath(pathList); - req.setDataSchemaForTree(dataSchemaForTree); - } + try (final AINodeClient ai = + AINodeClientManager.getInstance().borrowClient(AINodeClientManager.DEFAULT_AINODE_ID)) { + final TTrainingReq req = new TTrainingReq(); + req.setModelId(modelId); req.setParameters(parameters); - req.setTimeRanges(timeRanges); - final TSStatus executionStatus = client.createTraining(req); - if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != executionStatus.getCode()) { - future.setException(new IoTDBException(executionStatus)); + if (existingModelId != null) { + req.setExistingModelId(existingModelId); + } + if (existingModelId != null) { + req.setExistingModelId(existingModelId); + } + final TSStatus status = ai.createTrainingTask(req); + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != status.getCode()) { + future.setException(new IoTDBException(status)); } else { future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } - } catch (final ClientManagerException | TException e) { + } catch (final Exception e) { future.setException(e); } return future; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowAIDevicesTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowAIDevicesTask.java index d2cdd96759512..690f6f9485f23 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowAIDevicesTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowAIDevicesTask.java @@ -19,9 +19,9 @@ package org.apache.iotdb.db.queryengine.plan.execution.config.metadata.ai; +import org.apache.iotdb.ainode.rpc.thrift.TShowAIDevicesResp; import org.apache.iotdb.commons.schema.column.ColumnHeader; import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; -import org.apache.iotdb.confignode.rpc.thrift.TShowAIDevicesResp; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.common.header.DatasetHeaderFactory; import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowLoadedModelsTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowLoadedModelsTask.java index 5beb8a6a5aa64..c8c6f8938f759 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowLoadedModelsTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowLoadedModelsTask.java @@ -19,9 +19,9 @@ package org.apache.iotdb.db.queryengine.plan.execution.config.metadata.ai; +import org.apache.iotdb.ainode.rpc.thrift.TShowLoadedModelsResp; import org.apache.iotdb.commons.schema.column.ColumnHeader; import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; -import org.apache.iotdb.confignode.rpc.thrift.TShowLoadedModelResp; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.common.header.DatasetHeaderFactory; import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult; @@ -54,7 +54,7 @@ public ListenableFuture execute(IConfigTaskExecutor configTask } public static void buildTsBlock( - TShowLoadedModelResp resp, SettableFuture future) { + TShowLoadedModelsResp resp, SettableFuture future) { List outputDataTypes = ColumnHeaderConstant.showLoadedModelsColumnHeaders.stream() .map(ColumnHeader::getColumnType) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowModelsTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowModelsTask.java index 73c5b1b6f7dc9..c0d7f4ef20309 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowModelsTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ai/ShowModelsTask.java @@ -19,9 +19,9 @@ package org.apache.iotdb.db.queryengine.plan.execution.config.metadata.ai; +import org.apache.iotdb.ainode.rpc.thrift.TShowModelsResp; import org.apache.iotdb.commons.schema.column.ColumnHeader; import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; -import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.common.header.DatasetHeaderFactory; import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult; @@ -59,7 +59,7 @@ public ListenableFuture execute(IConfigTaskExecutor configTask return configTaskExecutor.showModels(modelName); } - public static void buildTsBlock(TShowModelResp resp, SettableFuture future) { + public static void buildTsBlock(TShowModelsResp resp, SettableFuture future) { List modelIdList = resp.getModelIdList(); Map modelTypeMap = resp.getModelTypeMap(); Map categoryMap = resp.getCategoryMap(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ForecastTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ForecastTableFunction.java index 5521062a24e1b..887d7c26d305e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ForecastTableFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ForecastTableFunction.java @@ -21,11 +21,10 @@ import org.apache.iotdb.ainode.rpc.thrift.TForecastResp; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.commons.client.IClientManager; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AINodeClientManager; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; import org.apache.iotdb.db.exception.sql.SemanticException; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.db.queryengine.plan.analyze.IModelFetcher; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.model.ModelInferenceDescriptor; import org.apache.iotdb.rpc.TSStatusCode; @@ -457,8 +456,7 @@ private static Map parseOptions(String options) { private static class ForecastDataProcessor implements TableFunctionDataProcessor { private static final TsBlockSerde SERDE = new TsBlockSerde(); - private static final IClientManager CLIENT_MANAGER = - AINodeClientManager.getInstance(); + private static final AINodeClientManager CLIENT_MANAGER = AINodeClientManager.getInstance(); private final TEndPoint targetAINode; private final String modelId; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/udf/UDTFForecast.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/udf/UDTFForecast.java index 22c2bce7b5ee2..e77e0641ae975 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/udf/UDTFForecast.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/udf/UDTFForecast.java @@ -21,10 +21,9 @@ import org.apache.iotdb.ainode.rpc.thrift.TForecastResp; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.commons.client.IClientManager; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AINodeClientManager; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClient; +import org.apache.iotdb.db.protocol.client.ainode.AINodeClientManager; import org.apache.iotdb.db.queryengine.plan.analyze.IModelFetcher; import org.apache.iotdb.db.queryengine.plan.analyze.ModelFetcher; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.model.ModelInferenceDescriptor; @@ -55,8 +54,7 @@ public class UDTFForecast implements UDTF { private static final TsBlockSerde serde = new TsBlockSerde(); - private static final IClientManager CLIENT_MANAGER = - AINodeClientManager.getInstance(); + private static final AINodeClientManager CLIENT_MANAGER = AINodeClientManager.getInstance(); private TEndPoint targetAINode = new TEndPoint("127.0.0.1", 10810); private String model_id; private int maxInputLength; diff --git a/iotdb-core/node-commons/pom.xml b/iotdb-core/node-commons/pom.xml index 9e74e999fc475..4a42198f5b57f 100644 --- a/iotdb-core/node-commons/pom.xml +++ b/iotdb-core/node-commons/pom.xml @@ -179,12 +179,6 @@ com.timecho.ratis ratis-common - - org.apache.iotdb - iotdb-thrift-ainode - 2.0.6-SNAPSHOT - compile - diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java index 32c6345dc27eb..106d67b6279d9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java @@ -20,8 +20,6 @@ package org.apache.iotdb.commons.client; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.commons.client.ainode.AINodeClient; -import org.apache.iotdb.commons.client.ainode.AsyncAINodeServiceClient; import org.apache.iotdb.commons.client.async.AsyncConfigNodeInternalServiceClient; import org.apache.iotdb.commons.client.async.AsyncDataNodeExternalServiceClient; import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient; @@ -325,56 +323,6 @@ public GenericKeyedObjectPool cre } } - public static class AsyncAINodeHeartbeatServiceClientPoolFactory - implements IClientPoolFactory { - @Override - public GenericKeyedObjectPool createClientPool( - ClientManager manager) { - GenericKeyedObjectPool clientPool = - new GenericKeyedObjectPool<>( - new AsyncAINodeServiceClient.Factory( - manager, - new ThriftClientProperty.Builder() - .setConnectionTimeoutMs(conf.getCnConnectionTimeoutInMS()) - .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) - .setPrintLogWhenEncounterException(false) - .build(), - ThreadName.ASYNC_DATANODE_HEARTBEAT_CLIENT_POOL.getName()), - new ClientPoolProperty.Builder() - .setMaxClientNumForEachNode(conf.getMaxClientNumForEachNode()) - .build() - .getConfig()); - ClientManagerMetrics.getInstance() - .registerClientManager(this.getClass().getSimpleName(), clientPool); - return clientPool; - } - } - - public static class AINodeClientPoolFactory - implements IClientPoolFactory { - - @Override - public GenericKeyedObjectPool createClientPool( - ClientManager manager) { - GenericKeyedObjectPool clientPool = - new GenericKeyedObjectPool<>( - new AINodeClient.Factory( - manager, - new ThriftClientProperty.Builder() - .setConnectionTimeoutMs(conf.getDnConnectionTimeoutInMS()) - .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .build()), - new ClientPoolProperty.Builder() - .setMaxClientNumForEachNode(conf.getMaxClientNumForEachNode()) - .build() - .getConfig()); - ClientManagerMetrics.getInstance() - .registerClientManager(this.getClass().getSimpleName(), clientPool); - return clientPool; - } - } - public static class SyncPipeConsensusServiceClientPoolFactory implements IClientPoolFactory { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AINodeClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AINodeClientManager.java deleted file mode 100644 index 3a06e478e7b5a..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ainode/AINodeClientManager.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.client.ainode; - -import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.commons.client.ClientPoolFactory; -import org.apache.iotdb.commons.client.IClientManager; - -public class AINodeClientManager { - private AINodeClientManager() { - // Empty constructor - } - - private static final class AINodeClientManagerHolder { - private static final IClientManager INSTANCE = - new IClientManager.Factory() - .createClientManager(new ClientPoolFactory.AINodeClientPoolFactory()); - } - - public static IClientManager getInstance() { - return AINodeClientManagerHolder.INSTANCE; - } -} diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 3ee3ca89bdcc1..d8f6318063ebb 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -659,6 +659,13 @@ struct TAINodeInfo { 4: required i32 internalPort } +// ----------- New messages ----------- + +struct TGetAINodeLocationResp { + 1: required common.TSStatus status + 2: optional common.TAINodeLocation aiNodeLocation +} + struct TShowDataNodes4InformationSchemaResp { 1: required common.TSStatus status 2: optional list dataNodesInfoList @@ -1098,42 +1105,6 @@ struct TDropModelReq { 1: required string modelId } -struct TShowModelReq { - 1: optional string modelId -} - -struct TShowModelResp { - 1: required common.TSStatus status - 2: optional list modelIdList - 3: optional map modelTypeMap - 4: optional map categoryMap - 5: optional map stateMap -} - -struct TShowLoadedModelReq { - 1: required list deviceIdList -} - -struct TShowLoadedModelResp { - 1: required common.TSStatus status - 2: required map> deviceLoadedModelsMap -} - -struct TShowAIDevicesResp { - 1: required common.TSStatus status - 2: required list deviceIdList -} - -struct TLoadModelReq { - 1: required string existingModelId - 2: required list deviceIdList -} - -struct TUnloadModelReq { - 1: required string modelId - 2: required list deviceIdList -} - struct TGetModelInfoReq { 1: required string modelId } @@ -1371,6 +1342,11 @@ service IConfigNodeRPCService { TAINodeConfigurationResp getAINodeConfiguration(i32 aiNodeId) + /** + * Return a reachable AINode location. + */ + TGetAINodeLocationResp getAINodeLocation() + /** * Get system configurations. i.e. configurations that is not associated with the DataNodeId */ @@ -2049,43 +2025,12 @@ service IConfigNodeRPCService { common.TSStatus dropModel(TDropModelReq req) /** - * Return the model table - */ - TShowModelResp showModel(TShowModelReq req) - - /** - * Return the loaded model table - */ - TShowLoadedModelResp showLoadedModel(TShowLoadedModelReq req) - - /** - * Return the available ai devices - */ - TShowAIDevicesResp showAIDevices() - - /** - * Load an existing model to specific devices - * - * @return SUCCESS_STATUS if the model loading task was submitted successfully - */ - common.TSStatus loadModel(TLoadModelReq req) - - /** - * Unload an existing model to specific devices - * - * @return SUCCESS_STATUS if the model unloading task was submitted successfully - */ - common.TSStatus unloadModel(TUnloadModelReq req) - - /** - * Return the model info by model_id - */ + * Return the model info by model_id + */ TGetModelInfoResp getModelInfo(TGetModelInfoReq req) common.TSStatus updateModelInfo(TUpdateModelInfoReq req) - common.TSStatus createTraining(TCreateTrainingReq req) - // ====================================================== // Quota // ====================================================== From 08c0eec5a91eff3a3f3dbff0cbbe39a1f08c7c75 Mon Sep 17 00:00:00 2001 From: Yongzao Date: Tue, 18 Nov 2025 00:15:32 +0800 Subject: [PATCH 065/180] [AINode] Fix bug that AINode cannot compile in Windows (#16767) (cherry picked from commit cd443ba9996abe1366b389e3d10f9ecb72cd8d12) --- iotdb-core/ainode/build_binary.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/iotdb-core/ainode/build_binary.py b/iotdb-core/ainode/build_binary.py index 1b5f99da21493..ff99fda09338d 100644 --- a/iotdb-core/ainode/build_binary.py +++ b/iotdb-core/ainode/build_binary.py @@ -324,7 +324,7 @@ def install_dependencies(venv_python, venv_dir, script_dir): shutil.rmtree(poetry_venv_path, ignore_errors=True) poetry_venv_path.parent.mkdir(parents=True, exist_ok=True) poetry_venv_path.symlink_to(venv_dir) - print(f"✓ Symlink created successfully") + print(f"Symlink created successfully") except Exception as e: print(f"WARNING: Failed to create symlink: {e}") print("Will try to use poetry install directly with VIRTUAL_ENV set") @@ -386,9 +386,7 @@ def install_dependencies(venv_python, venv_dir, script_dir): print("The symlink approach may not have worked. Please check the symlink.") sys.exit(1) else: - print( - f"✓ Poetry is correctly using virtual environment: {poetry_venv_path}" - ) + print(f"Poetry is correctly using virtual environment: {poetry_venv_path}") else: print("Warning: Could not verify poetry virtual environment path") print( @@ -465,12 +463,12 @@ def verify_poetry_env(): ) if test_result.returncode == 0: version = test_result.stdout.strip() - print(f" ✓ {package} {version} installed") + print(f"{package} {version} installed") else: error_msg = ( test_result.stderr.strip() if test_result.stderr else "Unknown error" ) - print(f" ✗ {package} NOT found in virtual environment: {error_msg}") + print(f"{package} NOT found in virtual environment: {error_msg}") missing_packages.append(package) if missing_packages: From 2078b4f706dc9672537b6993f83b46df5de3329d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 18 Nov 2025 16:02:04 +0800 Subject: [PATCH 066/180] fix (#16773) (cherry picked from commit bd97844bb7f54c9ecd8f4831a8fd3efc8400d996) --- .../org/apache/iotdb/db/conf/DataNodeMemoryConfig.java | 10 +++++++--- .../plan/analyze/cache/partition/PartitionCache.java | 2 +- .../metadata/fetcher/cache/TableDeviceSchemaCache.java | 2 +- .../fetcher/cache/TableDeviceSchemaCacheMetrics.java | 9 +++++---- .../schemaengine/rescon/MemSchemaEngineStatistics.java | 3 ++- 5 files changed, 16 insertions(+), 10 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/DataNodeMemoryConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/DataNodeMemoryConfig.java index 551d8a9031d63..373cec94a6632 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/DataNodeMemoryConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/DataNodeMemoryConfig.java @@ -32,6 +32,10 @@ public class DataNodeMemoryConfig { private static final Logger LOGGER = LoggerFactory.getLogger(DataNodeMemoryConfig.class); + public static final String SCHEMA_CACHE = "SchemaCache"; + public static final String SCHEMA_REGION = "SchemaRegion"; + public static final String PARTITION_CACHE = "PartitionCache"; + /** Reject proportion for system */ private double rejectProportion = 0.8; @@ -265,13 +269,13 @@ private void initSchemaMemoryAllocate( schemaRegionMemoryManager = schemaEngineMemoryManager.getOrCreateMemoryManager( - "SchemaRegion", schemaMemoryTotal * schemaMemoryProportion[0] / proportionSum); + SCHEMA_REGION, schemaMemoryTotal * schemaMemoryProportion[0] / proportionSum); schemaCacheMemoryManager = schemaEngineMemoryManager.getOrCreateMemoryManager( - "SchemaCache", schemaMemoryTotal * schemaMemoryProportion[1] / proportionSum); + SCHEMA_CACHE, schemaMemoryTotal * schemaMemoryProportion[1] / proportionSum); partitionCacheMemoryManager = schemaEngineMemoryManager.getOrCreateMemoryManager( - "PartitionCache", schemaMemoryTotal * schemaMemoryProportion[2] / proportionSum); + PARTITION_CACHE, schemaMemoryTotal * schemaMemoryProportion[2] / proportionSum); LOGGER.info( "allocateMemoryForSchemaRegion = {}", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java index df379b4789664..3c76bd08c1d81 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java @@ -128,7 +128,7 @@ public PartitionCache() { this.memoryBlock = memoryConfig .getPartitionCacheMemoryManager() - .exactAllocate("PartitionCache", MemoryBlockType.STATIC); + .exactAllocate(DataNodeMemoryConfig.PARTITION_CACHE, MemoryBlockType.STATIC); this.memoryBlock.allocate(this.memoryBlock.getTotalMemorySizeInBytes()); // TODO @spricoder: PartitionCache need to be controlled according to memory this.schemaPartitionCache = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/cache/TableDeviceSchemaCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/cache/TableDeviceSchemaCache.java index 90fc4fb0e94e4..acb9c5e61801e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/cache/TableDeviceSchemaCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/cache/TableDeviceSchemaCache.java @@ -112,7 +112,7 @@ private TableDeviceSchemaCache() { memoryBlock = memoryConfig .getSchemaCacheMemoryManager() - .exactAllocate("TableDeviceSchemaCache", MemoryBlockType.STATIC); + .exactAllocate(DataNodeMemoryConfig.SCHEMA_CACHE, MemoryBlockType.STATIC); dualKeyCache = new DualKeyCacheBuilder() .cacheEvictionPolicy( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/cache/TableDeviceSchemaCacheMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/cache/TableDeviceSchemaCacheMetrics.java index e65c418c0c1d2..5063150759f12 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/cache/TableDeviceSchemaCacheMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/cache/TableDeviceSchemaCacheMetrics.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.service.metric.enums.Metric; import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.conf.DataNodeMemoryConfig; import org.apache.iotdb.metrics.AbstractMetricService; import org.apache.iotdb.metrics.metricsets.IMetricSet; import org.apache.iotdb.metrics.utils.MetricLevel; @@ -47,7 +48,7 @@ public void bindTo(final AbstractMetricService metricService) { tableDeviceSchemaCache, TableDeviceSchemaCache::getHitCount, Tag.NAME.toString(), - "SchemaCache", + DataNodeMemoryConfig.SCHEMA_CACHE, Tag.TYPE.toString(), "hit"); metricService.createAutoGauge( @@ -56,7 +57,7 @@ public void bindTo(final AbstractMetricService metricService) { tableDeviceSchemaCache, TableDeviceSchemaCache::getRequestCount, Tag.NAME.toString(), - "SchemaCache", + DataNodeMemoryConfig.SCHEMA_CACHE, Tag.TYPE.toString(), "all"); metricService.createAutoGauge( @@ -86,14 +87,14 @@ public void unbindFrom(final AbstractMetricService metricService) { MetricType.AUTO_GAUGE, Metric.CACHE.toString(), Tag.NAME.toString(), - "SchemaCache", + DataNodeMemoryConfig.SCHEMA_CACHE, Tag.TYPE.toString(), "hit"); metricService.remove( MetricType.AUTO_GAUGE, Metric.CACHE.toString(), Tag.NAME.toString(), - "SchemaCache", + DataNodeMemoryConfig.SCHEMA_CACHE, Tag.TYPE.toString(), "all"); metricService.remove( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaEngineStatistics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaEngineStatistics.java index c50dd62ecea5c..61de4a1a8131d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaEngineStatistics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaEngineStatistics.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.memory.IMemoryBlock; import org.apache.iotdb.commons.memory.MemoryBlockType; +import org.apache.iotdb.db.conf.DataNodeMemoryConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.schemaengine.SchemaEngine; import org.apache.iotdb.db.schemaengine.template.ClusterTemplateManager; @@ -41,7 +42,7 @@ public class MemSchemaEngineStatistics implements ISchemaEngineStatistics { IoTDBDescriptor.getInstance() .getMemoryConfig() .getSchemaRegionMemoryManager() - .exactAllocate("SchemaRegion", MemoryBlockType.DYNAMIC); + .exactAllocate(DataNodeMemoryConfig.SCHEMA_REGION, MemoryBlockType.DYNAMIC); private final ClusterTemplateManager clusterTemplateManager = ClusterTemplateManager.getInstance(); From c110ab10c03fd88f89f57d8daff081fd09f8c7b7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 18 Nov 2025 22:48:34 +0800 Subject: [PATCH 067/180] Load: Fixed the bug that mods is not deleted in load tsFile when there are exceptions & Fixed the potential NPE in air gap agent close() method (#16775) (cherry picked from commit 2c381feda256c8441492203beaf88a0ff2f2b70c) --- .../protocol/airgap/IoTDBAirGapReceiverAgent.java | 5 ++++- ...leStatementDataTypeConvertExecutionVisitor.java | 4 ++-- ...eeStatementDataTypeConvertExecutionVisitor.java | 4 ++-- .../db/storageengine/load/LoadTsFileManager.java | 14 ++++++++++++++ 4 files changed, 22 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiverAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiverAgent.java index 644bda1835c81..1aa828a4f8c01 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiverAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiverAgent.java @@ -32,6 +32,7 @@ import java.io.IOException; import java.net.ServerSocket; import java.net.Socket; +import java.util.Objects; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -84,7 +85,9 @@ public void start() throws StartupException { @Override public void stop() { try { - serverSocket.close(); + if (Objects.nonNull(serverSocket)) { + serverSocket.close(); + } } catch (final IOException e) { LOGGER.warn("Failed to close IoTDBAirGapReceiverAgent's server socket", e); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeTableStatementDataTypeConvertExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeTableStatementDataTypeConvertExecutionVisitor.java index 4bb1f8d2e1773..d79b05f2fafe6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeTableStatementDataTypeConvertExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeTableStatementDataTypeConvertExecutionVisitor.java @@ -37,10 +37,10 @@ import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowsStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.LoadTsFileStatement; +import org.apache.iotdb.db.storageengine.load.LoadTsFileManager; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.tsfile.external.commons.io.FileUtils; import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -193,7 +193,7 @@ public Optional visitLoadFile( } if (loadTsFileStatement.isDeleteAfterLoad()) { - loadTsFileStatement.getTsFiles().forEach(FileUtils::deleteQuietly); + loadTsFileStatement.getTsFiles().forEach(LoadTsFileManager::cleanTsFile); } LOGGER.warn( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeTreeStatementDataTypeConvertExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeTreeStatementDataTypeConvertExecutionVisitor.java index e9707299e09a3..282b378a2d268 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeTreeStatementDataTypeConvertExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeTreeStatementDataTypeConvertExecutionVisitor.java @@ -36,9 +36,9 @@ import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowsStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.LoadTsFileStatement; +import org.apache.iotdb.db.storageengine.load.LoadTsFileManager; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.tsfile.external.commons.io.FileUtils; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.record.Tablet; import org.slf4j.Logger; @@ -153,7 +153,7 @@ file, new IoTDBTreePattern(null), Long.MIN_VALUE, Long.MAX_VALUE, null, null, tr } if (loadTsFileStatement.isDeleteAfterLoad()) { - loadTsFileStatement.getTsFiles().forEach(FileUtils::deleteQuietly); + loadTsFileStatement.getTsFiles().forEach(LoadTsFileManager::cleanTsFile); } LOGGER.warn( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java index f6203be7be324..30bb64cbd316e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java @@ -46,6 +46,7 @@ import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.flush.MemTableFlushTask; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; +import org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; import org.apache.iotdb.db.storageengine.load.active.ActiveLoadAgent; @@ -400,6 +401,19 @@ public static void updateWritePointCountMetrics( } } + public static void cleanTsFile(final File tsFile) { + try { + Files.deleteIfExists(tsFile.toPath()); + Files.deleteIfExists( + new File(tsFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).toPath()); + Files.deleteIfExists(ModificationFile.getExclusiveMods(tsFile).toPath()); + Files.deleteIfExists( + new File(tsFile.getAbsolutePath() + ModificationFileV1.FILE_SUFFIX).toPath()); + } catch (final IOException e) { + LOGGER.warn("Delete After Loading {} error.", tsFile, e); + } + } + private static class TsFileWriterManager { private final File taskDir; From 81b6d5922516e8c834d04b8efa79cd4fad2d585d Mon Sep 17 00:00:00 2001 From: Leo <165569073+yunbow30944@users.noreply.github.com> Date: Wed, 19 Nov 2025 09:23:19 +0800 Subject: [PATCH 068/180] [AINode] Fix bug of sundial and forecast udf (#16768) (cherry picked from commit 2b47be756ad8703ce3673973260983f10c4f94e3) --- .../ainode/core/model/sundial/modeling_sundial.py | 13 +++++++++---- .../ainode/core/model/timerxl/modeling_timer.py | 6 +++++- .../iotdb/db/queryengine/plan/udf/UDTFForecast.java | 1 - 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/iotdb-core/ainode/iotdb/ainode/core/model/sundial/modeling_sundial.py b/iotdb-core/ainode/iotdb/ainode/core/model/sundial/modeling_sundial.py index 544193e4d9c65..3ebf516f705e0 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/model/sundial/modeling_sundial.py +++ b/iotdb-core/ainode/iotdb/ainode/core/model/sundial/modeling_sundial.py @@ -616,7 +616,11 @@ def prepare_inputs_for_generation( if attention_mask is not None and attention_mask.shape[1] > ( input_ids.shape[1] // self.config.input_token_len ): - input_ids = input_ids[:, -(attention_mask.shape[1] - past_length) :] + input_ids = input_ids[ + :, + -(attention_mask.shape[1] - past_length) + * self.config.input_token_len :, + ] # 2 - If the past_length is smaller than input_ids', then input_ids holds all input tokens. We can discard # input_ids based on the past_length. elif past_length < (input_ids.shape[1] // self.config.input_token_len): @@ -629,9 +633,10 @@ def prepare_inputs_for_generation( position_ids = attention_mask.long().cumsum(-1) - 1 position_ids.masked_fill_(attention_mask == 0, 1) if past_key_values: - position_ids = position_ids[ - :, -(input_ids.shape[1] // self.config.input_token_len) : - ] + token_num = ( + input_ids.shape[1] + self.config.input_token_len - 1 + ) // self.config.input_token_len + position_ids = position_ids[:, -token_num:] # if `inputs_embeds` are passed, we only want to use them in the 1st generation step if inputs_embeds is not None and past_key_values is None: diff --git a/iotdb-core/ainode/iotdb/ainode/core/model/timerxl/modeling_timer.py b/iotdb-core/ainode/iotdb/ainode/core/model/timerxl/modeling_timer.py index 37bf56dfc59a7..0a33c682742aa 100644 --- a/iotdb-core/ainode/iotdb/ainode/core/model/timerxl/modeling_timer.py +++ b/iotdb-core/ainode/iotdb/ainode/core/model/timerxl/modeling_timer.py @@ -606,7 +606,11 @@ def prepare_inputs_for_generation( if attention_mask is not None and attention_mask.shape[1] > ( input_ids.shape[1] // self.config.input_token_len ): - input_ids = input_ids[:, -(attention_mask.shape[1] - past_length) :] + input_ids = input_ids[ + :, + -(attention_mask.shape[1] - past_length) + * self.config.input_token_len :, + ] # 2 - If the past_length is smaller than input_ids', then input_ids holds all input tokens. We can discard # input_ids based on the past_length. elif past_length < (input_ids.shape[1] // self.config.input_token_len): diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/udf/UDTFForecast.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/udf/UDTFForecast.java index e77e0641ae975..260410954d4c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/udf/UDTFForecast.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/udf/UDTFForecast.java @@ -114,7 +114,6 @@ public void beforeStart(UDFParameters parameters, UDTFConfigurations configurati } ModelInferenceDescriptor descriptor = modelFetcher.fetchModel(this.model_id); this.targetAINode = descriptor.getTargetAINode(); - this.maxInputLength = descriptor.getModelInformation().getInputShape()[0]; this.outputInterval = parameters.getLongOrDefault(OUTPUT_INTERVAL, DEFAULT_OUTPUT_INTERVAL); this.outputLength = From 842c4a5ffc8e987633594cb57e49ebab43cc97a2 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 20 Nov 2025 17:28:25 +0800 Subject: [PATCH 069/180] Fixed the bugs related to device auto-create alignment ignorance (#16780) * plan-first * refact * refa * fux * fix * ref * partial-fix * fix * may-final * fix * PBTree * fix * gras * err * bugfix (cherry picked from commit 9fd9d7e818a8fd6049b9caf3a417f698070bf429) --- .../IoTDBCreateAlignedTimeseriesIT.java | 22 +++++++++ .../db/it/schema/IoTDBCreateTimeseriesIT.java | 8 ++++ .../org/apache/iotdb/rpc/TSStatusCode.java | 1 - .../schemaregion/SchemaExecutionVisitor.java | 44 +++++++++++------- .../metadata/AlignedTimeSeriesException.java | 35 ++++++++++++++ .../executor/RegionWriteExecutor.java | 4 +- .../schema/AutoCreateSchemaExecutor.java | 46 ++++++++++++++----- .../impl/SchemaRegionMemoryImpl.java | 10 +++- .../impl/SchemaRegionPBTreeImpl.java | 10 +++- .../impl/mem/MTreeBelowSGMemoryImpl.java | 18 +++++++- .../impl/pbtree/MTreeBelowSGCachedImpl.java | 21 +++++++-- .../impl/CreateAlignedTimeSeriesPlanImpl.java | 10 ++++ .../req/impl/CreateTimeSeriesPlanImpl.java | 10 ++++ .../db/metadata/path/MeasurementPathTest.java | 4 +- .../iotdb/commons/path/MeasurementPath.java | 26 ----------- .../iotdb/commons/path/PartialPath.java | 26 +++++++++++ 16 files changed, 227 insertions(+), 68 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeSeriesException.java diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBCreateAlignedTimeseriesIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBCreateAlignedTimeseriesIT.java index bd470cfdb4d3f..7b784c034ab60 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBCreateAlignedTimeseriesIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBCreateAlignedTimeseriesIT.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.it.schema; import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; +import org.apache.iotdb.db.it.utils.TestUtils; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.itbase.category.ClusterIT; import org.apache.iotdb.itbase.category.LocalStandaloneIT; @@ -34,6 +35,9 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.util.Collections; + +import static org.junit.Assert.fail; /** * Notice that, all test begins with "IoTDB" is integration test. All test which will start the @@ -141,4 +145,22 @@ private void assertTimeseriesEquals(String[] timeSeriesArray) throws SQLExceptio } Assert.assertEquals(timeSeriesArray.length, count); } + + @Test + public void testDifferentDeviceAlignment() { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + // Should ignore the alignment difference + statement.execute("create aligned timeseries root.sg2.d (s2 int64, s3 int64)"); + // Should use the existing alignment + statement.execute("create timeseries root.sg2.d.s1 with datatype=INT64"); + statement.execute("insert into root.sg2.d (time, s4) values (-1, 1)"); + TestUtils.assertResultSetEqual( + statement.executeQuery("select * from root.sg2.d"), + "Time,root.sg2.d.s3,root.sg2.d.s4,root.sg2.d.s1,root.sg2.d.s2,", + Collections.singleton("-1,null,1.0,null,null,")); + } catch (SQLException ignored) { + fail(); + } + } } diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBCreateTimeseriesIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBCreateTimeseriesIT.java index ca309274b638c..b2b6f1c1078ad 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBCreateTimeseriesIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBCreateTimeseriesIT.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.it.schema; import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; +import org.apache.iotdb.db.it.utils.TestUtils; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.itbase.category.ClusterIT; import org.apache.iotdb.itbase.category.LocalStandaloneIT; @@ -35,6 +36,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -308,6 +310,12 @@ public void testDifferentDeviceAlignment() { statement.execute("create timeseries root.sg2.d.s1 with datatype=INT64"); // Should ignore the alignment difference statement.execute("create aligned timeseries root.sg2.d (s2 int64, s3 int64)"); + // Should use the existing alignment + statement.execute("insert into root.sg2.d (time, s4) aligned values (-1, 1)"); + TestUtils.assertResultSetEqual( + statement.executeQuery("select * from root.sg2.d"), + "Time,root.sg2.d.s3,root.sg2.d.s4,root.sg2.d.s1,root.sg2.d.s2,", + Collections.singleton("-1,null,1.0,null,null,")); } catch (SQLException ignored) { fail(); } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index 4fd90ca4b9492..246a1f669826f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -106,7 +106,6 @@ public enum TSStatusCode { WRITE_PROCESS_REJECT(606), OUT_OF_TTL(607), COMPACTION_ERROR(608), - @Deprecated ALIGNED_TIMESERIES_ERROR(609), WAL_ERROR(610), DISK_SPACE_INSUFFICIENT(611), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/schemaregion/SchemaExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/schemaregion/SchemaExecutionVisitor.java index 8d3c215583858..28ccff2f20d41 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/schemaregion/SchemaExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/schemaregion/SchemaExecutionVisitor.java @@ -189,7 +189,7 @@ public TSStatus visitInternalCreateTimeSeries( final PartialPath devicePath = node.getDevicePath(); final MeasurementGroup measurementGroup = node.getMeasurementGroup(); - final List alreadyExistingTimeSeries = new ArrayList<>(); + final List existingTimeSeriesAndAlignmentMismatch = new ArrayList<>(); final List failingStatus = new ArrayList<>(); if (node.isAligned()) { @@ -197,7 +197,7 @@ public TSStatus visitInternalCreateTimeSeries( devicePath, measurementGroup, schemaRegion, - alreadyExistingTimeSeries, + existingTimeSeriesAndAlignmentMismatch, failingStatus, node.isGeneratedByPipe()); } else { @@ -205,7 +205,7 @@ public TSStatus visitInternalCreateTimeSeries( devicePath, measurementGroup, schemaRegion, - alreadyExistingTimeSeries, + existingTimeSeriesAndAlignmentMismatch, failingStatus, node.isGeneratedByPipe()); } @@ -214,8 +214,8 @@ public TSStatus visitInternalCreateTimeSeries( return RpcUtils.getStatus(failingStatus); } - if (!alreadyExistingTimeSeries.isEmpty()) { - return RpcUtils.getStatus(alreadyExistingTimeSeries); + if (!existingTimeSeriesAndAlignmentMismatch.isEmpty()) { + return RpcUtils.getStatus(existingTimeSeriesAndAlignmentMismatch); } return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS, "Execute successfully"); @@ -227,7 +227,7 @@ public TSStatus visitInternalCreateMultiTimeSeries( PartialPath devicePath; MeasurementGroup measurementGroup; - final List alreadyExistingTimeSeries = new ArrayList<>(); + final List existingTimeSeriesAndAlignmentMismatch = new ArrayList<>(); final List failingStatus = new ArrayList<>(); for (final Map.Entry> deviceEntry : @@ -239,7 +239,7 @@ public TSStatus visitInternalCreateMultiTimeSeries( devicePath, measurementGroup, schemaRegion, - alreadyExistingTimeSeries, + existingTimeSeriesAndAlignmentMismatch, failingStatus, node.isGeneratedByPipe()); } else { @@ -247,7 +247,7 @@ public TSStatus visitInternalCreateMultiTimeSeries( devicePath, measurementGroup, schemaRegion, - alreadyExistingTimeSeries, + existingTimeSeriesAndAlignmentMismatch, failingStatus, node.isGeneratedByPipe()); } @@ -257,8 +257,8 @@ public TSStatus visitInternalCreateMultiTimeSeries( return RpcUtils.getStatus(failingStatus); } - if (!alreadyExistingTimeSeries.isEmpty()) { - return RpcUtils.getStatus(alreadyExistingTimeSeries); + if (!existingTimeSeriesAndAlignmentMismatch.isEmpty()) { + return RpcUtils.getStatus(existingTimeSeriesAndAlignmentMismatch); } return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS, "Execute successfully"); @@ -268,11 +268,12 @@ private void executeInternalCreateTimeSeries( final PartialPath devicePath, final MeasurementGroup measurementGroup, final ISchemaRegion schemaRegion, - final List alreadyExistingTimeSeries, + final List existingTimeSeriesAndAlignmentMismatch, final List failingStatus, final boolean withMerge) { final int size = measurementGroup.getMeasurements().size(); // todo implement batch creation of one device in SchemaRegion + boolean alignedIsSet = false; for (int i = 0; i < size; i++) { try { final ICreateTimeSeriesPlan createTimeSeriesPlan = @@ -283,11 +284,17 @@ private void executeInternalCreateTimeSeries( // Thus the original ones are not altered ((CreateTimeSeriesPlanImpl) createTimeSeriesPlan).setWithMerge(withMerge); schemaRegion.createTimeSeries(createTimeSeriesPlan, -1); + if (((CreateTimeSeriesPlanImpl) createTimeSeriesPlan).getAligned().get() && !alignedIsSet) { + existingTimeSeriesAndAlignmentMismatch.add( + new TSStatus(TSStatusCode.ALIGNED_TIMESERIES_ERROR.getStatusCode()) + .setMessage(PartialPath.transformDataToString(devicePath))); + alignedIsSet = true; + } } catch (final MeasurementAlreadyExistException e) { // There's no need to internal create time series. - alreadyExistingTimeSeries.add( + existingTimeSeriesAndAlignmentMismatch.add( RpcUtils.getStatus( - e.getErrorCode(), MeasurementPath.transformDataToString(e.getMeasurementPath()))); + e.getErrorCode(), PartialPath.transformDataToString(e.getMeasurementPath()))); } catch (final MetadataException e) { logger.warn("{}: MetaData error: ", e.getMessage(), e); failingStatus.add(RpcUtils.getStatus(e.getErrorCode(), e.getMessage())); @@ -299,7 +306,7 @@ private void executeInternalCreateAlignedTimeSeries( final PartialPath devicePath, final MeasurementGroup measurementGroup, final ISchemaRegion schemaRegion, - final List alreadyExistingTimeSeries, + final List existingTimeSeriesAndAlignmentMismatch, final List failingStatus, final boolean withMerge) { final List measurementList = measurementGroup.getMeasurements(); @@ -336,9 +343,9 @@ private void executeInternalCreateAlignedTimeSeries( // The existence check will be executed before truly creation // There's no need to internal create time series. final MeasurementPath measurementPath = e.getMeasurementPath(); - alreadyExistingTimeSeries.add( + existingTimeSeriesAndAlignmentMismatch.add( RpcUtils.getStatus( - e.getErrorCode(), MeasurementPath.transformDataToString(e.getMeasurementPath()))); + e.getErrorCode(), PartialPath.transformDataToString(e.getMeasurementPath()))); // remove the existing time series from plan final int index = measurementList.indexOf(measurementPath.getMeasurement()); @@ -389,6 +396,11 @@ private void executeInternalCreateAlignedTimeSeries( shouldRetry = false; } } + if (!((CreateAlignedTimeSeriesPlanImpl) createAlignedTimeSeriesPlan).getAligned().get()) { + existingTimeSeriesAndAlignmentMismatch.add( + new TSStatus(TSStatusCode.ALIGNED_TIMESERIES_ERROR.getStatusCode()) + .setMessage(PartialPath.transformDataToString(devicePath))); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeSeriesException.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeSeriesException.java new file mode 100644 index 0000000000000..f159376dfae1b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeSeriesException.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.exception.metadata; + +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.rpc.TSStatusCode; + +public class AlignedTimeSeriesException extends MetadataException { + + public AlignedTimeSeriesException(final boolean aligned, final String path) { + super( + String.format( + "TimeSeries under this device is%s aligned, please use createTimeSeries or change device. (Path: %s)", + aligned ? "" : " not", path), + TSStatusCode.ALIGNED_TIMESERIES_ERROR.getStatusCode(), + true); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java index 63af0423d1a9f..b5a831b90b2d0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java @@ -676,7 +676,7 @@ private RegionExecutionResult executeInternalCreateTimeSeries( alreadyExistingStatus.add( RpcUtils.getStatus( metadataException.getErrorCode(), - MeasurementPath.transformDataToString( + PartialPath.transformDataToString( ((MeasurementAlreadyExistException) metadataException) .getMeasurementPath()))); } else { @@ -774,7 +774,7 @@ private RegionExecutionResult executeInternalCreateMultiTimeSeries( alreadyExistingStatus.add( RpcUtils.getStatus( metadataException.getErrorCode(), - MeasurementPath.transformDataToString( + PartialPath.transformDataToString( ((MeasurementAlreadyExistException) metadataException) .getMeasurementPath()))); } else { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/AutoCreateSchemaExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/AutoCreateSchemaExecutor.java index 358614c9a58e8..a6a3301aea034 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/AutoCreateSchemaExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/AutoCreateSchemaExecutor.java @@ -29,6 +29,7 @@ import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.metadata.AlignedTimeSeriesException; import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; @@ -188,7 +189,7 @@ void autoCreateTimeSeries( } if (!devicesNeedAutoCreateTimeSeries.isEmpty()) { - internalCreateTimeSeries(schemaTree, devicesNeedAutoCreateTimeSeries, context); + internalCreateTimeSeries(schemaTree, devicesNeedAutoCreateTimeSeries, context, false); } } @@ -425,7 +426,7 @@ void autoCreateMissingMeasurements( } if (!devicesNeedAutoCreateTimeSeries.isEmpty()) { - internalCreateTimeSeries(schemaTree, devicesNeedAutoCreateTimeSeries, context); + internalCreateTimeSeries(schemaTree, devicesNeedAutoCreateTimeSeries, context, true); } } @@ -463,11 +464,15 @@ private void internalCreateTimeSeries( List compressors, boolean isAligned, MPPQueryContext context) { + final Map> input = + Collections.singletonMap(devicePath, new Pair<>(isAligned, null)); List measurementPathList = - executeInternalCreateTimeseriesStatement( + executeInternalCreateTimeSeriesStatement( + input, new InternalCreateTimeSeriesStatement( devicePath, measurements, tsDataTypes, encodings, compressors, isAligned), - context); + context, + false); Set alreadyExistingMeasurementIndexSet = measurementPathList.stream() @@ -488,13 +493,16 @@ private void internalCreateTimeSeries( null, null, null, - isAligned); + input.get(devicePath).getLeft()); } } - // Auto create timeseries and return the existing timeseries info - private List executeInternalCreateTimeseriesStatement( - final Statement statement, final MPPQueryContext context) { + // Auto create timeSeries and return the existing timeSeries info + private List executeInternalCreateTimeSeriesStatement( + final Map> devicesNeedAutoCreateTimeSeries, + final Statement statement, + final MPPQueryContext context, + final boolean isLoad) { final TSStatus status = AuthorityChecker.checkAuthority(statement, context); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { throw new IoTDBRuntimeException(status.getMessage(), status.getCode()); @@ -516,7 +524,18 @@ private List executeInternalCreateTimeseriesStatement( for (final TSStatus subStatus : executionResult.status.subStatus) { if (subStatus.code == TSStatusCode.TIMESERIES_ALREADY_EXIST.getStatusCode()) { alreadyExistingMeasurements.add( - MeasurementPath.parseDataFromString(subStatus.getMessage())); + (MeasurementPath) PartialPath.parseDataFromString(subStatus.getMessage())); + } else if (subStatus.code == TSStatusCode.ALIGNED_TIMESERIES_ERROR.getStatusCode()) { + final PartialPath devicePath = PartialPath.parseDataFromString(subStatus.getMessage()); + final Pair pair = + devicesNeedAutoCreateTimeSeries.get(devicePath); + if (!isLoad) { + pair.setLeft(!pair.getLeft()); + } else { + // Load does not tolerate the device alignment mismatch + throw new SemanticException( + new AlignedTimeSeriesException(!pair.getLeft(), devicePath.getFullPath())); + } } else { failedCreationSet.add(subStatus); } @@ -583,11 +602,13 @@ private void internalActivateTemplate( private void internalCreateTimeSeries( ClusterSchemaTree schemaTree, Map> devicesNeedAutoCreateTimeSeries, - MPPQueryContext context) { + MPPQueryContext context, + final boolean isLoad) { // Deep copy to avoid changes to the original map final List measurementPathList = - executeInternalCreateTimeseriesStatement( + executeInternalCreateTimeSeriesStatement( + devicesNeedAutoCreateTimeSeries, new InternalCreateMultiTimeSeriesStatement( devicesNeedAutoCreateTimeSeries.entrySet().stream() .collect( @@ -597,7 +618,8 @@ private void internalCreateTimeSeries( new Pair<>( entry.getValue().getLeft(), entry.getValue().getRight().deepCopy())))), - context); + context, + isLoad); schemaTree.appendMeasurementPaths(measurementPathList); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java index 39be89f093808..5f6486f8be4f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java @@ -702,7 +702,10 @@ public void createTimeSeries(final ICreateTimeSeriesPlan plan, long offset) plan instanceof CreateTimeSeriesPlanImpl && ((CreateTimeSeriesPlanImpl) plan).isWithMerge() || plan instanceof CreateTimeSeriesNode - && ((CreateTimeSeriesNode) plan).isGeneratedByPipe()); + && ((CreateTimeSeriesNode) plan).isGeneratedByPipe(), + plan instanceof CreateTimeSeriesPlanImpl + ? ((CreateTimeSeriesPlanImpl) plan).getAligned() + : null); // Should merge if (Objects.isNull(leafMNode)) { @@ -786,7 +789,10 @@ public void createAlignedTimeSeries(final ICreateAlignedTimeSeriesPlan plan) aliasList, (plan instanceof CreateAlignedTimeSeriesPlanImpl && ((CreateAlignedTimeSeriesPlanImpl) plan).isWithMerge()), - existingMeasurementIndexes); + existingMeasurementIndexes, + (plan instanceof CreateAlignedTimeSeriesPlanImpl + ? ((CreateAlignedTimeSeriesPlanImpl) plan).getAligned() + : null)); // update statistics and schemaDataTypeNumMap regionStatistics.addMeasurement(measurementMNodeList.size()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java index 944ffe0ea5357..a39b881fe1f4f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionPBTreeImpl.java @@ -645,7 +645,10 @@ public void createTimeSeries(final ICreateTimeSeriesPlan plan, long offset) plan.getProps(), plan.getAlias(), (plan instanceof CreateTimeSeriesPlanImpl - && ((CreateTimeSeriesPlanImpl) plan).isWithMerge())); + && ((CreateTimeSeriesPlanImpl) plan).isWithMerge()), + plan instanceof CreateTimeSeriesPlanImpl + ? ((CreateTimeSeriesPlanImpl) plan).getAligned() + : null); try { // Should merge @@ -756,7 +759,10 @@ public void createAlignedTimeSeries(final ICreateAlignedTimeSeriesPlan plan) aliasList, (plan instanceof CreateAlignedTimeSeriesPlanImpl && ((CreateAlignedTimeSeriesPlanImpl) plan).isWithMerge()), - existingMeasurementIndexes); + existingMeasurementIndexes, + (plan instanceof CreateAlignedTimeSeriesPlanImpl + ? ((CreateAlignedTimeSeriesPlanImpl) plan).getAligned() + : null)); try { // Update statistics and schemaDataTypeNumMap diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java index 15d51928d6406..5931023b3cf90 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java @@ -238,7 +238,8 @@ public IMeasurementMNode createTimeSeries( final CompressionType compressor, final Map props, final String alias, - final boolean withMerge) + final boolean withMerge, + final AtomicBoolean isAligned) throws MetadataException { final String[] nodeNames = path.getNodes(); if (nodeNames.length <= 2) { @@ -253,6 +254,12 @@ public IMeasurementMNode createTimeSeries( synchronized (this) { final IMemMNode device = checkAndAutoCreateDeviceNode(devicePath.getTailNode(), deviceParent); + if (device.isDevice() + && device.getAsDeviceMNode().isAlignedNullable() != null + && isAligned != null) { + isAligned.set(device.getAsDeviceMNode().isAligned()); + } + MetaFormatUtils.checkTimeseriesProps(path.getFullPath(), props); final String leafName = path.getMeasurement(); @@ -328,7 +335,8 @@ public List> createAlignedTimeSeries( final List compressors, final List aliasList, final boolean withMerge, - final Set existingMeasurementIndexes) + final Set existingMeasurementIndexes, + final AtomicBoolean isAligned) throws MetadataException { final List> measurementMNodeList = new ArrayList<>(); MetaFormatUtils.checkSchemaMeasurementNames(measurements); @@ -339,6 +347,12 @@ public List> createAlignedTimeSeries( synchronized (this) { final IMemMNode device = checkAndAutoCreateDeviceNode(devicePath.getTailNode(), deviceParent); + if (device.isDevice() + && device.getAsDeviceMNode().isAlignedNullable() != null + && isAligned != null) { + isAligned.set(device.getAsDeviceMNode().isAligned()); + } + for (int i = 0; i < measurements.size(); i++) { if (device.hasChild(measurements.get(i))) { final IMemMNode node = device.getChild(measurements.get(i)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java index d5187e14c9140..5127cd9e65532 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java @@ -266,7 +266,8 @@ public IMeasurementMNode createTimeSeries( String alias) throws MetadataException { IMeasurementMNode measurementMNode = - createTimeSeriesWithPinnedReturn(path, dataType, encoding, compressor, props, alias, false); + createTimeSeriesWithPinnedReturn( + path, dataType, encoding, compressor, props, alias, false, null); unPinMNode(measurementMNode.getAsMNode()); return measurementMNode; } @@ -289,7 +290,8 @@ public IMeasurementMNode createTimeSeriesWithPinnedReturn( final CompressionType compressor, final Map props, final String alias, - final boolean withMerge) + final boolean withMerge, + final AtomicBoolean isAligned) throws MetadataException { final String[] nodeNames = path.getNodes(); if (nodeNames.length <= 2) { @@ -305,6 +307,12 @@ public IMeasurementMNode createTimeSeriesWithPinnedReturn( synchronized (this) { ICachedMNode device = checkAndAutoCreateDeviceNode(devicePath.getTailNode(), deviceParent); + if (device.isDevice() + && device.getAsDeviceMNode().isAlignedNullable() != null + && isAligned != null) { + isAligned.set(device.getAsDeviceMNode().isAligned()); + } + try { MetaFormatUtils.checkTimeseriesProps(path.getFullPath(), props); @@ -387,7 +395,8 @@ public List> createAlignedTimeSeries( final List compressors, final List aliasList, final boolean withMerge, - final Set existingMeasurementIndexes) + final Set existingMeasurementIndexes, + final AtomicBoolean isAligned) throws MetadataException { final List> measurementMNodeList = new ArrayList<>(); MetaFormatUtils.checkSchemaMeasurementNames(measurements); @@ -399,6 +408,12 @@ public List> createAlignedTimeSeries( synchronized (this) { ICachedMNode device = checkAndAutoCreateDeviceNode(devicePath.getTailNode(), deviceParent); + if (device.isDevice() + && device.getAsDeviceMNode().isAlignedNullable() != null + && isAligned != null) { + isAligned.set(device.getAsDeviceMNode().isAligned()); + } + try { for (int i = 0; i < measurements.size(); i++) { if (store.hasChild(device, measurements.get(i))) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/impl/CreateAlignedTimeSeriesPlanImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/impl/CreateAlignedTimeSeriesPlanImpl.java index 4ffa8aca5b2f3..8a2c4c12e8c39 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/impl/CreateAlignedTimeSeriesPlanImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/impl/CreateAlignedTimeSeriesPlanImpl.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; public class CreateAlignedTimeSeriesPlanImpl implements ICreateAlignedTimeSeriesPlan { @@ -45,6 +46,7 @@ public class CreateAlignedTimeSeriesPlanImpl implements ICreateAlignedTimeSeries private List> attributesList; private List tagOffsets = null; private transient boolean withMerge; + private final transient AtomicBoolean aligned = new AtomicBoolean(true); public CreateAlignedTimeSeriesPlanImpl() {} @@ -194,4 +196,12 @@ public void setWithMerge(final boolean withMerge) { tagOffsets = Objects.nonNull(tagOffsets) ? new ArrayList<>(tagOffsets) : null; } } + + public void setAligned(final boolean aligned) { + this.aligned.set(aligned); + } + + public AtomicBoolean getAligned() { + return aligned; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/impl/CreateTimeSeriesPlanImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/impl/CreateTimeSeriesPlanImpl.java index 6234cc41affdd..73b47a4f5165d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/impl/CreateTimeSeriesPlanImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/write/req/impl/CreateTimeSeriesPlanImpl.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicBoolean; public class CreateTimeSeriesPlanImpl implements ICreateTimeSeriesPlan { @@ -41,6 +42,7 @@ public class CreateTimeSeriesPlanImpl implements ICreateTimeSeriesPlan { private Map attributes = null; private long tagOffset = -1; private transient boolean withMerge; + private final transient AtomicBoolean aligned = new AtomicBoolean(false); public CreateTimeSeriesPlanImpl() {} @@ -170,4 +172,12 @@ public boolean isWithMerge() { public void setWithMerge(final boolean withMerge) { this.withMerge = withMerge; } + + public void setAligned(final boolean aligned) { + this.aligned.set(aligned); + } + + public AtomicBoolean getAligned() { + return aligned; + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/path/MeasurementPathTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/path/MeasurementPathTest.java index cf5f8ab57ba7d..148a3c56bd4ca 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/path/MeasurementPathTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/path/MeasurementPathTest.java @@ -43,8 +43,8 @@ public void testTransformDataToString() throws IllegalPathException { new MeasurementPath( new PartialPath("root.sg.d.s"), new MeasurementSchema("s", TSDataType.INT32), true); rawPath.setMeasurementAlias("alias"); - String string = MeasurementPath.transformDataToString(rawPath); - MeasurementPath newPath = MeasurementPath.parseDataFromString(string); + String string = PartialPath.transformDataToString(rawPath); + MeasurementPath newPath = (MeasurementPath) PartialPath.parseDataFromString(string); Assert.assertEquals(rawPath.getFullPath(), newPath.getFullPath()); Assert.assertEquals(rawPath.getMeasurementAlias(), newPath.getMeasurementAlias()); Assert.assertEquals(rawPath.getMeasurementSchema(), newPath.getMeasurementSchema()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/MeasurementPath.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/MeasurementPath.java index f74a29f1815f4..0718366aabfc0 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/MeasurementPath.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/MeasurementPath.java @@ -33,12 +33,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.rmi.UnexpectedException; import java.util.ArrayList; import java.util.Arrays; @@ -339,23 +336,6 @@ public PartialPath transformToPartialPath() { return getDevicePath().concatNode(getTailNode()); } - /** - * In specific scenarios, like internal create timeseries, the message can only be passed as - * String format. - */ - public static String transformDataToString(MeasurementPath measurementPath) { - ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); - try { - measurementPath.serialize(dataOutputStream); - } catch (IOException ignored) { - // this exception won't happen. - } - byte[] bytes = byteArrayOutputStream.toByteArray(); - // must use single-byte char sets - return new String(bytes, StandardCharsets.ISO_8859_1); - } - @Override protected IDeviceID toDeviceID(String[] nodes) { // remove measurement @@ -363,12 +343,6 @@ protected IDeviceID toDeviceID(String[] nodes) { return super.toDeviceID(nodes); } - public static MeasurementPath parseDataFromString(String measurementPathData) { - return (MeasurementPath) - PathDeserializeUtil.deserialize( - ByteBuffer.wrap(measurementPathData.getBytes(StandardCharsets.ISO_8859_1))); - } - @Override protected PartialPath createPartialPath(String[] newPathNodes) { return new MeasurementPath(newPathNodes); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java index ae35769a1d081..b0f2c3fe05193 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PartialPath.java @@ -38,9 +38,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -1040,6 +1043,29 @@ public PartialPath transformToPartialPath() { return this; } + /** + * In specific scenarios, like internal create timeseries, the message can only be passed as + * String format. + */ + public static String transformDataToString(PartialPath partialPath) { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); + try { + partialPath.serialize(dataOutputStream); + } catch (IOException ignored) { + // this exception won't happen. + } + byte[] bytes = byteArrayOutputStream.toByteArray(); + // must use single-byte char sets + return new String(bytes, StandardCharsets.ISO_8859_1); + } + + public static PartialPath parseDataFromString(String measurementPathData) { + return (PartialPath) + PathDeserializeUtil.deserialize( + ByteBuffer.wrap(measurementPathData.getBytes(StandardCharsets.ISO_8859_1))); + } + /** Return true if the path ends with ** and no other nodes contain *. Otherwise, return false. */ public boolean isPrefixPath() { if (nodes.length <= 0) { From fe49c5ee0806f57a30cfff5c86d61fb653a51203 Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Thu, 20 Nov 2025 18:10:19 +0800 Subject: [PATCH 070/180] Load: Active Load supports ModV2 (#16769) (cherry picked from commit f708b96623861172ed3a4cf5a6d29433565549ef) --- .../thrift/IoTDBDataNodeReceiver.java | 4 +- .../plan/analyze/load/LoadTsFileAnalyzer.java | 5 +- .../plan/node/load/LoadSingleTsFileNode.java | 6 +-- .../load/active/ActiveLoadDirScanner.java | 20 ++------ .../load/active/ActiveLoadTsFileLoader.java | 6 ++- ...tementDataTypeConvertExecutionVisitor.java | 10 ++-- ...tementDataTypeConvertExecutionVisitor.java | 10 ++-- .../LoadUtil.java} | 49 ++++++++++++++++--- 8 files changed, 65 insertions(+), 45 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/{active/ActiveLoadUtil.java => util/LoadUtil.java} (74%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index 652530d11b481..4d4bf7b39268d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -101,7 +101,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.pipe.PipeEnrichedStatement; import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; import org.apache.iotdb.db.storageengine.load.active.ActiveLoadPathHelper; -import org.apache.iotdb.db.storageengine.load.active.ActiveLoadUtil; +import org.apache.iotdb.db.storageengine.load.util.LoadUtil; import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager; import org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType; import org.apache.iotdb.db.tools.schema.SRStatementGenerator; @@ -580,7 +580,7 @@ private TSStatus loadTsFileAsync(final String dataBaseName, final List a null, shouldMarkAsPipeRequest.get()); - if (!ActiveLoadUtil.loadFilesToActiveDir(loadAttributes, absolutePaths, true)) { + if (!LoadUtil.loadFilesToActiveDir(loadAttributes, absolutePaths, true)) { throw new PipeException("Load active listening pipe dir is not set."); } return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java index 076b529ab918a..8202de5c49947 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java @@ -43,9 +43,9 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileResourceUtils; import org.apache.iotdb.db.storageengine.load.active.ActiveLoadPathHelper; -import org.apache.iotdb.db.storageengine.load.active.ActiveLoadUtil; import org.apache.iotdb.db.storageengine.load.converter.LoadTsFileDataTypeConverter; import org.apache.iotdb.db.storageengine.load.metrics.LoadTsFileCostMetricsSet; +import org.apache.iotdb.db.storageengine.load.util.LoadUtil; import org.apache.iotdb.db.utils.TimestampPrecisionUtils; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -291,8 +291,7 @@ private boolean doAsyncLoad(final IAnalysis analysis) { tabletConversionThresholdBytes, isGeneratedByPipe); - if (ActiveLoadUtil.loadTsFileAsyncToActiveDir( - tsFiles, activeLoadAttributes, isDeleteAfterLoad)) { + if (LoadUtil.loadTsFileAsyncToActiveDir(tsFiles, activeLoadAttributes, isDeleteAfterLoad)) { analysis.setFinishQueryAfterAnalyze(true); setRealStatement(analysis); return true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java index c8691868c8266..604fda6e1e8d5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java @@ -30,8 +30,8 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; -import org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.load.util.LoadUtil; import org.apache.tsfile.exception.NotImplementedException; import org.apache.tsfile.file.metadata.IDeviceID; @@ -229,10 +229,10 @@ public void clean() { if (deleteAfterLoad) { Files.deleteIfExists(tsFile.toPath()); Files.deleteIfExists( - new File(tsFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).toPath()); + new File(LoadUtil.getTsFileResourcePath(tsFile.getAbsolutePath())).toPath()); Files.deleteIfExists(ModificationFile.getExclusiveMods(tsFile).toPath()); Files.deleteIfExists( - new File(tsFile.getAbsolutePath() + ModificationFileV1.FILE_SUFFIX).toPath()); + new File(LoadUtil.getTsFileModsV1Path(tsFile.getAbsolutePath())).toPath()); } } catch (final IOException e) { LOGGER.warn("Delete After Loading {} error.", tsFile, e); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java index 470cf702b226a..cedc4d5e29f63 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadDirScanner.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.db.storageengine.load.metrics.ActiveLoadingFilesNumberMetricsSet; import org.apache.iotdb.db.storageengine.load.metrics.ActiveLoadingFilesSizeMetricsSet; +import org.apache.iotdb.db.storageengine.load.util.LoadUtil; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.external.commons.io.FileUtils; @@ -51,9 +52,6 @@ public class ActiveLoadDirScanner extends ActiveLoadScheduledExecutorService { private static final Logger LOGGER = LoggerFactory.getLogger(ActiveLoadDirScanner.class); - private static final String RESOURCE = ".resource"; - private static final String MODS = ".mods"; - private final AtomicReference listeningDirsConfig = new AtomicReference<>(); private final Set listeningDirs = new CopyOnWriteArraySet<>(); @@ -110,11 +108,7 @@ private void scan() throws IOException { fileStream .filter(file -> !activeLoadTsFileLoader.isFilePendingOrLoading(file)) .filter(File::exists) - .map( - file -> - (file.getName().endsWith(RESOURCE) || file.getName().endsWith(MODS)) - ? getTsFilePath(file.getAbsolutePath()) - : file.getAbsolutePath()) + .map(file -> LoadUtil.getTsFilePath(file.getAbsolutePath())) .filter(this::isTsFileCompleted) .limit(currentAllowedPendingSize) .forEach( @@ -202,7 +196,7 @@ private void hotReloadActiveLoadDirs() { listeningDirsConfig.set(IOTDB_CONFIG.getLoadActiveListeningDirs()); listeningDirs.addAll(Arrays.asList(IOTDB_CONFIG.getLoadActiveListeningDirs())); - ActiveLoadUtil.updateLoadDiskSelector(); + LoadUtil.updateLoadDiskSelector(); } } } @@ -235,14 +229,6 @@ private void createDirectoriesIfNotExists(final String dirPath) { } } - private static String getTsFilePath(final String filePathWithResourceOrModsTail) { - return filePathWithResourceOrModsTail.endsWith(RESOURCE) - ? filePathWithResourceOrModsTail.substring( - 0, filePathWithResourceOrModsTail.length() - RESOURCE.length()) - : filePathWithResourceOrModsTail.substring( - 0, filePathWithResourceOrModsTail.length() - MODS.length()); - } - // Metrics public long countAndReportActiveListeningDirsFileNumber() { long totalFileCount = 0; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java index 20817c9414634..0e565afd70cd3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java @@ -40,6 +40,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.pipe.PipeEnrichedStatement; import org.apache.iotdb.db.storageengine.load.metrics.ActiveLoadingFilesNumberMetricsSet; import org.apache.iotdb.db.storageengine.load.metrics.ActiveLoadingFilesSizeMetricsSet; +import org.apache.iotdb.db.storageengine.load.util.LoadUtil; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.external.commons.io.FileUtils; @@ -297,8 +298,9 @@ private void handleOtherException( private void removeFileAndResourceAndModsToFailDir(final String filePath) { removeToFailDir(filePath); - removeToFailDir(filePath + ".resource"); - removeToFailDir(filePath + ".mods"); + removeToFailDir(LoadUtil.getTsFileResourcePath(filePath)); + removeToFailDir(LoadUtil.getTsFileModsV1Path(filePath)); + removeToFailDir(LoadUtil.getTsFileModsV2Path(filePath)); } private void removeToFailDir(final String filePath) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTableStatementDataTypeConvertExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTableStatementDataTypeConvertExecutionVisitor.java index 9a6be9737af83..a011490ca497b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTableStatementDataTypeConvertExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTableStatementDataTypeConvertExecutionVisitor.java @@ -27,9 +27,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AstVisitor; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LoadTsFile; import org.apache.iotdb.db.queryengine.plan.statement.Statement; -import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; -import org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.load.util.LoadUtil; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.rpc.TSStatusCode; @@ -125,9 +123,9 @@ public Optional visitLoadTsFile( tsfile -> { FileUtils.deleteQuietly(tsfile); final String tsFilePath = tsfile.getAbsolutePath(); - FileUtils.deleteQuietly(new File(tsFilePath + TsFileResource.RESOURCE_SUFFIX)); - FileUtils.deleteQuietly(new File(tsFilePath + ModificationFileV1.FILE_SUFFIX)); - FileUtils.deleteQuietly(new File(tsFilePath + ModificationFile.FILE_SUFFIX)); + FileUtils.deleteQuietly(new File(LoadUtil.getTsFileResourcePath(tsFilePath))); + FileUtils.deleteQuietly(new File(LoadUtil.getTsFileModsV1Path(tsFilePath))); + FileUtils.deleteQuietly(new File(LoadUtil.getTsFileModsV2Path(tsFilePath))); }); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTreeStatementDataTypeConvertExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTreeStatementDataTypeConvertExecutionVisitor.java index 2999d436c9593..226966454aaad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTreeStatementDataTypeConvertExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTreeStatementDataTypeConvertExecutionVisitor.java @@ -29,11 +29,9 @@ import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertMultiTabletsStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.LoadTsFileStatement; -import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; -import org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.load.memory.LoadTsFileMemoryBlock; import org.apache.iotdb.db.storageengine.load.memory.LoadTsFileMemoryManager; +import org.apache.iotdb.db.storageengine.load.util.LoadUtil; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.external.commons.io.FileUtils; @@ -178,9 +176,9 @@ public Optional visitLoadFile( tsfile -> { FileUtils.deleteQuietly(tsfile); final String tsFilePath = tsfile.getAbsolutePath(); - FileUtils.deleteQuietly(new File(tsFilePath + TsFileResource.RESOURCE_SUFFIX)); - FileUtils.deleteQuietly(new File(tsFilePath + ModificationFileV1.FILE_SUFFIX)); - FileUtils.deleteQuietly(new File(tsFilePath + ModificationFile.FILE_SUFFIX)); + FileUtils.deleteQuietly(new File(LoadUtil.getTsFileResourcePath(tsFilePath))); + FileUtils.deleteQuietly(new File(LoadUtil.getTsFileModsV1Path(tsFilePath))); + FileUtils.deleteQuietly(new File(LoadUtil.getTsFileModsV2Path(tsFilePath))); }); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/util/LoadUtil.java similarity index 74% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/util/LoadUtil.java index e3dbe43507d88..a3d29337b865b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/util/LoadUtil.java @@ -17,11 +17,15 @@ * under the License. */ -package org.apache.iotdb.db.storageengine.load.active; +package org.apache.iotdb.db.storageengine.load.util; import org.apache.iotdb.commons.utils.RetryUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; +import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; +import org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.load.active.ActiveLoadPathHelper; import org.apache.iotdb.db.storageengine.load.disk.ILoadDiskSelector; import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager; import org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType; @@ -40,9 +44,9 @@ import static org.apache.iotdb.commons.utils.FileUtils.copyFileWithMD5Check; import static org.apache.iotdb.commons.utils.FileUtils.moveFileWithMD5Check; -public class ActiveLoadUtil { +public class LoadUtil { - private static final Logger LOGGER = LoggerFactory.getLogger(ActiveLoadUtil.class); + private static final Logger LOGGER = LoggerFactory.getLogger(LoadUtil.class); private static volatile ILoadDiskSelector loadDiskSelector = updateLoadDiskSelector(); @@ -68,6 +72,37 @@ public static boolean loadTsFileAsyncToActiveDir( return true; } + public static String getTsFilePath(final String filePathWithResourceOrModsTail) { + if (filePathWithResourceOrModsTail.endsWith(TsFileResource.RESOURCE_SUFFIX)) { + return filePathWithResourceOrModsTail.substring( + 0, filePathWithResourceOrModsTail.length() - TsFileResource.RESOURCE_SUFFIX.length()); + } + + if (filePathWithResourceOrModsTail.endsWith(ModificationFileV1.FILE_SUFFIX)) { + return filePathWithResourceOrModsTail.substring( + 0, filePathWithResourceOrModsTail.length() - ModificationFileV1.FILE_SUFFIX.length()); + } + + if (filePathWithResourceOrModsTail.endsWith(ModificationFile.FILE_SUFFIX)) { + return filePathWithResourceOrModsTail.substring( + 0, filePathWithResourceOrModsTail.length() - ModificationFile.FILE_SUFFIX.length()); + } + + return filePathWithResourceOrModsTail; + } + + public static String getTsFileModsV1Path(final String tsFilePath) { + return tsFilePath + ModificationFileV1.FILE_SUFFIX; + } + + public static String getTsFileModsV2Path(final String tsFilePath) { + return tsFilePath + ModificationFile.FILE_SUFFIX; + } + + public static String getTsFileResourcePath(final String tsFilePath) { + return tsFilePath + TsFileResource.RESOURCE_SUFFIX; + } + private static boolean loadTsFilesToActiveDir( final Map loadAttributes, final File file, final boolean isDeleteAfterLoad) throws IOException { @@ -93,9 +128,11 @@ private static boolean loadTsFilesToActiveDir( final File targetDir = ActiveLoadPathHelper.resolveTargetDir(targetFilePath, attributes); loadTsFileAsyncToTargetDir( - targetDir, new File(file.getAbsolutePath() + ".resource"), isDeleteAfterLoad); + targetDir, new File(getTsFileResourcePath(file.getAbsolutePath())), isDeleteAfterLoad); + loadTsFileAsyncToTargetDir( + targetDir, new File(getTsFileModsV1Path(file.getAbsolutePath())), isDeleteAfterLoad); loadTsFileAsyncToTargetDir( - targetDir, new File(file.getAbsolutePath() + ".mods"), isDeleteAfterLoad); + targetDir, new File(getTsFileModsV2Path(file.getAbsolutePath())), isDeleteAfterLoad); loadTsFileAsyncToTargetDir(targetDir, file, isDeleteAfterLoad); return true; } @@ -182,7 +219,7 @@ public static ILoadDiskSelector updateLoadDiskSelector() { return new File(finalFolderManager.getNextFolder()); }); - ActiveLoadUtil.loadDiskSelector = loadDiskSelector; + LoadUtil.loadDiskSelector = loadDiskSelector; return loadDiskSelector; } } From 769f10374dd690d76832028bd779bf6f0b8a91e4 Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Thu, 20 Nov 2025 18:57:17 +0800 Subject: [PATCH 071/180] Load: Fixed the issue of TSFile parent directory being null and TSFile resource being updated during the Load process. (#16751) * Load: Fixed the issue of TSFile parent directory being null and TSFile resource being updated during the Load process. * add IT * fix * FIX * update * update (cherry picked from commit 0f2483fbe2ee16c4f34371417ad49b2dff71ce8d) --- .../apache/iotdb/db/it/IoTDBLoadTsFileIT.java | 43 +++++++++++++++++++ .../load/LoadTsFileDispatcherImpl.java | 12 ++++-- .../statement/crud/LoadTsFileStatement.java | 2 +- .../storageengine/dataregion/DataRegion.java | 6 ++- .../dataregion/tsfile/TsFileResource.java | 43 ++++++++++++++++++- .../load/disk/MinIOSelector.java | 10 +++-- 6 files changed, 105 insertions(+), 11 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java index 36a94174751c9..c54adb347117e 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; import org.apache.iotdb.db.it.utils.TestUtils; import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.it.utils.TsFileGenerator; import org.apache.iotdb.itbase.category.ClusterIT; @@ -67,6 +68,7 @@ import static org.apache.iotdb.db.it.utils.TestUtils.executeNonQuery; import static org.apache.iotdb.db.it.utils.TestUtils.grantUserSeriesPrivilege; import static org.apache.iotdb.db.it.utils.TestUtils.grantUserSystemPrivileges; +import static org.apache.iotdb.it.env.cluster.ClusterConstant.USER_DIR; @RunWith(IoTDBTestRunner.class) @Category({LocalStandaloneIT.class, ClusterIT.class}) @@ -742,6 +744,47 @@ public void testLoadWithOnNonStandardTsFileName() throws Exception { } } + @Test + public void testLoadWithRelativePathName() throws Exception { + DataNodeWrapper dataNodeWrapper = EnvFactory.getEnv().getDataNodeWrapper(0); + + registerSchema(); + + final long writtenPoint1; + // device 0, device 1, sg 0 + File relativePathFile = new File(System.getProperty(USER_DIR), "1-0-0-0.tsfile"); + try { + try (final TsFileGenerator generator = new TsFileGenerator(relativePathFile)) { + generator.registerTimeseries( + SchemaConfig.DEVICE_0, Collections.singletonList(SchemaConfig.MEASUREMENT_00)); + generator.generateData(SchemaConfig.DEVICE_0, 1, PARTITION_INTERVAL / 10_000, false); + writtenPoint1 = generator.getTotalNumber(); + } + + try (final Connection connection = + EnvFactory.getEnv().getConnectionWithSpecifiedDataNode(dataNodeWrapper); + final Statement statement = connection.createStatement()) { + + statement.execute(String.format("load \"%s\" sglevel=2", "1-0-0-0.tsfile")); + + try (final ResultSet resultSet = + statement.executeQuery("select count(*) from root.sg.** group by level=1,2")) { + if (resultSet.next()) { + final long sg1Count = resultSet.getLong("count(root.sg.test_0.*.*)"); + Assert.assertEquals(writtenPoint1, sg1Count); + } else { + Assert.fail("This ResultSet is empty."); + } + } + } + + } finally { + if (relativePathFile.exists()) { + relativePathFile.delete(); + } + } + } + @Test public void testLoadWithMods() throws Exception { final long writtenPoint1; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java index cfe6bb1a1a68c..7301ce0f406a2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java @@ -170,15 +170,22 @@ public void dispatchLocally(FragmentInstance instance) throws FragmentInstanceDi } } else if (planNode instanceof LoadSingleTsFileNode) { // do not need to split final TsFileResource tsFileResource = ((LoadSingleTsFileNode) planNode).getTsFileResource(); + final String filePath = tsFileResource.getTsFile().getAbsolutePath(); try { PipeDataNodeAgent.runtime().assignProgressIndexForTsFileLoad(tsFileResource); tsFileResource.setGeneratedByPipe(isGeneratedByPipe); tsFileResource.serialize(); + TsFileResource cloneTsFileResource = null; + try { + cloneTsFileResource = tsFileResource.shallowCloneForNative(); + } catch (CloneNotSupportedException e) { + cloneTsFileResource = tsFileResource.shallowClone(); + } StorageEngine.getInstance() .getDataRegion((DataRegionId) groupId) .loadNewTsFile( - tsFileResource, + cloneTsFileResource, ((LoadSingleTsFileNode) planNode).isDeleteAfterLoad(), isGeneratedByPipe, false); @@ -189,8 +196,7 @@ public void dispatchLocally(FragmentInstance instance) throws FragmentInstanceDi resultStatus.setMessage(e.getMessage()); throw new FragmentInstanceDispatchException(resultStatus); } catch (IOException e) { - LOGGER.warn( - "Serialize TsFileResource {} error.", tsFileResource.getTsFile().getAbsolutePath(), e); + LOGGER.warn("Serialize TsFileResource {} error.", filePath, e); TSStatus resultStatus = new TSStatus(); resultStatus.setCode(TSStatusCode.LOAD_FILE_ERROR.getStatusCode()); resultStatus.setMessage(e.getMessage()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java index 2d74925971c31..d1dff1bb9cf25 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java @@ -70,7 +70,7 @@ public class LoadTsFileStatement extends Statement { private List writePointCountList; public LoadTsFileStatement(String filePath) throws FileNotFoundException { - this.file = new File(filePath); + this.file = new File(filePath).getAbsoluteFile(); this.databaseLevel = IoTDBDescriptor.getInstance().getConfig().getDefaultDatabaseLevel(); this.verifySchema = true; this.deleteAfterLoad = false; 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 2c0c39032c437..43c395abc123a 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 @@ -3549,10 +3549,12 @@ public void loadNewTsFile( final boolean isGeneratedByPipe, final boolean isFromConsensus) throws LoadFileException { - final File tsfileToBeInserted = newTsFileResource.getTsFile(); + final File tsfileToBeInserted = newTsFileResource.getTsFile().getAbsoluteFile(); final long newFilePartitionId = newTsFileResource.getTimePartitionWithCheck(); - if (!TsFileValidator.getInstance().validateTsFile(newTsFileResource)) { + if (!TsFileValidator.getInstance().validateTsFile(newTsFileResource) + || !tsfileToBeInserted.exists() + || tsfileToBeInserted.getParentFile() == null) { throw new LoadFileException( "tsfile validate failed, " + newTsFileResource.getTsFile().getName()); } 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 35ea1660fde78..3169cf85ccb4d 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 @@ -96,7 +96,7 @@ import static org.apache.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX; @SuppressWarnings("java:S1135") // ignore todos -public class TsFileResource implements PersistentResource { +public class TsFileResource implements PersistentResource, Cloneable { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(TsFileResource.class) @@ -1585,4 +1585,45 @@ public Map>> getLastValues() { public void setLastValues(Map>> lastValues) { this.lastValues = lastValues; } + + public TsFileResource shallowClone() { + TsFileResource cloned = new TsFileResource(); + cloned.file = this.file; + cloned.timeIndex = this.timeIndex; + cloned.maxPlanIndex = this.maxPlanIndex; + cloned.minPlanIndex = this.minPlanIndex; + cloned.exclusiveModFileFuture = this.exclusiveModFileFuture; + cloned.sharedModFilePathFuture = this.sharedModFilePathFuture; + cloned.modFileManagement = this.modFileManagement; + cloned.exclusiveModFile = this.exclusiveModFile; + cloned.sharedModFile = this.sharedModFile; + cloned.sharedModFileOffset = this.sharedModFileOffset; + cloned.compactionModFile = this.compactionModFile; + cloned.isSeq = this.isSeq; + cloned.tsFileRepairStatus = this.tsFileRepairStatus; + cloned.settleTsFileCallBack = this.settleTsFileCallBack; + cloned.deviceTimeIndexRamSize = this.deviceTimeIndexRamSize; + cloned.tsFileSize = this.tsFileSize; + cloned.processor = this.processor; + cloned.originTsFileResource = this.originTsFileResource; + cloned.isGeneratedByPipeConsensus = this.isGeneratedByPipeConsensus; + cloned.isGeneratedByPipe = this.isGeneratedByPipe; + cloned.insertionCompactionCandidateStatus = this.insertionCompactionCandidateStatus; + cloned.tierLevel = this.tierLevel; + cloned.pathToChunkMetadataListMap = this.pathToChunkMetadataListMap; + cloned.pathToReadOnlyMemChunkMap = this.pathToReadOnlyMemChunkMap; + cloned.pathToTimeSeriesMetadataMap = this.pathToTimeSeriesMetadataMap; + cloned.lastValues = this.lastValues; + cloned.maxProgressIndex.set(this.maxProgressIndex.get()); + cloned.atomicStatus.set(this.atomicStatus.get()); + cloned.isEmpty.set(this.isEmpty.get()); + cloned.tsFileID = this.tsFileID; + cloned.prev = null; + cloned.next = null; + return cloned; + } + + public TsFileResource shallowCloneForNative() throws CloneNotSupportedException { + return (TsFileResource) clone(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/disk/MinIOSelector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/disk/MinIOSelector.java index 9956148b0835d..7ec8300058a9c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/disk/MinIOSelector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/disk/MinIOSelector.java @@ -81,10 +81,12 @@ public File selectTargetDirectory( throws DiskSpaceInsufficientException, LoadFileException { String fileDirRoot = null; try { - fileDirRoot = - Optional.ofNullable(FileStoreUtils.getFileStore(sourceDirectory.getCanonicalPath())) - .map(Object::toString) - .orElse(null); + if (sourceDirectory != null) { + fileDirRoot = + Optional.ofNullable(FileStoreUtils.getFileStore(sourceDirectory.getCanonicalPath())) + .map(Object::toString) + .orElse(null); + } } catch (Exception e) { logger.warn( "Exception occurs when reading target file's mount point {}", From e72ee9a247a3e72b8baf3cff579bfc4829017997 Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Thu, 20 Nov 2025 19:02:45 +0800 Subject: [PATCH 072/180] Load: Fixed parameter passing error in MoveFile function (#16748) * Load: Fixed parameter passing error in MoveFile function * update * add ut * update (cherry picked from commit b3509d6d6bf16203137bca4cb4b204e7cbd14654) --- .../apache/iotdb/commons/utils/FileUtils.java | 92 +++++++++++-------- .../iotdb/commons/utils/FileUtilsTest.java | 79 ++++++++++++++++ 2 files changed, 132 insertions(+), 39 deletions(-) create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/FileUtilsTest.java diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java index 5775eb658535a..baed8bcd537e7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java @@ -418,7 +418,7 @@ public static void moveFileWithMD5Check(final File sourceFile, final File target final String sourceFileName = sourceFile.getName(); final File targetFile = new File(targetDir, sourceFileName); if (targetFile.exists()) { - moveFile(sourceFile, targetFile); + moveFile(sourceFile, targetDir); } else { org.apache.tsfile.external.commons.io.FileUtils.moveFileToDirectory( sourceFile, targetDir, true); @@ -428,31 +428,37 @@ public static void moveFileWithMD5Check(final File sourceFile, final File target private static void moveFile(File sourceFile, File targetDir) throws IOException { String sourceFileName = sourceFile.getName(); final File exitsFile = new File(targetDir, sourceFileName); - try (final FileInputStream is1 = new FileInputStream(sourceFile); - final FileInputStream is2 = new FileInputStream(exitsFile); ) { - long sourceFileSize = is1.getChannel().size(); - long exitsFileSize = is2.getChannel().size(); - if (sourceFileSize != exitsFileSize) { - File file = renameWithSize(sourceFile, sourceFileSize, targetDir); - if (!file.exists()) { - moveFileRename(sourceFile, file); - return; - } - } - String sourceFileMD5 = DigestUtils.md5Hex(is1); - String exitsFileMD5 = DigestUtils.md5Hex(is2); + // First check file sizes + long sourceFileSize = sourceFile.length(); + long existsFileSize = exitsFile.length(); - if (sourceFileMD5.equals(exitsFileMD5)) { - org.apache.tsfile.external.commons.io.FileUtils.forceDelete(sourceFile); - LOGGER.info( - "Deleted the file {} because it already exists in the target directory: {}", - sourceFile.getName(), - targetDir.getAbsolutePath()); - } else { - File file = renameWithMD5(sourceFile, sourceFileMD5, targetDir); + if (sourceFileSize != existsFileSize) { + File file = renameWithSize(sourceFile, sourceFileSize, targetDir); + if (!file.exists()) { moveFileRename(sourceFile, file); } + return; + } + + // If sizes are equal, check MD5 + String sourceFileMD5; + String existsFileMD5; + try (final FileInputStream is1 = new FileInputStream(sourceFile); + final FileInputStream is2 = new FileInputStream(exitsFile); ) { + sourceFileMD5 = DigestUtils.md5Hex(is1); + existsFileMD5 = DigestUtils.md5Hex(is2); + } + + if (sourceFileMD5.equals(existsFileMD5)) { + org.apache.tsfile.external.commons.io.FileUtils.forceDelete(sourceFile); + LOGGER.info( + "Deleted the file {} because it already exists in the target directory: {}", + sourceFile.getName(), + targetDir.getAbsolutePath()); + } else { + File file = renameWithMD5(sourceFile, sourceFileMD5, targetDir); + moveFileRename(sourceFile, file); } } @@ -492,27 +498,35 @@ private static void copyFileWithMD5(final File sourceFile, final File targetDir) throws IOException { String sourceFileName = sourceFile.getName(); final File exitsFile = new File(targetDir, sourceFileName); - try (final FileInputStream is1 = new FileInputStream(sourceFile); - final FileInputStream is2 = new FileInputStream(exitsFile); ) { - long sourceFileSize = is1.getChannel().size(); - long exitsFileSize = is2.getChannel().size(); - if (sourceFileSize != exitsFileSize) { - File file = renameWithSize(sourceFile, sourceFileSize, targetDir); - if (!file.exists()) { - copyFileRename(sourceFile, file); - return; - } - } - String sourceFileMD5 = DigestUtils.md5Hex(is1); - String exitsFileMD5 = DigestUtils.md5Hex(is2); - if (sourceFileMD5.equals(exitsFileMD5)) { - return; - } - File file = renameWithMD5(sourceFile, sourceFileMD5, targetDir); + // First check file sizes + long sourceFileSize = sourceFile.length(); + long exitsFileSize = exitsFile.length(); + + if (sourceFileSize != exitsFileSize) { + File file = renameWithSize(sourceFile, sourceFileSize, targetDir); if (!file.exists()) { copyFileRename(sourceFile, file); } + return; + } + + // If sizes are equal, check MD5 + String sourceFileMD5; + String exitsFileMD5; + try (final FileInputStream is1 = new FileInputStream(sourceFile); + final FileInputStream is2 = new FileInputStream(exitsFile); ) { + sourceFileMD5 = DigestUtils.md5Hex(is1); + exitsFileMD5 = DigestUtils.md5Hex(is2); + } + + if (sourceFileMD5.equals(exitsFileMD5)) { + return; + } + + File file = renameWithMD5(sourceFile, sourceFileMD5, targetDir); + if (!file.exists()) { + copyFileRename(sourceFile, file); } } diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/FileUtilsTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/FileUtilsTest.java new file mode 100644 index 0000000000000..e25755f4c17ac --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/FileUtilsTest.java @@ -0,0 +1,79 @@ +/* + * 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.commons.utils; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.write.TsFileWriter; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Collections; + +public class FileUtilsTest { + private File tmpDir; + private File targetDir; + + @Before + public void setUp() throws Exception { + tmpDir = new File(Files.createTempDirectory("load").toUri()); + targetDir = new File(Files.createTempDirectory("target").toUri()); + } + + @After + public void tearDown() throws Exception { + tmpDir.delete(); + targetDir.delete(); + } + + @Test + public void testFileUtils() throws WriteProcessException, IOException { + File tstFile = new File(tmpDir, "1-1-0-0.tsfile"); + File tstFile2 = new File(tmpDir, "2-1-0-0.tsfile"); + generateFile(tstFile); + FileUtils.copyFile(tstFile, tstFile2); + FileUtils.moveFileWithMD5Check(tstFile, targetDir); + tstFile2.renameTo(tstFile); + FileUtils.moveFileWithMD5Check(tstFile, targetDir); + } + + private void generateFile(File tsfile) throws WriteProcessException, IOException { + try (TsFileWriter writer = new TsFileWriter(tsfile)) { + writer.registerAlignedTimeseries( + "root.test.d1", + Collections.singletonList(new MeasurementSchema("s1", TSDataType.BOOLEAN))); + Tablet tablet = + new Tablet( + "root.test.d1", + Collections.singletonList(new MeasurementSchema("s1", TSDataType.BOOLEAN))); + for (int i = 0; i < 5; i++) { + tablet.addTimestamp(i, i); + tablet.addValue(i, 0, true); + } + writer.writeTree(tablet); + } + } +} From 40355e34103a8166a61a8625ab9ce2c09366c437 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <78788603+Pengzna@users.noreply.github.com> Date: Thu, 20 Nov 2025 05:37:40 -0600 Subject: [PATCH 073/180] Refine CN consensus layer API for procedure robustness (#16303) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * refine cn consensus layer API for procedure * refine and fix the whole logic * fix comment * fix comment --------- Co-authored-by: 彭俊植 (cherry picked from commit 0e209e73c21137be44cebe3926c7f1fed62a0422) --- .../procedure/CompletedProcedureRecycler.java | 21 ++++- .../iotdb/confignode/procedure/Procedure.java | 27 ++++-- .../procedure/ProcedureExecutor.java | 88 +++++++++++++++---- .../procedure/TimeoutExecutorThread.java | 12 ++- .../procedure/store/ConfigProcedureStore.java | 26 ++++-- .../procedure/store/IProcedureStore.java | 10 +-- .../iotdb/commons/utils/RetryUtils.java | 59 ++++++++++++- 7 files changed, 203 insertions(+), 40 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/CompletedProcedureRecycler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/CompletedProcedureRecycler.java index cea15a1bfb687..179563cc3b3e5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/CompletedProcedureRecycler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/CompletedProcedureRecycler.java @@ -70,15 +70,30 @@ protected void periodicExecute(final Env env) { // Failed procedures aren't persisted in WAL. batchIds[batchCount++] = entry.getKey(); if (batchCount == batchIds.length) { - store.delete(batchIds, 0, batchCount); - batchCount = 0; + try { + store.delete(batchIds, 0, batchCount); + } catch (Exception e) { + LOG.error("Error deleting completed procedures {}.", proc, e); + // Do not remove from the completed map. Even this procedure may be restored + // unexpectedly in another new CN leader, we do not need to do anything else since + // procedures are idempotent. + continue; + } finally { + batchCount = 0; + } } it.remove(); LOG.trace("Evict completed {}", proc); } } if (batchCount > 0) { - store.delete(batchIds, 0, batchCount); + try { + store.delete(batchIds, 0, batchCount); + } catch (Exception e) { + // Even this procedure may be restored unexpectedly in another new CN leader, we do not need + // to do anything else since procedures are idempotent. + LOG.error("Error deleting completed procedures {}.", batchIds, e); + } } } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/Procedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/Procedure.java index 91af03d3971be..89e6e37e431a0 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/Procedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/Procedure.java @@ -24,6 +24,7 @@ import org.apache.iotdb.confignode.procedure.state.ProcedureLockState; import org.apache.iotdb.confignode.procedure.state.ProcedureState; import org.apache.iotdb.confignode.procedure.store.IProcedureStore; +import org.apache.iotdb.consensus.exception.ConsensusException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -196,7 +197,7 @@ public void deserialize(ByteBuffer byteBuffer) { byteBuffer.get(resultArr); } // has lock - if (byteBuffer.get() == 1) { + if (byteBuffer.get() == 1 && this.state != ProcedureState.ROLLEDBACK) { this.lockedWhenLoading(); } } @@ -300,8 +301,15 @@ public final ProcedureLockState doAcquireLock(Env env, IProcedureStore store) { } ProcedureLockState state = acquireLock(env); if (state == ProcedureLockState.LOCK_ACQUIRED) { - locked = true; - store.update(this); + try { + locked = true; + store.update(this); + } catch (Exception e) { + // Do not need to do anything else. New leader which restore this procedure from a wrong + // state will reexecute it and converge to the correct state since procedures are + // idempotent. + LOG.warn("pid={} Failed to persist lock state to store.", this.procId, e); + } } return state; } @@ -312,12 +320,19 @@ public final ProcedureLockState doAcquireLock(Env env, IProcedureStore store) { * @param env environment * @param store ProcedureStore */ - public final void doReleaseLock(Env env, IProcedureStore store) { + public final void doReleaseLock(Env env, IProcedureStore store) throws Exception { locked = false; - if (getState() != ProcedureState.ROLLEDBACK) { + if (getState() == ProcedureState.ROLLEDBACK) { + LOG.info("Force write unlock state to raft for pid={}", this.procId); + } + try { store.update(this); + // do not release lock when consensus layer is not working + releaseLock(env); + } catch (ConsensusException e) { + LOG.error("pid={} Failed to persist unlock state to store.", this.procId, e); + throw e; } - releaseLock(env); } public final void restoreLock(Env env) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/ProcedureExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/ProcedureExecutor.java index 0d8368583b4eb..efd37778efa4f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/ProcedureExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/ProcedureExecutor.java @@ -20,6 +20,7 @@ package org.apache.iotdb.confignode.procedure; import org.apache.iotdb.commons.concurrent.ThreadName; +import org.apache.iotdb.commons.utils.RetryUtils; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; @@ -221,7 +222,8 @@ Long getRootProcedureId(Procedure proc) { private void releaseLock(Procedure procedure, boolean force) { if (force || !procedure.holdLock(this.environment) || procedure.isFinished()) { - procedure.doReleaseLock(this.environment, store); + RetryUtils.executeWithEndlessBackoffRetry( + () -> procedure.doReleaseLock(this.environment, store), "procedure release lock"); } } @@ -477,7 +479,11 @@ private void countDownChildren(RootProcedureStack rootProcStack, Procedure } if (parent != null && parent.tryRunnable()) { // If success, means all its children have completed, move parent to front of the queue. - store.update(parent); + // Must endless retry here, since this step is not idempotent and can not be re-execute + // correctly in new CN leader. + RetryUtils.executeWithEndlessBackoffRetry( + () -> store.update(parent), "count down children procedure"); + // do not add this procedure when exception occurred scheduler.addFront(parent); LOG.info( "Finished subprocedure pid={}, resume processing ppid={}", @@ -506,21 +512,44 @@ private void updateStoreOnExecution( if (LOG.isDebugEnabled()) { LOG.debug("Stored {}, children {}", proc, Arrays.toString(subprocs)); } - store.update(subprocs); + try { + store.update(subprocs); + } catch (Exception e) { + // Do nothing since this step is idempotent. New CN leader can converge to the correct + // state when restore this procedure. + LOG.warn("Failed to update subprocs on execution", e); + } } else { LOG.debug("Store update {}", proc); if (proc.isFinished() && !proc.hasParent()) { final long[] childProcIds = rootProcStack.getSubprocedureIds(); if (childProcIds != null) { - store.delete(childProcIds); - for (long childProcId : childProcIds) { - procedures.remove(childProcId); + try { + store.delete(childProcIds); + // do not remove these procedures when exception occurred + for (long childProcId : childProcIds) { + procedures.remove(childProcId); + } + } catch (Exception e) { + // Do nothing since this step is idempotent. New CN leader can converge to the correct + // state when restore this procedure. + LOG.warn("Failed to delete subprocedures on execution", e); } } else { - store.update(proc); + try { + store.update(proc); + } catch (Exception e) { + LOG.warn("Failed to update procedure on execution", e); + } } } else { - store.update(proc); + try { + store.update(proc); + } catch (Exception e) { + // Do nothing since this step is idempotent. New CN leader can converge to the correct + // state when restore this procedure. + LOG.warn("Failed to update procedure on execution", e); + } } } } @@ -577,7 +606,9 @@ private ProcedureLockState executeRootStackRollback( if (exception == null) { exception = procedureStack.getException(); rootProcedure.setFailure(exception); - store.update(rootProcedure); + // Endless retry since this step is not idempotent. + RetryUtils.executeWithEndlessBackoffRetry( + () -> store.update(rootProcedure), "root procedure rollback"); } List> subprocStack = procedureStack.getSubproceduresStack(); int stackTail = subprocStack.size(); @@ -653,18 +684,37 @@ private void cleanupAfterRollback(Procedure procedure) { procedure.updateMetricsOnFinish(getEnvironment(), procedure.elapsedTime(), false); if (procedure.hasParent()) { - store.delete(procedure.getProcId()); - procedures.remove(procedure.getProcId()); + try { + store.delete(procedure.getProcId()); + // do not remove this procedure when exception occurred + procedures.remove(procedure.getProcId()); + } catch (Exception e) { + // Do nothing since this step is idempotent. New CN leader can converge to the correct + // state when restore this procedure. + LOG.warn("Failed to delete procedure on rollback", e); + } } else { final long[] childProcIds = rollbackStack.get(procedure.getProcId()).getSubprocedureIds(); - if (childProcIds != null) { - store.delete(childProcIds); - } else { - store.update(procedure); + try { + if (childProcIds != null) { + store.delete(childProcIds); + } else { + store.update(procedure); + } + } catch (Exception e) { + // Do nothing since this step is idempotent. New CN leader can converge to the correct + // state when restore this procedure. + LOG.warn("Failed to delete procedure on rollback", e); } } } else { - store.update(procedure); + try { + store.update(procedure); + } catch (Exception e) { + // Do nothing since this step is idempotent. New CN leader can converge to the correct + // state when restore this procedure. + LOG.warn("Failed to update procedure on rollback", e); + } } } @@ -916,7 +966,11 @@ public long submitProcedure(Procedure procedure) { procedure.setProcId(store.getNextProcId()); procedure.setProcRunnable(); // Commit the transaction - store.update(procedure); + try { + store.update(procedure); + } catch (Exception e) { + LOG.error("Failed to update store procedure {}", procedure, e); + } LOG.debug("{} is stored.", procedure); // Add the procedure to the executor return pushProcedure(procedure); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/TimeoutExecutorThread.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/TimeoutExecutorThread.java index 5aaf9a623f523..1614148abd308 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/TimeoutExecutorThread.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/TimeoutExecutorThread.java @@ -19,12 +19,16 @@ package org.apache.iotdb.confignode.procedure; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.concurrent.DelayQueue; import java.util.concurrent.Delayed; import java.util.concurrent.TimeUnit; public class TimeoutExecutorThread extends StoppableThread { + private static final Logger LOGGER = LoggerFactory.getLogger(TimeoutExecutorThread.class); private static final int DELAY_QUEUE_TIMEOUT = 20; private final ProcedureExecutor executor; private final DelayQueue> queue = new DelayQueue<>(); @@ -71,7 +75,13 @@ public void run() { long rootProcId = executor.getRootProcedureId(procedure); RootProcedureStack rollbackStack = executor.getRollbackStack(rootProcId); rollbackStack.abort(); - executor.getStore().update(procedure); + try { + executor.getStore().update(procedure); + } catch (Exception e) { + // Do nothing since new CN leader can converge to the correct state when restore this + // procedure. + LOGGER.warn("Failed to update procedure {}", procedure, e); + } executor.getScheduler().addFront(procedure); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ConfigProcedureStore.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ConfigProcedureStore.java index 393c1e93740eb..603a9a8a627d5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ConfigProcedureStore.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ConfigProcedureStore.java @@ -89,43 +89,55 @@ public long getNextProcId() { } @Override - public void update(Procedure procedure) { + public void update(Procedure procedure) throws Exception { Objects.requireNonNull(ProcedureFactory.getProcedureType(procedure), "Procedure type is null"); final UpdateProcedurePlan updateProcedurePlan = new UpdateProcedurePlan(procedure); try { configManager.getConsensusManager().write(updateProcedurePlan); } catch (ConsensusException e) { - LOG.warn("Failed in the write API executing the consensus layer due to: ", e); + LOG.warn( + "pid={} Failed in the write update API executing the consensus layer due to: ", + procedure.getProcId(), + e); + // In consensus layer API, do nothing but just throw an exception to let upper caller handle + // it. + throw e; } } @Override - public void update(Procedure[] subprocs) { + public void update(Procedure[] subprocs) throws Exception { for (Procedure subproc : subprocs) { update(subproc); } } @Override - public void delete(long procId) { + public void delete(long procId) throws Exception { DeleteProcedurePlan deleteProcedurePlan = new DeleteProcedurePlan(); deleteProcedurePlan.setProcId(procId); try { configManager.getConsensusManager().write(deleteProcedurePlan); } catch (ConsensusException e) { - LOG.warn("Failed in the write API executing the consensus layer due to: ", e); + LOG.warn( + "pid={} Failed in the write delete API executing the consensus layer due to: ", + procId, + e); + // In consensus layer API, do nothing but just throw an exception to let upper caller handle + // it. + throw e; } } @Override - public void delete(long[] childProcIds) { + public void delete(long[] childProcIds) throws Exception { for (long childProcId : childProcIds) { delete(childProcId); } } @Override - public void delete(long[] batchIds, int startIndex, int batchCount) { + public void delete(long[] batchIds, int startIndex, int batchCount) throws Exception { for (int i = startIndex; i < batchCount; i++) { delete(batchIds[i]); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/IProcedureStore.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/IProcedureStore.java index 8e8e715fd84fd..3dba6d29288d9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/IProcedureStore.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/IProcedureStore.java @@ -38,15 +38,15 @@ public interface IProcedureStore { long getNextProcId(); - void update(Procedure procedure); + void update(Procedure procedure) throws Exception; - void update(Procedure[] subprocs); + void update(Procedure[] subprocs) throws Exception; - void delete(long procId); + void delete(long procId) throws Exception; - void delete(long[] childProcIds); + void delete(long[] childProcIds) throws Exception; - void delete(long[] batchIds, int startIndex, int batchCount); + void delete(long[] batchIds, int startIndex, int batchCount) throws Exception; void cleanup(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/RetryUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/RetryUtils.java index cff60601edb3f..9b297cae0536c 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/RetryUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/RetryUtils.java @@ -22,10 +22,15 @@ import org.apache.iotdb.commons.exception.pipe.PipeConsensusRetryWithIncreasingIntervalException; import org.apache.iotdb.rpc.TSStatusCode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.net.ConnectException; public class RetryUtils { + private static final Logger LOGGER = LoggerFactory.getLogger(RetryUtils.class); + public interface CallableWithException { T call() throws E; } @@ -46,7 +51,7 @@ public static boolean notNeedRetryForConsensus(int statusCode) { return statusCode == TSStatusCode.PIPE_CONSENSUS_DEPRECATED_REQUEST.getStatusCode(); } - public static final int MAX_RETRIES = 3; + public static final int MAX_RETRIES = 5; public static T retryOnException( final CallableWithException callable) throws E { @@ -63,6 +68,58 @@ public static T retryOnException( } } + private static final long INITIAL_BACKOFF_MS = 100; + private static final long MAX_BACKOFF_MS = 60000; + + @FunctionalInterface + public interface OperationWithException { + void run() throws Exception; + } + + /** + * Exponential backoff retry helper method. + * + * @param operation The operation to execute. + * @param operationName A description of the operation (for logging). + */ + public static void executeWithEndlessBackoffRetry( + OperationWithException operation, String operationName) { + long currentBackoff = INITIAL_BACKOFF_MS; + int attempt = 0; + + // Endless retry + while (true) { + attempt++; + try { + operation.run(); + if (attempt > 1) { + LOGGER.info("Operation '{}' succeeded after {} attempts", operationName, attempt); + } + return; + } catch (Exception e) { + LOGGER.warn( + "Operation '{}' failed (attempt {}). Retrying in {}ms...", + operationName, + attempt, + currentBackoff, + e); + try { + Thread.sleep(currentBackoff); + } catch (InterruptedException ie) { + LOGGER.warn( + "Retry wait for operation '{}' was interrupted, stopping retries.", + operationName, + ie); + Thread.currentThread().interrupt(); + return; + } + + // Double the backoff, but cap it at the max to prevent overflow + currentBackoff = Math.min(currentBackoff * 2, MAX_BACKOFF_MS); + } + } + } + private RetryUtils() { // utility class } From b7e1c02338f5cae13ac8f49e0fc33a5453120cba Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 21 Nov 2025 09:18:06 +0800 Subject: [PATCH 074/180] Pipe: improved the memory calculation for tablets (#16739) * fix * fix * opti * fix * fix * fix function name * fix * fix * fix ut --------- Co-authored-by: Tian Jiang (cherry picked from commit 0ebdb878e57bc1754bbbcf8c59790919e6d256b4) --- .../it/session/IoTDBSessionRelationalIT.java | 2 + .../memory/InsertNodeMemoryEstimator.java | 55 +++------------- .../resource/memory/PipeMemoryWeightUtil.java | 62 +------------------ .../statement/crud/InsertBaseStatement.java | 2 +- .../statement/crud/InsertTabletStatement.java | 2 +- .../iotdb/db/utils/datastructure/TVList.java | 2 +- pom.xml | 2 +- 7 files changed, 16 insertions(+), 111 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/session/IoTDBSessionRelationalIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/session/IoTDBSessionRelationalIT.java index bba5681b58bb0..5a26ca510a064 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/session/IoTDBSessionRelationalIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/session/IoTDBSessionRelationalIT.java @@ -1812,6 +1812,7 @@ public void insertRelationalTabletWithAutoCastTest() dataTypes.remove(TSDataType.OBJECT); dataTypes.remove(TSDataType.VECTOR); dataTypes.remove(TSDataType.UNKNOWN); + dataTypes.remove(TSDataType.OBJECT); try { for (TSDataType from : dataTypes) { @@ -1903,6 +1904,7 @@ public void insertRelationalRowWithAutoCastTest() dataTypes.remove(TSDataType.OBJECT); dataTypes.remove(TSDataType.VECTOR); dataTypes.remove(TSDataType.UNKNOWN); + dataTypes.remove(TSDataType.OBJECT); for (TSDataType from : dataTypes) { for (TSDataType to : dataTypes) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java index 60076c6139e34..7f1d7357b02e4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java @@ -189,7 +189,7 @@ private static long calculateFullInsertNodeSize(final InsertNode node) { // MeasurementSchemas size += sizeOfMeasurementSchemas(node.getMeasurementSchemas()); // Measurement - size += sizeOfStringArray(node.getMeasurements()); + size += RamUsageEstimator.sizeOf(node.getMeasurements()); // dataTypes size += RamUsageEstimator.shallowSizeOf(node.getDataTypes()); // columnCategories @@ -231,7 +231,7 @@ private static long sizeOfInsertTabletNode(final InsertTabletNode node) { long size = INSERT_TABLET_NODE_SIZE; size += calculateFullInsertNodeSize(node); size += RamUsageEstimator.sizeOf(node.getTimes()); - size += sizeOfBitMapArray(node.getBitMaps()); + size += RamUsageEstimator.sizeOf(node.getBitMaps()); size += sizeOfColumns(node.getColumns(), node.getMeasurementSchemas()); final List range = node.getRange(); if (range != null) { @@ -247,7 +247,7 @@ private static long calculateInsertTabletNodeSizeExcludingSchemas(final InsertTa size += RamUsageEstimator.sizeOf(node.getTimes()); - size += sizeOfBitMapArray(node.getBitMaps()); + size += RamUsageEstimator.sizeOf(node.getBitMaps()); size += sizeOfColumns(node.getColumns(), node.getMeasurementSchemas()); @@ -390,7 +390,7 @@ private static long sizeOfRelationalInsertTabletNode(final RelationalInsertTable size += RamUsageEstimator.sizeOf(node.getTimes()); - size += sizeOfBitMapArray(node.getBitMaps()); + size += RamUsageEstimator.sizeOf(node.getBitMaps()); size += sizeOfColumns(node.getColumns(), node.getMeasurementSchemas()); @@ -413,7 +413,7 @@ public static long sizeOfPartialPath(final PartialPath partialPath) { if (nodes != null) { // Since fullPath may be lazy loaded, lazy loading will not be triggered here, so it is // assumed that the memory size of fullPath is the same as that of nodes. - size += sizeOfStringArray(nodes) * 2; + size += RamUsageEstimator.sizeOf(nodes) * 2; size += TsFileConstant.PATH_SEPARATOR.length() * (nodes.length - 1) + NUM_BYTES_OBJECT_HEADER; } return size; @@ -555,34 +555,6 @@ private static long sizeOfBinary(final Binary binary) { return Objects.nonNull(binary) ? binary.ramBytesUsed() : 0L; } - public static long sizeOfStringArray(final String[] values) { - return Objects.nonNull(values) ? RamUsageEstimator.sizeOf(values) : 0L; - } - - public static long sizeOfBitMapArray(BitMap[] bitMaps) { - if (bitMaps == null) { - return 0L; - } - long size = - RamUsageEstimator.alignObjectSize( - NUM_BYTES_ARRAY_HEADER + NUM_BYTES_OBJECT_REF * bitMaps.length); - for (BitMap bitMap : bitMaps) { - size += sizeOfBitMap(bitMap); - } - return size; - } - - private static long sizeOfBitMap(final BitMap bitMaps) { - if (bitMaps == null) { - return 0L; - } - long size = BIT_MAP_SIZE; - - size += - RamUsageEstimator.alignObjectSize(NUM_BYTES_ARRAY_HEADER + bitMaps.getByteArray().length); - return size; - } - public static long sizeOfColumns( final Object[] columns, final MeasurementSchema[] measurementSchemas) { // Directly calculate if measurementSchemas are absent @@ -632,7 +604,7 @@ public static long sizeOfColumns( case BLOB: case OBJECT: { - size += getBinarySize((Binary[]) columns[i]); + size += RamUsageEstimator.sizeOf((Binary[]) columns[i]); break; } } @@ -642,23 +614,10 @@ public static long sizeOfColumns( private static long getNumBytesUnknownObject(final Object obj) { return obj instanceof Binary[] - ? getBinarySize((Binary[]) obj) + ? RamUsageEstimator.sizeOf((Binary[]) obj) : RamUsageEstimator.sizeOfObject(obj); } - private static long getBinarySize(final Binary[] binaries) { - if (binaries == null) { - return 0L; - } - - long size = 0L; - for (Binary binary : binaries) { - size += InsertNodeMemoryEstimator.sizeOfBinary(binary); - } - - return size + RamUsageEstimator.shallowSizeOf(binaries); - } - public static long sizeOfValues( final Object[] values, final MeasurementSchema[] measurementSchemas) { // Directly calculate if measurementSchemas are absent 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 30b68e4174158..0ba204fcbd2b1 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 @@ -35,7 +35,6 @@ 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.BitMap; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.TsPrimitiveType; import org.apache.tsfile.write.record.Tablet; @@ -44,6 +43,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.tsfile.utils.RamUsageEstimator.NUM_BYTES_ARRAY_HEADER; import static org.apache.tsfile.utils.RamUsageEstimator.NUM_BYTES_OBJECT_REF; @@ -203,64 +203,8 @@ private static Pair calculateTabletRowCountAndMemoryBySize( } } - public static long calculateTabletSizeInBytes(Tablet tablet) { - long totalSizeInBytes = 0; - - if (tablet == null) { - return totalSizeInBytes; - } - - long[] timestamps = tablet.getTimestamps(); - Object[] tabletValues = tablet.getValues(); - - // timestamps - if (timestamps != null) { - totalSizeInBytes += timestamps.length * 8L; - } - - // values - final List timeseries = tablet.getSchemas(); - if (timeseries != null) { - for (int column = 0; column < timeseries.size(); column++) { - final IMeasurementSchema measurementSchema = timeseries.get(column); - if (measurementSchema == null) { - continue; - } - - final TSDataType tsDataType = measurementSchema.getType(); - if (tsDataType == null) { - continue; - } - - if (tsDataType.isBinary()) { - if (tabletValues == null || tabletValues.length <= column) { - continue; - } - final Binary[] values = ((Binary[]) tabletValues[column]); - if (values == null) { - continue; - } - for (Binary value : values) { - totalSizeInBytes += value == null ? 8 : value.ramBytesUsed(); - } - } else { - totalSizeInBytes += (long) tablet.getMaxRowNumber() * tsDataType.getDataTypeSize(); - } - } - } - - // bitMaps - BitMap[] bitMaps = tablet.getBitMaps(); - if (bitMaps != null) { - for (int i = 0; i < bitMaps.length; i++) { - totalSizeInBytes += bitMaps[i] == null ? 0 : bitMaps[i].getSize(); - } - } - - // estimate other dataStructures size - totalSizeInBytes += 100; - - return totalSizeInBytes; + public static long calculateTabletSizeInBytes(final Tablet tablet) { + return Objects.nonNull(tablet) ? tablet.ramBytesUsed() : 0L; } public static long calculateTableSchemaBytesUsed(TableSchema tableSchema) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java index 177c05bde1927..f115bde9d2d29 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java @@ -686,7 +686,7 @@ public long ramBytesUsed() { ramBytesUsed = InsertNodeMemoryEstimator.sizeOfPartialPath(devicePath) + InsertNodeMemoryEstimator.sizeOfMeasurementSchemas(measurementSchemas) - + InsertNodeMemoryEstimator.sizeOfStringArray(measurements) + + RamUsageEstimator.sizeOf(measurements) + RamUsageEstimator.shallowSizeOf(dataTypes) + RamUsageEstimator.shallowSizeOf(columnCategories) // We assume that the integers are all cached by JVM 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 efb4746b9b3d9..d3257d98d41ff 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 @@ -615,7 +615,7 @@ public void swapColumn(int src, int target) { protected long calculateBytesUsed() { return INSTANCE_SIZE + RamUsageEstimator.sizeOf(times) - + InsertNodeMemoryEstimator.sizeOfBitMapArray(nullBitMaps) + + RamUsageEstimator.sizeOf(nullBitMaps) + InsertNodeMemoryEstimator.sizeOfColumns(columns, measurementSchemas) + (Objects.nonNull(deviceIDs) ? Arrays.stream(deviceIDs) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index e20575be36278..a1f2842d34a9f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -973,7 +973,7 @@ public TsBlock nextBatch() { TSDataType dataType = getDataType(); int maxRowCountOfCurrentBatch = Math.min( - paginationController.hasLimit() + paginationController.hasSetLimit() ? (int) paginationController.getCurLimit() : Integer.MAX_VALUE, Math.min(maxNumberOfPointsInPage, rows - index)); diff --git a/pom.xml b/pom.xml index b997a87580c2e..71e0f017c5df2 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 0.14.1 1.9 1.5.6-3 - 2.2.0-251030-SNAPSHOT + 2.2.0-251113-SNAPSHOT 1.3.16 - 1.8.0 + 1.10.0 3.6.0 1.8 1.8 @@ -320,6 +320,18 @@ bcpkix-jdk18on ${bouncycastle.version} + + org.apache.tsfile + tsfile + ${tsfile.version} + + + + org.lz4 + lz4-java + + + org.apache.ratis ratis-server @@ -492,7 +504,7 @@ ${zstd-jni.version} - org.lz4 + at.yawk.lz4 lz4-java ${lz4-java.version} From c646f3b636ad18625a133fc996633b63b0950dc7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 6 Dec 2025 20:55:37 +0800 Subject: [PATCH 115/180] Bump at.yawk.lz4:lz4-java from 1.10.0 to 1.10.1 (#16874) Bumps [at.yawk.lz4:lz4-java](https://github.com/yawkat/lz4-java) from 1.10.0 to 1.10.1. - [Release notes](https://github.com/yawkat/lz4-java/releases) - [Changelog](https://github.com/yawkat/lz4-java/blob/main/CHANGES.md) - [Commits](https://github.com/yawkat/lz4-java/compare/v1.10.0...v1.10.1) --- updated-dependencies: - dependency-name: at.yawk.lz4:lz4-java dependency-version: 1.10.1 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> (cherry picked from commit 0bb9e96ece7f1bc2b08541f773ea346186e66e01) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index eecb8978762c8..4ba65896c97f2 100644 --- a/pom.xml +++ b/pom.xml @@ -115,7 +115,7 @@ 4.13.2 1.3.16 - 1.10.0 + 1.10.1 3.6.0 1.8 1.8 From aff27d12221ce0d9a7b801631bdc0d4f88c384c5 Mon Sep 17 00:00:00 2001 From: JackieTien97 Date: Mon, 8 Dec 2025 11:17:59 +0800 Subject: [PATCH 116/180] Bump tsfile version to 2.2.0-251208-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4ba65896c97f2..6841b71697820 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 0.14.1 1.9 1.5.6-3 - 2.2.0-251113-SNAPSHOT + 2.2.0-251208-SNAPSHOT