From 3315146bca8574720bf6c84aed54056265762c21 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Thu, 28 May 2026 18:46:55 +0800 Subject: [PATCH 01/32] plan --- .../DataNodeTableOperatorGenerator.java | 8 + .../node/DataNodePlanNodeDeserializer.java | 3 + .../plan/planner/plan/node/PlanVisitor.java | 5 + .../relational/planner/RelationPlanner.java | 65 +++ .../TableDistributedPlanGenerator.java | 11 + .../planner/node/ExternalTsFileScanNode.java | 146 ++++++ .../UnaliasSymbolReferences.java | 30 ++ .../plan/planner/plan/node/PlanNodeType.java | 1 + .../function/TableBuiltinTableFunction.java | 6 +- .../tvf/ReadTsFileTableFunction.java | 478 ++++++++++++++++++ 10 files changed, 752 insertions(+), 1 deletion(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index b5de9a5561e23..81bbfa60d131e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -122,6 +122,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExchangeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; @@ -1116,6 +1117,13 @@ public Operator visitDeviceTableScan( return tableScanOperator; } + @Override + public Operator visitExternalTsFileScan( + ExternalTsFileScanNode node, LocalExecutionPlanContext context) { + throw new UnsupportedOperationException( + "ExternalTsFileScanNode physical operator is not implemented yet"); + } + private SeriesScanOptions.Builder getSeriesScanOptionsBuilder( LocalExecutionPlanContext context, @NotNull Expression timePredicate) { SeriesScanOptions.Builder scanOptionsBuilder = new SeriesScanOptions.Builder(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/DataNodePlanNodeDeserializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/DataNodePlanNodeDeserializer.java index 2a9b9b6905c77..b79011de3ecf6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/DataNodePlanNodeDeserializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/DataNodePlanNodeDeserializer.java @@ -123,6 +123,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertTabletNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AlignedAggregationTreeDeviceViewScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; @@ -469,6 +470,8 @@ public PlanNode deserialize(ByteBuffer buffer, short nodeType) { return AlignedAggregationTreeDeviceViewScanNode.deserialize(buffer); case 1042: return NonAlignedAggregationTreeDeviceViewScanNode.deserialize(buffer); + case 1043: + return ExternalTsFileScanNode.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 54ee9d17a9420..73c41ab1079f8 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 @@ -124,6 +124,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AlignedAggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeAlignedDeviceViewScanNode; @@ -649,6 +650,10 @@ default R visitDeviceTableScan(DeviceTableScanNode node, C context) { return visitTableScan(node, context); } + default R visitExternalTsFileScan(ExternalTsFileScanNode node, C context) { + return visitTableScan(node, context); + } + default R visitInformationSchemaTableScan(InformationSchemaTableScanNode node, C context) { return visitTableScan(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 268cd43452df1..18e2c2d8cb7ba 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 @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.relational.analyzer.NodeRef; +import org.apache.iotdb.commons.queryengine.plan.relational.function.TableBuiltinTableFunction; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Assignments; @@ -88,6 +89,7 @@ import org.apache.iotdb.commons.queryengine.plan.relational.type.InternalTypeManager; import org.apache.iotdb.commons.queryengine.utils.cte.CteDataStore; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction; import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; @@ -120,6 +122,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.PredicateWithUncorrelatedScalarSubqueryReconstructor; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.rowpattern.rowpattern.RowPatternToIrRewriter; @@ -150,6 +153,7 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -1455,6 +1459,11 @@ public RelationPlan visitDelete(final Delete node, final Void context) { @Override public RelationPlan visitTableFunctionInvocation(TableFunctionInvocation node, Void context) { TableFunctionInvocationAnalysis functionAnalysis = analysis.getTableFunctionAnalysis(node); + if (TableBuiltinTableFunction.READ_TSFILE + .getFunctionName() + .equalsIgnoreCase(functionAnalysis.getFunctionName())) { + return planExternalTsFileScan(node, functionAnalysis); + } ImmutableList.Builder sources = ImmutableList.builder(); ImmutableList.Builder sourceProperties = @@ -1583,6 +1592,62 @@ public RelationPlan visitTableFunctionInvocation(TableFunctionInvocation node, V return new RelationPlan(root, analysis.getScope(node), outputSymbols.build(), outerContext); } + private RelationPlan planExternalTsFileScan( + TableFunctionInvocation node, TableFunctionInvocationAnalysis functionAnalysis) { + if (!(functionAnalysis.getTableFunctionHandle() + instanceof ReadTsFileTableFunction.ReadTsFileTableFunctionHandle)) { + throw new IllegalStateException("read_tsfile table function handle is invalid"); + } + + ReadTsFileTableFunction.ReadTsFileTableFunctionHandle handle = + (ReadTsFileTableFunction.ReadTsFileTableFunctionHandle) + functionAnalysis.getTableFunctionHandle(); + Scope scope = analysis.getScope(node); + RelationType relationType = scope.getRelationType(); + + ImmutableList.Builder outputSymbolsBuilder = ImmutableList.builder(); + ImmutableMap.Builder assignmentsBuilder = ImmutableMap.builder(); + for (int i = 0; i < relationType.getAllFieldCount(); i++) { + Field field = relationType.getFieldByIndex(i); + Symbol symbol = symbolAllocator.newSymbol(field); + outputSymbolsBuilder.add(symbol); + assignmentsBuilder.put( + symbol, + new ColumnSchema( + field.getName().orElse(null), + field.getType(), + field.isHidden(), + field.getColumnCategory())); + } + + List outputSymbols = outputSymbolsBuilder.build(); + Map assignments = assignmentsBuilder.build(); + QualifiedObjectName qualifiedObjectName = + createExternalTsFileQualifiedObjectName(handle.getTableName()); + analysis.addTableSchema(qualifiedObjectName, assignments); + + return new RelationPlan( + new ExternalTsFileScanNode( + idAllocator.genPlanNodeId(), + qualifiedObjectName, + outputSymbols, + assignments, + handle.getTsFilePaths()), + scope, + outputSymbols, + outerContext); + } + + private QualifiedObjectName createExternalTsFileQualifiedObjectName(String tableName) { + String normalizedTableName = tableName.toLowerCase(Locale.ENGLISH); + if (normalizedTableName.indexOf('.') >= 0) { + return QualifiedObjectName.valueOf(normalizedTableName); + } + String databaseName = + sessionInfo.getDatabaseName().orElse("external").toLowerCase(Locale.ENGLISH); + return new QualifiedObjectName(databaseName, normalizedTableName); + } + private static void stayConsistent( String[] measurements, MeasurementSchema[] measurementSchemas) { int minLength = Math.min(measurements.length, measurementSchemas.length); 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 d31996aceba89..709db77a38e34 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 @@ -96,6 +96,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CopyToNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; @@ -727,6 +728,16 @@ public List visitDeviceTableScan( } } + @Override + public List visitExternalTsFileScan( + final ExternalTsFileScanNode node, final PlanContext context) { + node.setRegionReplicaSet( + new TRegionReplicaSet( + null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation()))); + context.mostUsedRegion = node.getRegionReplicaSet(); + return Collections.singletonList(node); + } + private List constructDeviceTableScanByTags( final DeviceTableScanNode node, final PlanContext context) { DataPartition dataPartition = analysis.getDataPartitionInfo(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java new file mode 100644 index 0000000000000..7e2d4850b56e9 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.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.relational.planner.node; + +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.IPlanVisitor; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class ExternalTsFileScanNode extends TableScanNode { + private List tsFilePaths; + + protected ExternalTsFileScanNode() {} + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + List tsFilePaths) { + super(id, qualifiedObjectName, outputSymbols, assignments); + this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); + } + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + List tsFilePaths) { + super( + id, + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset); + this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); + } + + @Override + public R accept(IPlanVisitor visitor, C context) { + return ((PlanVisitor) visitor).visitExternalTsFileScan(this, context); + } + + @Override + public ExternalTsFileScanNode clone() { + return new ExternalTsFileScanNode( + getPlanNodeId(), + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + tsFilePaths); + } + + public List getTsFilePaths() { + return tsFilePaths; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + PlanNodeType.EXTERNAL_TSFILE_SCAN_NODE.serialize(byteBuffer); + TableScanNode.serializeMemberVariables(this, byteBuffer, true); + serializeTsFilePaths(byteBuffer); + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + PlanNodeType.EXTERNAL_TSFILE_SCAN_NODE.serialize(stream); + TableScanNode.serializeMemberVariables(this, stream, true); + serializeTsFilePaths(stream); + } + + private void serializeTsFilePaths(ByteBuffer byteBuffer) { + ReadWriteIOUtils.write(tsFilePaths.size(), byteBuffer); + for (String tsFilePath : tsFilePaths) { + ReadWriteIOUtils.write(tsFilePath, byteBuffer); + } + } + + private void serializeTsFilePaths(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(tsFilePaths.size(), stream); + for (String tsFilePath : tsFilePaths) { + ReadWriteIOUtils.write(tsFilePath, stream); + } + } + + public static ExternalTsFileScanNode deserialize(ByteBuffer byteBuffer) { + ExternalTsFileScanNode node = new ExternalTsFileScanNode(); + TableScanNode.deserializeMemberVariables(byteBuffer, node, true); + + int size = ReadWriteIOUtils.readInt(byteBuffer); + List tsFilePaths = new ArrayList<>(size); + while (size-- > 0) { + tsFilePaths.add(ReadWriteIOUtils.readString(byteBuffer)); + } + node.tsFilePaths = Collections.unmodifiableList(tsFilePaths); + + node.setPlanNodeId(PlanNodeId.deserialize(byteBuffer)); + return node; + } + + @Override + public String toString() { + return "ExternalTsFileScanNode-" + this.getPlanNodeId(); + } +} 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 6dda2d17503d0..62f88860aed9d 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 @@ -66,6 +66,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; @@ -270,6 +271,35 @@ public PlanAndMappings visitInformationSchemaTableScan( mapping); } + @Override + public PlanAndMappings visitExternalTsFileScan( + ExternalTsFileScanNode node, UnaliasContext context) { + Map mapping = new HashMap<>(context.getCorrelationMapping()); + SymbolMapper mapper = symbolMapper(mapping); + + List newOutputs = mapper.map(node.getOutputSymbols()); + + Map newAssignments = new HashMap<>(); + node.getAssignments() + .forEach( + (symbol, handle) -> { + Symbol newSymbol = mapper.map(symbol); + newAssignments.put(newSymbol, handle); + }); + + return new PlanAndMappings( + new ExternalTsFileScanNode( + node.getPlanNodeId(), + node.getQualifiedObjectName(), + newOutputs, + newAssignments, + node.getPushDownPredicate() == null ? null : mapper.map(node.getPushDownPredicate()), + node.getPushDownLimit(), + node.getPushDownOffset(), + node.getTsFilePaths()), + mapping); + } + @Override public PlanAndMappings visitCteScan(CteScanNode node, UnaliasContext context) { Map mapping = new HashMap<>(context.getCorrelationMapping()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/planner/plan/node/PlanNodeType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/planner/plan/node/PlanNodeType.java index 71eb2238f15f0..4fb68ac195958 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/planner/plan/node/PlanNodeType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/planner/plan/node/PlanNodeType.java @@ -200,6 +200,7 @@ public enum PlanNodeType { TABLE_DISK_USAGE_INFORMATION_SCHEMA_TABLE_SCAN_NODE((short) 1040), ALIGNED_AGGREGATION_TREE_DEVICE_VIEW_SCAN_NODE((short) 1041), NON_ALIGNED_AGGREGATION_TREE_DEVICE_VIEW_SCAN_NODE((short) 1042), + EXTERNAL_TSFILE_SCAN_NODE((short) 1043), RELATIONAL_INSERT_TABLET((short) 2000), RELATIONAL_INSERT_ROW((short) 2001), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/relational/function/TableBuiltinTableFunction.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/relational/function/TableBuiltinTableFunction.java index 43673a2e90ce1..fe54d1078cdd0 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/relational/function/TableBuiltinTableFunction.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/relational/function/TableBuiltinTableFunction.java @@ -26,6 +26,7 @@ import org.apache.iotdb.commons.udf.builtin.relational.tvf.CapacityTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.CumulateTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.HOPTableFunction; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.SessionTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.TumbleTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.VariationTableFunction; @@ -45,7 +46,8 @@ public enum TableBuiltinTableFunction { CAPACITY("capacity"), FORECAST("forecast"), PATTERN_MATCH("pattern_match"), - CLASSIFY("classify"); + CLASSIFY("classify"), + READ_TSFILE("read_tsfile"); private final String functionName; @@ -91,6 +93,8 @@ public static TableFunction getBuiltinTableFunction(String functionName) { return new ForecastTableFunction(); case "classify": return new ClassifyTableFunction(); + case "read_tsfile": + return new ReadTsFileTableFunction(); default: throw new UnsupportedOperationException( String.format(QueryMessages.UNSUPPORTED_TABLE_FUNCTION, functionName)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java new file mode 100644 index 0000000000000..91641c75a4626 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java @@ -0,0 +1,478 @@ +/* + * 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.udf.builtin.relational.tvf; + +import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; +import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; +import org.apache.iotdb.udf.api.exception.UDFException; +import org.apache.iotdb.udf.api.relational.TableFunction; +import org.apache.iotdb.udf.api.relational.table.TableFunctionAnalysis; +import org.apache.iotdb.udf.api.relational.table.TableFunctionHandle; +import org.apache.iotdb.udf.api.relational.table.TableFunctionProcessorProvider; +import org.apache.iotdb.udf.api.relational.table.argument.Argument; +import org.apache.iotdb.udf.api.relational.table.argument.DescribedSchema; +import org.apache.iotdb.udf.api.relational.table.argument.ScalarArgument; +import org.apache.iotdb.udf.api.relational.table.processor.TableFunctionLeafProcessor; +import org.apache.iotdb.udf.api.relational.table.specification.ParameterSpecification; +import org.apache.iotdb.udf.api.relational.table.specification.ScalarParameterSpecification; +import org.apache.iotdb.udf.api.type.Type; + +import org.apache.tsfile.block.column.ColumnBuilder; +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.enums.ColumnCategory; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.TableSchema; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; + +/** Reads one or more TsFiles as a table function source. */ +public class ReadTsFileTableFunction implements TableFunction { + private static final String TABLE_NAME_PARAMETER_NAME = "TABLE_NAME"; + private static final String TSFILE_PATHS_PARAMETER_NAME = "TSFILE_PATHS"; + + @Override + public List getArgumentsSpecifications() { + return Arrays.asList( + ScalarParameterSpecification.builder() + .name(TABLE_NAME_PARAMETER_NAME) + .type(Type.STRING) + .build(), + ScalarParameterSpecification.builder() + .name(TSFILE_PATHS_PARAMETER_NAME) + .type(Type.STRING) + .build()); + } + + @Override + public TableFunctionAnalysis analyze(Map arguments) throws UDFException { + String tableName = getRequiredStringArgument(arguments, TABLE_NAME_PARAMETER_NAME); + List tsFilePaths = + parseTsFilePaths(getRequiredStringArgument(arguments, TSFILE_PATHS_PARAMETER_NAME)); + TsFileSchemaCollection schemaCollection = + collectTsFilesAndResolveSchema(tableName, tsFilePaths); + if (schemaCollection.mergedTableSchema == null) { + throw new UDFArgumentNotValidException( + "No table schema found for table " + tableName + " in TsFiles"); + } + DescribedSchema outputSchema = convertToDescribedSchema(schemaCollection.mergedTableSchema); + + ReadTsFileTableFunctionHandle handle = + new ReadTsFileTableFunctionHandle( + tableName, + schemaCollection.tsFiles.stream() + .map(File::getAbsolutePath) + .collect(Collectors.toList()), + outputSchema); + + return TableFunctionAnalysis.builder().properColumnSchema(outputSchema).handle(handle).build(); + } + + @Override + public TableFunctionHandle createTableFunctionHandle() { + return new ReadTsFileTableFunctionHandle(); + } + + @Override + public TableFunctionProcessorProvider getProcessorProvider( + TableFunctionHandle tableFunctionHandle) { + ReadTsFileTableFunctionHandle handle = (ReadTsFileTableFunctionHandle) tableFunctionHandle; + return new TableFunctionProcessorProvider() { + @Override + public TableFunctionLeafProcessor getSplitProcessor() { + return new ReadTsFileLeafProcessor(handle); + } + }; + } + + private static String getRequiredStringArgument(Map arguments, String name) { + Argument argument = arguments.get(name); + if (!(argument instanceof ScalarArgument)) { + throw new UDFArgumentNotValidException("Missing scalar argument: " + name); + } + Object value = ((ScalarArgument) argument).getValue(); + if (!(value instanceof String) || ((String) value).trim().isEmpty()) { + throw new UDFArgumentNotValidException("Argument " + name + " should not be empty"); + } + return ((String) value).trim(); + } + + private static List parseTsFilePaths(String tsFilePaths) { + List paths = + Arrays.stream(tsFilePaths.split(",")) + .map(String::trim) + .filter(path -> !path.isEmpty()) + .collect(Collectors.toList()); + if (paths.isEmpty()) { + throw new UDFArgumentNotValidException( + "Argument " + TSFILE_PATHS_PARAMETER_NAME + " should contain at least one path"); + } + return paths; + } + + private static TsFileSchemaCollection collectTsFilesAndResolveSchema( + String tableName, List tsFilePaths) { + List tsFiles = new ArrayList<>(); + MergedTableSchemaBuilder schemaBuilder = null; + for (String tsFilePath : tsFilePaths) { + Path path = new File(tsFilePath).toPath(); + if (!Files.exists(path)) { + throw new UDFArgumentNotValidException("TsFile path does not exist: " + tsFilePath); + } + try (Stream walkedPaths = Files.walk(path)) { + Iterator iterator = walkedPaths.filter(Files::isRegularFile).iterator(); + while (iterator.hasNext()) { + Path filePath = iterator.next(); + TableSchema tableSchema = tryReadTableSchema(tableName, filePath.toFile()); + if (tableSchema == null) { + continue; + } + tsFiles.add(filePath.toFile()); + if (schemaBuilder == null) { + schemaBuilder = new MergedTableSchemaBuilder(tableName, tableSchema); + } else { + schemaBuilder.merge(tableSchema); + } + } + } catch (IOException e) { + throw new UDFArgumentNotValidException("Failed to scan TsFile path: " + tsFilePath); + } + } + if (tsFiles.isEmpty()) { + throw new UDFArgumentNotValidException("No valid TsFiles found"); + } + return new TsFileSchemaCollection( + tsFiles, schemaBuilder == null ? null : schemaBuilder.build()); + } + + private static TableSchema tryReadTableSchema(String tableName, File tsFile) { + if (!tsFile.canRead()) { + return null; + } + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + if (!reader.isComplete() || reader.readVersionNumber() != TSFileConfig.VERSION_NUMBER) { + return null; + } + Map tableSchemaMap = reader.getTableSchemaMap(); + return tableSchemaMap.get(tableName.toLowerCase(Locale.ENGLISH)); + } catch (Exception e) { + return null; + } + } + + private static class TsFileSchemaCollection { + private final List tsFiles; + private final TableSchema mergedTableSchema; + + private TsFileSchemaCollection(List tsFiles, TableSchema mergedTableSchema) { + this.tsFiles = tsFiles; + this.mergedTableSchema = mergedTableSchema; + } + } + + private static class MergedTableSchemaBuilder { + private final String tableName; + private IMeasurementSchema timeColumnSchema; + private final List tagColumnSchemas = new ArrayList<>(); + private final Map fieldColumnSchemaMap = new LinkedHashMap<>(); + + private MergedTableSchemaBuilder(String tableName, TableSchema tableSchema) { + this.tableName = tableName.toLowerCase(Locale.ENGLISH); + merge(tableSchema); + } + + private void merge(TableSchema tableSchema) { + IMeasurementSchema currentTimeColumn = null; + List currentTagColumns = new ArrayList<>(); + List currentFieldColumns = new ArrayList<>(); + List columnSchemas = tableSchema.getColumnSchemas(); + List columnCategories = tableSchema.getColumnTypes(); + + for (int i = 0; i < columnCategories.size(); i++) { + if (columnCategories.get(i) == ColumnCategory.TIME) { + if (currentTimeColumn != null) { + throw new UDFArgumentNotValidException( + "Multiple time columns found when merging table schema for table " + tableName); + } + currentTimeColumn = columnSchemas.get(i); + } else if (columnCategories.get(i) == ColumnCategory.TAG) { + currentTagColumns.add(columnSchemas.get(i)); + } else if (columnCategories.get(i) == ColumnCategory.FIELD) { + currentFieldColumns.add(columnSchemas.get(i)); + } + } + + mergeTimeColumn(currentTimeColumn); + mergeTagColumns(currentTagColumns); + mergeFieldColumns(currentFieldColumns); + } + + private void mergeTimeColumn(IMeasurementSchema currentTimeColumn) { + if (currentTimeColumn == null) { + return; + } + if (timeColumnSchema == null) { + timeColumnSchema = currentTimeColumn; + return; + } + if (!timeColumnSchema.getMeasurementName().equals(currentTimeColumn.getMeasurementName()) + || currentTimeColumn.getType() != TSDataType.TIMESTAMP) { + throw new UDFArgumentNotValidException( + "Time column conflicts when merging table schema for table " + tableName); + } + } + + private void mergeTagColumns(List currentTagColumns) { + int prefixLength = Math.min(tagColumnSchemas.size(), currentTagColumns.size()); + for (int i = 0; i < prefixLength; i++) { + if (!tagColumnSchemas + .get(i) + .getMeasurementName() + .equals(currentTagColumns.get(i).getMeasurementName())) { + throw new UDFArgumentNotValidException( + "Tag columns conflict when merging table schema for table " + tableName); + } + } + tagColumnSchemas.addAll(currentTagColumns.subList(prefixLength, currentTagColumns.size())); + } + + private void mergeFieldColumns(List currentFieldColumns) { + for (IMeasurementSchema fieldColumn : currentFieldColumns) { + String fieldName = fieldColumn.getMeasurementName().toLowerCase(Locale.ENGLISH); + IMeasurementSchema existingColumn = fieldColumnSchemaMap.get(fieldName); + if (existingColumn != null && existingColumn.getType() != fieldColumn.getType()) { + throw new UDFArgumentNotValidException( + "Field column " + + fieldColumn.getMeasurementName() + + " has conflicting data types when merging table schema for table " + + tableName); + } + fieldColumnSchemaMap.putIfAbsent(fieldName, fieldColumn); + } + } + + private TableSchema build() { + List columnSchemas = new ArrayList<>(); + List columnCategories = new ArrayList<>(); + columnSchemas.add( + timeColumnSchema == null + ? new MeasurementSchema(TIME_COLUMN_NAME, TSDataType.TIMESTAMP) + : timeColumnSchema); + columnCategories.add(ColumnCategory.TIME); + for (IMeasurementSchema tagColumnSchema : tagColumnSchemas) { + columnSchemas.add(tagColumnSchema); + columnCategories.add(ColumnCategory.TAG); + } + for (IMeasurementSchema fieldColumnSchema : fieldColumnSchemaMap.values()) { + columnSchemas.add(fieldColumnSchema); + columnCategories.add(ColumnCategory.FIELD); + } + return new TableSchema(tableName, columnSchemas, columnCategories); + } + } + + private static DescribedSchema convertToDescribedSchema(TableSchema tableSchema) { + DescribedSchema.Builder builder = DescribedSchema.builder(); + for (IMeasurementSchema columnSchema : tableSchema.getColumnSchemas()) { + builder.addField( + columnSchema.getMeasurementName(), + UDFDataTypeTransformer.transformToUDFDataType(columnSchema.getType())); + } + return builder.build(); + } + + private static class ReadTsFileLeafProcessor implements TableFunctionLeafProcessor { + private final ReadTsFileTableFunctionHandle handle; + private boolean finished; + + private ReadTsFileLeafProcessor(ReadTsFileTableFunctionHandle handle) { + this.handle = handle; + } + + @Override + public void beforeStart() { + // TODO: Open TsFile resources here. + finished = true; + } + + @Override + public void process(List columnBuilders) { + // TODO: Read one batch from TsFile resources and write values into column builders. + } + + @Override + public boolean isFinish() { + return finished; + } + + @Override + public void beforeDestroy() { + // TODO: Close TsFile resources here. + } + } + + public static class ReadTsFileTableFunctionHandle implements TableFunctionHandle { + private String tableName; + private List tsFilePaths; + private List outputColumnNames; + private List outputColumnTypes; + + public ReadTsFileTableFunctionHandle() { + this("", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); + } + + public ReadTsFileTableFunctionHandle( + String tableName, List tsFilePaths, DescribedSchema outputSchema) { + this( + tableName, + tsFilePaths, + outputSchema.getFields().stream() + .map(field -> field.getName().orElse("")) + .collect(Collectors.toList()), + outputSchema.getFields().stream() + .map(DescribedSchema.Field::getType) + .collect(Collectors.toList())); + } + + private ReadTsFileTableFunctionHandle( + String tableName, + List tsFilePaths, + List outputColumnNames, + List outputColumnTypes) { + if (outputColumnNames.size() != outputColumnTypes.size()) { + throw new IllegalArgumentException("Output column names and types size mismatch"); + } + this.tableName = tableName; + this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); + this.outputColumnNames = Collections.unmodifiableList(new ArrayList<>(outputColumnNames)); + this.outputColumnTypes = Collections.unmodifiableList(new ArrayList<>(outputColumnTypes)); + } + + public String getTableName() { + return tableName; + } + + public List getTsFilePaths() { + return tsFilePaths; + } + + public List getOutputColumnNames() { + return outputColumnNames; + } + + public List getOutputColumnTypes() { + return outputColumnTypes; + } + + @Override + public byte[] serialize() { + ByteBuffer buffer = ByteBuffer.allocate(calculateSerializeSize()); + writeString(buffer, tableName); + buffer.putInt(tsFilePaths.size()); + tsFilePaths.forEach(path -> writeString(buffer, path)); + buffer.putInt(outputColumnNames.size()); + for (int i = 0; i < outputColumnNames.size(); i++) { + writeString(buffer, outputColumnNames.get(i)); + buffer.put(outputColumnTypes.get(i).getType()); + } + return buffer.array(); + } + + @Override + public void deserialize(byte[] bytes) { + ByteBuffer buffer = ByteBuffer.wrap(bytes); + tableName = readString(buffer); + int size = buffer.getInt(); + List paths = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + paths.add(readString(buffer)); + } + tsFilePaths = Collections.unmodifiableList(paths); + size = buffer.getInt(); + List columnNames = new ArrayList<>(size); + List columnTypes = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + columnNames.add(readString(buffer)); + columnTypes.add(Type.valueOf(buffer.get())); + } + outputColumnNames = Collections.unmodifiableList(columnNames); + outputColumnTypes = Collections.unmodifiableList(columnTypes); + } + + @Override + public String toString() { + return "ReadTsFileTableFunctionHandle{" + + "tableName='" + + tableName + + '\'' + + ", tsFilePaths=" + + tsFilePaths + + ", outputColumnNames=" + + outputColumnNames + + ", outputColumnTypes=" + + outputColumnTypes + + '}'; + } + + private int calculateSerializeSize() { + int size = Integer.BYTES + tableName.getBytes(StandardCharsets.UTF_8).length; + size += Integer.BYTES; + for (String path : tsFilePaths) { + size += Integer.BYTES + path.getBytes(StandardCharsets.UTF_8).length; + } + size += Integer.BYTES; + for (String columnName : outputColumnNames) { + size += Integer.BYTES + columnName.getBytes(StandardCharsets.UTF_8).length + Byte.BYTES; + } + return size; + } + + private static void writeString(ByteBuffer buffer, String value) { + byte[] bytes = value.getBytes(StandardCharsets.UTF_8); + buffer.putInt(bytes.length); + buffer.put(bytes); + } + + private static String readString(ByteBuffer buffer) { + byte[] bytes = new byte[buffer.getInt()]; + buffer.get(bytes); + return new String(bytes, StandardCharsets.UTF_8); + } + } +} From 0026afbb8de1245e9d359b035d9c133aba8e99a7 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Fri, 29 May 2026 18:18:18 +0800 Subject: [PATCH 02/32] scan --- .../execution/driver/DataDriverContext.java | 16 + .../fragment/FragmentInstanceContext.java | 54 ++++ .../AbstractDeviceTableScanOperator.java | 78 +++++ .../relational/AbstractTableScanOperator.java | 72 ++--- .../ExternalTsFileSeriesScanUtil.java | 80 +++++ .../MultiTsFileResourceIterator.java | 203 ++++++++++++ ...rderedExternalTsFileTableScanOperator.java | 298 ++++++++++++++++++ .../source/relational/TableScanOperator.java | 2 +- .../TreeAlignedDeviceViewScanOperator.java | 2 +- ...rderedExternalTsFileTableScanOperator.java | 211 +++++++++++++ .../DataNodeTableOperatorGenerator.java | 102 +++++- .../plan/planner/LocalExecutionPlanner.java | 16 + .../relational/planner/RelationPlanner.java | 2 +- .../TableDistributedPlanGenerator.java | 64 ++++ .../iterative/rule/PruneTableScanColumns.java | 15 + .../planner/node/ExternalTsFileScanNode.java | 46 +++ .../TransformSortToStreamSort.java | 29 +- .../UnaliasSymbolReferences.java | 2 + .../dataregion/read/QueryDataSourceType.java | 3 +- .../tvf/ReadTsFileTableFunction.java | 40 ++- 20 files changed, 1266 insertions(+), 69 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDeviceTableScanOperator.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java index f895859d1adff..a2892b6fec0bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java @@ -41,6 +41,7 @@ public class DataDriverContext extends DriverContext { private List paths; private QueryDataSourceType queryDataSourceType = null; private Map deviceIDToContext; + private List externalTsFilePaths; // it will be set to null, after QueryDataSource being inited private List sourceOperators; @@ -49,6 +50,7 @@ public DataDriverContext(FragmentInstanceContext fragmentInstanceContext, int pi this.paths = new ArrayList<>(); this.sourceOperators = new ArrayList<>(); this.deviceIDToContext = null; + this.externalTsFilePaths = null; } public DataDriverContext(DataDriverContext parentContext, int pipelineId) { @@ -56,6 +58,7 @@ public DataDriverContext(DataDriverContext parentContext, int pipelineId) { this.paths = new ArrayList<>(); this.sourceOperators = new ArrayList<>(); this.deviceIDToContext = null; + this.externalTsFilePaths = null; } public void setQueryDataSourceType(QueryDataSourceType queryDataSourceType) { @@ -71,6 +74,19 @@ public void clearDeviceIDToContext() { deviceIDToContext = null; } + public void setExternalTsFilePaths(List externalTsFilePaths) { + this.externalTsFilePaths = externalTsFilePaths; + } + + public List getExternalTsFilePaths() { + return externalTsFilePaths; + } + + public void clearExternalTsFilePaths() { + // friendly for gc + externalTsFilePaths = null; + } + public void addPath(IFullPath path) { this.paths.add(path); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 2a0373cf6fdad..a57b99ca18c2e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -56,6 +56,8 @@ import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceType; import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; 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.tsfile.timeindex.FileTimeIndex; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsResp; @@ -69,6 +71,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.IOException; import java.time.ZoneId; import java.time.format.DateTimeParseException; import java.util.ArrayList; @@ -117,6 +121,8 @@ public class FragmentInstanceContext extends QueryContext { // Used for region scan, relating methods are to be added. private Map devicePathsToContext; + private List externalTsFilePaths; + // Shared by all scan operators in this fragment instance to avoid memory problem protected IQueryDataSource sharedQueryDataSource; @@ -612,6 +618,10 @@ public void setDevicePathsToContext(Map devicePathsToC this.devicePathsToContext = devicePathsToContext; } + public void setExternalTsFilePaths(List externalTsFilePaths) { + this.externalTsFilePaths = externalTsFilePaths; + } + public MemoryReservationManager getMemoryReservationContext() { return memoryReservationManager; } @@ -779,6 +789,43 @@ public boolean initRegionScanQueryDataSource(List pathList) { } } + public boolean initExternalTsFileQueryDataSource(List externalTsFilePaths) + throws QueryProcessException { + long startTime = System.nanoTime(); + try { + if (externalTsFilePaths == null || externalTsFilePaths.isEmpty()) { + this.sharedQueryDataSource = EMPTY_QUERY_DATA_SOURCE; + return true; + } + + List externalTsFileResources = new ArrayList<>(externalTsFilePaths.size()); + for (String externalTsFilePath : externalTsFilePaths) { + TsFileResource resource = + new TsFileResource(new File(externalTsFilePath), TsFileResourceStatus.NORMAL); + if (resource.resourceFileExists()) { + try { + resource.deserialize(); + } catch (IOException e) { + throw new QueryProcessException( + "Failed to deserialize external TsFile resource: " + + externalTsFilePath + + ", " + + e.getMessage()); + } + } else { + resource.setTimeIndex(new FileTimeIndex(Long.MIN_VALUE, Long.MAX_VALUE)); + } + externalTsFileResources.add(resource); + } + + this.sharedQueryDataSource = + new QueryDataSource(externalTsFileResources, Collections.emptyList()); + return true; + } finally { + addInitQueryDataSourceCost(System.nanoTime() - startTime); + } + } + public synchronized IQueryDataSource getSharedQueryDataSource() throws QueryProcessException { if (sharedQueryDataSource == null) { switch (queryDataSourceType) { @@ -804,6 +851,13 @@ public synchronized IQueryDataSource getSharedQueryDataSource() throws QueryProc return getUnfinishedQueryDataSource(); } break; + case EXTERNAL_TSFILE_SCAN: + if (initExternalTsFileQueryDataSource(externalTsFilePaths)) { + externalTsFilePaths = null; + } else { + return getUnfinishedQueryDataSource(); + } + break; default: throw new QueryProcessException( "Unsupported query data source type: " + queryDataSourceType); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDeviceTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDeviceTableScanOperator.java new file mode 100644 index 0000000000000..e4b119291aed1 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDeviceTableScanOperator.java @@ -0,0 +1,78 @@ +/* + * 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.execution.operator.source.relational; + +import org.apache.iotdb.calc.plan.planner.CommonOperatorUtils; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; + +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.util.List; + +import static org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter.DEVICE_NUMBER; + +public abstract class AbstractDeviceTableScanOperator extends AbstractTableScanOperator { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(AbstractDeviceTableScanOperator.class); + + protected List deviceEntries; + protected int deviceCount; + protected int currentDeviceIndex; + + protected AbstractDeviceTableScanOperator(AbstractTableScanOperatorParameter parameter) { + super(parameter); + this.deviceEntries = parameter.deviceEntries; + this.deviceCount = parameter.deviceEntries.size(); + this.currentDeviceIndex = 0; + this.operatorContext.recordSpecifiedInfo(DEVICE_NUMBER, Integer.toString(this.deviceCount)); + recordCurrentDeviceIndex(); + constructAlignedSeriesScanUtil(); + } + + @Override + protected boolean hasCurrentDeviceEntry() { + return currentDeviceIndex < deviceCount; + } + + @Override + protected DeviceEntry getCurrentDeviceEntry() { + return deviceEntries.get(currentDeviceIndex); + } + + @Override + protected boolean advanceDeviceEntry() { + currentDeviceIndex++; + return hasCurrentDeviceEntry(); + } + + @Override + protected void recordCurrentDeviceIndex() { + this.operatorContext.recordSpecifiedInfo( + CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); + } + + @Override + public long ramBytesUsed() { + return super.ramBytesUsed() + + INSTANCE_SIZE + - AbstractTableScanOperator.INSTANCE_SIZE + + RamUsageEstimator.sizeOfCollection(deviceEntries); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java index 52a21628f2a3b..45481e95f11d4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java @@ -19,7 +19,6 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; -import org.apache.iotdb.calc.plan.planner.CommonOperatorUtils; import org.apache.iotdb.commons.path.AlignedFullPath; import org.apache.iotdb.commons.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; @@ -51,30 +50,25 @@ import static org.apache.iotdb.commons.queryengine.plan.relational.type.InternalTypeManager.getTSDataType; import static org.apache.iotdb.db.queryengine.execution.operator.source.AlignedSeriesScanOperator.appendDataIntoBuilder; -import static org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter.DEVICE_NUMBER; public abstract class AbstractTableScanOperator extends AbstractSeriesScanOperator { - private static final long INSTANCE_SIZE = - RamUsageEstimator.shallowSizeOfInstance(TableScanOperator.class); + protected static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(AbstractTableScanOperator.class); private final List columnSchemas; private final int[] columnsIndexArray; - private final List deviceEntries; + protected final Ordering scanOrder; + protected final SeriesScanOptions seriesScanOptions; - private final int deviceCount; + protected final List measurementColumnNames; - private final Ordering scanOrder; - private final SeriesScanOptions seriesScanOptions; + protected final Set allSensors; - private final List measurementColumnNames; + protected final List measurementSchemas; - private final Set allSensors; - - private final List measurementSchemas; - - private final List measurementColumnTSDataTypes; + protected final List measurementColumnTSDataTypes; private TsBlockBuilder measurementDataBuilder; @@ -84,17 +78,11 @@ public abstract class AbstractTableScanOperator extends AbstractSeriesScanOperat private QueryDataSource queryDataSource; - private int currentDeviceIndex; - public AbstractTableScanOperator(AbstractTableScanOperatorParameter parameter) { this.sourceId = parameter.sourceId; this.operatorContext = parameter.context; - this.operatorContext.recordSpecifiedInfo( - DEVICE_NUMBER, Integer.toString(parameter.deviceEntries.size())); this.columnSchemas = parameter.columnSchemas; this.columnsIndexArray = parameter.columnsIndexArray; - this.deviceEntries = parameter.deviceEntries; - this.deviceCount = parameter.deviceEntries.size(); this.scanOrder = parameter.scanOrder; this.seriesScanOptions = parameter.seriesScanOptions; this.measurementColumnNames = parameter.measurementColumnNames; @@ -104,18 +92,12 @@ public AbstractTableScanOperator(AbstractTableScanOperatorParameter parameter) { parameter.measurementSchemas.stream() .map(IMeasurementSchema::getType) .collect(Collectors.toList()); - this.currentDeviceIndex = 0; - this.operatorContext.recordSpecifiedInfo( - CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING, Integer.toString(0)); - // allSensors include time and all field columns this.maxReturnSize = Math.min( maxReturnSize, allSensors.size() * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte()); this.maxTsBlockLineNum = parameter.maxTsBlockLineNum; - - constructAlignedSeriesScanUtil(); } @Override @@ -160,8 +142,7 @@ public TsBlock next() throws Exception { if (measurementDataBuilder.isEmpty() && measurementDataBlock == null && currentDeviceNoMoreData) { - currentDeviceIndex++; - prepareForNextDevice(); + moveToNextDevice(); } } catch (IOException e) { @@ -200,13 +181,13 @@ protected void buildResult(TsBlock tsBlock) { } private void constructResultTsBlock() { - DeviceEntry currentDeviceEntry = deviceEntries.get(currentDeviceIndex); + DeviceEntry currentDeviceEntry = getCurrentDeviceEntry(); this.resultTsBlock = MeasurementToTableViewAdaptorUtils.toTableBlock( measurementDataBlock, columnsIndexArray, columnSchemas, - deviceEntries.get(currentDeviceIndex), + currentDeviceEntry, idColumnIndex -> getNthIdColumnValue(currentDeviceEntry, idColumnIndex)); } @@ -220,7 +201,7 @@ public boolean hasNext() throws Exception { @Override public boolean isFinished() throws Exception { return (retainedTsBlock == null) - && (currentDeviceIndex >= deviceCount || seriesScanOptions.limitConsumedUp()); + && (!hasCurrentDeviceEntry() || seriesScanOptions.limitConsumedUp()); } @Override @@ -252,31 +233,37 @@ public void initQueryDataSource(IQueryDataSource dataSource) { this.measurementDataBuilder.setMaxTsBlockLineNumber(this.maxTsBlockLineNum); } - private void prepareForNextDevice() { - if (currentDeviceIndex < deviceCount) { + private void moveToNextDevice() { + if (advanceDeviceEntry()) { // construct AlignedSeriesScanUtil for next device constructAlignedSeriesScanUtil(); // reset QueryDataSource queryDataSource.reset(); this.seriesScanUtil.initQueryDataSource(queryDataSource); - this.operatorContext.recordSpecifiedInfo( - CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); + recordCurrentDeviceIndex(); } } - private void constructAlignedSeriesScanUtil() { - if (this.deviceEntries.isEmpty()) { + protected abstract boolean hasCurrentDeviceEntry(); + + protected abstract DeviceEntry getCurrentDeviceEntry(); + + protected abstract boolean advanceDeviceEntry(); + + protected abstract void recordCurrentDeviceIndex(); + + protected void constructAlignedSeriesScanUtil() { + if (!hasCurrentDeviceEntry()) { // no need to construct SeriesScanUtil, hasNext will return false return; } - if (this.deviceEntries.get(this.currentDeviceIndex) == null) { - throw new IllegalStateException( - "Device entries of index " + this.currentDeviceIndex + " in TableScanOperator is empty"); + if (getCurrentDeviceEntry() == null) { + throw new IllegalStateException("Current device entry in TableScanOperator is empty"); } - DeviceEntry deviceEntry = this.deviceEntries.get(this.currentDeviceIndex); + DeviceEntry deviceEntry = getCurrentDeviceEntry(); AlignedFullPath alignedPath = constructAlignedPath(deviceEntry, measurementColumnNames, measurementSchemas, allSensors); this.seriesScanUtil = @@ -304,8 +291,7 @@ public long ramBytesUsed() { + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(seriesScanUtil) + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext) + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(sourceId) - + (resultTsBlockBuilder == null ? 0 : resultTsBlockBuilder.getRetainedSizeInBytes()) - + RamUsageEstimator.sizeOfCollection(deviceEntries); + + (resultTsBlockBuilder == null ? 0 : resultTsBlockBuilder.getRetainedSizeInBytes()); } public static class AbstractTableScanOperatorParameter { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java new file mode 100644 index 0000000000000..c568a2768c951 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java @@ -0,0 +1,80 @@ +/* + * 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.execution.operator.source.relational; + +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.AlignedSeriesScanUtil; +import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; +import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; + +import java.io.IOException; +import java.util.List; + +public class ExternalTsFileSeriesScanUtil extends AlignedSeriesScanUtil { + + private final ExternalTsFileMetadataLoader metadataLoader; + + public ExternalTsFileSeriesScanUtil( + AlignedFullPath seriesPath, + Ordering scanOrder, + SeriesScanOptions scanOptions, + FragmentInstanceContext context, + boolean queryAllSensors, + List givenDataTypes, + ExternalTsFileMetadataLoader metadataLoader) { + super(seriesPath, scanOrder, scanOptions, context, queryAllSensors, givenDataTypes); + this.metadataLoader = metadataLoader; + } + + public ExternalTsFileSeriesScanUtil( + AlignedFullPath seriesPath, + Ordering scanOrder, + SeriesScanOptions scanOptions, + FragmentInstanceContext context, + boolean queryAllSensors, + List givenDataTypes, + MultiTsFileResourceIterator resourceIterator) { + this( + seriesPath, + scanOrder, + scanOptions, + context, + queryAllSensors, + givenDataTypes, + resourceIterator::loadTimeSeriesMetadata); + } + + @Override + protected AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, boolean isSeq) throws IOException { + return metadataLoader.loadTimeSeriesMetadata(resource, (AlignedFullPath) seriesPath); + } + + @FunctionalInterface + public interface ExternalTsFileMetadataLoader { + AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedFullPath) throws IOException; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java new file mode 100644 index 0000000000000..3748f16c1cdbe --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java @@ -0,0 +1,203 @@ +/* + * 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.execution.operator.source.relational; + +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; +import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.LazyTsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.LongConsumer; + +public class MultiTsFileResourceIterator { + + private final String tableName; + private final FragmentInstanceContext fragmentInstanceContext; + private final SeriesScanOptions seriesScanOptions; + private final Map deviceIteratorMap = + new HashMap<>(); + + private IDeviceID currentDevice; + + public MultiTsFileResourceIterator( + String tableName, + List seqResources, + List unseqResources, + Map resourceReaderMap, + FragmentInstanceContext fragmentInstanceContext, + SeriesScanOptions seriesScanOptions) { + this.tableName = tableName; + this.fragmentInstanceContext = fragmentInstanceContext; + this.seriesScanOptions = seriesScanOptions; + initDeviceIterators(seqResources, resourceReaderMap); + initDeviceIterators(unseqResources, resourceReaderMap); + } + + private void initDeviceIterators( + List resources, Map resourceReaderMap) { + for (TsFileResource resource : resources) { + try { + TsFileSequenceReader reader = resourceReaderMap.get(resource); + if (reader == null) { + throw new IllegalArgumentException( + "Missing external TsFile reader: " + resource.getTsFilePath()); + } + deviceIteratorMap.put(resource, new TsFileResourceDeviceIterator(resource, reader)); + } catch (IOException e) { + throw new RuntimeException( + "Failed to create device iterator for external TsFile: " + resource.getTsFilePath(), e); + } + } + } + + public boolean hasNextDevice() { + for (TsFileResourceDeviceIterator iterator : deviceIteratorMap.values()) { + if (iterator.hasNextDevice() + || (iterator.getCurrentDevice() != null + && !iterator.getCurrentDevice().equals(currentDevice))) { + return true; + } + } + return false; + } + + public IDeviceID nextDevice() { + IDeviceID nextDevice = null; + List exhaustedResources = new ArrayList<>(); + for (Map.Entry entry : + deviceIteratorMap.entrySet()) { + TsFileResource resource = entry.getKey(); + TsFileResourceDeviceIterator iterator = entry.getValue(); + if (iterator.getCurrentDevice() == null + || iterator.getCurrentDevice().equals(currentDevice)) { + if (iterator.hasNextDevice()) { + if (iterator.nextDevice() == null) { + exhaustedResources.add(resource); + continue; + } + } else { + exhaustedResources.add(resource); + continue; + } + } + if (nextDevice == null || nextDevice.compareTo(iterator.getCurrentDevice()) > 0) { + nextDevice = iterator.getCurrentDevice(); + } + } + for (TsFileResource resource : exhaustedResources) { + deviceIteratorMap.remove(resource); + } + currentDevice = nextDevice; + return currentDevice; + } + + public IDeviceID getCurrentDevice() { + return currentDevice; + } + + public AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedPath) throws IOException { + TsFileResourceDeviceIterator iterator = deviceIteratorMap.get(resource); + if (iterator == null + || currentDevice == null + || !currentDevice.equals(iterator.getCurrentDevice())) { + return null; + } + return iterator.loadTimeSeriesMetadata(alignedPath); + } + + public long[] getCurrentDeviceMeasurementNodeOffset(TsFileResource resource) { + TsFileResourceDeviceIterator iterator = deviceIteratorMap.get(resource); + if (iterator == null + || currentDevice == null + || !currentDevice.equals(iterator.getCurrentDevice())) { + return null; + } + return iterator.getCurrentDeviceMeasurementNodeOffset(); + } + + private boolean isDeviceMatched(IDeviceID deviceID) { + return tableName.equalsIgnoreCase(deviceID.getTableName()); + } + + private class TsFileResourceDeviceIterator { + + private final TsFileResource resource; + private final LazyTsFileDeviceIterator deviceIterator; + private IDeviceID currentDevice; + + private TsFileResourceDeviceIterator(TsFileResource resource, TsFileSequenceReader reader) + throws IOException { + this.resource = resource; + LongConsumer ioSizeRecorder = + fragmentInstanceContext.getQueryStatistics().getLoadTimeSeriesMetadataActualIOSize() + ::addAndGet; + this.deviceIterator = new LazyTsFileDeviceIterator(reader, tableName, ioSizeRecorder); + } + + private boolean hasNextDevice() { + return deviceIterator.hasNext(); + } + + private IDeviceID nextDevice() { + while (deviceIterator.hasNext()) { + IDeviceID nextDevice = deviceIterator.next(); + if (isDeviceMatched(nextDevice)) { + currentDevice = nextDevice; + return currentDevice; + } + } + currentDevice = null; + return null; + } + + private IDeviceID getCurrentDevice() { + return currentDevice; + } + + private long[] getCurrentDeviceMeasurementNodeOffset() { + return deviceIterator.getCurrentDeviceMeasurementNodeOffset(); + } + + private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata(AlignedFullPath alignedPath) + throws IOException { + // TODO: Pass getCurrentDeviceMeasurementNodeOffset() to FileLoaderUtils after this branch + // supports offset-based metadata loading. + return FileLoaderUtils.loadAlignedTimeSeriesMetadata( + resource, + alignedPath, + fragmentInstanceContext, + seriesScanOptions.getGlobalTimeFilter(), + resource.isSeq(), + fragmentInstanceContext.isIgnoreAllNullRows()); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java new file mode 100644 index 0000000000000..3c0ff7a087a9b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java @@ -0,0 +1,298 @@ +/* + * 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.execution.operator.source.relational; + +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.OrderingScheme; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.TsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static org.apache.iotdb.calc.plan.planner.CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING; + +public class OrderedExternalTsFileTableScanOperator extends AbstractTableScanOperator { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(OrderedExternalTsFileTableScanOperator.class); + + private final String tableName; + private final Map assignments; + private final OrderingScheme pushedOrderingScheme; + private final Map> deviceMeasurementNodeOffsetMap = + new HashMap<>(); + + private DeviceEntry currentDeviceEntry; + private int currentDeviceIndex; + private List sortedDeviceEntries = new ArrayList<>(); + + public OrderedExternalTsFileTableScanOperator( + AbstractTableScanOperatorParameter parameter, + String tableName, + Map assignments, + OrderingScheme pushedOrderingScheme) { + super(parameter); + this.tableName = tableName; + this.assignments = assignments; + this.pushedOrderingScheme = pushedOrderingScheme; + this.currentDeviceIndex = 0; + } + + @Override + String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { + int segmentOffset = + deviceEntry.getDeviceID().segmentNum() > 0 + && tableName.equalsIgnoreCase((String) deviceEntry.getNthSegment(0)) + ? 1 + : 0; + Object segment = deviceEntry.getNthSegment(idColumnIndex + segmentOffset); + return segment == null ? null : (String) segment; + } + + @Override + public void initQueryDataSource(IQueryDataSource dataSource) { + super.initQueryDataSource(dataSource); + sortedDeviceEntries = collectSortedDeviceEntries((QueryDataSource) dataSource); + currentDeviceEntry = sortedDeviceEntries.isEmpty() ? null : sortedDeviceEntries.get(0); + recordCurrentDeviceIndex(); + constructAlignedSeriesScanUtil(); + if (seriesScanUtil != null) { + seriesScanUtil.initQueryDataSource((QueryDataSource) dataSource); + } + } + + private List collectSortedDeviceEntries(QueryDataSource queryDataSource) { + List deviceInfos = collectDeviceInfos(queryDataSource); + deviceInfos.sort(createDeviceInfoComparator()); + + List deviceEntries = new ArrayList<>(deviceInfos.size()); + Set visitedDevices = new LinkedHashSet<>(); + for (ExternalTsFileDeviceInfo deviceInfo : deviceInfos) { + deviceMeasurementNodeOffsetMap + .computeIfAbsent(deviceInfo.resource, ignored -> new HashMap<>()) + .put(deviceInfo.deviceID, deviceInfo.deviceMeasurementNodeOffset); + if (visitedDevices.add(deviceInfo.deviceID)) { + deviceEntries.add(new AlignedDeviceEntry(deviceInfo.deviceID, new Binary[0])); + } + } + return deviceEntries; + } + + private List collectDeviceInfos(QueryDataSource queryDataSource) { + List deviceInfos = new ArrayList<>(); + for (TsFileResource resource : getAllResources(queryDataSource)) { + collectDeviceInfos(resource, deviceInfos); + } + return deviceInfos; + } + + private void collectDeviceInfos( + TsFileResource resource, List deviceInfos) { + try (TsFileSequenceReader reader = new TsFileSequenceReader(resource.getTsFilePath())) { + TsFileDeviceIterator deviceIterator = + reader.getTableDevicesIteratorWithIsAligned(tableName, contextValue -> {}); + while (deviceIterator.hasNext()) { + Pair deviceInfo = deviceIterator.next(); + IDeviceID deviceID = deviceInfo.left; + if (!isDeviceMatched(deviceID)) { + continue; + } + deviceInfos.add( + new ExternalTsFileDeviceInfo( + deviceID, resource, deviceIterator.getCurrentDeviceMeasurementNodeOffset())); + } + } catch (IOException e) { + throw new RuntimeException( + "Failed to collect devices from external TsFile: " + resource.getTsFilePath(), e); + } + } + + private List getAllResources(QueryDataSource queryDataSource) { + List resources = + new ArrayList<>( + queryDataSource.getSeqResources().size() + queryDataSource.getUnseqResources().size()); + resources.addAll(queryDataSource.getSeqResources()); + resources.addAll(queryDataSource.getUnseqResources()); + return resources; + } + + private boolean isDeviceMatched(IDeviceID deviceID) { + return tableName.equalsIgnoreCase(deviceID.getTableName()); + } + + private Comparator createDeviceInfoComparator() { + Comparator comparator = null; + for (Symbol symbol : pushedOrderingScheme.getOrderBy()) { + if (TableScanNode.isTimeColumn(symbol, assignments)) { + continue; + } + int tagIndex = getTagIndex(symbol); + final int deviceSegmentIndex = tagIndex + 1; + Comparator valueComparator = + pushedOrderingScheme.getOrdering(symbol).isNullsFirst() + ? Comparator.nullsFirst(Comparator.naturalOrder()) + : Comparator.nullsLast(Comparator.naturalOrder()); + Comparator currentComparator = + Comparator.comparing( + deviceInfo -> getDeviceSegment(deviceInfo.deviceID, deviceSegmentIndex), + valueComparator); + if (!pushedOrderingScheme.getOrdering(symbol).isAscending()) { + currentComparator = currentComparator.reversed(); + } + comparator = + comparator == null ? currentComparator : comparator.thenComparing(currentComparator); + } + return comparator == null + ? Comparator.comparing(deviceInfo -> deviceInfo.deviceID) + : comparator.thenComparing(deviceInfo -> deviceInfo.deviceID); + } + + private String getDeviceSegment(IDeviceID deviceID, int deviceSegmentIndex) { + return deviceSegmentIndex < deviceID.segmentNum() + ? (String) deviceID.segment(deviceSegmentIndex) + : null; + } + + private int getTagIndex(Symbol symbol) { + int tagIndex = 0; + for (Map.Entry entry : assignments.entrySet()) { + if (entry.getValue().getColumnCategory() != TsTableColumnCategory.TAG) { + continue; + } + if (entry.getKey().equals(symbol)) { + return tagIndex; + } + tagIndex++; + } + throw new IllegalArgumentException("Unexpected external TsFile ordering symbol: " + symbol); + } + + @Override + protected boolean hasCurrentDeviceEntry() { + return currentDeviceEntry != null; + } + + @Override + protected DeviceEntry getCurrentDeviceEntry() { + return currentDeviceEntry; + } + + @Override + protected boolean advanceDeviceEntry() { + currentDeviceIndex++; + currentDeviceEntry = + currentDeviceIndex < sortedDeviceEntries.size() + ? sortedDeviceEntries.get(currentDeviceIndex) + : null; + return currentDeviceEntry != null; + } + + @Override + protected void recordCurrentDeviceIndex() { + operatorContext.recordSpecifiedInfo( + CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); + } + + @Override + protected void constructAlignedSeriesScanUtil() { + if (!hasCurrentDeviceEntry()) { + return; + } + + DeviceEntry deviceEntry = getCurrentDeviceEntry(); + if (deviceEntry == null) { + throw new IllegalStateException("Current device entry in TableScanOperator is empty"); + } + + this.seriesScanUtil = + new ExternalTsFileSeriesScanUtil( + constructAlignedPath( + deviceEntry, measurementColumnNames, measurementSchemas, allSensors), + scanOrder, + seriesScanOptions, + ((OperatorContext) operatorContext).getInstanceContext(), + true, + measurementColumnTSDataTypes, + this::loadTimeSeriesMetadata); + } + + private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedPath) throws IOException { + Optional deviceMeasurementNodeOffset = + Optional.ofNullable(deviceMeasurementNodeOffsetMap) + .map(map -> map.get(resource)) + .map(map -> map.get(alignedPath.getDeviceId())); + // TODO: Use deviceMeasurementNodeOffset after FileLoaderUtils supports offset-based metadata + // loading in this branch. + return FileLoaderUtils.loadAlignedTimeSeriesMetadata( + resource, + alignedPath, + ((OperatorContext) operatorContext).getInstanceContext(), + seriesScanOptions.getGlobalTimeFilter(), + resource.isSeq(), + ((OperatorContext) operatorContext).getInstanceContext().isIgnoreAllNullRows()); + } + + @Override + public long ramBytesUsed() { + return super.ramBytesUsed() + + INSTANCE_SIZE + - AbstractTableScanOperator.INSTANCE_SIZE + + RamUsageEstimator.sizeOfMap(deviceMeasurementNodeOffsetMap) + + RamUsageEstimator.sizeOfCollection(sortedDeviceEntries); + } + + private static class ExternalTsFileDeviceInfo { + private final IDeviceID deviceID; + private final TsFileResource resource; + private final long[] deviceMeasurementNodeOffset; + + private ExternalTsFileDeviceInfo( + IDeviceID deviceID, TsFileResource resource, long[] deviceMeasurementNodeOffset) { + this.deviceID = deviceID; + this.resource = resource; + this.deviceMeasurementNodeOffset = deviceMeasurementNodeOffset; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java index 11f10ef3dae94..a50e978b377d0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java @@ -21,7 +21,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; -public class TableScanOperator extends AbstractTableScanOperator { +public class TableScanOperator extends AbstractDeviceTableScanOperator { public TableScanOperator(AbstractTableScanOperatorParameter parameter) { super(parameter); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TreeAlignedDeviceViewScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TreeAlignedDeviceViewScanOperator.java index 03d0e998e7193..12feaccc42f20 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TreeAlignedDeviceViewScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TreeAlignedDeviceViewScanOperator.java @@ -23,7 +23,7 @@ import org.apache.tsfile.file.metadata.IDeviceID; -public class TreeAlignedDeviceViewScanOperator extends AbstractTableScanOperator { +public class TreeAlignedDeviceViewScanOperator extends AbstractDeviceTableScanOperator { private final IDeviceID.TreeDeviceIdColumnValueExtractor extractor; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java new file mode 100644 index 0000000000000..518a2fc010290 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java @@ -0,0 +1,211 @@ +/* + * 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.execution.operator.source.relational; + +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.utils.EncryptDBUtils; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.iotdb.calc.plan.planner.CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING; + +public class UnorderedExternalTsFileTableScanOperator extends AbstractTableScanOperator { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(UnorderedExternalTsFileTableScanOperator.class); + + private final String tableName; + + private MultiTsFileResourceIterator deviceIterator; + private Map resourceReaderMap = Collections.emptyMap(); + private DeviceEntry currentDeviceEntry; + private int currentDeviceIndex; + + public UnorderedExternalTsFileTableScanOperator( + AbstractTableScanOperatorParameter parameter, String tableName) { + super(parameter); + this.tableName = tableName; + this.currentDeviceIndex = 0; + } + + @Override + String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { + int segmentOffset = + deviceEntry.getDeviceID().segmentNum() > 0 + && tableName.equalsIgnoreCase((String) deviceEntry.getNthSegment(0)) + ? 1 + : 0; + Object segment = deviceEntry.getNthSegment(idColumnIndex + segmentOffset); + return segment == null ? null : (String) segment; + } + + @Override + public void initQueryDataSource(IQueryDataSource dataSource) { + super.initQueryDataSource(dataSource); + + QueryDataSource queryDataSource = (QueryDataSource) dataSource; + initDeviceIterator(queryDataSource); + currentDeviceEntry = nextDeviceEntry(); + recordCurrentDeviceIndex(); + constructAlignedSeriesScanUtil(); + if (seriesScanUtil != null) { + seriesScanUtil.initQueryDataSource(queryDataSource); + } + } + + private void initDeviceIterator(QueryDataSource queryDataSource) { + resourceReaderMap = createResourceReaderMap(getAllResources(queryDataSource)); + deviceIterator = + new MultiTsFileResourceIterator( + tableName, + queryDataSource.getSeqResources(), + queryDataSource.getUnseqResources(), + resourceReaderMap, + ((OperatorContext) operatorContext).getInstanceContext(), + seriesScanOptions); + } + + private Map createResourceReaderMap( + List resources) { + Map readerMap = new HashMap<>(resources.size()); + for (TsFileResource resource : resources) { + try { + readerMap.put( + resource, + new TsFileSequenceReader( + resource.getTsFilePath(), + ((OperatorContext) operatorContext) + .getInstanceContext() + .getQueryStatistics() + .getLoadTimeSeriesMetadataActualIOSize() + ::addAndGet, + EncryptDBUtils.getFirstEncryptParamFromTSFilePath(resource.getTsFilePath()))); + } catch (IOException e) { + closeResourceReaders(readerMap); + throw new RuntimeException( + "Failed to open external TsFile reader: " + resource.getTsFilePath(), e); + } + } + return readerMap; + } + + private List getAllResources(QueryDataSource queryDataSource) { + List resources = + new ArrayList<>( + queryDataSource.getSeqResources().size() + queryDataSource.getUnseqResources().size()); + resources.addAll(queryDataSource.getSeqResources()); + resources.addAll(queryDataSource.getUnseqResources()); + return resources; + } + + private DeviceEntry nextDeviceEntry() { + if (deviceIterator == null || !deviceIterator.hasNextDevice()) { + return null; + } + IDeviceID nextDevice = deviceIterator.nextDevice(); + return nextDevice == null ? null : new AlignedDeviceEntry(nextDevice, new Binary[0]); + } + + @Override + protected boolean hasCurrentDeviceEntry() { + return currentDeviceEntry != null; + } + + @Override + protected DeviceEntry getCurrentDeviceEntry() { + return currentDeviceEntry; + } + + @Override + protected boolean advanceDeviceEntry() { + currentDeviceIndex++; + currentDeviceEntry = nextDeviceEntry(); + return currentDeviceEntry != null; + } + + @Override + protected void recordCurrentDeviceIndex() { + operatorContext.recordSpecifiedInfo( + CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); + } + + @Override + protected void constructAlignedSeriesScanUtil() { + if (!hasCurrentDeviceEntry()) { + return; + } + + DeviceEntry deviceEntry = getCurrentDeviceEntry(); + if (deviceEntry == null) { + throw new IllegalStateException("Current device entry in TableScanOperator is empty"); + } + + this.seriesScanUtil = + new ExternalTsFileSeriesScanUtil( + constructAlignedPath( + deviceEntry, measurementColumnNames, measurementSchemas, allSensors), + scanOrder, + seriesScanOptions, + ((OperatorContext) operatorContext).getInstanceContext(), + true, + measurementColumnTSDataTypes, + deviceIterator); + } + + @Override + public void close() throws Exception { + closeResourceReaders(resourceReaderMap); + resourceReaderMap = Collections.emptyMap(); + deviceIterator = null; + super.close(); + } + + @Override + public long ramBytesUsed() { + return super.ramBytesUsed() + + INSTANCE_SIZE + - AbstractTableScanOperator.INSTANCE_SIZE + + RamUsageEstimator.sizeOfMap(resourceReaderMap); + } + + private void closeResourceReaders(Map readerMap) { + for (TsFileSequenceReader reader : readerMap.values()) { + try { + reader.close(); + } catch (IOException ignored) { + // ignore close failure + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index 81bbfa60d131e..e24528ce5c734 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -40,6 +40,7 @@ import org.apache.iotdb.commons.queryengine.common.SessionInfo; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.planner.plan.parameter.InputLocation; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; @@ -97,11 +98,13 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.relational.DeviceIteratorScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.InformationSchemaTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.LastQueryAggTableScanOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.OrderedExternalTsFileTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TreeAlignedDeviceViewAggregationScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TreeAlignedDeviceViewScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TreeNonAlignedDeviceViewAggregationScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TreeToTableViewAdaptorOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.UnorderedExternalTsFileTableScanOperator; import org.apache.iotdb.db.queryengine.plan.analyze.cache.schema.DataNodeTTLCache; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.CountSchemaMergeNode; @@ -136,6 +139,7 @@ import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.IDeviceSchemaInfo; import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; import org.apache.iotdb.db.schemaengine.table.DataNodeTreeViewSchemaUtils; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -445,7 +449,8 @@ public Operator visitTreeNonAlignedDeviceViewScan( "PushDownOffset should not be set when isPushLimitToEachDevice is true."); } CommonTableScanOperatorParameters commonParameter = - new CommonTableScanOperatorParameters(node, fieldColumnsRenameMap, true); + new CommonTableScanOperatorParameters( + node, fieldColumnsRenameMap, true, node.getTagAndAttributeIndexMap()); List measurementSchemas = commonParameter.measurementSchemas; List measurementSchemaIndex2Symbols = commonParameter.measurementSchemaIndex2Symbol; List measurementColumnNames = commonParameter.measurementColumnNames; @@ -855,9 +860,10 @@ private static class CommonTableScanOperatorParameters { int idx; private CommonTableScanOperatorParameters( - DeviceTableScanNode node, + TableScanNode node, Map fieldColumnsRenameMap, - boolean keepNonOutputMeasurementColumns) { + boolean keepNonOutputMeasurementColumns, + Map tagAndAttributeColumnsIndexMap) { outputColumnNames = node.getOutputSymbols(); int outputColumnCount = keepNonOutputMeasurementColumns ? node.getAssignments().size() : outputColumnNames.size(); @@ -865,7 +871,7 @@ private CommonTableScanOperatorParameters( symbolInputs = new ArrayList<>(outputColumnCount); columnsIndexArray = new int[outputColumnCount]; columnSchemaMap = node.getAssignments(); - tagAndAttributeColumnsIndexMap = node.getTagAndAttributeIndexMap(); + this.tagAndAttributeColumnsIndexMap = tagAndAttributeColumnsIndexMap; measurementColumnNames = new ArrayList<>(); measurementColumnsIndexMap = new HashMap<>(); measurementSchemas = new ArrayList<>(); @@ -1022,7 +1028,8 @@ private void addSource( long viewTTL) { CommonTableScanOperatorParameters commonParameter = - new CommonTableScanOperatorParameters(node, fieldColumnsRenameMap, false); + new CommonTableScanOperatorParameters( + node, fieldColumnsRenameMap, false, node.getTagAndAttributeIndexMap()); List measurementSchemas = commonParameter.measurementSchemas; List measurementColumnNames = commonParameter.measurementColumnNames; List columnSchemas = commonParameter.columnSchemas; @@ -1120,8 +1127,89 @@ public Operator visitDeviceTableScan( @Override public Operator visitExternalTsFileScan( ExternalTsFileScanNode node, LocalExecutionPlanContext context) { - throw new UnsupportedOperationException( - "ExternalTsFileScanNode physical operator is not implemented yet"); + AbstractTableScanOperator.AbstractTableScanOperatorParameter parameter = + constructExternalTsFileTableScanOperatorParameter(node, context); + + AbstractTableScanOperator externalTsFileTableScanOperator = + node.getPushedOrderingScheme().isPresent() + ? new OrderedExternalTsFileTableScanOperator( + parameter, + node.getQualifiedObjectName().getObjectName(), + node.getAssignments(), + node.getPushedOrderingScheme().get()) + : new UnorderedExternalTsFileTableScanOperator( + parameter, node.getQualifiedObjectName().getObjectName()); + + context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); + + DataDriverContext dataDriverContext = (DataDriverContext) context.getDriverContext(); + dataDriverContext.addSourceOperator(externalTsFileTableScanOperator); + dataDriverContext.setExternalTsFilePaths(node.getTsFilePaths()); + dataDriverContext.setQueryDataSourceType(QueryDataSourceType.EXTERNAL_TSFILE_SCAN); + dataDriverContext.setInputDriver(true); + + return externalTsFileTableScanOperator; + } + + private AbstractTableScanOperator.AbstractTableScanOperatorParameter + constructExternalTsFileTableScanOperatorParameter( + ExternalTsFileScanNode node, LocalExecutionPlanContext context) { + CommonTableScanOperatorParameters commonParameter = + new CommonTableScanOperatorParameters( + node, Collections.emptyMap(), false, buildTagAndAttributeColumnsIndexMap(node)); + SeriesScanOptions seriesScanOptions = + buildSeriesScanOptions( + context, + commonParameter.columnSchemaMap, + commonParameter.measurementColumnNames, + commonParameter.measurementColumnsIndexMap, + commonParameter.timeColumnName, + Optional.empty(), + node.getPushDownLimit(), + node.getPushDownOffset(), + false, + node.getPushDownPredicate()); + + OperatorContext operatorContext = + addOperatorContext( + context, + node.getPlanNodeId(), + node.getPushedOrderingScheme().isPresent() + ? OrderedExternalTsFileTableScanOperator.class.getSimpleName() + : UnorderedExternalTsFileTableScanOperator.class.getSimpleName()); + + Set allSensors = new HashSet<>(commonParameter.measurementColumnNames); + // for time column + allSensors.add(""); + + return new AbstractTableScanOperator.AbstractTableScanOperatorParameter( + allSensors, + operatorContext, + node.getPlanNodeId(), + commonParameter.columnSchemas, + commonParameter.columnsIndexArray, + Collections.emptyList(), + node.getScanOrder(), + seriesScanOptions, + commonParameter.measurementColumnNames, + commonParameter.measurementSchemas, + TSFileDescriptor.getInstance().getConfig().getMaxTsBlockLineNumber()); + } + + private static Map buildTagAndAttributeColumnsIndexMap(TableScanNode node) { + Map tagAndAttributeColumnsIndexMap = new HashMap<>(); + int index = 0; + for (Map.Entry entry : node.getAssignments().entrySet()) { + switch (entry.getValue().getColumnCategory()) { + case TAG: + case ATTRIBUTE: + tagAndAttributeColumnsIndexMap.put(entry.getKey(), index++); + break; + default: + break; + } + } + return tagAndAttributeColumnsIndexMap; } private SeriesScanOptions.Builder getSeriesScanOptionsBuilder( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanner.java index 95f28052c1fd5..f05c49b466d54 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanner.java @@ -124,6 +124,7 @@ public List plan( instanceContext.setSourcePaths(collectSourcePaths(context)); instanceContext.setDevicePathsToContext(collectDevicePathsToContext(context)); + instanceContext.setExternalTsFilePaths(collectExternalTsFilePaths(context)); instanceContext.setQueryDataSourceType( getQueryDataSourceType((DataDriverContext) context.getDriverContext())); @@ -264,6 +265,21 @@ private List collectSourcePaths(LocalExecutionPlanContext context) { return sourcePaths; } + private List collectExternalTsFilePaths(LocalExecutionPlanContext context) { + List externalTsFilePaths = new ArrayList<>(); + context + .getPipelineDriverFactories() + .forEach( + pipeline -> { + DataDriverContext dataDriverContext = (DataDriverContext) pipeline.getDriverContext(); + if (dataDriverContext.getExternalTsFilePaths() != null) { + externalTsFilePaths.addAll(dataDriverContext.getExternalTsFilePaths()); + } + dataDriverContext.clearExternalTsFilePaths(); + }); + return externalTsFilePaths; + } + public synchronized boolean forceAllocateFreeMemoryForOperators(long memoryInBytes) { // TODO @spricoder: consider a better way if (OPERATORS_MEMORY_BLOCK.getFreeMemoryInBytes() - memoryInBytes 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 18e2c2d8cb7ba..f857775cc45f0 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 @@ -1617,7 +1617,7 @@ private RelationPlan planExternalTsFileScan( field.getName().orElse(null), field.getType(), field.isHidden(), - field.getColumnCategory())); + handle.getOutputColumnCategories().get(i))); } List outputSymbols = outputSymbolsBuilder.build(); 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 709db77a38e34..6c54af74445b0 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 @@ -735,6 +735,9 @@ public List visitExternalTsFileScan( new TRegionReplicaSet( null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation()))); context.mostUsedRegion = node.getRegionReplicaSet(); + if (context.hasSortProperty) { + processExternalTsFileSortProperty(node, context); + } return Collections.singletonList(node); } @@ -1935,6 +1938,67 @@ private void processSortProperty( } } + private void processExternalTsFileSortProperty( + final ExternalTsFileScanNode externalTsFileScanNode, final PlanContext context) { + final OrderingScheme expectedOrderingScheme = context.expectedOrderingScheme; + final List newOrderingSymbols = new ArrayList<>(); + final List newSortOrders = new ArrayList<>(); + boolean lastIsTimeRelated = false; + + for (final Symbol symbol : expectedOrderingScheme.getOrderBy()) { + if (externalTsFileScanNode.isTimeColumn(symbol)) { + if (!expectedOrderingScheme.getOrdering(symbol).isAscending()) { + externalTsFileScanNode.setScanOrder(Ordering.DESC); + } + newOrderingSymbols.add(symbol); + newSortOrders.add(expectedOrderingScheme.getOrdering(symbol)); + lastIsTimeRelated = true; + break; + } + + final ColumnSchema columnSchema = externalTsFileScanNode.getAssignments().get(symbol); + if (columnSchema == null || columnSchema.getColumnCategory() != TsTableColumnCategory.TAG) { + break; + } + + newOrderingSymbols.add(symbol); + newSortOrders.add(expectedOrderingScheme.getOrdering(symbol)); + } + + if (newOrderingSymbols.isEmpty()) { + return; + } + + OrderingScheme pushedOrderingScheme = + new OrderingScheme( + newOrderingSymbols, + IntStream.range(0, newOrderingSymbols.size()) + .boxed() + .collect(Collectors.toMap(newOrderingSymbols::get, newSortOrders::get))); + externalTsFileScanNode.setPushedOrderingScheme(pushedOrderingScheme); + + if (lastIsTimeRelated) { + if (newOrderingSymbols.size() > 1 + && newOrderingSymbols.size() == expectedOrderingScheme.getOrderBy().size() + && isOrderByAllIdsAndTime( + analysis.getTableColumnSchema(externalTsFileScanNode.getQualifiedObjectName()), + externalTsFileScanNode.getAssignments(), + new OrderingScheme( + newOrderingSymbols.subList(0, newOrderingSymbols.size() - 1), + IntStream.range(0, newOrderingSymbols.size() - 1) + .boxed() + .collect(Collectors.toMap(newOrderingSymbols::get, newSortOrders::get))), + newOrderingSymbols.size() - 2)) { + nodeOrderingMap.put(externalTsFileScanNode.getPlanNodeId(), pushedOrderingScheme); + } + return; + } + + if (newOrderingSymbols.size() == expectedOrderingScheme.getOrderBy().size()) { + nodeOrderingMap.put(externalTsFileScanNode.getPlanNodeId(), pushedOrderingScheme); + } + } + private Optional createTreeDeviceIdColumnValueExtractor(DeviceTableScanNode node) { if (node instanceof TreeDeviceViewScanNode diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java index f0969147fa587..2c8654da4683d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolsExtractor; 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.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; @@ -128,6 +129,20 @@ public static Optional pruneColumns(TableScanNode node, Set re deviceTableScanNode.isPushLimitToEachDevice(), deviceTableScanNode.containsNonAlignedDevice())); } + } else if (node instanceof ExternalTsFileScanNode) { + ExternalTsFileScanNode externalTsFileScanNode = (ExternalTsFileScanNode) node; + return Optional.of( + new ExternalTsFileScanNode( + externalTsFileScanNode.getPlanNodeId(), + externalTsFileScanNode.getQualifiedObjectName(), + newOutputs, + newAssignments, + externalTsFileScanNode.getPushDownPredicate(), + externalTsFileScanNode.getPushDownLimit(), + externalTsFileScanNode.getPushDownOffset(), + externalTsFileScanNode.getScanOrder(), + externalTsFileScanNode.getPushedOrderingScheme().orElse(null), + externalTsFileScanNode.getTsFilePaths())); } else if (node instanceof InformationSchemaTableScanNode) { // For the convenience of process in execution stage, column-prune for // InformationSchemaTableScanNode is diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java index 7e2d4850b56e9..ed39c7b4ed072 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -25,9 +25,11 @@ import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.OrderingScheme; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -38,9 +40,12 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; public class ExternalTsFileScanNode extends TableScanNode { private List tsFilePaths; + private Ordering scanOrder = Ordering.ASC; + private OrderingScheme pushedOrderingScheme; protected ExternalTsFileScanNode() {} @@ -62,6 +67,8 @@ public ExternalTsFileScanNode( Expression pushDownPredicate, long pushDownLimit, long pushDownOffset, + Ordering scanOrder, + OrderingScheme pushedOrderingScheme, List tsFilePaths) { super( id, @@ -71,6 +78,8 @@ public ExternalTsFileScanNode( pushDownPredicate, pushDownLimit, pushDownOffset); + this.scanOrder = scanOrder; + this.pushedOrderingScheme = pushedOrderingScheme; this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); } @@ -89,6 +98,8 @@ public ExternalTsFileScanNode clone() { pushDownPredicate, pushDownLimit, pushDownOffset, + scanOrder, + pushedOrderingScheme, tsFilePaths); } @@ -96,10 +107,33 @@ public List getTsFilePaths() { return tsFilePaths; } + public Ordering getScanOrder() { + return scanOrder; + } + + public void setScanOrder(Ordering scanOrder) { + this.scanOrder = scanOrder; + } + + public Optional getPushedOrderingScheme() { + return Optional.ofNullable(pushedOrderingScheme); + } + + public void setPushedOrderingScheme(OrderingScheme pushedOrderingScheme) { + this.pushedOrderingScheme = pushedOrderingScheme; + } + @Override protected void serializeAttributes(ByteBuffer byteBuffer) { PlanNodeType.EXTERNAL_TSFILE_SCAN_NODE.serialize(byteBuffer); TableScanNode.serializeMemberVariables(this, byteBuffer, true); + ReadWriteIOUtils.write(scanOrder.ordinal(), byteBuffer); + if (pushedOrderingScheme == null) { + ReadWriteIOUtils.write(false, byteBuffer); + } else { + ReadWriteIOUtils.write(true, byteBuffer); + pushedOrderingScheme.serialize(byteBuffer); + } serializeTsFilePaths(byteBuffer); } @@ -107,6 +141,13 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { protected void serializeAttributes(DataOutputStream stream) throws IOException { PlanNodeType.EXTERNAL_TSFILE_SCAN_NODE.serialize(stream); TableScanNode.serializeMemberVariables(this, stream, true); + ReadWriteIOUtils.write(scanOrder.ordinal(), stream); + if (pushedOrderingScheme == null) { + ReadWriteIOUtils.write(false, stream); + } else { + ReadWriteIOUtils.write(true, stream); + pushedOrderingScheme.serialize(stream); + } serializeTsFilePaths(stream); } @@ -128,6 +169,11 @@ public static ExternalTsFileScanNode deserialize(ByteBuffer byteBuffer) { ExternalTsFileScanNode node = new ExternalTsFileScanNode(); TableScanNode.deserializeMemberVariables(byteBuffer, node, true); + node.scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)]; + if (ReadWriteIOUtils.readBool(byteBuffer)) { + node.pushedOrderingScheme = OrderingScheme.deserialize(byteBuffer); + } + int size = ReadWriteIOUtils.readInt(byteBuffer); List tsFilePaths = new ArrayList<>(size); while (size-- > 0) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java index 4b4866aa1faab..e6f5962f2e523 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.planner.OrderingScheme; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; @@ -35,11 +36,12 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import java.util.Map; -import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode.isTimeColumn; +import static org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode.isTimeColumn; /** * Optimization phase: Logical plan planning. @@ -91,9 +93,13 @@ public PlanNode visitSort(SortNode node, Context context) { } context.setCanTransform(false); - DeviceTableScanNode deviceTableScanNode = context.getTableScanNode(); + TableScanNode tableScanNode = context.getTableScanNode(); + if (tableScanNode == null) { + node.setChild(child); + return node; + } Map tableColumnSchema = - analysis.getTableColumnSchema(deviceTableScanNode.getQualifiedObjectName()); + analysis.getTableColumnSchema(tableScanNode.getQualifiedObjectName()); OrderingScheme orderingScheme = node.getOrderingScheme(); int streamSortIndex = -1; @@ -110,10 +116,7 @@ public PlanNode visitSort(SortNode node, Context context) { if (streamSortIndex >= 0) { boolean orderByAllIdsAndTime = isOrderByAllIdsAndTime( - tableColumnSchema, - deviceTableScanNode.getAssignments(), - orderingScheme, - streamSortIndex); + tableColumnSchema, tableScanNode.getAssignments(), orderingScheme, streamSortIndex); return new StreamSortNode( queryContext.getQueryId().genPlanNodeId(), @@ -144,6 +147,12 @@ public PlanNode visitDeviceTableScan(DeviceTableScanNode node, Context context) return node; } + @Override + public PlanNode visitExternalTsFileScan(ExternalTsFileScanNode node, Context context) { + context.setTableScanNode(node); + return node; + } + @Override public PlanNode visitInformationSchemaTableScan( InformationSchemaTableScanNode node, Context context) { @@ -200,15 +209,15 @@ public static boolean isOrderByAllIdsAndTime( } private static class Context { - private DeviceTableScanNode tableScanNode; + private TableScanNode tableScanNode; private boolean canTransform = true; - public DeviceTableScanNode getTableScanNode() { + public TableScanNode getTableScanNode() { return tableScanNode; } - public void setTableScanNode(DeviceTableScanNode tableScanNode) { + public void setTableScanNode(TableScanNode tableScanNode) { this.tableScanNode = tableScanNode; } 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 62f88860aed9d..3dd10fa036105 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 @@ -296,6 +296,8 @@ public PlanAndMappings visitExternalTsFileScan( node.getPushDownPredicate() == null ? null : mapper.map(node.getPushDownPredicate()), node.getPushDownLimit(), node.getPushDownOffset(), + node.getScanOrder(), + node.getPushedOrderingScheme().map(mapper::map).orElse(null), node.getTsFilePaths()), mapping); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java index d80ce27ad7a1e..1214fd2b89ca0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java @@ -21,5 +21,6 @@ public enum QueryDataSourceType { SERIES_SCAN, DEVICE_REGION_SCAN, - TIME_SERIES_REGION_SCAN + TIME_SERIES_REGION_SCAN, + EXTERNAL_TSFILE_SCAN } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java index 91641c75a4626..82c4133b33923 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java @@ -19,6 +19,7 @@ package org.apache.iotdb.commons.udf.builtin.relational.tvf; +import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; import org.apache.iotdb.udf.api.exception.UDFException; @@ -99,6 +100,9 @@ public TableFunctionAnalysis analyze(Map arguments) throws UDF schemaCollection.tsFiles.stream() .map(File::getAbsolutePath) .collect(Collectors.toList()), + schemaCollection.mergedTableSchema.getColumnTypes().stream() + .map(TsTableColumnCategory::fromTsFileColumnCategory) + .collect(Collectors.toList()), outputSchema); return TableFunctionAnalysis.builder().properColumnSchema(outputSchema).handle(handle).build(); @@ -352,13 +356,22 @@ public static class ReadTsFileTableFunctionHandle implements TableFunctionHandle private List tsFilePaths; private List outputColumnNames; private List outputColumnTypes; + private List outputColumnCategories; public ReadTsFileTableFunctionHandle() { - this("", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); + this( + "", + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList()); } public ReadTsFileTableFunctionHandle( - String tableName, List tsFilePaths, DescribedSchema outputSchema) { + String tableName, + List tsFilePaths, + List outputColumnCategories, + DescribedSchema outputSchema) { this( tableName, tsFilePaths, @@ -367,21 +380,28 @@ public ReadTsFileTableFunctionHandle( .collect(Collectors.toList()), outputSchema.getFields().stream() .map(DescribedSchema.Field::getType) - .collect(Collectors.toList())); + .collect(Collectors.toList()), + outputColumnCategories); } private ReadTsFileTableFunctionHandle( String tableName, List tsFilePaths, List outputColumnNames, - List outputColumnTypes) { + List outputColumnTypes, + List outputColumnCategories) { if (outputColumnNames.size() != outputColumnTypes.size()) { throw new IllegalArgumentException("Output column names and types size mismatch"); } + if (outputColumnNames.size() != outputColumnCategories.size()) { + throw new IllegalArgumentException("Output column names and categories size mismatch"); + } this.tableName = tableName; this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); this.outputColumnNames = Collections.unmodifiableList(new ArrayList<>(outputColumnNames)); this.outputColumnTypes = Collections.unmodifiableList(new ArrayList<>(outputColumnTypes)); + this.outputColumnCategories = + Collections.unmodifiableList(new ArrayList<>(outputColumnCategories)); } public String getTableName() { @@ -400,6 +420,10 @@ public List getOutputColumnTypes() { return outputColumnTypes; } + public List getOutputColumnCategories() { + return outputColumnCategories; + } + @Override public byte[] serialize() { ByteBuffer buffer = ByteBuffer.allocate(calculateSerializeSize()); @@ -410,6 +434,7 @@ public byte[] serialize() { for (int i = 0; i < outputColumnNames.size(); i++) { writeString(buffer, outputColumnNames.get(i)); buffer.put(outputColumnTypes.get(i).getType()); + buffer.put(outputColumnCategories.get(i).getCategory()); } return buffer.array(); } @@ -427,12 +452,15 @@ public void deserialize(byte[] bytes) { size = buffer.getInt(); List columnNames = new ArrayList<>(size); List columnTypes = new ArrayList<>(size); + List columnCategories = new ArrayList<>(size); for (int i = 0; i < size; i++) { columnNames.add(readString(buffer)); columnTypes.add(Type.valueOf(buffer.get())); + columnCategories.add(TsTableColumnCategory.deserialize(buffer.get())); } outputColumnNames = Collections.unmodifiableList(columnNames); outputColumnTypes = Collections.unmodifiableList(columnTypes); + outputColumnCategories = Collections.unmodifiableList(columnCategories); } @Override @@ -447,6 +475,8 @@ public String toString() { + outputColumnNames + ", outputColumnTypes=" + outputColumnTypes + + ", outputColumnCategories=" + + outputColumnCategories + '}'; } @@ -458,7 +488,7 @@ private int calculateSerializeSize() { } size += Integer.BYTES; for (String columnName : outputColumnNames) { - size += Integer.BYTES + columnName.getBytes(StandardCharsets.UTF_8).length + Byte.BYTES; + size += Integer.BYTES + columnName.getBytes(StandardCharsets.UTF_8).length + 2 * Byte.BYTES; } return size; } From 48d86c920af5ad5aec943ca7fe9ae2a6dacd24c0 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Mon, 1 Jun 2026 11:22:56 +0800 Subject: [PATCH 03/32] push down tag predicate --- .../fragment/FragmentInstanceContext.java | 2 +- .../operator/source/SeriesScanUtil.java | 46 ++++++----- .../ExternalTsFileDeviceFilterVisitor.java | 50 ++++++++++++ .../ExternalTsFileSeriesScanUtil.java | 6 ++ .../MultiTsFileResourceIterator.java | 12 ++- ...rderedExternalTsFileTableScanOperator.java | 16 +++- ...rderedExternalTsFileTableScanOperator.java | 8 +- .../DataNodeTableOperatorGenerator.java | 36 ++++++++- .../iterative/rule/PruneTableScanColumns.java | 2 + .../planner/node/ExternalTsFileScanNode.java | 80 +++++++++++++++++++ .../PushPredicateIntoTableScan.java | 23 +++++- .../UnaliasSymbolReferences.java | 2 + 12 files changed, 251 insertions(+), 32 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index a57b99ca18c2e..67b36a6d23d87 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -819,7 +819,7 @@ public boolean initExternalTsFileQueryDataSource(List externalTsFilePath } this.sharedQueryDataSource = - new QueryDataSource(externalTsFileResources, Collections.emptyList()); + new QueryDataSource(Collections.emptyList(), externalTsFileResources); return true; } finally { addInitQueryDataSourceCost(System.nanoTime() - startTime); 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 6e6eb76905bc9..0591e1234c30c 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 @@ -208,27 +208,7 @@ public void initQueryDataSource(QueryDataSource dataSource) { dataSource.fillOrderIndexes(deviceID, orderUtils.getAscending()); this.dataSource = dataSource; - // updated filter concerning TTL - // IgnoreAllNullRows is false indicating that the current query is a table model query. - // In most cases, We can use this condition to determine from which model to obtain the ttl - // of the current device. However, it should be noted that for tree model data queried using - // table view, ttl also needs to be obtained from the tree model. - if (context.isIgnoreAllNullRows() || scanOptions.isTableViewForTreeModel()) { - if (deviceID != EMPTY_DEVICE_ID) { - long ttl = DataNodeTTLCache.getInstance().getTTLForTree(deviceID); - scanOptions.setTTLForTreeDevice(ttl); - } - } else { - if (scanOptions.timeFilterNeedUpdatedByTtl()) { - String databaseName = dataSource.getDatabaseName(); - long ttl = - databaseName == null - ? Long.MAX_VALUE - : DataNodeTTLCache.getInstance() - .getTTLForTable(databaseName, deviceID.getTableName()); - scanOptions.setTTLForTableDevice(ttl); - } - } + updateFilterUsingTTL(dataSource); // init file index orderUtils.setCurSeqFileIndex(dataSource); @@ -254,6 +234,30 @@ public void initQueryDataSource(QueryDataSource dataSource) { } } + protected void updateFilterUsingTTL(QueryDataSource dataSource) { + // updated filter concerning TTL + // IgnoreAllNullRows is false indicating that the current query is a table model query. + // In most cases, We can use this condition to determine from which model to obtain the ttl + // of the current device. However, it should be noted that for tree model data queried using + // table view, ttl also needs to be obtained from the tree model. + if (context.isIgnoreAllNullRows() || scanOptions.isTableViewForTreeModel()) { + if (deviceID != EMPTY_DEVICE_ID) { + long ttl = DataNodeTTLCache.getInstance().getTTLForTree(deviceID); + scanOptions.setTTLForTreeDevice(ttl); + } + } else { + if (scanOptions.timeFilterNeedUpdatedByTtl()) { + String databaseName = dataSource.getDatabaseName(); + long ttl = + databaseName == null + ? Long.MAX_VALUE + : DataNodeTTLCache.getInstance() + .getTTLForTable(databaseName, deviceID.getTableName()); + scanOptions.setTTLForTableDevice(ttl); + } + } + } + protected PriorityMergeReader getPriorityMergeReader() { return new PriorityMergeReader(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java new file mode 100644 index 0000000000000..005491d2b6ef7 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java @@ -0,0 +1,50 @@ +/* + * 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.execution.operator.source.relational; + +import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.commons.schema.filter.SchemaFilterVisitor; +import org.apache.iotdb.commons.schema.filter.impl.StringValueFilterVisitor; +import org.apache.iotdb.commons.schema.filter.impl.singlechild.AttributeFilter; +import org.apache.iotdb.commons.schema.filter.impl.singlechild.TagFilter; + +import org.apache.tsfile.file.metadata.IDeviceID; + +public class ExternalTsFileDeviceFilterVisitor extends SchemaFilterVisitor { + + @Override + protected Boolean visitNode(final SchemaFilter filter, final IDeviceID deviceID) { + throw new UnsupportedOperationException( + "The schema filter type " + filter.getSchemaFilterType() + " is not supported"); + } + + @Override + public Boolean visitTagFilter(final TagFilter filter, final IDeviceID deviceID) { + final int index = filter.getIndex() + 1; + final String value = index < deviceID.segmentNum() ? (String) deviceID.segment(index) : null; + return filter.getChild().accept(StringValueFilterVisitor.getInstance(), value); + } + + @Override + public Boolean visitAttributeFilter(final AttributeFilter filter, final IDeviceID deviceID) { + throw new UnsupportedOperationException( + "Attribute filter is not supported for external TsFile device filtering"); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java index c568a2768c951..f8e7766ffbe83 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.AlignedSeriesScanUtil; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.enums.TSDataType; @@ -72,6 +73,11 @@ protected AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( return metadataLoader.loadTimeSeriesMetadata(resource, (AlignedFullPath) seriesPath); } + @Override + protected void updateFilterUsingTTL(QueryDataSource dataSource) { + // External TsFiles are not managed by IoTDB metadata, so no table/tree TTL applies here. + } + @FunctionalInterface public interface ExternalTsFileMetadataLoader { AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java index 3748f16c1cdbe..67bddc4facb55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; @@ -42,6 +43,9 @@ public class MultiTsFileResourceIterator { private final String tableName; private final FragmentInstanceContext fragmentInstanceContext; private final SeriesScanOptions seriesScanOptions; + private final SchemaFilter deviceFilter; + private final ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = + new ExternalTsFileDeviceFilterVisitor(); private final Map deviceIteratorMap = new HashMap<>(); @@ -53,10 +57,12 @@ public MultiTsFileResourceIterator( List unseqResources, Map resourceReaderMap, FragmentInstanceContext fragmentInstanceContext, - SeriesScanOptions seriesScanOptions) { + SeriesScanOptions seriesScanOptions, + SchemaFilter deviceFilter) { this.tableName = tableName; this.fragmentInstanceContext = fragmentInstanceContext; this.seriesScanOptions = seriesScanOptions; + this.deviceFilter = deviceFilter; initDeviceIterators(seqResources, resourceReaderMap); initDeviceIterators(unseqResources, resourceReaderMap); } @@ -145,7 +151,9 @@ public long[] getCurrentDeviceMeasurementNodeOffset(TsFileResource resource) { } private boolean isDeviceMatched(IDeviceID deviceID) { - return tableName.equalsIgnoreCase(deviceID.getTableName()); + return tableName.equalsIgnoreCase(deviceID.getTableName()) + && (deviceFilter == null + || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID))); } private class TsFileResourceDeviceIterator { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java index 3c0ff7a087a9b..1476ed889aadc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.planner.OrderingScheme; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; @@ -60,6 +61,9 @@ public class OrderedExternalTsFileTableScanOperator extends AbstractTableScanOpe private final String tableName; private final Map assignments; private final OrderingScheme pushedOrderingScheme; + private final SchemaFilter deviceFilter; + private final ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = + new ExternalTsFileDeviceFilterVisitor(); private final Map> deviceMeasurementNodeOffsetMap = new HashMap<>(); @@ -71,11 +75,13 @@ public OrderedExternalTsFileTableScanOperator( AbstractTableScanOperatorParameter parameter, String tableName, Map assignments, - OrderingScheme pushedOrderingScheme) { + OrderingScheme pushedOrderingScheme, + SchemaFilter deviceFilter) { super(parameter); this.tableName = tableName; this.assignments = assignments; this.pushedOrderingScheme = pushedOrderingScheme; + this.deviceFilter = deviceFilter; this.currentDeviceIndex = 0; } @@ -138,6 +144,9 @@ private void collectDeviceInfos( if (!isDeviceMatched(deviceID)) { continue; } + if (!isDeviceFilterMatched(deviceID)) { + continue; + } deviceInfos.add( new ExternalTsFileDeviceInfo( deviceID, resource, deviceIterator.getCurrentDeviceMeasurementNodeOffset())); @@ -161,6 +170,11 @@ private boolean isDeviceMatched(IDeviceID deviceID) { return tableName.equalsIgnoreCase(deviceID.getTableName()); } + private boolean isDeviceFilterMatched(IDeviceID deviceID) { + return deviceFilter == null + || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID)); + } + private Comparator createDeviceInfoComparator() { Comparator comparator = null; for (Symbol symbol : pushedOrderingScheme.getOrderBy()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java index 518a2fc010290..b1eb554a3ef46 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; @@ -46,6 +47,7 @@ public class UnorderedExternalTsFileTableScanOperator extends AbstractTableScanO RamUsageEstimator.shallowSizeOfInstance(UnorderedExternalTsFileTableScanOperator.class); private final String tableName; + private final SchemaFilter deviceFilter; private MultiTsFileResourceIterator deviceIterator; private Map resourceReaderMap = Collections.emptyMap(); @@ -53,9 +55,10 @@ public class UnorderedExternalTsFileTableScanOperator extends AbstractTableScanO private int currentDeviceIndex; public UnorderedExternalTsFileTableScanOperator( - AbstractTableScanOperatorParameter parameter, String tableName) { + AbstractTableScanOperatorParameter parameter, String tableName, SchemaFilter deviceFilter) { super(parameter); this.tableName = tableName; + this.deviceFilter = deviceFilter; this.currentDeviceIndex = 0; } @@ -93,7 +96,8 @@ private void initDeviceIterator(QueryDataSource queryDataSource) { queryDataSource.getUnseqResources(), resourceReaderMap, ((OperatorContext) operatorContext).getInstanceContext(), - seriesScanOptions); + seriesScanOptions, + deviceFilter); } private Map createResourceReaderMap( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index e24528ce5c734..3f678f7abddf0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -51,7 +51,9 @@ import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.LongLiteral; import org.apache.iotdb.commons.queryengine.plan.relational.type.InternalTypeManager; import org.apache.iotdb.commons.queryengine.utils.TimestampPrecisionUtils; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.schema.table.TsTable; +import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -111,6 +113,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.sink.IdentitySinkNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.ConvertPredicateToTimeFilterVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.schema.ConvertSchemaPredicateToFilterVisitor; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.metadata.NonAlignedDeviceEntry; @@ -1129,6 +1132,7 @@ public Operator visitExternalTsFileScan( ExternalTsFileScanNode node, LocalExecutionPlanContext context) { AbstractTableScanOperator.AbstractTableScanOperatorParameter parameter = constructExternalTsFileTableScanOperatorParameter(node, context); + SchemaFilter deviceFilter = constructExternalTsFileDeviceFilter(node); AbstractTableScanOperator externalTsFileTableScanOperator = node.getPushedOrderingScheme().isPresent() @@ -1136,9 +1140,10 @@ public Operator visitExternalTsFileScan( parameter, node.getQualifiedObjectName().getObjectName(), node.getAssignments(), - node.getPushedOrderingScheme().get()) + node.getPushedOrderingScheme().get(), + deviceFilter) : new UnorderedExternalTsFileTableScanOperator( - parameter, node.getQualifiedObjectName().getObjectName()); + parameter, node.getQualifiedObjectName().getObjectName(), deviceFilter); context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); @@ -1151,6 +1156,31 @@ public Operator visitExternalTsFileScan( return externalTsFileTableScanOperator; } + private SchemaFilter constructExternalTsFileDeviceFilter(ExternalTsFileScanNode node) { + if (!node.getTagPredicate().isPresent()) { + return null; + } + TsTable table = new TsTable(node.getQualifiedObjectName().getObjectName()); + for (Map.Entry entry : node.getAssignments().entrySet()) { + ColumnSchema columnSchema = entry.getValue(); + if (columnSchema.getColumnCategory() == TsTableColumnCategory.TAG) { + table.addColumnSchema( + new TagColumnSchema(entry.getKey().getName(), getTSDataType(columnSchema.getType()))); + } + } + SchemaFilter deviceFilter = + node.getTagPredicate() + .get() + .accept( + new ConvertSchemaPredicateToFilterVisitor(), + new ConvertSchemaPredicateToFilterVisitor.Context(table)); + if (deviceFilter == null) { + throw new UnsupportedOperationException( + "Unsupported external TsFile device filter: " + node.getTagPredicate().get()); + } + return deviceFilter; + } + private AbstractTableScanOperator.AbstractTableScanOperatorParameter constructExternalTsFileTableScanOperatorParameter( ExternalTsFileScanNode node, LocalExecutionPlanContext context) { @@ -1164,7 +1194,7 @@ public Operator visitExternalTsFileScan( commonParameter.measurementColumnNames, commonParameter.measurementColumnsIndexMap, commonParameter.timeColumnName, - Optional.empty(), + node.getTimePredicate(), node.getPushDownLimit(), node.getPushDownOffset(), false, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java index 2c8654da4683d..97dc0672b55fe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java @@ -140,6 +140,8 @@ public static Optional pruneColumns(TableScanNode node, Set re externalTsFileScanNode.getPushDownPredicate(), externalTsFileScanNode.getPushDownLimit(), externalTsFileScanNode.getPushDownOffset(), + externalTsFileScanNode.getTagPredicate().orElse(null), + externalTsFileScanNode.getTimePredicate().orElse(null), externalTsFileScanNode.getScanOrder(), externalTsFileScanNode.getPushedOrderingScheme().orElse(null), externalTsFileScanNode.getTsFilePaths())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java index ed39c7b4ed072..e60cebfdf29f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -44,6 +44,8 @@ public class ExternalTsFileScanNode extends TableScanNode { private List tsFilePaths; + private Expression tagPredicate; + private Expression timePredicate; private Ordering scanOrder = Ordering.ASC; private OrderingScheme pushedOrderingScheme; @@ -70,6 +72,34 @@ public ExternalTsFileScanNode( Ordering scanOrder, OrderingScheme pushedOrderingScheme, List tsFilePaths) { + this( + id, + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + null, + null, + scanOrder, + pushedOrderingScheme, + tsFilePaths); + } + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + Expression tagPredicate, + Expression timePredicate, + Ordering scanOrder, + OrderingScheme pushedOrderingScheme, + List tsFilePaths) { super( id, qualifiedObjectName, @@ -78,6 +108,8 @@ public ExternalTsFileScanNode( pushDownPredicate, pushDownLimit, pushDownOffset); + this.tagPredicate = tagPredicate; + this.timePredicate = timePredicate; this.scanOrder = scanOrder; this.pushedOrderingScheme = pushedOrderingScheme; this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); @@ -98,6 +130,8 @@ public ExternalTsFileScanNode clone() { pushDownPredicate, pushDownLimit, pushDownOffset, + tagPredicate, + timePredicate, scanOrder, pushedOrderingScheme, tsFilePaths); @@ -107,6 +141,22 @@ public List getTsFilePaths() { return tsFilePaths; } + public Optional getTagPredicate() { + return Optional.ofNullable(tagPredicate); + } + + public void setTagPredicate(Expression tagPredicate) { + this.tagPredicate = tagPredicate; + } + + public Optional getTimePredicate() { + return Optional.ofNullable(timePredicate); + } + + public void setTimePredicate(Expression timePredicate) { + this.timePredicate = timePredicate; + } + public Ordering getScanOrder() { return scanOrder; } @@ -127,6 +177,8 @@ public void setPushedOrderingScheme(OrderingScheme pushedOrderingScheme) { protected void serializeAttributes(ByteBuffer byteBuffer) { PlanNodeType.EXTERNAL_TSFILE_SCAN_NODE.serialize(byteBuffer); TableScanNode.serializeMemberVariables(this, byteBuffer, true); + serializePredicate(tagPredicate, byteBuffer); + serializePredicate(timePredicate, byteBuffer); ReadWriteIOUtils.write(scanOrder.ordinal(), byteBuffer); if (pushedOrderingScheme == null) { ReadWriteIOUtils.write(false, byteBuffer); @@ -141,6 +193,8 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { protected void serializeAttributes(DataOutputStream stream) throws IOException { PlanNodeType.EXTERNAL_TSFILE_SCAN_NODE.serialize(stream); TableScanNode.serializeMemberVariables(this, stream, true); + serializePredicate(tagPredicate, stream); + serializePredicate(timePredicate, stream); ReadWriteIOUtils.write(scanOrder.ordinal(), stream); if (pushedOrderingScheme == null) { ReadWriteIOUtils.write(false, stream); @@ -165,10 +219,36 @@ private void serializeTsFilePaths(DataOutputStream stream) throws IOException { } } + private void serializePredicate(Expression predicate, ByteBuffer byteBuffer) { + if (predicate == null) { + ReadWriteIOUtils.write(false, byteBuffer); + } else { + ReadWriteIOUtils.write(true, byteBuffer); + Expression.serialize(predicate, byteBuffer); + } + } + + private void serializePredicate(Expression predicate, DataOutputStream stream) + throws IOException { + if (predicate == null) { + ReadWriteIOUtils.write(false, stream); + } else { + ReadWriteIOUtils.write(true, stream); + Expression.serialize(predicate, stream); + } + } + public static ExternalTsFileScanNode deserialize(ByteBuffer byteBuffer) { ExternalTsFileScanNode node = new ExternalTsFileScanNode(); TableScanNode.deserializeMemberVariables(byteBuffer, node, true); + if (ReadWriteIOUtils.readBool(byteBuffer)) { + node.tagPredicate = Expression.deserialize(byteBuffer); + } + if (ReadWriteIOUtils.readBool(byteBuffer)) { + node.timePredicate = Expression.deserialize(byteBuffer); + } + node.scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)]; if (ReadWriteIOUtils.readBool(byteBuffer)) { node.pushedOrderingScheme = OrderingScheme.deserialize(byteBuffer); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index aff3a1445843a..361e2d540bb54 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -72,6 +72,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolsExtractor; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.ReplaceSymbolInExpression; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; @@ -471,12 +472,22 @@ public PlanNode visitDeviceTableScan( return combineFilterAndScan(tableScanNode, context.inheritedPredicate); } + @Override + public PlanNode visitExternalTsFileScan( + ExternalTsFileScanNode tableScanNode, RewriteContext context) { + if (TRUE_LITERAL.equals(context.inheritedPredicate)) { + return tableScanNode; + } + + return combineFilterAndScan(tableScanNode, context.inheritedPredicate); + } + public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression predicate) { SplitExpression splitExpression = tableScanNode instanceof InformationSchemaTableScanNode ? splitPredicateForInformationSchemaTable( (InformationSchemaTableScanNode) tableScanNode, predicate) - : splitPredicate((DeviceTableScanNode) tableScanNode, predicate); + : splitPredicate(tableScanNode, predicate); // exist expressions can push down to scan operator if (!splitExpression.getExpressionsCanPushDown().isEmpty()) { @@ -492,6 +503,8 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre Boolean hasValueFilter = resultPair.getRight(); if (tableScanNode instanceof DeviceTableScanNode && resultPair.left != null) { ((DeviceTableScanNode) tableScanNode).setTimePredicate(resultPair.left); + } else if (tableScanNode instanceof ExternalTsFileScanNode && resultPair.left != null) { + ((ExternalTsFileScanNode) tableScanNode).setTimePredicate(resultPair.left); } if (Boolean.TRUE.equals(hasValueFilter)) { if (pushDownPredicate instanceof LogicalExpression @@ -510,6 +523,12 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre if (tableScanNode instanceof DeviceTableScanNode) { getDeviceEntriesWithDataPartitions( (DeviceTableScanNode) tableScanNode, splitExpression.getMetadataExpressions()); + } else if (tableScanNode instanceof ExternalTsFileScanNode) { + ((ExternalTsFileScanNode) tableScanNode) + .setTagPredicate( + splitExpression.getMetadataExpressions().isEmpty() + ? null + : combineConjuncts(splitExpression.getMetadataExpressions())); } // exist expressions can not push down to scan operator @@ -621,7 +640,7 @@ private SplitExpression splitPredicateForInformationSchemaTable( Collections.emptyList(), expressionsCanPushDown, expressionsCannotPushDown, null); } - private SplitExpression splitPredicate(DeviceTableScanNode node, Expression predicate) { + private SplitExpression splitPredicate(TableScanNode node, Expression predicate) { Set idOrAttributeColumnNames = new HashSet<>(node.getAssignments().size()); Set timeOrMeasurementColumnNames = new HashSet<>(node.getAssignments().size()); String timeColumnName = null; 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 3dd10fa036105..389a91f300bf9 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 @@ -296,6 +296,8 @@ public PlanAndMappings visitExternalTsFileScan( node.getPushDownPredicate() == null ? null : mapper.map(node.getPushDownPredicate()), node.getPushDownLimit(), node.getPushDownOffset(), + node.getTagPredicate().map(mapper::map).orElse(null), + node.getTimePredicate().map(mapper::map).orElse(null), node.getScanOrder(), node.getPushedOrderingScheme().map(mapper::map).orElse(null), node.getTsFilePaths()), From 3d37f72ca0c498c73be1f7c60fcce33d1627de22 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Mon, 1 Jun 2026 11:45:24 +0800 Subject: [PATCH 04/32] fix explain analyze file num --- .../queryengine/execution/fragment/FragmentInstanceContext.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 67b36a6d23d87..62aa75f85c09c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -820,6 +820,7 @@ public boolean initExternalTsFileQueryDataSource(List externalTsFilePath this.sharedQueryDataSource = new QueryDataSource(Collections.emptyList(), externalTsFileResources); + closedUnseqFileNum = externalTsFileResources.size(); return true; } finally { addInitQueryDataSourceCost(System.nanoTime() - startTime); From 539196705470844b38af998c0ead4a00ecd24494 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Wed, 3 Jun 2026 14:51:29 +0800 Subject: [PATCH 05/32] fix --- .../execution/driver/DataDriverContext.java | 16 ------------- .../fragment/FragmentInstanceContext.java | 8 +++++-- .../MultiTsFileResourceIterator.java | 5 ++-- ...rderedExternalTsFileTableScanOperator.java | 24 +++++++++---------- .../DataNodeTableOperatorGenerator.java | 2 +- .../plan/planner/LocalExecutionPlanner.java | 16 ------------- .../planner/plan/node/PlanGraphPrinter.java | 21 ++++++++++++++++ ...ConvertSchemaPredicateToFilterVisitor.java | 21 ++++++++++++++-- 8 files changed, 60 insertions(+), 53 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java index a2892b6fec0bf..f895859d1adff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java @@ -41,7 +41,6 @@ public class DataDriverContext extends DriverContext { private List paths; private QueryDataSourceType queryDataSourceType = null; private Map deviceIDToContext; - private List externalTsFilePaths; // it will be set to null, after QueryDataSource being inited private List sourceOperators; @@ -50,7 +49,6 @@ public DataDriverContext(FragmentInstanceContext fragmentInstanceContext, int pi this.paths = new ArrayList<>(); this.sourceOperators = new ArrayList<>(); this.deviceIDToContext = null; - this.externalTsFilePaths = null; } public DataDriverContext(DataDriverContext parentContext, int pipelineId) { @@ -58,7 +56,6 @@ public DataDriverContext(DataDriverContext parentContext, int pipelineId) { this.paths = new ArrayList<>(); this.sourceOperators = new ArrayList<>(); this.deviceIDToContext = null; - this.externalTsFilePaths = null; } public void setQueryDataSourceType(QueryDataSourceType queryDataSourceType) { @@ -74,19 +71,6 @@ public void clearDeviceIDToContext() { deviceIDToContext = null; } - public void setExternalTsFilePaths(List externalTsFilePaths) { - this.externalTsFilePaths = externalTsFilePaths; - } - - public List getExternalTsFilePaths() { - return externalTsFilePaths; - } - - public void clearExternalTsFilePaths() { - // friendly for gc - externalTsFilePaths = null; - } - public void addPath(IFullPath path) { this.paths.add(path); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 62aa75f85c09c..56214b971f572 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -618,8 +618,12 @@ public void setDevicePathsToContext(Map devicePathsToC this.devicePathsToContext = devicePathsToContext; } - public void setExternalTsFilePaths(List externalTsFilePaths) { - this.externalTsFilePaths = externalTsFilePaths; + public void addExternalTsFilePaths(List externalTsFilePaths) { + if (this.externalTsFilePaths == null) { + this.externalTsFilePaths = new ArrayList<>(externalTsFilePaths); + return; + } + this.externalTsFilePaths.addAll(externalTsFilePaths); } public MemoryReservationManager getMemoryReservationContext() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java index 67bddc4facb55..c55d336a2246b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java @@ -151,9 +151,8 @@ public long[] getCurrentDeviceMeasurementNodeOffset(TsFileResource resource) { } private boolean isDeviceMatched(IDeviceID deviceID) { - return tableName.equalsIgnoreCase(deviceID.getTableName()) - && (deviceFilter == null - || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID))); + return deviceFilter == null + || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID)); } private class TsFileResourceDeviceIterator { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java index 1476ed889aadc..3b84e1a190858 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java @@ -36,10 +36,9 @@ import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.TsFileDeviceIterator; +import org.apache.tsfile.read.LazyTsFileDeviceIterator; import org.apache.tsfile.read.TsFileSequenceReader; import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.RamUsageEstimator; import java.io.IOException; @@ -136,17 +135,20 @@ private List collectDeviceInfos(QueryDataSource queryD private void collectDeviceInfos( TsFileResource resource, List deviceInfos) { try (TsFileSequenceReader reader = new TsFileSequenceReader(resource.getTsFilePath())) { - TsFileDeviceIterator deviceIterator = - reader.getTableDevicesIteratorWithIsAligned(tableName, contextValue -> {}); + LazyTsFileDeviceIterator deviceIterator = + new LazyTsFileDeviceIterator( + reader, + tableName, + ((OperatorContext) operatorContext) + .getInstanceContext() + .getQueryStatistics() + .getLoadTimeSeriesMetadataActualIOSize() + ::addAndGet); while (deviceIterator.hasNext()) { - Pair deviceInfo = deviceIterator.next(); - IDeviceID deviceID = deviceInfo.left; + IDeviceID deviceID = deviceIterator.next(); if (!isDeviceMatched(deviceID)) { continue; } - if (!isDeviceFilterMatched(deviceID)) { - continue; - } deviceInfos.add( new ExternalTsFileDeviceInfo( deviceID, resource, deviceIterator.getCurrentDeviceMeasurementNodeOffset())); @@ -167,10 +169,6 @@ private List getAllResources(QueryDataSource queryDataSource) { } private boolean isDeviceMatched(IDeviceID deviceID) { - return tableName.equalsIgnoreCase(deviceID.getTableName()); - } - - private boolean isDeviceFilterMatched(IDeviceID deviceID) { return deviceFilter == null || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index 3f678f7abddf0..a857e06330488 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -1149,9 +1149,9 @@ public Operator visitExternalTsFileScan( DataDriverContext dataDriverContext = (DataDriverContext) context.getDriverContext(); dataDriverContext.addSourceOperator(externalTsFileTableScanOperator); - dataDriverContext.setExternalTsFilePaths(node.getTsFilePaths()); dataDriverContext.setQueryDataSourceType(QueryDataSourceType.EXTERNAL_TSFILE_SCAN); dataDriverContext.setInputDriver(true); + context.getInstanceContext().addExternalTsFilePaths(node.getTsFilePaths()); return externalTsFileTableScanOperator; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanner.java index f05c49b466d54..95f28052c1fd5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanner.java @@ -124,7 +124,6 @@ public List plan( instanceContext.setSourcePaths(collectSourcePaths(context)); instanceContext.setDevicePathsToContext(collectDevicePathsToContext(context)); - instanceContext.setExternalTsFilePaths(collectExternalTsFilePaths(context)); instanceContext.setQueryDataSourceType( getQueryDataSourceType((DataDriverContext) context.getDriverContext())); @@ -265,21 +264,6 @@ private List collectSourcePaths(LocalExecutionPlanContext context) { return sourcePaths; } - private List collectExternalTsFilePaths(LocalExecutionPlanContext context) { - List externalTsFilePaths = new ArrayList<>(); - context - .getPipelineDriverFactories() - .forEach( - pipeline -> { - DataDriverContext dataDriverContext = (DataDriverContext) pipeline.getDriverContext(); - if (dataDriverContext.getExternalTsFilePaths() != null) { - externalTsFilePaths.addAll(dataDriverContext.getExternalTsFilePaths()); - } - dataDriverContext.clearExternalTsFilePaths(); - }); - return externalTsFilePaths; - } - public synchronized boolean forceAllocateFreeMemoryForOperators(long memoryInBytes) { // TODO @spricoder: consider a better way if (OPERATORS_MEMORY_BLOCK.getFreeMemoryInBytes() - memoryInBytes 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 e38ce40d48bb3..4338fe8ad4333 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 @@ -97,6 +97,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExchangeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; @@ -661,6 +662,10 @@ public List visitTableScan(TableScanNode node, GraphContext context) { if (node instanceof DeviceTableScanNode) { deviceTableScanNode = (DeviceTableScanNode) node; } + ExternalTsFileScanNode externalTsFileScanNode = null; + if (node instanceof ExternalTsFileScanNode) { + externalTsFileScanNode = (ExternalTsFileScanNode) node; + } List boxValue = new ArrayList<>(); boxValue.add(node.toString()); @@ -676,6 +681,22 @@ public List visitTableScan(TableScanNode node, GraphContext context) { String.format("TimePredicate: %s", deviceTableScanNode.getTimePredicate().get())); } } + if (externalTsFileScanNode != null) { + boxValue.add(String.format("ScanOrder: %s", externalTsFileScanNode.getScanOrder())); + boxValue.add(String.format("TsFilePaths: %s", externalTsFileScanNode.getTsFilePaths())); + externalTsFileScanNode + .getPushedOrderingScheme() + .ifPresent( + orderingScheme -> + boxValue.add(String.format("PushedOrderingScheme: %s", orderingScheme))); + externalTsFileScanNode + .getTimePredicate() + .ifPresent( + timePredicate -> boxValue.add(String.format("TimePredicate: %s", timePredicate))); + externalTsFileScanNode + .getTagPredicate() + .ifPresent(tagPredicate -> boxValue.add(String.format("TagPredicate: %s", tagPredicate))); + } if (node.getPushDownPredicate() != null) { boxValue.add(String.format("PushDownPredicate: %s", node.getPushDownPredicate())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java index 510c748aceea4..7bcf09fd73a3d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java @@ -55,6 +55,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -234,8 +235,24 @@ public SchemaFilter visitNullIfExpression(final NullIfExpression node, final Con } @Override - public SchemaFilter visitBetweenPredicate(final BetweenPredicate node, final Context context) { - return visitExpression(node, context); + public @Nullable SchemaFilter visitBetweenPredicate( + final BetweenPredicate node, final Context context) { + final SchemaFilter lowerBoundFilter = + new ComparisonExpression( + ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, + node.getMin(), + node.getValue()) + .accept(this, context); + final SchemaFilter upperBoundFilter = + new ComparisonExpression( + ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, + node.getValue(), + node.getMax()) + .accept(this, context); + if (Objects.isNull(lowerBoundFilter) || Objects.isNull(upperBoundFilter)) { + return null; + } + return new AndFilter(Arrays.asList(lowerBoundFilter, upperBoundFilter)); } private SchemaFilter wrapTagOrAttributeFilter( From 194e609befce9dbc28f1b73e6223f9f2f80ca69a Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Thu, 4 Jun 2026 11:11:34 +0800 Subject: [PATCH 06/32] FileReaderManager --- .../fragment/FragmentInstanceContext.java | 16 +++- .../execution/fragment/QueryContext.java | 4 + .../operator/source/FileLoaderUtils.java | 10 ++- ...rderedExternalTsFileTableScanOperator.java | 6 +- ...ConvertSchemaPredicateToFilterVisitor.java | 8 +- .../buffer/BloomFilterCache.java | 24 ++++- .../db/storageengine/buffer/ChunkCache.java | 31 +++++-- .../buffer/TimeSeriesMetadataCache.java | 27 +++++- .../read/control/FileReaderManager.java | 88 +++++++++++++++++++ .../reader/chunk/DiskAlignedChunkLoader.java | 6 +- .../read/reader/chunk/DiskChunkLoader.java | 6 +- 11 files changed, 196 insertions(+), 30 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 56214b971f572..654df2301d884 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -122,6 +122,7 @@ public class FragmentInstanceContext extends QueryContext { private Map devicePathsToContext; private List externalTsFilePaths; + private List externalTsFileResources; // Shared by all scan operators in this fragment instance to avoid memory problem protected IQueryDataSource sharedQueryDataSource; @@ -236,6 +237,11 @@ public void setQueryDataSourceType(QueryDataSourceType queryDataSourceType) { this.queryDataSourceType = queryDataSourceType; } + @Override + public boolean isExternalTsFileScan() { + return queryDataSourceType == QueryDataSourceType.EXTERNAL_TSFILE_SCAN; + } + @TestOnly public static FragmentInstanceContext createFragmentInstanceContext( FragmentInstanceId id, FragmentInstanceStateMachine stateMachine) { @@ -802,7 +808,7 @@ public boolean initExternalTsFileQueryDataSource(List externalTsFilePath return true; } - List externalTsFileResources = new ArrayList<>(externalTsFilePaths.size()); + externalTsFileResources = new ArrayList<>(externalTsFilePaths.size()); for (String externalTsFilePath : externalTsFilePaths) { TsFileResource resource = new TsFileResource(new File(externalTsFilePath), TsFileResourceStatus.NORMAL); @@ -820,6 +826,7 @@ public boolean initExternalTsFileQueryDataSource(List externalTsFilePath resource.setTimeIndex(new FileTimeIndex(Long.MIN_VALUE, Long.MAX_VALUE)); } externalTsFileResources.add(resource); + FileReaderManager.getInstance().increaseExternalFileReaderReference(externalTsFilePath); } this.sharedQueryDataSource = @@ -1075,6 +1082,13 @@ public synchronized void releaseResource() { unClosedFilePaths = null; } + if (externalTsFileResources != null) { + for (TsFileResource tsFile : externalTsFileResources) { + FileReaderManager.getInstance().decreaseExternalFileReaderReference(tsFile.getTsFilePath()); + } + externalTsFileResources = null; + } + // release TVList/AlignedTVList owned by current query releaseTVListOwnedByQuery(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index d3edbe2e0d063..f89d8ddefe508 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -97,6 +97,10 @@ public QueryContext(long queryId, boolean debug, long startTime, long timeout) { this.timeout = timeout; } + public boolean isExternalTsFileScan() { + return false; + } + // Only used for query with table data(Tree view is not included) public boolean collectTable(String table) { // In the current version (2025.08.14), there is only one table under one FI diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java index 254c061187a43..dcc38e49a61e7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java @@ -110,7 +110,8 @@ public static TimeseriesMetadata loadTimeSeriesMetadata( context.ignoreNotExistsDevice() || resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE, context.isDebug(), - context); + context, + context.isExternalTsFileScan()); if (timeSeriesMetadata != null) { long t2 = System.nanoTime(); List pathModifications = @@ -297,6 +298,7 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr boolean isDebug = context.isDebug(); String filePath = resource.getTsFilePath(); IDeviceID deviceId = alignedPath.getDeviceId(); + boolean isExternalTsFile = context.isExternalTsFileScan(); // when resource.getTimeIndexType() == 1, TsFileResource.timeIndexType is deviceTimeIndex // we should not ignore the non-exist of device in TsFileMetadata @@ -308,7 +310,8 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr context.ignoreNotExistsDevice() || resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE, isDebug, - context); + context, + isExternalTsFile); if (timeColumn != null) { // only need time column, like count_time aggregation if (valueMeasurementList.isEmpty()) { @@ -337,7 +340,8 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr context.ignoreNotExistsDevice() || resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE, isDebug, - context); + context, + isExternalTsFile); exist = (exist || (valueColumn != null)); valueTimeSeriesMetadataList.add(valueColumn); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java index 3b84e1a190858..2b1fdf561dfc6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java @@ -140,9 +140,9 @@ private void collectDeviceInfos( reader, tableName, ((OperatorContext) operatorContext) - .getInstanceContext() - .getQueryStatistics() - .getLoadTimeSeriesMetadataActualIOSize() + .getInstanceContext() + .getQueryStatistics() + .getLoadTimeSeriesMetadataActualIOSize() ::addAndGet); while (deviceIterator.hasNext()) { IDeviceID deviceID = deviceIterator.next(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java index 7bcf09fd73a3d..a7781bc1e64d9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java @@ -239,15 +239,11 @@ public SchemaFilter visitNullIfExpression(final NullIfExpression node, final Con final BetweenPredicate node, final Context context) { final SchemaFilter lowerBoundFilter = new ComparisonExpression( - ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, - node.getMin(), - node.getValue()) + ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, node.getMin(), node.getValue()) .accept(this, context); final SchemaFilter upperBoundFilter = new ComparisonExpression( - ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, - node.getValue(), - node.getMax()) + ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, node.getValue(), node.getMax()) .accept(this, context); if (Objects.isNull(lowerBoundFilter) || Objects.isNull(upperBoundFilter)) { return null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java index 380583efbdb9f..eb0ae5554a57c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java @@ -101,9 +101,20 @@ public BloomFilter get( LongConsumer cacheHitAdder, LongConsumer cacheMissAdder) throws IOException { - BloomFilterLoader loader = new BloomFilterLoader(ioSizeRecorder); + return get(key, debug, ioSizeRecorder, cacheHitAdder, cacheMissAdder, false); + } + + public BloomFilter get( + BloomFilterCacheKey key, + boolean debug, + LongConsumer ioSizeRecorder, + LongConsumer cacheHitAdder, + LongConsumer cacheMissAdder, + boolean externalTsFile) + throws IOException { + BloomFilterLoader loader = new BloomFilterLoader(ioSizeRecorder, externalTsFile); try { - if (!CACHE_ENABLE) { + if (!CACHE_ENABLE || externalTsFile) { return loader.apply(key); } @@ -203,9 +214,15 @@ private static class BloomFilterLoader implements Function timeRangeList, + Statistics chunkStatistic, + QueryContext queryContext, + boolean externalTsFile) + throws IOException { LongConsumer ioSizeRecorder = queryContext.getQueryStatistics().getLoadChunkActualIOSize()::addAndGet; LongConsumer cacheHitAdder = @@ -143,7 +154,8 @@ public Chunk get( queryContext.isDebug(), ioSizeRecorder, cacheHitAdder, - cacheMissAdder); + cacheMissAdder, + externalTsFile); } private Chunk get( @@ -153,12 +165,13 @@ private Chunk get( boolean debug, LongConsumer ioSizeRecorder, LongConsumer cacheHitAdder, - LongConsumer cacheMissAdder) + LongConsumer cacheMissAdder, + boolean externalTsFile) throws IOException { long startTime = System.nanoTime(); - ChunkLoader chunkLoader = new ChunkLoader(ioSizeRecorder); + ChunkLoader chunkLoader = new ChunkLoader(ioSizeRecorder, externalTsFile); try { - if (!CACHE_ENABLE) { + if (!CACHE_ENABLE || externalTsFile) { Chunk chunk = chunkLoader.apply(chunkCacheKey); return constructChunk(chunk, timeRangeList, chunkStatistic); } @@ -297,9 +310,15 @@ private static class ChunkLoader implements Function { private boolean cacheMiss = false; private final LongConsumer ioSizeRecorder; + private final boolean externalTsFile; private ChunkLoader(LongConsumer ioSizeRecorder) { + this(ioSizeRecorder, false); + } + + private ChunkLoader(LongConsumer ioSizeRecorder, boolean externalTsFile) { this.ioSizeRecorder = ioSizeRecorder; + this.externalTsFile = externalTsFile; } @Override @@ -310,7 +329,7 @@ public Chunk apply(ChunkCacheKey key) { cacheMiss = true; TsFileSequenceReader reader = FileReaderManager.getInstance() - .get(key.getFilePath(), key.tsFileID, key.closed, ioSizeRecorder); + .get(key.getFilePath(), key.tsFileID, key.closed, ioSizeRecorder, externalTsFile); Chunk chunk = reader.readMemChunk(key.offsetOfChunkHeader, ioSizeRecorder); // to save memory footprint, we don't save measurementId in ChunkHeader of Chunk chunk.getHeader().setMeasurementID(null); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java index cc3b2ee5aff2c..ba26db0f1d711 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java @@ -119,6 +119,19 @@ public TimeseriesMetadata get( boolean debug, QueryContext queryContext) throws IOException { + return get(filePath, key, allSensors, ignoreNotExists, debug, queryContext, false); + } + + @SuppressWarnings({"squid:S1860", "squid:S6541", "squid:S3776"}) // Suppress synchronize warning + public TimeseriesMetadata get( + String filePath, + TimeSeriesMetadataCacheKey key, + Set allSensors, + boolean ignoreNotExists, + boolean debug, + QueryContext queryContext, + boolean externalTsFile) + throws IOException { long startTime = System.nanoTime(); long loadBloomFilterTime = 0; LongConsumer timeSeriesMetadataIoSizeRecorder = @@ -127,14 +140,14 @@ public TimeseriesMetadata get( queryContext.getQueryStatistics().getLoadBloomFilterActualIOSize()::addAndGet; boolean cacheHit = true; try { - if (!CACHE_ENABLE) { + if (!CACHE_ENABLE || externalTsFile) { String deviceStringFormat = key.device.toString(); cacheHit = false; // bloom filter part TsFileSequenceReader reader = FileReaderManager.getInstance() - .get(filePath, key.tsFileID, true, bloomFilterIoSizeRecorder); + .get(filePath, key.tsFileID, true, bloomFilterIoSizeRecorder, externalTsFile); BloomFilter bloomFilter = reader.readBloomFilter(bloomFilterIoSizeRecorder); queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount().incrementAndGet(); if (bloomFilter != null @@ -182,7 +195,8 @@ public TimeseriesMetadata get( queryContext.getQueryStatistics().getLoadBloomFilterFromCacheCount() ::addAndGet, queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount() - ::addAndGet); + ::addAndGet, + externalTsFile); if (bloomFilter != null && !bloomFilter.contains( deviceStringFormat + TsFileConstant.PATH_SEPARATOR + key.measurement)) { @@ -196,7 +210,12 @@ public TimeseriesMetadata get( loadBloomFilterTime = System.nanoTime() - loadBloomFilterStartTime; TsFileSequenceReader reader = FileReaderManager.getInstance() - .get(filePath, key.tsFileID, true, timeSeriesMetadataIoSizeRecorder); + .get( + filePath, + key.tsFileID, + true, + timeSeriesMetadataIoSizeRecorder, + externalTsFile); List timeSeriesMetadataList = reader.readTimeseriesMetadata( key.device, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java index 3952c7480a59d..d2dac0b775bc6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java @@ -80,11 +80,19 @@ public class FileReaderManager { */ private Map unclosedReferenceMap; + /** External TsFile readers. The key is the file path. */ + private Map externalFileReaderMap; + + /** Reference count of external TsFile readers. */ + private Map externalReferenceMap; + private FileReaderManager() { closedFileReaderMap = new ConcurrentHashMap<>(); unclosedFileReaderMap = new ConcurrentHashMap<>(); closedReferenceMap = new ConcurrentHashMap<>(); unclosedReferenceMap = new ConcurrentHashMap<>(); + externalFileReaderMap = new ConcurrentHashMap<>(); + externalReferenceMap = new ConcurrentHashMap<>(); } public static FileReaderManager getInstance() { @@ -137,6 +145,20 @@ public synchronized TsFileSequenceReader get(String filePath, TsFileID tsFileID, public synchronized TsFileSequenceReader get( String filePath, TsFileID tsFileID, boolean isClosed, LongConsumer ioSizeRecorder) throws IOException { + return get(filePath, tsFileID, isClosed, ioSizeRecorder, false); + } + + @SuppressWarnings("squid:S2095") + public synchronized TsFileSequenceReader get( + String filePath, + TsFileID tsFileID, + boolean isClosed, + LongConsumer ioSizeRecorder, + boolean isExternalTsFile) + throws IOException { + if (isExternalTsFile) { + return getExternalTsFileReader(filePath, ioSizeRecorder); + } Map readerMap = !isClosed ? unclosedFileReaderMap : closedFileReaderMap; @@ -170,6 +192,25 @@ public synchronized TsFileSequenceReader get( return readerMap.get(tsFileID); } + private TsFileSequenceReader getExternalTsFileReader(String filePath, LongConsumer ioSizeRecorder) + throws IOException { + TsFileSequenceReader reader = externalFileReaderMap.get(filePath); + if (reader == null) { + int currentOpenedReaderCount = externalFileReaderMap.size(); + if (currentOpenedReaderCount >= MAX_CACHED_FILE_SIZE + && (currentOpenedReaderCount % PRINT_INTERVAL == 0)) { + logger.warn(StorageEngineMessages.QUERY_OPENED_FILES, externalFileReaderMap.size()); + } + reader = + new TsFileSequenceReader( + filePath, + ioSizeRecorder, + EncryptDBUtils.getFirstEncryptParamFromTSFilePath(filePath)); + externalFileReaderMap.put(filePath, reader); + } + return reader; + } + /** * Increase the reference count of the reader specified by filePath. Only when the reference count * of a reader equals zero, the reader can be closed and removed. @@ -189,6 +230,10 @@ public void increaseFileReaderReference(TsFileResource tsFile, boolean isClosed) } } + public synchronized void increaseExternalFileReaderReference(String filePath) { + externalReferenceMap.computeIfAbsent(filePath, k -> new AtomicInteger()).getAndIncrement(); + } + /** * Decrease the reference count of the reader specified by filePath. This method is latch-free. * Only when the reference count of a reader equals zero, the reader can be closed and removed. @@ -207,6 +252,38 @@ public void decreaseFileReaderReference(TsFileResource tsFile, boolean isClosed) tsFile.readUnlock(); } + public synchronized void decreaseExternalFileReaderReference(String filePath) { + AtomicInteger reference = externalReferenceMap.get(filePath); + if (reference != null && reference.decrementAndGet() == 0) { + closeUnUsedExternalReaderAndRemoveRef(filePath); + } + } + + private void closeUnUsedExternalReaderAndRemoveRef(String readerKey) { + synchronized (this) { + AtomicInteger reference = externalReferenceMap.get(readerKey); + if (reference != null && reference.get() != 0) { + return; + } + + TsFileSequenceReader reader = externalFileReaderMap.get(readerKey); + if (reader != null) { + try { + reader.close(); + } catch (IOException e) { + logger.error( + StorageEngineMessages.CANNOT_CLOSE_TSFILE_SEQUENCE_READER, reader.getFileName(), e); + } + } + externalFileReaderMap.remove(readerKey); + externalReferenceMap.remove(readerKey); + if (resourceLogger.isDebugEnabled()) { + resourceLogger.debug( + "{} externalTsFileReader is closed because of no reference.", readerKey); + } + } + } + private void closeUnUsedReaderAndRemoveRef( String tsFilePath, TsFileID tsFileID, boolean isClosed) { Map readerMap = @@ -263,6 +340,17 @@ public synchronized void closeAndRemoveAllOpenedReaders() throws IOException { unclosedReferenceMap.remove(entry.getKey()); iterator.remove(); } + Iterator> externalIterator = + externalFileReaderMap.entrySet().iterator(); + while (externalIterator.hasNext()) { + Map.Entry entry = externalIterator.next(); + entry.getValue().close(); + if (resourceLogger.isDebugEnabled()) { + resourceLogger.debug("{} externalTsFileReader is closed.", entry.getKey()); + } + externalReferenceMap.remove(entry.getKey()); + externalIterator.remove(); + } } /** This method is only for unit tests. */ diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java index 7c1e9d1263518..2ea8ec0c6bab9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java @@ -92,7 +92,8 @@ public IChunkReader getChunkReader(IChunkMetadata chunkMetaData, Filter globalTi resource.isClosed()), timeChunkMetadata.getDeleteIntervalList(), timeChunkMetadata.getStatistics(), - context); + context, + context.isExternalTsFileScan()); List valueChunkList = new ArrayList<>(); for (IChunkMetadata valueChunkMetadata : alignedChunkMetadata.getValueChunkMetadataList()) { Chunk chunk = @@ -107,7 +108,8 @@ public IChunkReader getChunkReader(IChunkMetadata chunkMetaData, Filter globalTi resource.isClosed()), valueChunkMetadata.getDeleteIntervalList(), valueChunkMetadata.getStatistics(), - context); + context, + context.isExternalTsFileScan()); final TsFileID tsFileID = getTsFileID(); if (chunk != null && tsFileID.regionId > 0 diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java index 7d259a9e50534..73355bed0c85b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java @@ -64,7 +64,8 @@ public Chunk loadChunk(ChunkMetadata chunkMetaData) throws IOException { resource.isClosed()), chunkMetaData.getDeleteIntervalList(), chunkMetaData.getStatistics(), - context); + context, + context.isExternalTsFileScan()); } @Override @@ -87,7 +88,8 @@ public IChunkReader getChunkReader(IChunkMetadata chunkMetaData, Filter globalTi resource.isClosed()), chunkMetaData.getDeleteIntervalList(), chunkMetaData.getStatistics(), - context); + context, + context.isExternalTsFileScan()); byte chunkType = chunk.getHeader().getChunkType(); if (chunkType != MetaMarker.CHUNK_HEADER && chunkType != MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) { From 342c4e1e17d9b6c60934186eb3ec6083e8358aad Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Thu, 4 Jun 2026 18:38:44 +0800 Subject: [PATCH 07/32] impl 2 --- .../ExternalTsFileTableScanOperator.java | 181 ++++++++++++++ ...rderedExternalTsFileTableScanOperator.java | 71 +++++- .../DataNodeTableOperatorGenerator.java | 52 +--- .../planner/plan/node/PlanGraphPrinter.java | 8 - .../TableDistributedPlanGenerator.java | 144 +++++++++++- .../iterative/rule/PruneTableScanColumns.java | 6 +- .../planner/node/ExternalTsFileScanNode.java | 222 +++++++++--------- .../PushPredicateIntoTableScan.java | 94 +++++++- .../UnaliasSymbolReferences.java | 6 +- .../tvf/ReadTsFileTableFunction.java | 47 ++-- 10 files changed, 617 insertions(+), 214 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java new file mode 100644 index 0000000000000..0c68ec96c9bae --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java @@ -0,0 +1,181 @@ +/* + * 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.execution.operator.source.relational; + +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.iotdb.calc.plan.planner.CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING; + +public class ExternalTsFileTableScanOperator extends AbstractTableScanOperator { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(ExternalTsFileTableScanOperator.class); + + private final String tableName; + private final List deviceEntries; + private final List> deviceOffsets; + + private DeviceEntry currentDeviceEntry; + private List currentDeviceOffsets; + private int currentDeviceIndex; + + public ExternalTsFileTableScanOperator( + AbstractTableScanOperatorParameter parameter, + String tableName, + List deviceEntries, + List> deviceOffsets) { + super(parameter); + this.tableName = tableName; + this.deviceEntries = new ArrayList<>(deviceEntries); + this.deviceOffsets = new ArrayList<>(deviceOffsets); + this.currentDeviceIndex = 0; + } + + @Override + String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { + int segmentOffset = + deviceEntry.getDeviceID().segmentNum() > 0 + && tableName.equalsIgnoreCase((String) deviceEntry.getNthSegment(0)) + ? 1 + : 0; + Object segment = deviceEntry.getNthSegment(idColumnIndex + segmentOffset); + return segment == null ? null : (String) segment; + } + + @Override + public void initQueryDataSource(IQueryDataSource dataSource) { + super.initQueryDataSource(dataSource); + currentDeviceEntry = nextDeviceEntry(); + recordCurrentDeviceIndex(); + constructAlignedSeriesScanUtil(); + if (seriesScanUtil != null) { + seriesScanUtil.initQueryDataSource((QueryDataSource) dataSource); + } + } + + private DeviceEntry nextDeviceEntry() { + if (currentDeviceIndex >= deviceEntries.size()) { + currentDeviceOffsets = null; + return null; + } + DeviceEntry deviceEntry = deviceEntries.get(currentDeviceIndex); + currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); + return deviceEntry; + } + + @Override + protected boolean hasCurrentDeviceEntry() { + return currentDeviceEntry != null; + } + + @Override + protected DeviceEntry getCurrentDeviceEntry() { + return currentDeviceEntry; + } + + @Override + protected boolean advanceDeviceEntry() { + currentDeviceIndex++; + currentDeviceEntry = nextDeviceEntry(); + return currentDeviceEntry != null; + } + + @Override + protected void recordCurrentDeviceIndex() { + operatorContext.recordSpecifiedInfo( + CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); + } + + @Override + protected void constructAlignedSeriesScanUtil() { + if (!hasCurrentDeviceEntry()) { + return; + } + + DeviceEntry deviceEntry = getCurrentDeviceEntry(); + if (deviceEntry == null) { + throw new IllegalStateException("Current device entry in TableScanOperator is empty"); + } + + this.seriesScanUtil = + new ExternalTsFileSeriesScanUtil( + constructAlignedPath( + deviceEntry, measurementColumnNames, measurementSchemas, allSensors), + scanOrder, + seriesScanOptions, + ((OperatorContext) operatorContext).getInstanceContext(), + true, + measurementColumnTSDataTypes, + this::loadTimeSeriesMetadata); + } + + private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedPath) throws IOException { + if (currentDeviceOffsets == null + || !getCurrentDeviceEntry().getDeviceID().equals(alignedPath.getDeviceId())) { + return null; + } + + long[] deviceMeasurementNodeOffset = getDeviceMeasurementNodeOffset(resource.getTsFilePath()); + if (deviceMeasurementNodeOffset == null) { + return null; + } + // TODO: Use deviceMeasurementNodeOffset after FileLoaderUtils supports offset-based metadata + // loading in this branch. + return FileLoaderUtils.loadAlignedTimeSeriesMetadata( + resource, + alignedPath, + ((OperatorContext) operatorContext).getInstanceContext(), + seriesScanOptions.getGlobalTimeFilter(), + resource.isSeq(), + ((OperatorContext) operatorContext).getInstanceContext().isIgnoreAllNullRows()); + } + + private long[] getDeviceMeasurementNodeOffset(String tsFilePath) { + for (ExternalTsFileDeviceOffset offset : currentDeviceOffsets) { + if (tsFilePath.equals(offset.getTsFilePath())) { + return offset.getDeviceMeasurementNodeOffset(); + } + } + return null; + } + + @Override + public long ramBytesUsed() { + return super.ramBytesUsed() + + INSTANCE_SIZE + - AbstractTableScanOperator.INSTANCE_SIZE + + RamUsageEstimator.sizeOfCollection(deviceEntries) + + RamUsageEstimator.sizeOfCollection(deviceOffsets); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java index b1eb554a3ef46..215fdd66f33fe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java @@ -19,8 +19,11 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; +import org.apache.iotdb.commons.path.AlignedFullPath; import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; @@ -28,6 +31,7 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.utils.EncryptDBUtils; +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.TsFileSequenceReader; import org.apache.tsfile.utils.Binary; @@ -48,17 +52,28 @@ public class UnorderedExternalTsFileTableScanOperator extends AbstractTableScanO private final String tableName; private final SchemaFilter deviceFilter; + private final List deviceEntries; + private final List> deviceOffsets; + private final ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = + new ExternalTsFileDeviceFilterVisitor(); private MultiTsFileResourceIterator deviceIterator; private Map resourceReaderMap = Collections.emptyMap(); private DeviceEntry currentDeviceEntry; + private List currentDeviceOffsets; private int currentDeviceIndex; public UnorderedExternalTsFileTableScanOperator( - AbstractTableScanOperatorParameter parameter, String tableName, SchemaFilter deviceFilter) { + AbstractTableScanOperatorParameter parameter, + String tableName, + SchemaFilter deviceFilter, + List deviceEntries, + List> deviceOffsets) { super(parameter); this.tableName = tableName; this.deviceFilter = deviceFilter; + this.deviceEntries = deviceEntries; + this.deviceOffsets = deviceOffsets; this.currentDeviceIndex = 0; } @@ -78,7 +93,9 @@ public void initQueryDataSource(IQueryDataSource dataSource) { super.initQueryDataSource(dataSource); QueryDataSource queryDataSource = (QueryDataSource) dataSource; - initDeviceIterator(queryDataSource); + if (deviceEntries.isEmpty()) { + initDeviceIterator(queryDataSource); + } currentDeviceEntry = nextDeviceEntry(); recordCurrentDeviceIndex(); constructAlignedSeriesScanUtil(); @@ -134,6 +151,19 @@ private List getAllResources(QueryDataSource queryDataSource) { } private DeviceEntry nextDeviceEntry() { + if (!deviceEntries.isEmpty()) { + while (currentDeviceIndex < deviceEntries.size()) { + currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); + DeviceEntry deviceEntry = deviceEntries.get(currentDeviceIndex); + if (isDeviceMatched(deviceEntry.getDeviceID())) { + return deviceEntry; + } + currentDeviceIndex++; + } + currentDeviceOffsets = null; + return null; + } + if (deviceIterator == null || !deviceIterator.hasNextDevice()) { return null; } @@ -184,7 +214,42 @@ protected void constructAlignedSeriesScanUtil() { ((OperatorContext) operatorContext).getInstanceContext(), true, measurementColumnTSDataTypes, - deviceIterator); + deviceEntries.isEmpty() + ? deviceIterator::loadTimeSeriesMetadata + : this::loadTimeSeriesMetadata); + } + + private boolean isDeviceMatched(IDeviceID deviceID) { + return deviceFilter == null + || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID)); + } + + private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedPath) throws IOException { + if (currentDeviceOffsets == null + || !getCurrentDeviceEntry().getDeviceID().equals(alignedPath.getDeviceId())) { + return null; + } + if (!containsCurrentDevice(resource)) { + return null; + } + return FileLoaderUtils.loadAlignedTimeSeriesMetadata( + resource, + alignedPath, + ((OperatorContext) operatorContext).getInstanceContext(), + seriesScanOptions.getGlobalTimeFilter(), + resource.isSeq(), + ((OperatorContext) operatorContext).getInstanceContext().isIgnoreAllNullRows()); + } + + private boolean containsCurrentDevice(TsFileResource resource) { + String tsFilePath = resource.getTsFilePath(); + for (ExternalTsFileDeviceOffset offset : currentDeviceOffsets) { + if (tsFilePath.equals(offset.getTsFilePath())) { + return true; + } + } + return false; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index a857e06330488..6faca66d7aa74 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -51,9 +51,7 @@ import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.LongLiteral; import org.apache.iotdb.commons.queryengine.plan.relational.type.InternalTypeManager; import org.apache.iotdb.commons.queryengine.utils.TimestampPrecisionUtils; -import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.schema.table.TsTable; -import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -98,22 +96,20 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.relational.CteScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.DefaultAggTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.DeviceIteratorScanOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.InformationSchemaTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.LastQueryAggTableScanOperator; -import org.apache.iotdb.db.queryengine.execution.operator.source.relational.OrderedExternalTsFileTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TreeAlignedDeviceViewAggregationScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TreeAlignedDeviceViewScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TreeNonAlignedDeviceViewAggregationScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TreeToTableViewAdaptorOperator; -import org.apache.iotdb.db.queryengine.execution.operator.source.relational.UnorderedExternalTsFileTableScanOperator; import org.apache.iotdb.db.queryengine.plan.analyze.cache.schema.DataNodeTTLCache; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.CountSchemaMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.sink.IdentitySinkNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.ConvertPredicateToTimeFilterVisitor; -import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.schema.ConvertSchemaPredicateToFilterVisitor; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.metadata.NonAlignedDeviceEntry; @@ -1132,18 +1128,13 @@ public Operator visitExternalTsFileScan( ExternalTsFileScanNode node, LocalExecutionPlanContext context) { AbstractTableScanOperator.AbstractTableScanOperatorParameter parameter = constructExternalTsFileTableScanOperatorParameter(node, context); - SchemaFilter deviceFilter = constructExternalTsFileDeviceFilter(node); AbstractTableScanOperator externalTsFileTableScanOperator = - node.getPushedOrderingScheme().isPresent() - ? new OrderedExternalTsFileTableScanOperator( - parameter, - node.getQualifiedObjectName().getObjectName(), - node.getAssignments(), - node.getPushedOrderingScheme().get(), - deviceFilter) - : new UnorderedExternalTsFileTableScanOperator( - parameter, node.getQualifiedObjectName().getObjectName(), deviceFilter); + new ExternalTsFileTableScanOperator( + parameter, + node.getQualifiedObjectName().getObjectName(), + node.getDeviceEntries(), + node.getDeviceOffsets()); context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); @@ -1156,31 +1147,6 @@ public Operator visitExternalTsFileScan( return externalTsFileTableScanOperator; } - private SchemaFilter constructExternalTsFileDeviceFilter(ExternalTsFileScanNode node) { - if (!node.getTagPredicate().isPresent()) { - return null; - } - TsTable table = new TsTable(node.getQualifiedObjectName().getObjectName()); - for (Map.Entry entry : node.getAssignments().entrySet()) { - ColumnSchema columnSchema = entry.getValue(); - if (columnSchema.getColumnCategory() == TsTableColumnCategory.TAG) { - table.addColumnSchema( - new TagColumnSchema(entry.getKey().getName(), getTSDataType(columnSchema.getType()))); - } - } - SchemaFilter deviceFilter = - node.getTagPredicate() - .get() - .accept( - new ConvertSchemaPredicateToFilterVisitor(), - new ConvertSchemaPredicateToFilterVisitor.Context(table)); - if (deviceFilter == null) { - throw new UnsupportedOperationException( - "Unsupported external TsFile device filter: " + node.getTagPredicate().get()); - } - return deviceFilter; - } - private AbstractTableScanOperator.AbstractTableScanOperatorParameter constructExternalTsFileTableScanOperatorParameter( ExternalTsFileScanNode node, LocalExecutionPlanContext context) { @@ -1202,11 +1168,7 @@ private SchemaFilter constructExternalTsFileDeviceFilter(ExternalTsFileScanNode OperatorContext operatorContext = addOperatorContext( - context, - node.getPlanNodeId(), - node.getPushedOrderingScheme().isPresent() - ? OrderedExternalTsFileTableScanOperator.class.getSimpleName() - : UnorderedExternalTsFileTableScanOperator.class.getSimpleName()); + context, node.getPlanNodeId(), ExternalTsFileTableScanOperator.class.getSimpleName()); Set allSensors = new HashSet<>(commonParameter.measurementColumnNames); // for time column 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 4338fe8ad4333..3ab8b283d0b31 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 @@ -684,18 +684,10 @@ public List visitTableScan(TableScanNode node, GraphContext context) { if (externalTsFileScanNode != null) { boxValue.add(String.format("ScanOrder: %s", externalTsFileScanNode.getScanOrder())); boxValue.add(String.format("TsFilePaths: %s", externalTsFileScanNode.getTsFilePaths())); - externalTsFileScanNode - .getPushedOrderingScheme() - .ifPresent( - orderingScheme -> - boxValue.add(String.format("PushedOrderingScheme: %s", orderingScheme))); externalTsFileScanNode .getTimePredicate() .ifPresent( timePredicate -> boxValue.add(String.format("TimePredicate: %s", timePredicate))); - externalTsFileScanNode - .getTagPredicate() - .ifPresent(tagPredicate -> boxValue.add(String.format("TagPredicate: %s", tagPredicate))); } if (node.getPushDownPredicate() != null) { 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 6c54af74445b0..d6cc700a14f7f 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 @@ -73,6 +73,7 @@ import org.apache.iotdb.commons.schema.table.InformationSchema; import org.apache.iotdb.commons.schema.table.TsTable; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -731,16 +732,74 @@ public List visitDeviceTableScan( @Override public List visitExternalTsFileScan( final ExternalTsFileScanNode node, final PlanContext context) { - node.setRegionReplicaSet( - new TRegionReplicaSet( - null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation()))); + TRegionReplicaSet localRegionReplicaSet = + new TRegionReplicaSet(null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())); + node.setRegionReplicaSet(localRegionReplicaSet); context.mostUsedRegion = node.getRegionReplicaSet(); if (context.hasSortProperty) { processExternalTsFileSortProperty(node, context); + return Collections.singletonList(node); + } + + List splitNodes = splitExternalTsFileScanByDeviceEntries(node, localRegionReplicaSet); + if (!splitNodes.isEmpty()) { + return splitNodes; } return Collections.singletonList(node); } + private List splitExternalTsFileScanByDeviceEntries( + final ExternalTsFileScanNode node, final TRegionReplicaSet localRegionReplicaSet) { + List deviceEntries = node.getDeviceEntries(); + if (deviceEntries.size() <= 1) { + return Collections.emptyList(); + } + + int splitCount = + Math.min( + deviceEntries.size(), + IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism()); + if (splitCount <= 1) { + return Collections.emptyList(); + } + + List> splitDeviceEntries = new ArrayList<>(splitCount); + List>> splitDeviceOffsets = new ArrayList<>(splitCount); + for (int i = 0; i < splitCount; i++) { + splitDeviceEntries.add(new ArrayList<>()); + splitDeviceOffsets.add(new ArrayList<>()); + } + for (int i = 0; i < deviceEntries.size(); i++) { + splitDeviceEntries.get(i % splitCount).add(deviceEntries.get(i)); + splitDeviceOffsets.get(i % splitCount).add(node.getDeviceOffsets().get(i)); + } + + List result = new ArrayList<>(splitCount); + for (int i = 0; i < splitDeviceEntries.size(); i++) { + List entries = splitDeviceEntries.get(i); + if (entries.isEmpty()) { + continue; + } + ExternalTsFileScanNode splitNode = + new ExternalTsFileScanNode( + queryId.genPlanNodeId(), + node.getQualifiedObjectName(), + node.getOutputSymbols(), + node.getAssignments(), + node.getPushDownPredicate(), + node.getPushDownLimit(), + node.getPushDownOffset(), + node.getTimePredicate().orElse(null), + node.getScanOrder(), + node.getTsFilePaths(), + entries, + splitDeviceOffsets.get(i)); + splitNode.setRegionReplicaSet(localRegionReplicaSet); + result.add(splitNode); + } + return result; + } + private List constructDeviceTableScanByTags( final DeviceTableScanNode node, final PlanContext context) { DataPartition dataPartition = analysis.getDataPartitionInfo(); @@ -1969,13 +2028,32 @@ private void processExternalTsFileSortProperty( return; } - OrderingScheme pushedOrderingScheme = + OrderingScheme orderingScheme = new OrderingScheme( newOrderingSymbols, IntStream.range(0, newOrderingSymbols.size()) .boxed() .collect(Collectors.toMap(newOrderingSymbols::get, newSortOrders::get))); - externalTsFileScanNode.setPushedOrderingScheme(pushedOrderingScheme); + + Comparator comparator = + createExternalTsFileDeviceEntryComparator( + externalTsFileScanNode, newOrderingSymbols, newSortOrders); + if (comparator != null) { + Map> offsetMap = new HashMap<>(); + List deviceEntries = externalTsFileScanNode.getDeviceEntries(); + List> deviceOffsets = + externalTsFileScanNode.getDeviceOffsets(); + for (int i = 0; i < deviceEntries.size(); i++) { + offsetMap.put(deviceEntries.get(i).getDeviceID(), deviceOffsets.get(i)); + } + externalTsFileScanNode.getDeviceEntries().sort(comparator); + List> sortedDeviceOffsets = + new ArrayList<>(externalTsFileScanNode.getDeviceEntries().size()); + for (DeviceEntry deviceEntry : externalTsFileScanNode.getDeviceEntries()) { + sortedDeviceOffsets.add(offsetMap.get(deviceEntry.getDeviceID())); + } + externalTsFileScanNode.setDeviceOffsets(sortedDeviceOffsets); + } if (lastIsTimeRelated) { if (newOrderingSymbols.size() > 1 @@ -1989,14 +2067,66 @@ && isOrderByAllIdsAndTime( .boxed() .collect(Collectors.toMap(newOrderingSymbols::get, newSortOrders::get))), newOrderingSymbols.size() - 2)) { - nodeOrderingMap.put(externalTsFileScanNode.getPlanNodeId(), pushedOrderingScheme); + nodeOrderingMap.put(externalTsFileScanNode.getPlanNodeId(), orderingScheme); } return; } if (newOrderingSymbols.size() == expectedOrderingScheme.getOrderBy().size()) { - nodeOrderingMap.put(externalTsFileScanNode.getPlanNodeId(), pushedOrderingScheme); + nodeOrderingMap.put(externalTsFileScanNode.getPlanNodeId(), orderingScheme); + } + } + + private Comparator createExternalTsFileDeviceEntryComparator( + ExternalTsFileScanNode externalTsFileScanNode, + List orderingSymbols, + List sortOrders) { + Comparator comparator = null; + for (int i = 0; i < orderingSymbols.size(); i++) { + Symbol symbol = orderingSymbols.get(i); + if (externalTsFileScanNode.isTimeColumn(symbol)) { + break; + } + int tagIndex = getExternalTsFileTagIndex(externalTsFileScanNode, symbol); + final int deviceSegmentIndex = tagIndex + 1; + SortOrder sortOrder = sortOrders.get(i); + Comparator valueComparator = + sortOrder.isNullsFirst() + ? Comparator.nullsFirst(Comparator.naturalOrder()) + : Comparator.nullsLast(Comparator.naturalOrder()); + Comparator currentComparator = + Comparator.comparing( + deviceEntry -> getDeviceSegment(deviceEntry.getDeviceID(), deviceSegmentIndex), + valueComparator); + if (!sortOrder.isAscending()) { + currentComparator = currentComparator.reversed(); + } + comparator = + comparator == null ? currentComparator : comparator.thenComparing(currentComparator); } + return comparator == null ? null : comparator.thenComparing(DeviceEntry::getDeviceID); + } + + private int getExternalTsFileTagIndex( + ExternalTsFileScanNode externalTsFileScanNode, Symbol symbol) { + int tagIndex = 0; + for (Map.Entry entry : + externalTsFileScanNode.getAssignments().entrySet()) { + if (entry.getValue().getColumnCategory() != TsTableColumnCategory.TAG) { + continue; + } + if (entry.getKey().equals(symbol)) { + return tagIndex; + } + tagIndex++; + } + throw new IllegalArgumentException("Unexpected external TsFile ordering symbol: " + symbol); + } + + private String getDeviceSegment(IDeviceID deviceID, int deviceSegmentIndex) { + return deviceSegmentIndex < deviceID.segmentNum() + ? (String) deviceID.segment(deviceSegmentIndex) + : null; } private Optional diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java index 5a7ab6eadfa08..42c1c8025a749 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java @@ -188,11 +188,11 @@ public static Optional pruneColumns(TableScanNode node, Set re externalTsFileScanNode.getPushDownPredicate(), externalTsFileScanNode.getPushDownLimit(), externalTsFileScanNode.getPushDownOffset(), - externalTsFileScanNode.getTagPredicate().orElse(null), externalTsFileScanNode.getTimePredicate().orElse(null), externalTsFileScanNode.getScanOrder(), - externalTsFileScanNode.getPushedOrderingScheme().orElse(null), - externalTsFileScanNode.getTsFilePaths())); + externalTsFileScanNode.getTsFilePaths(), + externalTsFileScanNode.getDeviceEntries(), + externalTsFileScanNode.getDeviceOffsets())); } else if (node instanceof InformationSchemaTableScanNode) { // For the convenience of process in execution stage, column-prune for // InformationSchemaTableScanNode is diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java index e60cebfdf29f5..71184f21eed2e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -21,18 +21,16 @@ import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.IPlanVisitor; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; -import org.apache.iotdb.commons.queryengine.plan.relational.planner.OrderingScheme; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; -import org.apache.tsfile.utils.ReadWriteIOUtils; - import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -44,10 +42,10 @@ public class ExternalTsFileScanNode extends TableScanNode { private List tsFilePaths; - private Expression tagPredicate; private Expression timePredicate; private Ordering scanOrder = Ordering.ASC; - private OrderingScheme pushedOrderingScheme; + private List deviceEntries = Collections.emptyList(); + private List> deviceOffsets = Collections.emptyList(); protected ExternalTsFileScanNode() {} @@ -57,8 +55,28 @@ public ExternalTsFileScanNode( List outputSymbols, Map assignments, List tsFilePaths) { + this( + id, + qualifiedObjectName, + outputSymbols, + assignments, + tsFilePaths, + Collections.emptyList(), + Collections.emptyList()); + } + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + List tsFilePaths, + List deviceEntries, + List> deviceOffsets) { super(id, qualifiedObjectName, outputSymbols, assignments); this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); + this.deviceEntries = new ArrayList<>(deviceEntries); + this.deviceOffsets = copyDeviceOffsets(deviceOffsets); } public ExternalTsFileScanNode( @@ -70,7 +88,6 @@ public ExternalTsFileScanNode( long pushDownLimit, long pushDownOffset, Ordering scanOrder, - OrderingScheme pushedOrderingScheme, List tsFilePaths) { this( id, @@ -80,11 +97,35 @@ public ExternalTsFileScanNode( pushDownPredicate, pushDownLimit, pushDownOffset, - null, + scanOrder, + tsFilePaths, + Collections.emptyList()); + } + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + Ordering scanOrder, + List tsFilePaths, + List deviceEntries) { + this( + id, + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset, null, scanOrder, - pushedOrderingScheme, - tsFilePaths); + tsFilePaths, + deviceEntries, + Collections.emptyList()); } public ExternalTsFileScanNode( @@ -95,11 +136,37 @@ public ExternalTsFileScanNode( Expression pushDownPredicate, long pushDownLimit, long pushDownOffset, - Expression tagPredicate, Expression timePredicate, Ordering scanOrder, - OrderingScheme pushedOrderingScheme, List tsFilePaths) { + this( + id, + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + timePredicate, + scanOrder, + tsFilePaths, + Collections.emptyList(), + Collections.emptyList()); + } + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + Expression timePredicate, + Ordering scanOrder, + List tsFilePaths, + List deviceEntries, + List> deviceOffsets) { super( id, qualifiedObjectName, @@ -108,11 +175,11 @@ public ExternalTsFileScanNode( pushDownPredicate, pushDownLimit, pushDownOffset); - this.tagPredicate = tagPredicate; this.timePredicate = timePredicate; this.scanOrder = scanOrder; - this.pushedOrderingScheme = pushedOrderingScheme; this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); + this.deviceEntries = new ArrayList<>(deviceEntries); + this.deviceOffsets = copyDeviceOffsets(deviceOffsets); } @Override @@ -130,23 +197,41 @@ public ExternalTsFileScanNode clone() { pushDownPredicate, pushDownLimit, pushDownOffset, - tagPredicate, timePredicate, scanOrder, - pushedOrderingScheme, - tsFilePaths); + tsFilePaths, + deviceEntries, + deviceOffsets); } public List getTsFilePaths() { return tsFilePaths; } - public Optional getTagPredicate() { - return Optional.ofNullable(tagPredicate); + public List getDeviceEntries() { + return deviceEntries; } - public void setTagPredicate(Expression tagPredicate) { - this.tagPredicate = tagPredicate; + public void setDeviceEntries(List deviceEntries) { + this.deviceEntries = new ArrayList<>(deviceEntries); + } + + public List> getDeviceOffsets() { + return deviceOffsets; + } + + public void setDeviceOffsets(List> deviceOffsets) { + this.deviceOffsets = copyDeviceOffsets(deviceOffsets); + } + + private static List> copyDeviceOffsets( + List> deviceOffsets) { + List> copiedDeviceOffsets = + new ArrayList<>(deviceOffsets.size()); + for (List offsets : deviceOffsets) { + copiedDeviceOffsets.add(new ArrayList<>(offsets)); + } + return copiedDeviceOffsets; } public Optional getTimePredicate() { @@ -165,104 +250,21 @@ public void setScanOrder(Ordering scanOrder) { this.scanOrder = scanOrder; } - public Optional getPushedOrderingScheme() { - return Optional.ofNullable(pushedOrderingScheme); - } - - public void setPushedOrderingScheme(OrderingScheme pushedOrderingScheme) { - this.pushedOrderingScheme = pushedOrderingScheme; - } - @Override protected void serializeAttributes(ByteBuffer byteBuffer) { - PlanNodeType.EXTERNAL_TSFILE_SCAN_NODE.serialize(byteBuffer); - TableScanNode.serializeMemberVariables(this, byteBuffer, true); - serializePredicate(tagPredicate, byteBuffer); - serializePredicate(timePredicate, byteBuffer); - ReadWriteIOUtils.write(scanOrder.ordinal(), byteBuffer); - if (pushedOrderingScheme == null) { - ReadWriteIOUtils.write(false, byteBuffer); - } else { - ReadWriteIOUtils.write(true, byteBuffer); - pushedOrderingScheme.serialize(byteBuffer); - } - serializeTsFilePaths(byteBuffer); + throw new UnsupportedOperationException( + "ExternalTsFileScanNode cannot be serialized because it reads local external TsFiles"); } @Override protected void serializeAttributes(DataOutputStream stream) throws IOException { - PlanNodeType.EXTERNAL_TSFILE_SCAN_NODE.serialize(stream); - TableScanNode.serializeMemberVariables(this, stream, true); - serializePredicate(tagPredicate, stream); - serializePredicate(timePredicate, stream); - ReadWriteIOUtils.write(scanOrder.ordinal(), stream); - if (pushedOrderingScheme == null) { - ReadWriteIOUtils.write(false, stream); - } else { - ReadWriteIOUtils.write(true, stream); - pushedOrderingScheme.serialize(stream); - } - serializeTsFilePaths(stream); - } - - private void serializeTsFilePaths(ByteBuffer byteBuffer) { - ReadWriteIOUtils.write(tsFilePaths.size(), byteBuffer); - for (String tsFilePath : tsFilePaths) { - ReadWriteIOUtils.write(tsFilePath, byteBuffer); - } - } - - private void serializeTsFilePaths(DataOutputStream stream) throws IOException { - ReadWriteIOUtils.write(tsFilePaths.size(), stream); - for (String tsFilePath : tsFilePaths) { - ReadWriteIOUtils.write(tsFilePath, stream); - } - } - - private void serializePredicate(Expression predicate, ByteBuffer byteBuffer) { - if (predicate == null) { - ReadWriteIOUtils.write(false, byteBuffer); - } else { - ReadWriteIOUtils.write(true, byteBuffer); - Expression.serialize(predicate, byteBuffer); - } - } - - private void serializePredicate(Expression predicate, DataOutputStream stream) - throws IOException { - if (predicate == null) { - ReadWriteIOUtils.write(false, stream); - } else { - ReadWriteIOUtils.write(true, stream); - Expression.serialize(predicate, stream); - } + throw new UnsupportedOperationException( + "ExternalTsFileScanNode cannot be serialized because it reads local external TsFiles"); } public static ExternalTsFileScanNode deserialize(ByteBuffer byteBuffer) { - ExternalTsFileScanNode node = new ExternalTsFileScanNode(); - TableScanNode.deserializeMemberVariables(byteBuffer, node, true); - - if (ReadWriteIOUtils.readBool(byteBuffer)) { - node.tagPredicate = Expression.deserialize(byteBuffer); - } - if (ReadWriteIOUtils.readBool(byteBuffer)) { - node.timePredicate = Expression.deserialize(byteBuffer); - } - - node.scanOrder = Ordering.values()[ReadWriteIOUtils.readInt(byteBuffer)]; - if (ReadWriteIOUtils.readBool(byteBuffer)) { - node.pushedOrderingScheme = OrderingScheme.deserialize(byteBuffer); - } - - int size = ReadWriteIOUtils.readInt(byteBuffer); - List tsFilePaths = new ArrayList<>(size); - while (size-- > 0) { - tsFilePaths.add(ReadWriteIOUtils.readString(byteBuffer)); - } - node.tsFilePaths = Collections.unmodifiableList(tsFilePaths); - - node.setPlanNodeId(PlanNodeId.deserialize(byteBuffer)); - return node; + throw new UnsupportedOperationException( + "ExternalTsFileScanNode cannot be deserialized because it reads local external TsFiles"); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 361e2d540bb54..05df24c0cd6bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -47,12 +47,18 @@ import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.NullLiteral; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.StringLiteral; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.SymbolReference; +import org.apache.iotdb.commons.queryengine.plan.relational.type.InternalTypeManager; import org.apache.iotdb.commons.queryengine.utils.TimestampPrecisionUtils; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.schema.table.InformationSchema; +import org.apache.iotdb.commons.schema.table.TsTable; +import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileDeviceFilterVisitor; import org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; @@ -61,6 +67,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.ConvertPredicateToTimeFilterVisitor; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicateCombineIntoTableScanChecker; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicatePushIntoMetadataChecker; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.schema.ConvertSchemaPredicateToFilterVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.metadata.NonAlignedDeviceEntry; @@ -79,18 +87,24 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.LazyTsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; import org.apache.tsfile.read.common.type.Type; import org.apache.tsfile.read.filter.basic.Filter; +import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.Pair; import javax.annotation.Nullable; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -118,6 +132,7 @@ import static org.apache.iotdb.commons.schema.table.InformationSchema.CURRENT_QUERIES; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.ATTRIBUTE; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.FIELD; +import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TAG; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TIME; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.PARTITION_FETCHER; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.SCHEMA_FETCHER; @@ -476,6 +491,7 @@ public PlanNode visitDeviceTableScan( public PlanNode visitExternalTsFileScan( ExternalTsFileScanNode tableScanNode, RewriteContext context) { if (TRUE_LITERAL.equals(context.inheritedPredicate)) { + collectExternalTsFileDeviceTasks(tableScanNode, Collections.emptyList()); return tableScanNode; } @@ -524,11 +540,8 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre getDeviceEntriesWithDataPartitions( (DeviceTableScanNode) tableScanNode, splitExpression.getMetadataExpressions()); } else if (tableScanNode instanceof ExternalTsFileScanNode) { - ((ExternalTsFileScanNode) tableScanNode) - .setTagPredicate( - splitExpression.getMetadataExpressions().isEmpty() - ? null - : combineConjuncts(splitExpression.getMetadataExpressions())); + collectExternalTsFileDeviceTasks( + (ExternalTsFileScanNode) tableScanNode, splitExpression.getMetadataExpressions()); } // exist expressions can not push down to scan operator @@ -545,6 +558,77 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre return tableScanNode; } + private void collectExternalTsFileDeviceTasks( + ExternalTsFileScanNode tableScanNode, List metadataExpressions) { + SchemaFilter deviceFilter = + constructExternalTsFileDeviceFilter(tableScanNode, metadataExpressions); + ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = + new ExternalTsFileDeviceFilterVisitor(); + Map> deviceOffsetMap = new LinkedHashMap<>(); + for (String tsFilePath : tableScanNode.getTsFilePaths()) { + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFilePath)) { + LazyTsFileDeviceIterator deviceIterator = + new LazyTsFileDeviceIterator( + reader, tableScanNode.getQualifiedObjectName().getObjectName(), ignored -> {}); + while (deviceIterator.hasNext()) { + IDeviceID deviceID = deviceIterator.next(); + if (deviceFilter != null + && !Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID))) { + continue; + } + deviceOffsetMap + .computeIfAbsent(deviceID, ignored -> new ArrayList<>()) + .add( + new ExternalTsFileDeviceOffset( + tsFilePath, deviceIterator.getCurrentDeviceMeasurementNodeOffset())); + } + } catch (IOException e) { + throw new SemanticException( + "Failed to collect devices from external TsFile: " + tsFilePath); + } + } + List deviceEntries = new ArrayList<>(deviceOffsetMap.size()); + List> deviceOffsets = + new ArrayList<>(deviceOffsetMap.size()); + for (Map.Entry> entry : + deviceOffsetMap.entrySet()) { + deviceEntries.add(new AlignedDeviceEntry(entry.getKey(), new Binary[0])); + deviceOffsets.add(entry.getValue()); + } + tableScanNode.setDeviceEntries(deviceEntries); + tableScanNode.setDeviceOffsets(deviceOffsets); + } + + private SchemaFilter constructExternalTsFileDeviceFilter( + ExternalTsFileScanNode tableScanNode, List metadataExpressions) { + if (metadataExpressions.isEmpty()) { + return null; + } + TsTable table = new TsTable(tableScanNode.getQualifiedObjectName().getObjectName()); + for (Map.Entry entry : tableScanNode.getAssignments().entrySet()) { + ColumnSchema columnSchema = entry.getValue(); + if (TAG.equals(columnSchema.getColumnCategory())) { + table.addColumnSchema( + new TagColumnSchema( + entry.getKey().getName(), + InternalTypeManager.getTSDataType(columnSchema.getType()))); + } + } + Expression predicate = + metadataExpressions.size() == 1 + ? metadataExpressions.get(0) + : new LogicalExpression(LogicalExpression.Operator.AND, metadataExpressions); + SchemaFilter deviceFilter = + predicate.accept( + new ConvertSchemaPredicateToFilterVisitor(), + new ConvertSchemaPredicateToFilterVisitor.Context(table)); + if (deviceFilter == null) { + throw new UnsupportedOperationException( + "Unsupported external TsFile device filter: " + predicate); + } + return deviceFilter; + } + interface InformationSchemaTablePredicatePushDownChecker { boolean canPushDown(Expression expression); } 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 389a91f300bf9..c2f98dd01199d 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 @@ -296,11 +296,11 @@ public PlanAndMappings visitExternalTsFileScan( node.getPushDownPredicate() == null ? null : mapper.map(node.getPushDownPredicate()), node.getPushDownLimit(), node.getPushDownOffset(), - node.getTagPredicate().map(mapper::map).orElse(null), node.getTimePredicate().map(mapper::map).orElse(null), node.getScanOrder(), - node.getPushedOrderingScheme().map(mapper::map).orElse(null), - node.getTsFilePaths()), + node.getTsFilePaths(), + node.getDeviceEntries(), + node.getDeviceOffsets()), mapping); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java index 82c4133b33923..d2f92f6e28793 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java @@ -30,12 +30,10 @@ import org.apache.iotdb.udf.api.relational.table.argument.Argument; import org.apache.iotdb.udf.api.relational.table.argument.DescribedSchema; import org.apache.iotdb.udf.api.relational.table.argument.ScalarArgument; -import org.apache.iotdb.udf.api.relational.table.processor.TableFunctionLeafProcessor; import org.apache.iotdb.udf.api.relational.table.specification.ParameterSpecification; import org.apache.iotdb.udf.api.relational.table.specification.ScalarParameterSpecification; import org.apache.iotdb.udf.api.type.Type; -import org.apache.tsfile.block.column.ColumnBuilder; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.ColumnCategory; import org.apache.tsfile.enums.TSDataType; @@ -116,13 +114,8 @@ public TableFunctionHandle createTableFunctionHandle() { @Override public TableFunctionProcessorProvider getProcessorProvider( TableFunctionHandle tableFunctionHandle) { - ReadTsFileTableFunctionHandle handle = (ReadTsFileTableFunctionHandle) tableFunctionHandle; - return new TableFunctionProcessorProvider() { - @Override - public TableFunctionLeafProcessor getSplitProcessor() { - return new ReadTsFileLeafProcessor(handle); - } - }; + throw new UnsupportedOperationException( + "read_tsfile must be planned as an ExternalTsFileScanNode"); } private static String getRequiredStringArgument(Map arguments, String name) { @@ -321,33 +314,27 @@ private static DescribedSchema convertToDescribedSchema(TableSchema tableSchema) return builder.build(); } - private static class ReadTsFileLeafProcessor implements TableFunctionLeafProcessor { - private final ReadTsFileTableFunctionHandle handle; - private boolean finished; - - private ReadTsFileLeafProcessor(ReadTsFileTableFunctionHandle handle) { - this.handle = handle; - } + public static class ExternalTsFileDeviceOffset { - @Override - public void beforeStart() { - // TODO: Open TsFile resources here. - finished = true; - } + private final String tsFilePath; + private final long[] deviceMeasurementNodeOffset; - @Override - public void process(List columnBuilders) { - // TODO: Read one batch from TsFile resources and write values into column builders. + public ExternalTsFileDeviceOffset(String tsFilePath, long[] deviceMeasurementNodeOffset) { + this.tsFilePath = tsFilePath; + this.deviceMeasurementNodeOffset = + deviceMeasurementNodeOffset == null + ? null + : Arrays.copyOf(deviceMeasurementNodeOffset, deviceMeasurementNodeOffset.length); } - @Override - public boolean isFinish() { - return finished; + public String getTsFilePath() { + return tsFilePath; } - @Override - public void beforeDestroy() { - // TODO: Close TsFile resources here. + public long[] getDeviceMeasurementNodeOffset() { + return deviceMeasurementNodeOffset == null + ? null + : Arrays.copyOf(deviceMeasurementNodeOffset, deviceMeasurementNodeOffset.length); } } From 0bc87465afb77e38d5e6c2f18d1491b84151272d Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Thu, 4 Jun 2026 19:02:52 +0800 Subject: [PATCH 08/32] extends device table scan node --- .../planner/plan/node/PlanGraphPrinter.java | 8 +- .../iterative/rule/PruneTableScanColumns.java | 43 +++++---- .../planner/node/ExternalTsFileScanNode.java | 88 +++++++++++-------- .../PushAggregationIntoTableScan.java | 2 + .../PushPredicateIntoTableScan.java | 14 +-- .../UnaliasSymbolReferences.java | 7 +- 6 files changed, 94 insertions(+), 68 deletions(-) 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 3ab8b283d0b31..b6399820f8ba0 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 @@ -682,12 +682,8 @@ public List visitTableScan(TableScanNode node, GraphContext context) { } } if (externalTsFileScanNode != null) { - boxValue.add(String.format("ScanOrder: %s", externalTsFileScanNode.getScanOrder())); - boxValue.add(String.format("TsFilePaths: %s", externalTsFileScanNode.getTsFilePaths())); - externalTsFileScanNode - .getTimePredicate() - .ifPresent( - timePredicate -> boxValue.add(String.format("TimePredicate: %s", timePredicate))); + boxValue.add( + String.format("TsFileNumber: %s", externalTsFileScanNode.getTsFilePaths().size())); } if (node.getPushDownPredicate() != null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java index 42c1c8025a749..a8a36c6328e9d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java @@ -83,7 +83,32 @@ public static Optional pruneColumns(TableScanNode node, Set re .forEach(symbol -> newAssignments.put(symbol, node.getAssignments().get(symbol))); } - if (node instanceof DeviceTableScanNode) { + if (node instanceof ExternalTsFileScanNode) { + ExternalTsFileScanNode externalTsFileScanNode = (ExternalTsFileScanNode) node; + externalTsFileScanNode + .getTimePredicate() + .ifPresent( + timePredicate -> + SymbolsExtractor.extractUnique(timePredicate) + .forEach( + symbol -> newAssignments.put(symbol, node.getAssignments().get(symbol)))); + ExternalTsFileScanNode prunedNode = + new ExternalTsFileScanNode( + externalTsFileScanNode.getPlanNodeId(), + externalTsFileScanNode.getQualifiedObjectName(), + newOutputs, + newAssignments, + externalTsFileScanNode.getPushDownPredicate(), + externalTsFileScanNode.getPushDownLimit(), + externalTsFileScanNode.getPushDownOffset(), + externalTsFileScanNode.getTimePredicate().orElse(null), + externalTsFileScanNode.getScanOrder(), + externalTsFileScanNode.getTsFilePaths(), + externalTsFileScanNode.getDeviceEntries(), + externalTsFileScanNode.getDeviceOffsets()); + prunedNode.setRegionReplicaSet(externalTsFileScanNode.getRegionReplicaSet()); + return Optional.of(prunedNode); + } else if (node instanceof DeviceTableScanNode) { DeviceTableScanNode deviceTableScanNode = (DeviceTableScanNode) node; // add time entry if TimePredicate exists deviceTableScanNode @@ -177,22 +202,6 @@ public static Optional pruneColumns(TableScanNode node, Set re prunedNode.setRegionReplicaSet(deviceTableScanNode.getRegionReplicaSet()); return Optional.of(prunedNode); } - } else if (node instanceof ExternalTsFileScanNode) { - ExternalTsFileScanNode externalTsFileScanNode = (ExternalTsFileScanNode) node; - return Optional.of( - new ExternalTsFileScanNode( - externalTsFileScanNode.getPlanNodeId(), - externalTsFileScanNode.getQualifiedObjectName(), - newOutputs, - newAssignments, - externalTsFileScanNode.getPushDownPredicate(), - externalTsFileScanNode.getPushDownLimit(), - externalTsFileScanNode.getPushDownOffset(), - externalTsFileScanNode.getTimePredicate().orElse(null), - externalTsFileScanNode.getScanOrder(), - externalTsFileScanNode.getTsFilePaths(), - externalTsFileScanNode.getDeviceEntries(), - externalTsFileScanNode.getDeviceOffsets())); } else if (node instanceof InformationSchemaTableScanNode) { // For the convenience of process in execution stage, column-prune for // InformationSchemaTableScanNode is diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java index 71184f21eed2e..0b88692575fd4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -21,7 +21,6 @@ import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.IPlanVisitor; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; @@ -38,13 +37,11 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Optional; -public class ExternalTsFileScanNode extends TableScanNode { +import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TAG; + +public class ExternalTsFileScanNode extends DeviceTableScanNode { private List tsFilePaths; - private Expression timePredicate; - private Ordering scanOrder = Ordering.ASC; - private List deviceEntries = Collections.emptyList(); private List> deviceOffsets = Collections.emptyList(); protected ExternalTsFileScanNode() {} @@ -73,7 +70,7 @@ public ExternalTsFileScanNode( List tsFilePaths, List deviceEntries, List> deviceOffsets) { - super(id, qualifiedObjectName, outputSymbols, assignments); + super(id, qualifiedObjectName, outputSymbols, assignments, buildTagIndexMap(assignments)); this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); this.deviceEntries = new ArrayList<>(deviceEntries); this.deviceOffsets = copyDeviceOffsets(deviceOffsets); @@ -167,18 +164,51 @@ public ExternalTsFileScanNode( List tsFilePaths, List deviceEntries, List> deviceOffsets) { + this( + id, + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + timePredicate, + scanOrder, + false, + tsFilePaths, + deviceEntries, + deviceOffsets); + } + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + Expression timePredicate, + Ordering scanOrder, + boolean pushLimitToEachDevice, + List tsFilePaths, + List deviceEntries, + List> deviceOffsets) { super( id, qualifiedObjectName, outputSymbols, assignments, + new ArrayList<>(deviceEntries), + buildTagIndexMap(assignments), + scanOrder, + timePredicate, pushDownPredicate, pushDownLimit, - pushDownOffset); - this.timePredicate = timePredicate; - this.scanOrder = scanOrder; + pushDownOffset, + pushLimitToEachDevice, + false); this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); - this.deviceEntries = new ArrayList<>(deviceEntries); this.deviceOffsets = copyDeviceOffsets(deviceOffsets); } @@ -199,6 +229,7 @@ public ExternalTsFileScanNode clone() { pushDownOffset, timePredicate, scanOrder, + pushLimitToEachDevice, tsFilePaths, deviceEntries, deviceOffsets); @@ -208,14 +239,6 @@ public List getTsFilePaths() { return tsFilePaths; } - public List getDeviceEntries() { - return deviceEntries; - } - - public void setDeviceEntries(List deviceEntries) { - this.deviceEntries = new ArrayList<>(deviceEntries); - } - public List> getDeviceOffsets() { return deviceOffsets; } @@ -234,22 +257,6 @@ private static List> copyDeviceOffsets( return copiedDeviceOffsets; } - public Optional getTimePredicate() { - return Optional.ofNullable(timePredicate); - } - - public void setTimePredicate(Expression timePredicate) { - this.timePredicate = timePredicate; - } - - public Ordering getScanOrder() { - return scanOrder; - } - - public void setScanOrder(Ordering scanOrder) { - this.scanOrder = scanOrder; - } - @Override protected void serializeAttributes(ByteBuffer byteBuffer) { throw new UnsupportedOperationException( @@ -271,4 +278,15 @@ public static ExternalTsFileScanNode deserialize(ByteBuffer byteBuffer) { public String toString() { return "ExternalTsFileScanNode-" + this.getPlanNodeId(); } + + private static Map buildTagIndexMap(Map assignments) { + Map tagIndexMap = new java.util.HashMap<>(); + int tagIndex = 0; + for (Map.Entry entry : assignments.entrySet()) { + if (TAG.equals(entry.getValue().getColumnCategory())) { + tagIndexMap.put(entry.getKey(), tagIndex++); + } + } + return tagIndexMap; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java index 30ed96dd7a973..b5072bfd5c6bd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java @@ -34,6 +34,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolAllocator; 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.ExternalTsFileScanNode; import com.google.common.collect.ImmutableSet; import org.apache.tsfile.utils.Pair; @@ -104,6 +105,7 @@ public PlanNode visitAggregation(AggregationNode node, Context context) { // only optimize AggregationNode with raw DeviceTableScanNode if (tableScanNode == null + || tableScanNode instanceof ExternalTsFileScanNode || tableScanNode instanceof AggregationTableScanNode) { // no need to optimize return node; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 05df24c0cd6bf..af6ddc5db1cc1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -517,10 +517,10 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre Pair resultPair = extractGlobalTimeFilter(pushDownPredicate, splitExpression.getTimeColumnName()); Boolean hasValueFilter = resultPair.getRight(); - if (tableScanNode instanceof DeviceTableScanNode && resultPair.left != null) { - ((DeviceTableScanNode) tableScanNode).setTimePredicate(resultPair.left); - } else if (tableScanNode instanceof ExternalTsFileScanNode && resultPair.left != null) { + if (tableScanNode instanceof ExternalTsFileScanNode && resultPair.left != null) { ((ExternalTsFileScanNode) tableScanNode).setTimePredicate(resultPair.left); + } else if (tableScanNode instanceof DeviceTableScanNode && resultPair.left != null) { + ((DeviceTableScanNode) tableScanNode).setTimePredicate(resultPair.left); } if (Boolean.TRUE.equals(hasValueFilter)) { if (pushDownPredicate instanceof LogicalExpression @@ -536,12 +536,12 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre } // do index scan after expressionCanPushDown is processed - if (tableScanNode instanceof DeviceTableScanNode) { - getDeviceEntriesWithDataPartitions( - (DeviceTableScanNode) tableScanNode, splitExpression.getMetadataExpressions()); - } else if (tableScanNode instanceof ExternalTsFileScanNode) { + if (tableScanNode instanceof ExternalTsFileScanNode) { collectExternalTsFileDeviceTasks( (ExternalTsFileScanNode) tableScanNode, splitExpression.getMetadataExpressions()); + } else if (tableScanNode instanceof DeviceTableScanNode) { + getDeviceEntriesWithDataPartitions( + (DeviceTableScanNode) tableScanNode, splitExpression.getMetadataExpressions()); } // exist expressions can not push down to scan operator 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 c2f98dd01199d..b72b05fbff52a 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 @@ -287,7 +287,7 @@ public PlanAndMappings visitExternalTsFileScan( newAssignments.put(newSymbol, handle); }); - return new PlanAndMappings( + ExternalTsFileScanNode rewrittenNode = new ExternalTsFileScanNode( node.getPlanNodeId(), node.getQualifiedObjectName(), @@ -300,8 +300,9 @@ public PlanAndMappings visitExternalTsFileScan( node.getScanOrder(), node.getTsFilePaths(), node.getDeviceEntries(), - node.getDeviceOffsets()), - mapping); + node.getDeviceOffsets()); + rewrittenNode.setRegionReplicaSet(node.getRegionReplicaSet()); + return new PlanAndMappings(rewrittenNode, mapping); } @Override From 66bcfbf2008c445a64082b930f7b53465499bdda Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Fri, 5 Jun 2026 11:27:52 +0800 Subject: [PATCH 09/32] fix --- .../ExternalTsFileTableScanOperator.java | 74 ++------- .../DataNodeTableOperatorGenerator.java | 69 +------- .../TableDistributedPlanGenerator.java | 151 +----------------- .../planner/node/DeviceTableScanNode.java | 5 + .../planner/node/ExternalTsFileScanNode.java | 23 +++ 5 files changed, 56 insertions(+), 266 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java index 0c68ec96c9bae..daccb199d2894 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java @@ -24,8 +24,6 @@ import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; -import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; -import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; @@ -35,30 +33,26 @@ import java.util.ArrayList; import java.util.List; -import static org.apache.iotdb.calc.plan.planner.CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING; - -public class ExternalTsFileTableScanOperator extends AbstractTableScanOperator { +public class ExternalTsFileTableScanOperator extends TableScanOperator { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(ExternalTsFileTableScanOperator.class); + private static final long ABSTRACT_DEVICE_TABLE_SCAN_OPERATOR_INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(AbstractDeviceTableScanOperator.class); private final String tableName; - private final List deviceEntries; private final List> deviceOffsets; - private DeviceEntry currentDeviceEntry; - private List currentDeviceOffsets; - private int currentDeviceIndex; - public ExternalTsFileTableScanOperator( AbstractTableScanOperatorParameter parameter, String tableName, - List deviceEntries, List> deviceOffsets) { super(parameter); this.tableName = tableName; - this.deviceEntries = new ArrayList<>(deviceEntries); this.deviceOffsets = new ArrayList<>(deviceOffsets); - this.currentDeviceIndex = 0; + if (deviceCount != this.deviceOffsets.size()) { + throw new IllegalArgumentException( + "The size of external TsFile device offsets should be equal to device entries"); + } } @Override @@ -72,50 +66,6 @@ String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { return segment == null ? null : (String) segment; } - @Override - public void initQueryDataSource(IQueryDataSource dataSource) { - super.initQueryDataSource(dataSource); - currentDeviceEntry = nextDeviceEntry(); - recordCurrentDeviceIndex(); - constructAlignedSeriesScanUtil(); - if (seriesScanUtil != null) { - seriesScanUtil.initQueryDataSource((QueryDataSource) dataSource); - } - } - - private DeviceEntry nextDeviceEntry() { - if (currentDeviceIndex >= deviceEntries.size()) { - currentDeviceOffsets = null; - return null; - } - DeviceEntry deviceEntry = deviceEntries.get(currentDeviceIndex); - currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); - return deviceEntry; - } - - @Override - protected boolean hasCurrentDeviceEntry() { - return currentDeviceEntry != null; - } - - @Override - protected DeviceEntry getCurrentDeviceEntry() { - return currentDeviceEntry; - } - - @Override - protected boolean advanceDeviceEntry() { - currentDeviceIndex++; - currentDeviceEntry = nextDeviceEntry(); - return currentDeviceEntry != null; - } - - @Override - protected void recordCurrentDeviceIndex() { - operatorContext.recordSpecifiedInfo( - CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); - } - @Override protected void constructAlignedSeriesScanUtil() { if (!hasCurrentDeviceEntry()) { @@ -141,12 +91,14 @@ protected void constructAlignedSeriesScanUtil() { private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( TsFileResource resource, AlignedFullPath alignedPath) throws IOException { + List currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); if (currentDeviceOffsets == null || !getCurrentDeviceEntry().getDeviceID().equals(alignedPath.getDeviceId())) { return null; } - long[] deviceMeasurementNodeOffset = getDeviceMeasurementNodeOffset(resource.getTsFilePath()); + long[] deviceMeasurementNodeOffset = + getDeviceMeasurementNodeOffset(currentDeviceOffsets, resource.getTsFilePath()); if (deviceMeasurementNodeOffset == null) { return null; } @@ -161,7 +113,8 @@ private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( ((OperatorContext) operatorContext).getInstanceContext().isIgnoreAllNullRows()); } - private long[] getDeviceMeasurementNodeOffset(String tsFilePath) { + private long[] getDeviceMeasurementNodeOffset( + List currentDeviceOffsets, String tsFilePath) { for (ExternalTsFileDeviceOffset offset : currentDeviceOffsets) { if (tsFilePath.equals(offset.getTsFilePath())) { return offset.getDeviceMeasurementNodeOffset(); @@ -174,8 +127,7 @@ private long[] getDeviceMeasurementNodeOffset(String tsFilePath) { public long ramBytesUsed() { return super.ramBytesUsed() + INSTANCE_SIZE - - AbstractTableScanOperator.INSTANCE_SIZE - + RamUsageEstimator.sizeOfCollection(deviceEntries) + - ABSTRACT_DEVICE_TABLE_SCAN_OPERATOR_INSTANCE_SIZE + RamUsageEstimator.sizeOfCollection(deviceOffsets); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index 6faca66d7aa74..5ceeef9f86b3c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -1127,14 +1127,16 @@ public Operator visitDeviceTableScan( public Operator visitExternalTsFileScan( ExternalTsFileScanNode node, LocalExecutionPlanContext context) { AbstractTableScanOperator.AbstractTableScanOperatorParameter parameter = - constructExternalTsFileTableScanOperatorParameter(node, context); + constructAbstractTableScanOperatorParameter( + node, + context, + ExternalTsFileTableScanOperator.class.getSimpleName(), + Collections.emptyMap(), + Long.MAX_VALUE); AbstractTableScanOperator externalTsFileTableScanOperator = new ExternalTsFileTableScanOperator( - parameter, - node.getQualifiedObjectName().getObjectName(), - node.getDeviceEntries(), - node.getDeviceOffsets()); + parameter, node.getQualifiedObjectName().getObjectName(), node.getDeviceOffsets()); context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); @@ -1147,63 +1149,6 @@ public Operator visitExternalTsFileScan( return externalTsFileTableScanOperator; } - private AbstractTableScanOperator.AbstractTableScanOperatorParameter - constructExternalTsFileTableScanOperatorParameter( - ExternalTsFileScanNode node, LocalExecutionPlanContext context) { - CommonTableScanOperatorParameters commonParameter = - new CommonTableScanOperatorParameters( - node, Collections.emptyMap(), false, buildTagAndAttributeColumnsIndexMap(node)); - SeriesScanOptions seriesScanOptions = - buildSeriesScanOptions( - context, - commonParameter.columnSchemaMap, - commonParameter.measurementColumnNames, - commonParameter.measurementColumnsIndexMap, - commonParameter.timeColumnName, - node.getTimePredicate(), - node.getPushDownLimit(), - node.getPushDownOffset(), - false, - node.getPushDownPredicate()); - - OperatorContext operatorContext = - addOperatorContext( - context, node.getPlanNodeId(), ExternalTsFileTableScanOperator.class.getSimpleName()); - - Set allSensors = new HashSet<>(commonParameter.measurementColumnNames); - // for time column - allSensors.add(""); - - return new AbstractTableScanOperator.AbstractTableScanOperatorParameter( - allSensors, - operatorContext, - node.getPlanNodeId(), - commonParameter.columnSchemas, - commonParameter.columnsIndexArray, - Collections.emptyList(), - node.getScanOrder(), - seriesScanOptions, - commonParameter.measurementColumnNames, - commonParameter.measurementSchemas, - TSFileDescriptor.getInstance().getConfig().getMaxTsBlockLineNumber()); - } - - private static Map buildTagAndAttributeColumnsIndexMap(TableScanNode node) { - Map tagAndAttributeColumnsIndexMap = new HashMap<>(); - int index = 0; - for (Map.Entry entry : node.getAssignments().entrySet()) { - switch (entry.getValue().getColumnCategory()) { - case TAG: - case ATTRIBUTE: - tagAndAttributeColumnsIndexMap.put(entry.getKey(), index++); - break; - default: - break; - } - } - return tagAndAttributeColumnsIndexMap; - } - private SeriesScanOptions.Builder getSeriesScanOptionsBuilder( LocalExecutionPlanContext context, @NotNull Expression timePredicate) { SeriesScanOptions.Builder scanOptionsBuilder = new SeriesScanOptions.Builder(); 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 d6cc700a14f7f..104ed577882f7 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 @@ -736,23 +736,19 @@ public List visitExternalTsFileScan( new TRegionReplicaSet(null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())); node.setRegionReplicaSet(localRegionReplicaSet); context.mostUsedRegion = node.getRegionReplicaSet(); + List resultNodes = + splitExternalTsFileScanByDeviceEntries(node, localRegionReplicaSet); if (context.hasSortProperty) { - processExternalTsFileSortProperty(node, context); - return Collections.singletonList(node); - } - - List splitNodes = splitExternalTsFileScanByDeviceEntries(node, localRegionReplicaSet); - if (!splitNodes.isEmpty()) { - return splitNodes; + processSortProperty(node, resultNodes, context); } - return Collections.singletonList(node); + return resultNodes; } private List splitExternalTsFileScanByDeviceEntries( final ExternalTsFileScanNode node, final TRegionReplicaSet localRegionReplicaSet) { List deviceEntries = node.getDeviceEntries(); if (deviceEntries.size() <= 1) { - return Collections.emptyList(); + return Collections.singletonList(node); } int splitCount = @@ -760,7 +756,7 @@ private List splitExternalTsFileScanByDeviceEntries( deviceEntries.size(), IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism()); if (splitCount <= 1) { - return Collections.emptyList(); + return Collections.singletonList(node); } List> splitDeviceEntries = new ArrayList<>(splitCount); @@ -791,6 +787,7 @@ private List splitExternalTsFileScanByDeviceEntries( node.getPushDownOffset(), node.getTimePredicate().orElse(null), node.getScanOrder(), + node.isPushLimitToEachDevice(), node.getTsFilePaths(), entries, splitDeviceOffsets.get(i)); @@ -1992,141 +1989,9 @@ private void processSortProperty( newOrderingScheme.ifPresent( orderingScheme -> nodeOrderingMap.put(scanNode.getPlanNodeId(), orderingScheme)); if (comparator != null) { - scanNode.getDeviceEntries().sort(comparator); - } - } - } - - private void processExternalTsFileSortProperty( - final ExternalTsFileScanNode externalTsFileScanNode, final PlanContext context) { - final OrderingScheme expectedOrderingScheme = context.expectedOrderingScheme; - final List newOrderingSymbols = new ArrayList<>(); - final List newSortOrders = new ArrayList<>(); - boolean lastIsTimeRelated = false; - - for (final Symbol symbol : expectedOrderingScheme.getOrderBy()) { - if (externalTsFileScanNode.isTimeColumn(symbol)) { - if (!expectedOrderingScheme.getOrdering(symbol).isAscending()) { - externalTsFileScanNode.setScanOrder(Ordering.DESC); - } - newOrderingSymbols.add(symbol); - newSortOrders.add(expectedOrderingScheme.getOrdering(symbol)); - lastIsTimeRelated = true; - break; - } - - final ColumnSchema columnSchema = externalTsFileScanNode.getAssignments().get(symbol); - if (columnSchema == null || columnSchema.getColumnCategory() != TsTableColumnCategory.TAG) { - break; - } - - newOrderingSymbols.add(symbol); - newSortOrders.add(expectedOrderingScheme.getOrdering(symbol)); - } - - if (newOrderingSymbols.isEmpty()) { - return; - } - - OrderingScheme orderingScheme = - new OrderingScheme( - newOrderingSymbols, - IntStream.range(0, newOrderingSymbols.size()) - .boxed() - .collect(Collectors.toMap(newOrderingSymbols::get, newSortOrders::get))); - - Comparator comparator = - createExternalTsFileDeviceEntryComparator( - externalTsFileScanNode, newOrderingSymbols, newSortOrders); - if (comparator != null) { - Map> offsetMap = new HashMap<>(); - List deviceEntries = externalTsFileScanNode.getDeviceEntries(); - List> deviceOffsets = - externalTsFileScanNode.getDeviceOffsets(); - for (int i = 0; i < deviceEntries.size(); i++) { - offsetMap.put(deviceEntries.get(i).getDeviceID(), deviceOffsets.get(i)); - } - externalTsFileScanNode.getDeviceEntries().sort(comparator); - List> sortedDeviceOffsets = - new ArrayList<>(externalTsFileScanNode.getDeviceEntries().size()); - for (DeviceEntry deviceEntry : externalTsFileScanNode.getDeviceEntries()) { - sortedDeviceOffsets.add(offsetMap.get(deviceEntry.getDeviceID())); - } - externalTsFileScanNode.setDeviceOffsets(sortedDeviceOffsets); - } - - if (lastIsTimeRelated) { - if (newOrderingSymbols.size() > 1 - && newOrderingSymbols.size() == expectedOrderingScheme.getOrderBy().size() - && isOrderByAllIdsAndTime( - analysis.getTableColumnSchema(externalTsFileScanNode.getQualifiedObjectName()), - externalTsFileScanNode.getAssignments(), - new OrderingScheme( - newOrderingSymbols.subList(0, newOrderingSymbols.size() - 1), - IntStream.range(0, newOrderingSymbols.size() - 1) - .boxed() - .collect(Collectors.toMap(newOrderingSymbols::get, newSortOrders::get))), - newOrderingSymbols.size() - 2)) { - nodeOrderingMap.put(externalTsFileScanNode.getPlanNodeId(), orderingScheme); - } - return; - } - - if (newOrderingSymbols.size() == expectedOrderingScheme.getOrderBy().size()) { - nodeOrderingMap.put(externalTsFileScanNode.getPlanNodeId(), orderingScheme); - } - } - - private Comparator createExternalTsFileDeviceEntryComparator( - ExternalTsFileScanNode externalTsFileScanNode, - List orderingSymbols, - List sortOrders) { - Comparator comparator = null; - for (int i = 0; i < orderingSymbols.size(); i++) { - Symbol symbol = orderingSymbols.get(i); - if (externalTsFileScanNode.isTimeColumn(symbol)) { - break; - } - int tagIndex = getExternalTsFileTagIndex(externalTsFileScanNode, symbol); - final int deviceSegmentIndex = tagIndex + 1; - SortOrder sortOrder = sortOrders.get(i); - Comparator valueComparator = - sortOrder.isNullsFirst() - ? Comparator.nullsFirst(Comparator.naturalOrder()) - : Comparator.nullsLast(Comparator.naturalOrder()); - Comparator currentComparator = - Comparator.comparing( - deviceEntry -> getDeviceSegment(deviceEntry.getDeviceID(), deviceSegmentIndex), - valueComparator); - if (!sortOrder.isAscending()) { - currentComparator = currentComparator.reversed(); - } - comparator = - comparator == null ? currentComparator : comparator.thenComparing(currentComparator); - } - return comparator == null ? null : comparator.thenComparing(DeviceEntry::getDeviceID); - } - - private int getExternalTsFileTagIndex( - ExternalTsFileScanNode externalTsFileScanNode, Symbol symbol) { - int tagIndex = 0; - for (Map.Entry entry : - externalTsFileScanNode.getAssignments().entrySet()) { - if (entry.getValue().getColumnCategory() != TsTableColumnCategory.TAG) { - continue; + scanNode.sortDeviceEntries(comparator); } - if (entry.getKey().equals(symbol)) { - return tagIndex; - } - tagIndex++; } - throw new IllegalArgumentException("Unexpected external TsFile ordering symbol: " + symbol); - } - - private String getDeviceSegment(IDeviceID deviceID, int deviceSegmentIndex) { - return deviceSegmentIndex < deviceID.segmentNum() - ? (String) deviceID.segment(deviceSegmentIndex) - : null; } private Optional diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/DeviceTableScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/DeviceTableScanNode.java index c02c74058afee..e4fceb9818b9c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/DeviceTableScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/DeviceTableScanNode.java @@ -41,6 +41,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -275,6 +276,10 @@ public void appendDeviceEntry(DeviceEntry deviceEntry) { this.deviceEntries.add(deviceEntry); } + public void sortDeviceEntries(Comparator comparator) { + this.deviceEntries.sort(comparator); + } + public void setPushLimitToEachDevice(boolean pushLimitToEachDevice) { this.pushLimitToEachDevice = pushLimitToEachDevice; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java index 0b88692575fd4..f21a7a755c0bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -35,8 +35,10 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.stream.IntStream; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TAG; @@ -247,6 +249,27 @@ public void setDeviceOffsets(List> deviceOffset this.deviceOffsets = copyDeviceOffsets(deviceOffsets); } + @Override + public void sortDeviceEntries(Comparator comparator) { + int[] indexes = + IntStream.range(0, deviceEntries.size()) + .boxed() + .sorted( + (left, right) -> + comparator.compare(deviceEntries.get(left), deviceEntries.get(right))) + .mapToInt(Integer::intValue) + .toArray(); + List sortedDeviceEntries = new ArrayList<>(deviceEntries.size()); + List> sortedDeviceOffsets = + new ArrayList<>(deviceOffsets.size()); + for (int index : indexes) { + sortedDeviceEntries.add(deviceEntries.get(index)); + sortedDeviceOffsets.add(deviceOffsets.get(index)); + } + this.deviceEntries = sortedDeviceEntries; + this.deviceOffsets = sortedDeviceOffsets; + } + private static List> copyDeviceOffsets( List> deviceOffsets) { List> copiedDeviceOffsets = From 110b98a1ad906f8e66d7366d78672a742b6af336 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Fri, 5 Jun 2026 11:34:28 +0800 Subject: [PATCH 10/32] delete old operator --- ...rderedExternalTsFileTableScanOperator.java | 310 ------------------ ...rderedExternalTsFileTableScanOperator.java | 280 ---------------- 2 files changed, 590 deletions(-) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java deleted file mode 100644 index 2b1fdf561dfc6..0000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/OrderedExternalTsFileTableScanOperator.java +++ /dev/null @@ -1,310 +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.queryengine.execution.operator.source.relational; - -import org.apache.iotdb.commons.path.AlignedFullPath; -import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; -import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; -import org.apache.iotdb.commons.queryengine.plan.relational.planner.OrderingScheme; -import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; -import org.apache.iotdb.commons.schema.filter.SchemaFilter; -import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; -import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; -import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; -import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; -import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; -import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; -import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; - -import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; -import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.LazyTsFileDeviceIterator; -import org.apache.tsfile.read.TsFileSequenceReader; -import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.utils.RamUsageEstimator; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.HashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; - -import static org.apache.iotdb.calc.plan.planner.CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING; - -public class OrderedExternalTsFileTableScanOperator extends AbstractTableScanOperator { - private static final long INSTANCE_SIZE = - RamUsageEstimator.shallowSizeOfInstance(OrderedExternalTsFileTableScanOperator.class); - - private final String tableName; - private final Map assignments; - private final OrderingScheme pushedOrderingScheme; - private final SchemaFilter deviceFilter; - private final ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = - new ExternalTsFileDeviceFilterVisitor(); - private final Map> deviceMeasurementNodeOffsetMap = - new HashMap<>(); - - private DeviceEntry currentDeviceEntry; - private int currentDeviceIndex; - private List sortedDeviceEntries = new ArrayList<>(); - - public OrderedExternalTsFileTableScanOperator( - AbstractTableScanOperatorParameter parameter, - String tableName, - Map assignments, - OrderingScheme pushedOrderingScheme, - SchemaFilter deviceFilter) { - super(parameter); - this.tableName = tableName; - this.assignments = assignments; - this.pushedOrderingScheme = pushedOrderingScheme; - this.deviceFilter = deviceFilter; - this.currentDeviceIndex = 0; - } - - @Override - String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { - int segmentOffset = - deviceEntry.getDeviceID().segmentNum() > 0 - && tableName.equalsIgnoreCase((String) deviceEntry.getNthSegment(0)) - ? 1 - : 0; - Object segment = deviceEntry.getNthSegment(idColumnIndex + segmentOffset); - return segment == null ? null : (String) segment; - } - - @Override - public void initQueryDataSource(IQueryDataSource dataSource) { - super.initQueryDataSource(dataSource); - sortedDeviceEntries = collectSortedDeviceEntries((QueryDataSource) dataSource); - currentDeviceEntry = sortedDeviceEntries.isEmpty() ? null : sortedDeviceEntries.get(0); - recordCurrentDeviceIndex(); - constructAlignedSeriesScanUtil(); - if (seriesScanUtil != null) { - seriesScanUtil.initQueryDataSource((QueryDataSource) dataSource); - } - } - - private List collectSortedDeviceEntries(QueryDataSource queryDataSource) { - List deviceInfos = collectDeviceInfos(queryDataSource); - deviceInfos.sort(createDeviceInfoComparator()); - - List deviceEntries = new ArrayList<>(deviceInfos.size()); - Set visitedDevices = new LinkedHashSet<>(); - for (ExternalTsFileDeviceInfo deviceInfo : deviceInfos) { - deviceMeasurementNodeOffsetMap - .computeIfAbsent(deviceInfo.resource, ignored -> new HashMap<>()) - .put(deviceInfo.deviceID, deviceInfo.deviceMeasurementNodeOffset); - if (visitedDevices.add(deviceInfo.deviceID)) { - deviceEntries.add(new AlignedDeviceEntry(deviceInfo.deviceID, new Binary[0])); - } - } - return deviceEntries; - } - - private List collectDeviceInfos(QueryDataSource queryDataSource) { - List deviceInfos = new ArrayList<>(); - for (TsFileResource resource : getAllResources(queryDataSource)) { - collectDeviceInfos(resource, deviceInfos); - } - return deviceInfos; - } - - private void collectDeviceInfos( - TsFileResource resource, List deviceInfos) { - try (TsFileSequenceReader reader = new TsFileSequenceReader(resource.getTsFilePath())) { - LazyTsFileDeviceIterator deviceIterator = - new LazyTsFileDeviceIterator( - reader, - tableName, - ((OperatorContext) operatorContext) - .getInstanceContext() - .getQueryStatistics() - .getLoadTimeSeriesMetadataActualIOSize() - ::addAndGet); - while (deviceIterator.hasNext()) { - IDeviceID deviceID = deviceIterator.next(); - if (!isDeviceMatched(deviceID)) { - continue; - } - deviceInfos.add( - new ExternalTsFileDeviceInfo( - deviceID, resource, deviceIterator.getCurrentDeviceMeasurementNodeOffset())); - } - } catch (IOException e) { - throw new RuntimeException( - "Failed to collect devices from external TsFile: " + resource.getTsFilePath(), e); - } - } - - private List getAllResources(QueryDataSource queryDataSource) { - List resources = - new ArrayList<>( - queryDataSource.getSeqResources().size() + queryDataSource.getUnseqResources().size()); - resources.addAll(queryDataSource.getSeqResources()); - resources.addAll(queryDataSource.getUnseqResources()); - return resources; - } - - private boolean isDeviceMatched(IDeviceID deviceID) { - return deviceFilter == null - || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID)); - } - - private Comparator createDeviceInfoComparator() { - Comparator comparator = null; - for (Symbol symbol : pushedOrderingScheme.getOrderBy()) { - if (TableScanNode.isTimeColumn(symbol, assignments)) { - continue; - } - int tagIndex = getTagIndex(symbol); - final int deviceSegmentIndex = tagIndex + 1; - Comparator valueComparator = - pushedOrderingScheme.getOrdering(symbol).isNullsFirst() - ? Comparator.nullsFirst(Comparator.naturalOrder()) - : Comparator.nullsLast(Comparator.naturalOrder()); - Comparator currentComparator = - Comparator.comparing( - deviceInfo -> getDeviceSegment(deviceInfo.deviceID, deviceSegmentIndex), - valueComparator); - if (!pushedOrderingScheme.getOrdering(symbol).isAscending()) { - currentComparator = currentComparator.reversed(); - } - comparator = - comparator == null ? currentComparator : comparator.thenComparing(currentComparator); - } - return comparator == null - ? Comparator.comparing(deviceInfo -> deviceInfo.deviceID) - : comparator.thenComparing(deviceInfo -> deviceInfo.deviceID); - } - - private String getDeviceSegment(IDeviceID deviceID, int deviceSegmentIndex) { - return deviceSegmentIndex < deviceID.segmentNum() - ? (String) deviceID.segment(deviceSegmentIndex) - : null; - } - - private int getTagIndex(Symbol symbol) { - int tagIndex = 0; - for (Map.Entry entry : assignments.entrySet()) { - if (entry.getValue().getColumnCategory() != TsTableColumnCategory.TAG) { - continue; - } - if (entry.getKey().equals(symbol)) { - return tagIndex; - } - tagIndex++; - } - throw new IllegalArgumentException("Unexpected external TsFile ordering symbol: " + symbol); - } - - @Override - protected boolean hasCurrentDeviceEntry() { - return currentDeviceEntry != null; - } - - @Override - protected DeviceEntry getCurrentDeviceEntry() { - return currentDeviceEntry; - } - - @Override - protected boolean advanceDeviceEntry() { - currentDeviceIndex++; - currentDeviceEntry = - currentDeviceIndex < sortedDeviceEntries.size() - ? sortedDeviceEntries.get(currentDeviceIndex) - : null; - return currentDeviceEntry != null; - } - - @Override - protected void recordCurrentDeviceIndex() { - operatorContext.recordSpecifiedInfo( - CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); - } - - @Override - protected void constructAlignedSeriesScanUtil() { - if (!hasCurrentDeviceEntry()) { - return; - } - - DeviceEntry deviceEntry = getCurrentDeviceEntry(); - if (deviceEntry == null) { - throw new IllegalStateException("Current device entry in TableScanOperator is empty"); - } - - this.seriesScanUtil = - new ExternalTsFileSeriesScanUtil( - constructAlignedPath( - deviceEntry, measurementColumnNames, measurementSchemas, allSensors), - scanOrder, - seriesScanOptions, - ((OperatorContext) operatorContext).getInstanceContext(), - true, - measurementColumnTSDataTypes, - this::loadTimeSeriesMetadata); - } - - private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( - TsFileResource resource, AlignedFullPath alignedPath) throws IOException { - Optional deviceMeasurementNodeOffset = - Optional.ofNullable(deviceMeasurementNodeOffsetMap) - .map(map -> map.get(resource)) - .map(map -> map.get(alignedPath.getDeviceId())); - // TODO: Use deviceMeasurementNodeOffset after FileLoaderUtils supports offset-based metadata - // loading in this branch. - return FileLoaderUtils.loadAlignedTimeSeriesMetadata( - resource, - alignedPath, - ((OperatorContext) operatorContext).getInstanceContext(), - seriesScanOptions.getGlobalTimeFilter(), - resource.isSeq(), - ((OperatorContext) operatorContext).getInstanceContext().isIgnoreAllNullRows()); - } - - @Override - public long ramBytesUsed() { - return super.ramBytesUsed() - + INSTANCE_SIZE - - AbstractTableScanOperator.INSTANCE_SIZE - + RamUsageEstimator.sizeOfMap(deviceMeasurementNodeOffsetMap) - + RamUsageEstimator.sizeOfCollection(sortedDeviceEntries); - } - - private static class ExternalTsFileDeviceInfo { - private final IDeviceID deviceID; - private final TsFileResource resource; - private final long[] deviceMeasurementNodeOffset; - - private ExternalTsFileDeviceInfo( - IDeviceID deviceID, TsFileResource resource, long[] deviceMeasurementNodeOffset) { - this.deviceID = deviceID; - this.resource = resource; - this.deviceMeasurementNodeOffset = deviceMeasurementNodeOffset; - } - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java deleted file mode 100644 index 215fdd66f33fe..0000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/UnorderedExternalTsFileTableScanOperator.java +++ /dev/null @@ -1,280 +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.queryengine.execution.operator.source.relational; - -import org.apache.iotdb.commons.path.AlignedFullPath; -import org.apache.iotdb.commons.schema.filter.SchemaFilter; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; -import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; -import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; -import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; -import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; -import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; -import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; -import org.apache.iotdb.db.utils.EncryptDBUtils; - -import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; -import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.TsFileSequenceReader; -import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.utils.RamUsageEstimator; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.iotdb.calc.plan.planner.CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING; - -public class UnorderedExternalTsFileTableScanOperator extends AbstractTableScanOperator { - private static final long INSTANCE_SIZE = - RamUsageEstimator.shallowSizeOfInstance(UnorderedExternalTsFileTableScanOperator.class); - - private final String tableName; - private final SchemaFilter deviceFilter; - private final List deviceEntries; - private final List> deviceOffsets; - private final ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = - new ExternalTsFileDeviceFilterVisitor(); - - private MultiTsFileResourceIterator deviceIterator; - private Map resourceReaderMap = Collections.emptyMap(); - private DeviceEntry currentDeviceEntry; - private List currentDeviceOffsets; - private int currentDeviceIndex; - - public UnorderedExternalTsFileTableScanOperator( - AbstractTableScanOperatorParameter parameter, - String tableName, - SchemaFilter deviceFilter, - List deviceEntries, - List> deviceOffsets) { - super(parameter); - this.tableName = tableName; - this.deviceFilter = deviceFilter; - this.deviceEntries = deviceEntries; - this.deviceOffsets = deviceOffsets; - this.currentDeviceIndex = 0; - } - - @Override - String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { - int segmentOffset = - deviceEntry.getDeviceID().segmentNum() > 0 - && tableName.equalsIgnoreCase((String) deviceEntry.getNthSegment(0)) - ? 1 - : 0; - Object segment = deviceEntry.getNthSegment(idColumnIndex + segmentOffset); - return segment == null ? null : (String) segment; - } - - @Override - public void initQueryDataSource(IQueryDataSource dataSource) { - super.initQueryDataSource(dataSource); - - QueryDataSource queryDataSource = (QueryDataSource) dataSource; - if (deviceEntries.isEmpty()) { - initDeviceIterator(queryDataSource); - } - currentDeviceEntry = nextDeviceEntry(); - recordCurrentDeviceIndex(); - constructAlignedSeriesScanUtil(); - if (seriesScanUtil != null) { - seriesScanUtil.initQueryDataSource(queryDataSource); - } - } - - private void initDeviceIterator(QueryDataSource queryDataSource) { - resourceReaderMap = createResourceReaderMap(getAllResources(queryDataSource)); - deviceIterator = - new MultiTsFileResourceIterator( - tableName, - queryDataSource.getSeqResources(), - queryDataSource.getUnseqResources(), - resourceReaderMap, - ((OperatorContext) operatorContext).getInstanceContext(), - seriesScanOptions, - deviceFilter); - } - - private Map createResourceReaderMap( - List resources) { - Map readerMap = new HashMap<>(resources.size()); - for (TsFileResource resource : resources) { - try { - readerMap.put( - resource, - new TsFileSequenceReader( - resource.getTsFilePath(), - ((OperatorContext) operatorContext) - .getInstanceContext() - .getQueryStatistics() - .getLoadTimeSeriesMetadataActualIOSize() - ::addAndGet, - EncryptDBUtils.getFirstEncryptParamFromTSFilePath(resource.getTsFilePath()))); - } catch (IOException e) { - closeResourceReaders(readerMap); - throw new RuntimeException( - "Failed to open external TsFile reader: " + resource.getTsFilePath(), e); - } - } - return readerMap; - } - - private List getAllResources(QueryDataSource queryDataSource) { - List resources = - new ArrayList<>( - queryDataSource.getSeqResources().size() + queryDataSource.getUnseqResources().size()); - resources.addAll(queryDataSource.getSeqResources()); - resources.addAll(queryDataSource.getUnseqResources()); - return resources; - } - - private DeviceEntry nextDeviceEntry() { - if (!deviceEntries.isEmpty()) { - while (currentDeviceIndex < deviceEntries.size()) { - currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); - DeviceEntry deviceEntry = deviceEntries.get(currentDeviceIndex); - if (isDeviceMatched(deviceEntry.getDeviceID())) { - return deviceEntry; - } - currentDeviceIndex++; - } - currentDeviceOffsets = null; - return null; - } - - if (deviceIterator == null || !deviceIterator.hasNextDevice()) { - return null; - } - IDeviceID nextDevice = deviceIterator.nextDevice(); - return nextDevice == null ? null : new AlignedDeviceEntry(nextDevice, new Binary[0]); - } - - @Override - protected boolean hasCurrentDeviceEntry() { - return currentDeviceEntry != null; - } - - @Override - protected DeviceEntry getCurrentDeviceEntry() { - return currentDeviceEntry; - } - - @Override - protected boolean advanceDeviceEntry() { - currentDeviceIndex++; - currentDeviceEntry = nextDeviceEntry(); - return currentDeviceEntry != null; - } - - @Override - protected void recordCurrentDeviceIndex() { - operatorContext.recordSpecifiedInfo( - CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); - } - - @Override - protected void constructAlignedSeriesScanUtil() { - if (!hasCurrentDeviceEntry()) { - return; - } - - DeviceEntry deviceEntry = getCurrentDeviceEntry(); - if (deviceEntry == null) { - throw new IllegalStateException("Current device entry in TableScanOperator is empty"); - } - - this.seriesScanUtil = - new ExternalTsFileSeriesScanUtil( - constructAlignedPath( - deviceEntry, measurementColumnNames, measurementSchemas, allSensors), - scanOrder, - seriesScanOptions, - ((OperatorContext) operatorContext).getInstanceContext(), - true, - measurementColumnTSDataTypes, - deviceEntries.isEmpty() - ? deviceIterator::loadTimeSeriesMetadata - : this::loadTimeSeriesMetadata); - } - - private boolean isDeviceMatched(IDeviceID deviceID) { - return deviceFilter == null - || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID)); - } - - private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( - TsFileResource resource, AlignedFullPath alignedPath) throws IOException { - if (currentDeviceOffsets == null - || !getCurrentDeviceEntry().getDeviceID().equals(alignedPath.getDeviceId())) { - return null; - } - if (!containsCurrentDevice(resource)) { - return null; - } - return FileLoaderUtils.loadAlignedTimeSeriesMetadata( - resource, - alignedPath, - ((OperatorContext) operatorContext).getInstanceContext(), - seriesScanOptions.getGlobalTimeFilter(), - resource.isSeq(), - ((OperatorContext) operatorContext).getInstanceContext().isIgnoreAllNullRows()); - } - - private boolean containsCurrentDevice(TsFileResource resource) { - String tsFilePath = resource.getTsFilePath(); - for (ExternalTsFileDeviceOffset offset : currentDeviceOffsets) { - if (tsFilePath.equals(offset.getTsFilePath())) { - return true; - } - } - return false; - } - - @Override - public void close() throws Exception { - closeResourceReaders(resourceReaderMap); - resourceReaderMap = Collections.emptyMap(); - deviceIterator = null; - super.close(); - } - - @Override - public long ramBytesUsed() { - return super.ramBytesUsed() - + INSTANCE_SIZE - - AbstractTableScanOperator.INSTANCE_SIZE - + RamUsageEstimator.sizeOfMap(resourceReaderMap); - } - - private void closeResourceReaders(Map readerMap) { - for (TsFileSequenceReader reader : readerMap.values()) { - try { - reader.close(); - } catch (IOException ignored) { - // ignore close failure - } - } - } -} From 4be16d3042a8f65a267a8f26ead6ffeb59989c72 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Fri, 5 Jun 2026 17:16:47 +0800 Subject: [PATCH 11/32] agg scan --- .../AbstractAggTableScanOperator.java | 2 +- .../AbstractDefaultAggTableScanOperator.java | 2 +- .../ExternalTsFileAggTableScanOperator.java | 111 ++++++++++ .../ExternalTsFileSeriesScanUtil.java | 42 ++++ .../ExternalTsFileTableScanOperator.java | 31 +-- .../DataNodeTableOperatorGenerator.java | 29 +++ .../plan/planner/plan/node/PlanVisitor.java | 5 + .../TableDistributedPlanGenerator.java | 77 +++++++ .../node/AggregationTableScanNode.java | 50 +++++ .../ExternalTsFileAggregationScanNode.java | 189 ++++++++++++++++++ .../PushAggregationIntoTableScan.java | 29 ++- 11 files changed, 534 insertions(+), 33 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java index 1f37ce9f10472..e3030c0752cef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java @@ -96,7 +96,7 @@ public abstract class AbstractAggTableScanOperator extends AbstractDataSourceOpe protected SeriesScanOptions seriesScanOptions; private final boolean ascending; - private final Ordering scanOrder; + protected final Ordering scanOrder; // Some special data types(like BLOB) cannot use statistics protected final boolean canUseStatistics; private final long cachedRawDataSize; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java index f00b0049ef663..797d4491e417c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java @@ -34,7 +34,7 @@ public abstract class AbstractDefaultAggTableScanOperator extends AbstractAggTableScanOperator { - private static final long INSTANCE_SIZE = + protected static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(AbstractDefaultAggTableScanOperator.class); protected AbstractDefaultAggTableScanOperator(AbstractAggTableScanOperatorParameter parameter) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java new file mode 100644 index 0000000000000..79a101a6a79f3 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.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.execution.operator.source.relational; + +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.SeriesScanUtil; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.constructAlignedPath; + +public class ExternalTsFileAggTableScanOperator extends DefaultAggTableScanOperator { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(ExternalTsFileAggTableScanOperator.class); + + private final String tableName; + private final List> deviceOffsets; + + public ExternalTsFileAggTableScanOperator( + AbstractAggTableScanOperatorParameter parameter, + String tableName, + List> deviceOffsets) { + super(parameter); + this.tableName = tableName; + this.deviceOffsets = new ArrayList<>(deviceOffsets); + if (deviceCount != this.deviceOffsets.size()) { + throw new IllegalArgumentException( + "The size of external TsFile device offsets should be equal to device entries"); + } + } + + @Override + String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { + int segmentOffset = + deviceEntry.getDeviceID().segmentNum() > 0 + && tableName.equalsIgnoreCase((String) deviceEntry.getNthSegment(0)) + ? 1 + : 0; + Object segment = deviceEntry.getNthSegment(idColumnIndex + segmentOffset); + return segment == null ? null : (String) segment; + } + + @Override + protected void constructAlignedSeriesScanUtil() { + DeviceEntry deviceEntry = + deviceEntries.isEmpty() || deviceEntries.get(currentDeviceIndex) == null + ? new AlignedDeviceEntry(SeriesScanUtil.EMPTY_DEVICE_ID, new Binary[0]) + : deviceEntries.get(currentDeviceIndex); + this.seriesScanUtil = + new ExternalTsFileSeriesScanUtil( + constructAlignedPath( + deviceEntry, measurementColumnNames, measurementSchemas, allSensors), + scanOrder, + seriesScanOptions, + ((OperatorContext) operatorContext).getInstanceContext(), + true, + measurementColumnTSDataTypes, + this::loadTimeSeriesMetadata); + } + + private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedPath) throws IOException { + if (deviceEntries.isEmpty() || currentDeviceIndex >= deviceEntries.size()) { + return null; + } + List currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); + return ExternalTsFileSeriesScanUtil.loadTimeSeriesMetadata( + resource, + alignedPath, + deviceEntries.get(currentDeviceIndex).getDeviceID(), + currentDeviceOffsets, + ((OperatorContext) operatorContext).getInstanceContext(), + seriesScanOptions.getGlobalTimeFilter()); + } + + @Override + public long ramBytesUsed() { + return super.ramBytesUsed() + + INSTANCE_SIZE + - AbstractDefaultAggTableScanOperator.INSTANCE_SIZE + + RamUsageEstimator.sizeOfCollection(deviceOffsets); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java index f8e7766ffbe83..f46e3b0434681 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java @@ -20,8 +20,10 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.operator.source.AlignedSeriesScanUtil; +import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; @@ -29,6 +31,8 @@ import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.filter.basic.Filter; import java.io.IOException; import java.util.List; @@ -78,6 +82,44 @@ protected void updateFilterUsingTTL(QueryDataSource dataSource) { // External TsFiles are not managed by IoTDB metadata, so no table/tree TTL applies here. } + static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, + AlignedFullPath alignedPath, + IDeviceID currentDeviceID, + List currentDeviceOffsets, + FragmentInstanceContext context, + Filter globalTimeFilter) + throws IOException { + if (currentDeviceOffsets == null || !currentDeviceID.equals(alignedPath.getDeviceId())) { + return null; + } + + long[] deviceMeasurementNodeOffset = + getDeviceMeasurementNodeOffset(currentDeviceOffsets, resource.getTsFilePath()); + if (deviceMeasurementNodeOffset == null) { + return null; + } + // TODO: Use deviceMeasurementNodeOffset after FileLoaderUtils supports offset-based metadata + // loading in this branch. + return FileLoaderUtils.loadAlignedTimeSeriesMetadata( + resource, + alignedPath, + context, + globalTimeFilter, + resource.isSeq(), + context.isIgnoreAllNullRows()); + } + + private static long[] getDeviceMeasurementNodeOffset( + List currentDeviceOffsets, String tsFilePath) { + for (ExternalTsFileDeviceOffset offset : currentDeviceOffsets) { + if (tsFilePath.equals(offset.getTsFilePath())) { + return offset.getDeviceMeasurementNodeOffset(); + } + } + return null; + } + @FunctionalInterface public interface ExternalTsFileMetadataLoader { AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java index daccb199d2894..2ad90b1d46082 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java @@ -22,7 +22,6 @@ import org.apache.iotdb.commons.path.AlignedFullPath; import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; -import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -92,35 +91,13 @@ protected void constructAlignedSeriesScanUtil() { private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( TsFileResource resource, AlignedFullPath alignedPath) throws IOException { List currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); - if (currentDeviceOffsets == null - || !getCurrentDeviceEntry().getDeviceID().equals(alignedPath.getDeviceId())) { - return null; - } - - long[] deviceMeasurementNodeOffset = - getDeviceMeasurementNodeOffset(currentDeviceOffsets, resource.getTsFilePath()); - if (deviceMeasurementNodeOffset == null) { - return null; - } - // TODO: Use deviceMeasurementNodeOffset after FileLoaderUtils supports offset-based metadata - // loading in this branch. - return FileLoaderUtils.loadAlignedTimeSeriesMetadata( + return ExternalTsFileSeriesScanUtil.loadTimeSeriesMetadata( resource, alignedPath, + getCurrentDeviceEntry().getDeviceID(), + currentDeviceOffsets, ((OperatorContext) operatorContext).getInstanceContext(), - seriesScanOptions.getGlobalTimeFilter(), - resource.isSeq(), - ((OperatorContext) operatorContext).getInstanceContext().isIgnoreAllNullRows()); - } - - private long[] getDeviceMeasurementNodeOffset( - List currentDeviceOffsets, String tsFilePath) { - for (ExternalTsFileDeviceOffset offset : currentDeviceOffsets) { - if (tsFilePath.equals(offset.getTsFilePath())) { - return offset.getDeviceMeasurementNodeOffset(); - } - } - return null; + seriesScanOptions.getGlobalTimeFilter()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index 5ceeef9f86b3c..85835931f243f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -96,6 +96,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.relational.CteScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.DefaultAggTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.DeviceIteratorScanOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileAggTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.InformationSchemaTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.LastQueryAggTableScanOperator; @@ -124,6 +125,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExchangeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; @@ -1628,6 +1630,33 @@ public Operator visitAggregationTableScan( } } + @Override + public Operator visitExternalTsFileAggregationScan( + ExternalTsFileAggregationScanNode node, LocalExecutionPlanContext context) { + AbstractAggTableScanOperator.AbstractAggTableScanOperatorParameter parameter = + constructAbstractAggTableScanOperatorParameter(node, context); + + ExternalTsFileAggTableScanOperator aggTableScanOperator = + new ExternalTsFileAggTableScanOperator( + parameter, node.getQualifiedObjectName().getObjectName(), node.getDeviceOffsets()); + + context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); + addSource( + aggTableScanOperator, + context, + node, + parameter.getMeasurementColumnNames(), + parameter.getMeasurementSchemas(), + parameter.getAllSensors(), + ExternalTsFileAggregationScanNode.class.getSimpleName()); + + DataDriverContext dataDriverContext = (DataDriverContext) context.getDriverContext(); + dataDriverContext.setQueryDataSourceType(QueryDataSourceType.EXTERNAL_TSFILE_SCAN); + context.getInstanceContext().addExternalTsFilePaths(node.getTsFilePaths()); + + return aggTableScanOperator; + } + private LastQueryAggTableScanOperator constructLastQueryAggTableScanOperator( AggregationTableScanNode node, AbstractAggTableScanOperator.AbstractAggTableScanOperatorParameter parameter, 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 73c41ab1079f8..81596f2d38c10 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 @@ -124,6 +124,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AlignedAggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; @@ -654,6 +655,10 @@ default R visitExternalTsFileScan(ExternalTsFileScanNode node, C context) { return visitTableScan(node, context); } + default R visitExternalTsFileAggregationScan(ExternalTsFileAggregationScanNode node, C context) { + return visitAggregationTableScan(node, context); + } + default R visitInformationSchemaTableScan(InformationSchemaTableScanNode node, C context) { return visitTableScan(node, context); } 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 104ed577882f7..105dc1c1935d2 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 @@ -97,6 +97,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CopyToNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; @@ -797,6 +798,82 @@ private List splitExternalTsFileScanByDeviceEntries( return result; } + @Override + public List visitExternalTsFileAggregationScan( + ExternalTsFileAggregationScanNode node, PlanContext context) { + TRegionReplicaSet localRegionReplicaSet = + new TRegionReplicaSet(null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())); + node.setRegionReplicaSet(localRegionReplicaSet); + context.mostUsedRegion = node.getRegionReplicaSet(); + List resultNodes = + splitExternalTsFileAggregationScanByDeviceEntries(node, localRegionReplicaSet); + if (context.hasSortProperty) { + processSortProperty(node, resultNodes, context); + } + return resultNodes; + } + + private List splitExternalTsFileAggregationScanByDeviceEntries( + final ExternalTsFileAggregationScanNode node, final TRegionReplicaSet localRegionReplicaSet) { + List deviceEntries = node.getDeviceEntries(); + if (deviceEntries.size() <= 1) { + return Collections.singletonList(node); + } + + int splitCount = + Math.min( + deviceEntries.size(), + IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism()); + if (splitCount <= 1) { + return Collections.singletonList(node); + } + + List> splitDeviceEntries = new ArrayList<>(splitCount); + List>> splitDeviceOffsets = new ArrayList<>(splitCount); + for (int i = 0; i < splitCount; i++) { + splitDeviceEntries.add(new ArrayList<>()); + splitDeviceOffsets.add(new ArrayList<>()); + } + for (int i = 0; i < deviceEntries.size(); i++) { + splitDeviceEntries.get(i % splitCount).add(deviceEntries.get(i)); + splitDeviceOffsets.get(i % splitCount).add(node.getDeviceOffsets().get(i)); + } + + List result = new ArrayList<>(splitCount); + for (int i = 0; i < splitDeviceEntries.size(); i++) { + List entries = splitDeviceEntries.get(i); + if (entries.isEmpty()) { + continue; + } + ExternalTsFileAggregationScanNode splitNode = + new ExternalTsFileAggregationScanNode( + queryId.genPlanNodeId(), + node.getQualifiedObjectName(), + node.getOutputSymbols(), + node.getAssignments(), + entries, + node.getTagAndAttributeIndexMap(), + node.getScanOrder(), + node.getTimePredicate().orElse(null), + node.getPushDownPredicate(), + node.getPushDownLimit(), + node.getPushDownOffset(), + node.isPushLimitToEachDevice(), + node.containsNonAlignedDevice(), + node.getProjection(), + node.getAggregations(), + node.getGroupingSets(), + node.getPreGroupedSymbols(), + node.getStep(), + node.getGroupIdSymbol(), + node.getTsFilePaths(), + splitDeviceOffsets.get(i)); + splitNode.setRegionReplicaSet(localRegionReplicaSet); + result.add(splitNode); + } + return result; + } + private List constructDeviceTableScanByTags( final DeviceTableScanNode node, final PlanContext context) { DataPartition dataPartition = analysis.getDataPartitionInfo(); 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 e5298d01cd34d..19057bc71461e 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 @@ -295,6 +295,31 @@ public static AggregationTableScanNode combineAggregationAndTableScan( AggregationNode aggregationNode, ProjectNode projectNode, DeviceTableScanNode tableScanNode) { + if (tableScanNode instanceof ExternalTsFileScanNode) { + ExternalTsFileScanNode externalTsFileScanNode = (ExternalTsFileScanNode) tableScanNode; + return new ExternalTsFileAggregationScanNode( + id, + tableScanNode.getQualifiedObjectName(), + tableScanNode.getOutputSymbols(), + tableScanNode.getAssignments(), + tableScanNode.getDeviceEntries(), + tableScanNode.getTagAndAttributeIndexMap(), + tableScanNode.getScanOrder(), + tableScanNode.getTimePredicate().orElse(null), + tableScanNode.getPushDownPredicate(), + tableScanNode.getPushDownLimit(), + tableScanNode.getPushDownOffset(), + tableScanNode.isPushLimitToEachDevice(), + tableScanNode.containsNonAlignedDevice(), + projectNode == null ? null : projectNode.getAssignments(), + aggregationNode.getAggregations(), + aggregationNode.getGroupingSets(), + aggregationNode.getPreGroupedSymbols(), + aggregationNode.getStep(), + aggregationNode.getGroupIdSymbol(), + externalTsFileScanNode.getTsFilePaths(), + externalTsFileScanNode.getDeviceOffsets()); + } if (tableScanNode instanceof TreeDeviceViewScanNode) { TreeDeviceViewScanNode treeDeviceViewScanNode = (TreeDeviceViewScanNode) tableScanNode; return new AggregationTreeDeviceViewScanNode( @@ -349,6 +374,31 @@ public static AggregationTableScanNode combineAggregationAndTableScan( ProjectNode projectNode, DeviceTableScanNode tableScanNode, AggregationNode.Step step) { + if (tableScanNode instanceof ExternalTsFileScanNode) { + ExternalTsFileScanNode externalTsFileScanNode = (ExternalTsFileScanNode) tableScanNode; + return new ExternalTsFileAggregationScanNode( + id, + tableScanNode.getQualifiedObjectName(), + tableScanNode.getOutputSymbols(), + tableScanNode.getAssignments(), + tableScanNode.getDeviceEntries(), + tableScanNode.getTagAndAttributeIndexMap(), + tableScanNode.getScanOrder(), + tableScanNode.getTimePredicate().orElse(null), + tableScanNode.getPushDownPredicate(), + tableScanNode.getPushDownLimit(), + tableScanNode.getPushDownOffset(), + tableScanNode.isPushLimitToEachDevice(), + tableScanNode.containsNonAlignedDevice(), + projectNode == null ? null : projectNode.getAssignments(), + aggregationNode.getAggregations(), + aggregationNode.getGroupingSets(), + aggregationNode.getPreGroupedSymbols(), + step, + aggregationNode.getGroupIdSymbol(), + externalTsFileScanNode.getTsFilePaths(), + externalTsFileScanNode.getDeviceOffsets()); + } if (tableScanNode instanceof TreeDeviceViewScanNode) { TreeDeviceViewScanNode treeDeviceViewScanNode = (TreeDeviceViewScanNode) tableScanNode; return new AggregationTreeDeviceViewScanNode( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java new file mode 100644 index 0000000000000..04434ad498ef8 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java @@ -0,0 +1,189 @@ +/* + * 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.commons.queryengine.plan.planner.plan.node.IPlanVisitor; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Assignments; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.node.AggregationNode; +import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.IntStream; + +public class ExternalTsFileAggregationScanNode extends AggregationTableScanNode { + private List tsFilePaths; + private List> deviceOffsets = Collections.emptyList(); + + public ExternalTsFileAggregationScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + List deviceEntries, + Map tagAndAttributeIndexMap, + Ordering scanOrder, + Expression timePredicate, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + boolean pushLimitToEachDevice, + boolean containsNonAlignedDevice, + Assignments projection, + Map aggregations, + AggregationNode.GroupingSetDescriptor groupingSets, + List preGroupedSymbols, + AggregationNode.Step step, + Optional groupIdSymbol, + List tsFilePaths, + List> deviceOffsets) { + super( + id, + qualifiedObjectName, + outputSymbols, + assignments, + deviceEntries, + tagAndAttributeIndexMap, + scanOrder, + timePredicate, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + pushLimitToEachDevice, + containsNonAlignedDevice, + projection, + aggregations, + groupingSets, + preGroupedSymbols, + step, + groupIdSymbol); + this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); + this.deviceOffsets = copyDeviceOffsets(deviceOffsets); + } + + protected ExternalTsFileAggregationScanNode() {} + + @Override + public R accept(IPlanVisitor visitor, C context) { + return ((PlanVisitor) visitor).visitExternalTsFileAggregationScan(this, context); + } + + @Override + public ExternalTsFileAggregationScanNode clone() { + return new ExternalTsFileAggregationScanNode( + id, + qualifiedObjectName, + outputSymbols, + assignments, + deviceEntries, + tagAndAttributeIndexMap, + scanOrder, + timePredicate, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + pushLimitToEachDevice, + containsNonAlignedDevice, + projection, + aggregations, + groupingSets, + preGroupedSymbols, + step, + groupIdSymbol, + tsFilePaths, + deviceOffsets); + } + + public List getTsFilePaths() { + return tsFilePaths; + } + + public List> getDeviceOffsets() { + return deviceOffsets; + } + + @Override + public void sortDeviceEntries(Comparator comparator) { + int[] indexes = + IntStream.range(0, deviceEntries.size()) + .boxed() + .sorted( + (left, right) -> + comparator.compare(deviceEntries.get(left), deviceEntries.get(right))) + .mapToInt(Integer::intValue) + .toArray(); + List sortedDeviceEntries = new ArrayList<>(deviceEntries.size()); + List> sortedDeviceOffsets = + new ArrayList<>(deviceOffsets.size()); + for (int index : indexes) { + sortedDeviceEntries.add(deviceEntries.get(index)); + sortedDeviceOffsets.add(deviceOffsets.get(index)); + } + this.deviceEntries = sortedDeviceEntries; + this.deviceOffsets = sortedDeviceOffsets; + } + + private static List> copyDeviceOffsets( + List> deviceOffsets) { + List> copiedDeviceOffsets = + new ArrayList<>(deviceOffsets.size()); + for (List offsets : deviceOffsets) { + copiedDeviceOffsets.add(new ArrayList<>(offsets)); + } + return copiedDeviceOffsets; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + throw new UnsupportedOperationException( + "ExternalTsFileAggregationScanNode cannot be serialized because it reads local external TsFiles"); + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + throw new UnsupportedOperationException( + "ExternalTsFileAggregationScanNode cannot be serialized because it reads local external TsFiles"); + } + + public static ExternalTsFileAggregationScanNode deserialize(ByteBuffer byteBuffer) { + throw new UnsupportedOperationException( + "ExternalTsFileAggregationScanNode cannot be deserialized because it reads local external TsFiles"); + } + + @Override + public String toString() { + return "ExternalTsFileAggregationScanNode-" + this.getPlanNodeId(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java index b5072bfd5c6bd..c650a46119cbd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java @@ -30,6 +30,7 @@ import org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolAllocator; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; @@ -70,6 +71,7 @@ public PlanNode optimize(PlanNode plan, PlanOptimizer.Context context) { new Rewriter(), new Context( context.getQueryContext().getQueryId(), + context.getAnalysis(), context.getMetadata(), context.sessionInfo(), context.getSymbolAllocator())); @@ -105,7 +107,6 @@ public PlanNode visitAggregation(AggregationNode node, Context context) { // only optimize AggregationNode with raw DeviceTableScanNode if (tableScanNode == null - || tableScanNode instanceof ExternalTsFileScanNode || tableScanNode instanceof AggregationTableScanNode) { // no need to optimize return node; } @@ -116,6 +117,7 @@ public PlanNode visitAggregation(AggregationNode node, Context context) { node.getGroupingKeys(), projectNode, tableScanNode, + context.analysis, context.session, context.metadata); if (pushDownLevel == PushDownLevel.NOOP) { // no push-down @@ -140,6 +142,7 @@ private PushDownLevel calculatePushDownLevel( List groupingKeys, ProjectNode projectNode, DeviceTableScanNode tableScanNode, + Analysis analysis, SessionInfo session, Metadata metadata) { boolean hasProject = projectNode != null; @@ -195,7 +198,7 @@ private PushDownLevel calculatePushDownLevel( return PushDownLevel.NOOP; } else if (singleDeviceEntry || ImmutableSet.copyOf(groupingKeys) - .containsAll(getTagColumnsInTableStore(tableScanNode, metadata, session))) { + .containsAll(getTagColumnsInTableStore(tableScanNode, analysis, metadata, session))) { // If all tag columns appear in groupingKeys and no Measurement column appears, we can push // down completely. return PushDownLevel.COMPLETE; @@ -205,7 +208,19 @@ private PushDownLevel calculatePushDownLevel( } private List getTagColumnsInTableStore( - DeviceTableScanNode tableScanNode, Metadata metadata, SessionInfo session) { + DeviceTableScanNode tableScanNode, + Analysis analysis, + Metadata metadata, + SessionInfo session) { + if (tableScanNode instanceof ExternalTsFileScanNode) { + return analysis + .getTableColumnSchema(tableScanNode.getQualifiedObjectName()) + .entrySet() + .stream() + .filter(entry -> entry.getValue().getColumnCategory() == TAG) + .map(Map.Entry::getKey) + .collect(Collectors.toList()); + } return Objects.requireNonNull( metadata.getTableSchema(session, tableScanNode.getQualifiedObjectName()).orElse(null)) .getColumns() @@ -242,13 +257,19 @@ private enum PushDownLevel { private static class Context { private final QueryId queryId; + private final Analysis analysis; private final Metadata metadata; private final SessionInfo session; private final SymbolAllocator symbolAllocator; public Context( - QueryId queryId, Metadata metadata, SessionInfo session, SymbolAllocator symbolAllocator) { + QueryId queryId, + Analysis analysis, + Metadata metadata, + SessionInfo session, + SymbolAllocator symbolAllocator) { this.queryId = queryId; + this.analysis = analysis; this.metadata = metadata; this.session = session; this.symbolAllocator = symbolAllocator; From 93f9a5027de24e3d08c7010193145960160a3b63 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Mon, 8 Jun 2026 16:44:05 +0800 Subject: [PATCH 12/32] modify parameter name --- ...formationSchemaContentSupplierFactory.java | 8 +- .../config/metadata/ShowFunctionsTask.java | 6 + .../DataNodeTableBuiltinTableFunction.java | 67 ++++++++++ .../tvf/ReadTsFileTableFunction.java | 124 +++++++++++++++--- .../metadata/TableMetadataImpl.java | 11 ++ .../relational/planner/RelationPlanner.java | 6 +- .../function/TableBuiltinTableFunction.java | 6 +- 7 files changed, 201 insertions(+), 27 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java rename iotdb-core/{node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational => datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function}/tvf/ReadTsFileTableFunction.java (79%) 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 ca9c09ecb9548..43efe7c7415bb 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 @@ -86,6 +86,7 @@ import org.apache.iotdb.db.queryengine.plan.execution.IQueryExecution; import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.ShowCreateViewTask; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.security.AccessControl; @@ -133,6 +134,7 @@ import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.iotdb.commons.conf.IoTDBConstant.FUNCTION_STATE_AVAILABLE; import static org.apache.iotdb.commons.conf.IoTDBConstant.FUNCTION_TYPE_BUILTIN_AGG_FUNC; @@ -957,7 +959,11 @@ public boolean hasNext() { TableBuiltinAggregationFunction.getBuiltInAggregateFunctionName().iterator(); } else if (functionType.equals(BINARY_MAP.get(FUNCTION_TYPE_BUILTIN_AGG_FUNC))) { functionType = BINARY_MAP.get(FUNCTION_TYPE_BUILTIN_TABLE_FUNC); - nameIterator = TableBuiltinTableFunction.getBuiltInTableFunctionName().iterator(); + nameIterator = + Stream.concat( + TableBuiltinTableFunction.getBuiltInTableFunctionName().stream(), + DataNodeTableBuiltinTableFunction.getBuiltInTableFunctionName().stream()) + .iterator(); } else { return false; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ShowFunctionsTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ShowFunctionsTask.java index d9fcf3a979c76..9866b77cf13ca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ShowFunctionsTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ShowFunctionsTask.java @@ -36,6 +36,7 @@ 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.relational.function.DataNodeTableBuiltinTableFunction; import org.apache.iotdb.db.queryengine.plan.udf.TreeUDFUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -158,6 +159,11 @@ private static void buildTableModelTsBlock( TableBuiltinTableFunction.getBuiltInTableFunctionName(), BINARY_MAP.get(FUNCTION_TYPE_BUILTIN_TABLE_FUNC), BINARY_MAP.get(FUNCTION_STATE_AVAILABLE)); + appendFunctions( + builder, + DataNodeTableBuiltinTableFunction.getBuiltInTableFunctionName(), + BINARY_MAP.get(FUNCTION_TYPE_BUILTIN_TABLE_FUNC), + BINARY_MAP.get(FUNCTION_STATE_AVAILABLE)); DatasetHeader datasetHeader = DatasetHeaderFactory.getShowFunctionsHeader(); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS, builder.build(), datasetHeader)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java new file mode 100644 index 0000000000000..858c2ab3926f4 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java @@ -0,0 +1,67 @@ +/* + * 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.function; + +import org.apache.iotdb.commons.exception.SemanticException; +import org.apache.iotdb.commons.i18n.QueryMessages; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.ReadTsFileTableFunction; +import org.apache.iotdb.udf.api.relational.TableFunction; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; + +public enum DataNodeTableBuiltinTableFunction { + READ_TSFILE("read_tsfile"); + + private final String functionName; + + DataNodeTableBuiltinTableFunction(String functionName) { + this.functionName = functionName; + } + + public String getFunctionName() { + return functionName; + } + + private static final Set BUILT_IN_TABLE_FUNCTION_NAME = + new HashSet<>( + Arrays.stream(DataNodeTableBuiltinTableFunction.values()) + .map(DataNodeTableBuiltinTableFunction::getFunctionName) + .collect(Collectors.toList())); + + public static Set getBuiltInTableFunctionName() { + return BUILT_IN_TABLE_FUNCTION_NAME; + } + + public static boolean isBuiltInTableFunction(String functionName) { + return BUILT_IN_TABLE_FUNCTION_NAME.contains(functionName.toLowerCase()); + } + + public static TableFunction getBuiltinTableFunction(String functionName) { + switch (functionName.toLowerCase()) { + case "read_tsfile": + return new ReadTsFileTableFunction(); + default: + throw new SemanticException(String.format(QueryMessages.UNKNOWN_FUNCTION, functionName)); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ReadTsFileTableFunction.java similarity index 79% rename from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ReadTsFileTableFunction.java index d2f92f6e28793..71ed1b3670fc3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/relational/tvf/ReadTsFileTableFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ReadTsFileTableFunction.java @@ -17,10 +17,11 @@ * under the License. */ -package org.apache.iotdb.commons.udf.builtin.relational.tvf; +package org.apache.iotdb.db.queryengine.plan.relational.function.tvf; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; import org.apache.iotdb.udf.api.exception.UDFException; import org.apache.iotdb.udf.api.relational.TableFunction; @@ -48,6 +49,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -64,37 +66,38 @@ /** Reads one or more TsFiles as a table function source. */ public class ReadTsFileTableFunction implements TableFunction { private static final String TABLE_NAME_PARAMETER_NAME = "TABLE_NAME"; - private static final String TSFILE_PATHS_PARAMETER_NAME = "TSFILE_PATHS"; + private static final String PATHS_PARAMETER_NAME = "PATHS"; @Override public List getArgumentsSpecifications() { return Arrays.asList( + ScalarParameterSpecification.builder().name(PATHS_PARAMETER_NAME).type(Type.STRING).build(), ScalarParameterSpecification.builder() .name(TABLE_NAME_PARAMETER_NAME) .type(Type.STRING) - .build(), - ScalarParameterSpecification.builder() - .name(TSFILE_PATHS_PARAMETER_NAME) - .type(Type.STRING) + .defaultValue("") .build()); } @Override public TableFunctionAnalysis analyze(Map arguments) throws UDFException { - String tableName = getRequiredStringArgument(arguments, TABLE_NAME_PARAMETER_NAME); + String tableName = getOptionalStringArgument(arguments, TABLE_NAME_PARAMETER_NAME); List tsFilePaths = - parseTsFilePaths(getRequiredStringArgument(arguments, TSFILE_PATHS_PARAMETER_NAME)); + parseTsFilePaths(getRequiredStringArgument(arguments, PATHS_PARAMETER_NAME)); + checkTsFilePathsAreOutsideDataDirs(tsFilePaths); TsFileSchemaCollection schemaCollection = - collectTsFilesAndResolveSchema(tableName, tsFilePaths); + collectTsFilesAndResolveSchema(tableName.isEmpty() ? null : tableName, tsFilePaths); if (schemaCollection.mergedTableSchema == null) { throw new UDFArgumentNotValidException( - "No table schema found for table " + tableName + " in TsFiles"); + tableName.isEmpty() + ? "No table schema found in TsFiles" + : "No table schema found for table " + tableName + " in TsFiles"); } DescribedSchema outputSchema = convertToDescribedSchema(schemaCollection.mergedTableSchema); ReadTsFileTableFunctionHandle handle = new ReadTsFileTableFunctionHandle( - tableName, + schemaCollection.tableName, schemaCollection.tsFiles.stream() .map(File::getAbsolutePath) .collect(Collectors.toList()), @@ -130,6 +133,21 @@ private static String getRequiredStringArgument(Map arguments, return ((String) value).trim(); } + private static String getOptionalStringArgument(Map arguments, String name) { + Argument argument = arguments.get(name); + if (argument == null) { + return ""; + } + if (!(argument instanceof ScalarArgument)) { + throw new UDFArgumentNotValidException("Invalid scalar argument: " + name); + } + Object value = ((ScalarArgument) argument).getValue(); + if (!(value instanceof String)) { + throw new UDFArgumentNotValidException("Argument " + name + " should be a string"); + } + return ((String) value).trim(); + } + private static List parseTsFilePaths(String tsFilePaths) { List paths = Arrays.stream(tsFilePaths.split(",")) @@ -138,15 +156,44 @@ private static List parseTsFilePaths(String tsFilePaths) { .collect(Collectors.toList()); if (paths.isEmpty()) { throw new UDFArgumentNotValidException( - "Argument " + TSFILE_PATHS_PARAMETER_NAME + " should contain at least one path"); + "Argument " + PATHS_PARAMETER_NAME + " should contain at least one path"); } return paths; } + private static void checkTsFilePathsAreOutsideDataDirs(List tsFilePaths) { + List dataDirs = + Arrays.stream(IoTDBDescriptor.getInstance().getConfig().getDataDirs()) + .map(ReadTsFileTableFunction::normalizePath) + .collect(Collectors.toList()); + for (String tsFilePath : tsFilePaths) { + Path normalizedTsFilePath = normalizePath(tsFilePath); + for (Path dataDir : dataDirs) { + if (normalizedTsFilePath.startsWith(dataDir) || dataDir.startsWith(normalizedTsFilePath)) { + throw new UDFArgumentNotValidException( + String.format( + "read_tsfile path %s is not allowed because it may access IoTDB data directory %s", + tsFilePath, dataDir)); + } + } + } + } + + private static Path normalizePath(String path) { + Path normalizedPath = Paths.get(path).toAbsolutePath().normalize(); + try { + return normalizedPath.toRealPath(); + } catch (IOException e) { + return normalizedPath; + } + } + private static TsFileSchemaCollection collectTsFilesAndResolveSchema( - String tableName, List tsFilePaths) { + String specifiedTableName, List tsFilePaths) { List tsFiles = new ArrayList<>(); MergedTableSchemaBuilder schemaBuilder = null; + String resolvedTableName = + specifiedTableName == null ? null : specifiedTableName.toLowerCase(Locale.ENGLISH); for (String tsFilePath : tsFilePaths) { Path path = new File(tsFilePath).toPath(); if (!Files.exists(path)) { @@ -156,13 +203,25 @@ private static TsFileSchemaCollection collectTsFilesAndResolveSchema( Iterator iterator = walkedPaths.filter(Files::isRegularFile).iterator(); while (iterator.hasNext()) { Path filePath = iterator.next(); - TableSchema tableSchema = tryReadTableSchema(tableName, filePath.toFile()); + TableSchema tableSchema = + specifiedTableName == null + ? tryInferTableSchema(filePath.toFile()) + : tryReadTableSchema(specifiedTableName, filePath.toFile()); if (tableSchema == null) { continue; } + String currentTableName = tableSchema.getTableName().toLowerCase(Locale.ENGLISH); + if (resolvedTableName == null) { + resolvedTableName = currentTableName; + } else if (!resolvedTableName.equals(currentTableName)) { + throw new UDFArgumentNotValidException( + String.format( + "Cannot infer table name from TsFiles because multiple tables are found: %s and %s", + resolvedTableName, currentTableName)); + } tsFiles.add(filePath.toFile()); if (schemaBuilder == null) { - schemaBuilder = new MergedTableSchemaBuilder(tableName, tableSchema); + schemaBuilder = new MergedTableSchemaBuilder(resolvedTableName, tableSchema); } else { schemaBuilder.merge(tableSchema); } @@ -174,8 +233,7 @@ private static TsFileSchemaCollection collectTsFilesAndResolveSchema( if (tsFiles.isEmpty()) { throw new UDFArgumentNotValidException("No valid TsFiles found"); } - return new TsFileSchemaCollection( - tsFiles, schemaBuilder == null ? null : schemaBuilder.build()); + return new TsFileSchemaCollection(resolvedTableName, tsFiles, schemaBuilder.build()); } private static TableSchema tryReadTableSchema(String tableName, File tsFile) { @@ -193,11 +251,41 @@ private static TableSchema tryReadTableSchema(String tableName, File tsFile) { } } + private static TableSchema tryInferTableSchema(File tsFile) { + if (!tsFile.canRead()) { + return null; + } + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + if (!reader.isComplete() || reader.readVersionNumber() != TSFileConfig.VERSION_NUMBER) { + return null; + } + Map tableSchemaMap = reader.getTableSchemaMap(); + if (tableSchemaMap.isEmpty()) { + throw new UDFArgumentNotValidException( + "Cannot infer table name from TsFile because no table schema is found in " + + tsFile.getAbsolutePath()); + } + if (tableSchemaMap.size() > 1) { + throw new UDFArgumentNotValidException( + "Cannot infer table name from TsFile because multiple tables are found in " + + tsFile.getAbsolutePath()); + } + return tableSchemaMap.values().iterator().next(); + } catch (UDFArgumentNotValidException e) { + throw e; + } catch (Exception e) { + return null; + } + } + private static class TsFileSchemaCollection { + private final String tableName; private final List tsFiles; private final TableSchema mergedTableSchema; - private TsFileSchemaCollection(List tsFiles, TableSchema mergedTableSchema) { + private TsFileSchemaCollection( + String tableName, List tsFiles, TableSchema mergedTableSchema) { + this.tableName = tableName; this.tsFiles = tsFiles; this.mergedTableSchema = mergedTableSchema; } 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 898b226ffa135..1305f8db4a0a0 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 @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.partition.SchemaPartition; import org.apache.iotdb.commons.queryengine.common.SessionInfo; import org.apache.iotdb.commons.queryengine.plan.relational.function.OperatorType; +import org.apache.iotdb.commons.queryengine.plan.relational.function.TableFunctionFactory; import org.apache.iotdb.commons.queryengine.plan.relational.function.arithmetic.AdditionResolver; import org.apache.iotdb.commons.queryengine.plan.relational.function.arithmetic.DivisionResolver; import org.apache.iotdb.commons.queryengine.plan.relational.function.arithmetic.ModulusResolver; @@ -52,6 +53,7 @@ import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.plan.analyze.ClusterPartitionFetcher; import org.apache.iotdb.db.queryengine.plan.analyze.IPartitionFetcher; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.TableDeviceSchemaFetcher; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.TableDeviceSchemaValidator; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.TableHeaderSchemaValidator; @@ -62,6 +64,7 @@ import org.apache.iotdb.udf.api.customizer.parameter.FunctionArguments; import org.apache.iotdb.udf.api.relational.AggregateFunction; import org.apache.iotdb.udf.api.relational.ScalarFunction; +import org.apache.iotdb.udf.api.relational.TableFunction; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.common.type.ObjectType; @@ -95,6 +98,14 @@ public class TableMetadataImpl implements Metadata { private final DataNodeTableCache tableCache = DataNodeTableCache.getInstance(); + @Override + public TableFunction getTableFunction(String functionName) { + if (DataNodeTableBuiltinTableFunction.isBuiltInTableFunction(functionName)) { + return DataNodeTableBuiltinTableFunction.getBuiltinTableFunction(functionName); + } + return TableFunctionFactory.getTableFunction(functionName); + } + @Override public boolean tableExists(final QualifiedObjectName name) { return tableCache.getTable(name.getDatabaseName(), name.getObjectName(), false) != null; 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 f857775cc45f0..5855b15922f2b 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 @@ -24,7 +24,6 @@ import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.relational.analyzer.NodeRef; -import org.apache.iotdb.commons.queryengine.plan.relational.function.TableBuiltinTableFunction; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Assignments; @@ -89,7 +88,6 @@ import org.apache.iotdb.commons.queryengine.plan.relational.type.InternalTypeManager; import org.apache.iotdb.commons.queryengine.utils.cte.CteDataStore; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction; import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; @@ -116,6 +114,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Scope; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableArgumentAnalysis; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableFunctionInvocationAnalysis; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.ReadTsFileTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TreeDeviceViewSchema; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.IrUtils; @@ -1459,7 +1459,7 @@ public RelationPlan visitDelete(final Delete node, final Void context) { @Override public RelationPlan visitTableFunctionInvocation(TableFunctionInvocation node, Void context) { TableFunctionInvocationAnalysis functionAnalysis = analysis.getTableFunctionAnalysis(node); - if (TableBuiltinTableFunction.READ_TSFILE + if (DataNodeTableBuiltinTableFunction.READ_TSFILE .getFunctionName() .equalsIgnoreCase(functionAnalysis.getFunctionName())) { return planExternalTsFileScan(node, functionAnalysis); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/relational/function/TableBuiltinTableFunction.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/relational/function/TableBuiltinTableFunction.java index fe54d1078cdd0..43673a2e90ce1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/relational/function/TableBuiltinTableFunction.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/plan/relational/function/TableBuiltinTableFunction.java @@ -26,7 +26,6 @@ import org.apache.iotdb.commons.udf.builtin.relational.tvf.CapacityTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.CumulateTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.HOPTableFunction; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.SessionTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.TumbleTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.tvf.VariationTableFunction; @@ -46,8 +45,7 @@ public enum TableBuiltinTableFunction { CAPACITY("capacity"), FORECAST("forecast"), PATTERN_MATCH("pattern_match"), - CLASSIFY("classify"), - READ_TSFILE("read_tsfile"); + CLASSIFY("classify"); private final String functionName; @@ -93,8 +91,6 @@ public static TableFunction getBuiltinTableFunction(String functionName) { return new ForecastTableFunction(); case "classify": return new ClassifyTableFunction(); - case "read_tsfile": - return new ReadTsFileTableFunction(); default: throw new UnsupportedOperationException( String.format(QueryMessages.UNSUPPORTED_TABLE_FUNCTION, functionName)); From 9cf449d0eefd4743b78269da9914b863305b9971 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Wed, 10 Jun 2026 09:54:49 +0800 Subject: [PATCH 13/32] add external tsfile resource --- .../queryengine/common/MPPQueryContext.java | 51 ++ .../fragment/FragmentInstanceContext.java | 4 - .../ExternalTsFileAggTableScanOperator.java | 59 +- .../ExternalTsFileSeriesScanUtil.java | 15 +- .../ExternalTsFileTableScanOperator.java | 59 +- .../plan/execution/QueryExecution.java | 1 + .../DataNodeTableOperatorGenerator.java | 10 +- .../node/DataNodePlanNodeDeserializer.java | 3 - .../DataNodeTableBuiltinTableFunction.java | 10 +- .../ExternalTsFileQueryResource.java | 634 ++++++++++++++++++ .../ReadTsFileTableFunction.java | 30 +- .../relational/planner/RelationPlanner.java | 26 +- .../TableDistributedPlanGenerator.java | 204 +++--- .../iterative/rule/PruneTableScanColumns.java | 9 +- .../node/AggregationTableScanNode.java | 54 +- .../planner/node/DeviceTableScanNode.java | 5 - .../ExternalTsFileAggregationScanNode.java | 93 ++- .../planner/node/ExternalTsFileScanNode.java | 238 ++----- .../PushPredicateIntoTableScan.java | 48 +- .../UnaliasSymbolReferences.java | 9 +- 20 files changed, 1057 insertions(+), 505 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/{ => readTsFile}/ReadTsFileTableFunction.java (95%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 1e5bd2bff7706..8b1303ef44ea3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -32,20 +32,27 @@ import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.commons.queryengine.utils.cte.CteDataStore; +import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.analyze.Analysis; import org.apache.iotdb.db.queryengine.plan.analyze.PredicateUtils; import org.apache.iotdb.db.queryengine.plan.analyze.QueryType; import org.apache.iotdb.db.queryengine.plan.analyze.TypeProvider; import org.apache.iotdb.db.queryengine.plan.analyze.lock.SchemaLockType; import org.apache.iotdb.db.queryengine.plan.planner.memory.NotThreadSafeMemoryReservationManager; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; import org.apache.iotdb.db.queryengine.statistics.QueryPlanStatistics; import com.google.common.collect.ImmutableList; import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.nio.file.Paths; import java.time.ZoneId; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -62,6 +69,8 @@ * info and so on. */ public class MPPQueryContext implements IAuditEntity { + private static final Logger LOGGER = LoggerFactory.getLogger(MPPQueryContext.class); + private String sql; private final QueryId queryId; @@ -158,6 +167,8 @@ public enum ExplainType { // Tables in the subquery private final Map, List> subQueryTables = new HashMap<>(); + private List externalTsFileQueryResources; + @TestOnly public MPPQueryContext(QueryId queryId) { this.queryId = queryId; @@ -243,6 +254,46 @@ public QueryId getQueryId() { return queryId; } + public ExternalTsFileQueryResource createExternalTsFileQueryResource( + String tableName, List tsFilePaths) { + if (externalTsFileQueryResources == null) { + externalTsFileQueryResources = new ArrayList<>(); + } + ExternalTsFileQueryResource externalTsFileQueryResource = + new ExternalTsFileQueryResource( + queryId, + Paths.get(IoTDBDescriptor.getInstance().getConfig().getSortTmpDir()) + .resolve(ExternalTsFileQueryResource.EXTERNAL_TSFILE_TMP_DIR) + .resolve(queryId.getId()) + .resolve(String.valueOf(externalTsFileQueryResources.size())), + tableName, + tsFilePaths, + ignored -> {}, + true); + externalTsFileQueryResources.add(externalTsFileQueryResource); + return externalTsFileQueryResource; + } + + public void releaseExternalTsFileQueryResources() { + if (externalTsFileQueryResources == null) { + return; + } + for (ExternalTsFileQueryResource externalTsFileQueryResource : externalTsFileQueryResources) { + try { + externalTsFileQueryResource.close(); + } catch (Exception e) { + LOGGER.warn("Failed to release external TsFile query resource", e); + } + } + FileUtils.deleteFileOrDirectory( + Paths.get(IoTDBDescriptor.getInstance().getConfig().getSortTmpDir()) + .resolve(ExternalTsFileQueryResource.EXTERNAL_TSFILE_TMP_DIR) + .resolve(queryId.getId()) + .toFile(), + true); + externalTsFileQueryResources = null; + } + public long getLocalQueryId() { return localQueryId; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 654df2301d884..8ae3e885f5e6b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -826,7 +826,6 @@ public boolean initExternalTsFileQueryDataSource(List externalTsFilePath resource.setTimeIndex(new FileTimeIndex(Long.MIN_VALUE, Long.MAX_VALUE)); } externalTsFileResources.add(resource); - FileReaderManager.getInstance().increaseExternalFileReaderReference(externalTsFilePath); } this.sharedQueryDataSource = @@ -1083,9 +1082,6 @@ public synchronized void releaseResource() { } if (externalTsFileResources != null) { - for (TsFileResource tsFile : externalTsFileResources) { - FileReaderManager.getInstance().decreaseExternalFileReaderReference(tsFile.getTsFilePath()); - } externalTsFileResources = null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java index 79a101a6a79f3..3b4b27041032c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java @@ -20,9 +20,11 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; import org.apache.iotdb.commons.path.AlignedFullPath; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.SeriesScanUtil; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceOffset; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.MultiWayMergeReader; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -32,7 +34,7 @@ import org.apache.tsfile.utils.RamUsageEstimator; import java.io.IOException; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.constructAlignedPath; @@ -42,19 +44,21 @@ public class ExternalTsFileAggTableScanOperator extends DefaultAggTableScanOpera RamUsageEstimator.shallowSizeOfInstance(ExternalTsFileAggTableScanOperator.class); private final String tableName; - private final List> deviceOffsets; + private final ExternalTsFileQueryResource externalTsFileQueryResource; + private final int deviceTaskPartitionIndex; + private MultiWayMergeReader deviceTaskReader; + private int loadedDeviceOffsetIndex = -1; + private List currentDeviceOffsets = Collections.emptyList(); public ExternalTsFileAggTableScanOperator( AbstractAggTableScanOperatorParameter parameter, String tableName, - List> deviceOffsets) { + ExternalTsFileQueryResource externalTsFileQueryResource, + int deviceTaskPartitionIndex) { super(parameter); this.tableName = tableName; - this.deviceOffsets = new ArrayList<>(deviceOffsets); - if (deviceCount != this.deviceOffsets.size()) { - throw new IllegalArgumentException( - "The size of external TsFile device offsets should be equal to device entries"); - } + this.externalTsFileQueryResource = externalTsFileQueryResource; + this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; } @Override @@ -91,21 +95,52 @@ private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( if (deviceEntries.isEmpty() || currentDeviceIndex >= deviceEntries.size()) { return null; } - List currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); return ExternalTsFileSeriesScanUtil.loadTimeSeriesMetadata( resource, alignedPath, deviceEntries.get(currentDeviceIndex).getDeviceID(), - currentDeviceOffsets, + getCurrentDeviceOffsets(), + externalTsFileQueryResource.getTsFilePaths(), ((OperatorContext) operatorContext).getInstanceContext(), seriesScanOptions.getGlobalTimeFilter()); } + private List getCurrentDeviceOffsets() throws IOException { + if (loadedDeviceOffsetIndex == currentDeviceIndex) { + return currentDeviceOffsets; + } + if (deviceTaskReader == null) { + deviceTaskReader = + externalTsFileQueryResource.getMultiWayMergeReader(deviceTaskPartitionIndex); + } + DeviceEntry currentDeviceEntry = deviceEntries.get(currentDeviceIndex); + while (deviceTaskReader.hasNextDevice()) { + DeviceEntry deviceEntry = deviceTaskReader.nextDevice(); + if (deviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { + currentDeviceOffsets = deviceTaskReader.getCurrentDeviceOffsets(); + loadedDeviceOffsetIndex = currentDeviceIndex; + return currentDeviceOffsets; + } + } + currentDeviceOffsets = Collections.emptyList(); + loadedDeviceOffsetIndex = currentDeviceIndex; + return currentDeviceOffsets; + } + + @Override + public void close() throws Exception { + if (deviceTaskReader != null) { + deviceTaskReader.close(); + deviceTaskReader = null; + } + super.close(); + } + @Override public long ramBytesUsed() { return super.ramBytesUsed() + INSTANCE_SIZE - AbstractDefaultAggTableScanOperator.INSTANCE_SIZE - + RamUsageEstimator.sizeOfCollection(deviceOffsets); + + RamUsageEstimator.sizeOfCollection(currentDeviceOffsets); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java index f46e3b0434681..6ad6aa2af3f78 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java @@ -20,11 +20,11 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; import org.apache.iotdb.commons.path.AlignedFullPath; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.operator.source.AlignedSeriesScanUtil; import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceOffset; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -86,7 +86,8 @@ static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( TsFileResource resource, AlignedFullPath alignedPath, IDeviceID currentDeviceID, - List currentDeviceOffsets, + List currentDeviceOffsets, + List tsFilePaths, FragmentInstanceContext context, Filter globalTimeFilter) throws IOException { @@ -95,7 +96,7 @@ static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( } long[] deviceMeasurementNodeOffset = - getDeviceMeasurementNodeOffset(currentDeviceOffsets, resource.getTsFilePath()); + getDeviceMeasurementNodeOffset(currentDeviceOffsets, tsFilePaths, resource.getTsFilePath()); if (deviceMeasurementNodeOffset == null) { return null; } @@ -111,10 +112,10 @@ static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( } private static long[] getDeviceMeasurementNodeOffset( - List currentDeviceOffsets, String tsFilePath) { - for (ExternalTsFileDeviceOffset offset : currentDeviceOffsets) { - if (tsFilePath.equals(offset.getTsFilePath())) { - return offset.getDeviceMeasurementNodeOffset(); + List currentDeviceOffsets, List tsFilePaths, String tsFilePath) { + for (DeviceOffset offset : currentDeviceOffsets) { + if (tsFilePath.equals(tsFilePaths.get(offset.getFileIndex()))) { + return offset.getMeasurementNodeOffset(); } } return null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java index 2ad90b1d46082..ad14540e29173 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java @@ -20,8 +20,10 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; import org.apache.iotdb.commons.path.AlignedFullPath; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceOffset; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.MultiWayMergeReader; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -29,7 +31,7 @@ import org.apache.tsfile.utils.RamUsageEstimator; import java.io.IOException; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; public class ExternalTsFileTableScanOperator extends TableScanOperator { @@ -39,19 +41,21 @@ public class ExternalTsFileTableScanOperator extends TableScanOperator { RamUsageEstimator.shallowSizeOfInstance(AbstractDeviceTableScanOperator.class); private final String tableName; - private final List> deviceOffsets; + private final ExternalTsFileQueryResource externalTsFileQueryResource; + private final int deviceTaskPartitionIndex; + private MultiWayMergeReader deviceTaskReader; + private int loadedDeviceOffsetIndex = -1; + private List currentDeviceOffsets = Collections.emptyList(); public ExternalTsFileTableScanOperator( AbstractTableScanOperatorParameter parameter, String tableName, - List> deviceOffsets) { + ExternalTsFileQueryResource externalTsFileQueryResource, + int deviceTaskPartitionIndex) { super(parameter); this.tableName = tableName; - this.deviceOffsets = new ArrayList<>(deviceOffsets); - if (deviceCount != this.deviceOffsets.size()) { - throw new IllegalArgumentException( - "The size of external TsFile device offsets should be equal to device entries"); - } + this.externalTsFileQueryResource = externalTsFileQueryResource; + this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; } @Override @@ -90,21 +94,52 @@ protected void constructAlignedSeriesScanUtil() { private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( TsFileResource resource, AlignedFullPath alignedPath) throws IOException { - List currentDeviceOffsets = deviceOffsets.get(currentDeviceIndex); return ExternalTsFileSeriesScanUtil.loadTimeSeriesMetadata( resource, alignedPath, getCurrentDeviceEntry().getDeviceID(), - currentDeviceOffsets, + getCurrentDeviceOffsets(), + externalTsFileQueryResource.getTsFilePaths(), ((OperatorContext) operatorContext).getInstanceContext(), seriesScanOptions.getGlobalTimeFilter()); } + private List getCurrentDeviceOffsets() throws IOException { + if (loadedDeviceOffsetIndex == currentDeviceIndex) { + return currentDeviceOffsets; + } + if (deviceTaskReader == null) { + deviceTaskReader = + externalTsFileQueryResource.getMultiWayMergeReader(deviceTaskPartitionIndex); + } + DeviceEntry currentDeviceEntry = getCurrentDeviceEntry(); + while (deviceTaskReader.hasNextDevice()) { + DeviceEntry deviceEntry = deviceTaskReader.nextDevice(); + if (deviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { + currentDeviceOffsets = deviceTaskReader.getCurrentDeviceOffsets(); + loadedDeviceOffsetIndex = currentDeviceIndex; + return currentDeviceOffsets; + } + } + currentDeviceOffsets = Collections.emptyList(); + loadedDeviceOffsetIndex = currentDeviceIndex; + return currentDeviceOffsets; + } + + @Override + public void close() throws Exception { + if (deviceTaskReader != null) { + deviceTaskReader.close(); + deviceTaskReader = null; + } + super.close(); + } + @Override public long ramBytesUsed() { return super.ramBytesUsed() + INSTANCE_SIZE - ABSTRACT_DEVICE_TABLE_SCAN_OPERATOR_INSTANCE_SIZE - + RamUsageEstimator.sizeOfCollection(deviceOffsets); + + RamUsageEstimator.sizeOfCollection(currentDeviceOffsets); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java index a5f96e3c6c0b3..0a921dd2dd819 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java @@ -392,6 +392,7 @@ private void releaseResource(Throwable t) { } cleanUpResultHandle(); } + context.releaseExternalTsFileQueryResources(); } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index 85835931f243f..b0fc9e1d5546c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -1138,7 +1138,10 @@ public Operator visitExternalTsFileScan( AbstractTableScanOperator externalTsFileTableScanOperator = new ExternalTsFileTableScanOperator( - parameter, node.getQualifiedObjectName().getObjectName(), node.getDeviceOffsets()); + parameter, + node.getQualifiedObjectName().getObjectName(), + node.getExternalTsFileQueryResource(), + node.getDeviceTaskPartitionIndex()); context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); @@ -1638,7 +1641,10 @@ public Operator visitExternalTsFileAggregationScan( ExternalTsFileAggTableScanOperator aggTableScanOperator = new ExternalTsFileAggTableScanOperator( - parameter, node.getQualifiedObjectName().getObjectName(), node.getDeviceOffsets()); + parameter, + node.getQualifiedObjectName().getObjectName(), + node.getExternalTsFileQueryResource(), + node.getDeviceTaskPartitionIndex()); context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); addSource( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/DataNodePlanNodeDeserializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/DataNodePlanNodeDeserializer.java index b79011de3ecf6..2a9b9b6905c77 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/DataNodePlanNodeDeserializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/DataNodePlanNodeDeserializer.java @@ -123,7 +123,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertTabletNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AlignedAggregationTreeDeviceViewScanNode; -import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; @@ -470,8 +469,6 @@ public PlanNode deserialize(ByteBuffer buffer, short nodeType) { return AlignedAggregationTreeDeviceViewScanNode.deserialize(buffer); case 1042: return NonAlignedAggregationTreeDeviceViewScanNode.deserialize(buffer); - case 1043: - return ExternalTsFileScanNode.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/relational/function/DataNodeTableBuiltinTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java index 858c2ab3926f4..8e4d0830d415e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java @@ -21,11 +21,10 @@ import org.apache.iotdb.commons.exception.SemanticException; import org.apache.iotdb.commons.i18n.QueryMessages; -import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.ReadTsFileTableFunction; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ReadTsFileTableFunction; import org.apache.iotdb.udf.api.relational.TableFunction; import java.util.Arrays; -import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; @@ -43,10 +42,9 @@ public String getFunctionName() { } private static final Set BUILT_IN_TABLE_FUNCTION_NAME = - new HashSet<>( - Arrays.stream(DataNodeTableBuiltinTableFunction.values()) - .map(DataNodeTableBuiltinTableFunction::getFunctionName) - .collect(Collectors.toList())); + Arrays.stream(DataNodeTableBuiltinTableFunction.values()) + .map(DataNodeTableBuiltinTableFunction::getFunctionName) + .collect(Collectors.toSet()); public static Set getBuiltInTableFunctionName() { return BUILT_IN_TABLE_FUNCTION_NAME; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java new file mode 100644 index 0000000000000..8371c6a16873f --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -0,0 +1,634 @@ +/* + * 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.function.tvf.readTsFile; + +import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileDeviceFilterVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.LazyTsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.Closeable; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.function.LongConsumer; + +import static java.util.Objects.requireNonNull; + +/** + * Query-scoped resource for external TsFile scans. + * + *

Reader instances are owned by {@link FileReaderManager}; this class only balances reference + * increments/decrements for the query's external TsFile paths and deletes the query temporary + * directory. + */ +public class ExternalTsFileQueryResource implements AutoCloseable { + + public static final String EXTERNAL_TSFILE_TMP_DIR = "external-tsfile"; + private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 8L * 1024 * 1024; + private static final long DEVICE_OFFSET_INSTANCE_SIZE_IN_BYTES = 32L; + + private final QueryId queryId; + private final Path queryTempRoot; + private final String tableName; + private final List tsFilePaths; + private final LongConsumer ioSizeRecorder; + private final List deviceEntries = new ArrayList<>(); + private List deviceTaskPartitions = Collections.emptyList(); + private Comparator deviceEntryComparator; + + private boolean readersRetained; + private boolean closed; + + public ExternalTsFileQueryResource( + QueryId queryId, + Path tempRoot, + String tableName, + List tsFilePaths, + LongConsumer ioSizeRecorder, + boolean useExactTempRoot) { + this.queryId = queryId; + this.queryTempRoot = + useExactTempRoot + ? requireNonNull(tempRoot, "tempRoot is null") + : requireNonNull(tempRoot, "tempRoot is null") + .resolve(requireNonNull(queryId, "queryId is null").getId()); + this.tableName = tableName; + this.tsFilePaths = + Collections.unmodifiableList(new ArrayList<>(requireNonNull(tsFilePaths, "tsFilePaths"))); + this.ioSizeRecorder = requireNonNull(ioSizeRecorder, "ioSizeRecorder is null"); + } + + public synchronized void collectDeviceEntries( + SchemaFilter schemaFilter, Comparator comparator, int partitionCount) { + checkNotClosed(); + retainFileReaderReferences(); + ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = new ExternalTsFileDeviceFilterVisitor(); + try (DeviceCollector deviceCollector = new DeviceCollector()) { + List partitions = createDeviceTaskPartitions(partitionCount); + while (deviceCollector.hasNextDevice()) { + IDeviceID deviceID = deviceCollector.nextDevice(); + if (schemaFilter != null + && !Boolean.TRUE.equals(schemaFilter.accept(deviceFilterVisitor, deviceID))) { + continue; + } + DeviceEntry deviceEntry = new AlignedDeviceEntry(deviceID, new Binary[0]); + int deviceEntryIndex = deviceEntries.size(); + deviceEntries.add(deviceEntry); + DeviceTask deviceTask = + new DeviceTask( + deviceEntryIndex, new ArrayList<>(deviceCollector.getCurrentDeviceOffsets())); + DeviceTaskPartition partition = + partitions.get(Math.floorMod(deviceID.hashCode(), partitions.size())); + partition.add(deviceTask); + if (partition.getEstimatedSizeInBytes() >= DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES) { + partition.flush(comparator); + } + } + deviceEntryComparator = comparator; + collectDeviceTaskPartitions(partitions, comparator); + } + } + + public synchronized MultiWayMergeReader getMultiWayMergeReader(int partitionIndex) { + checkNotClosed(); + DeviceTaskPartition partition = getDeviceTaskPartition(partitionIndex); + try { + return new DeviceTaskRunReader(partition.getRunFiles(), deviceEntries, deviceEntryComparator); + } catch (IOException e) { + throw new RuntimeException("Failed to create external TsFile device task run reader", e); + } + } + + public List getTsFilePaths() { + return tsFilePaths; + } + + public List getDeviceEntries() { + return deviceEntries; + } + + public List getDeviceTaskPartitions() { + return deviceTaskPartitions; + } + + private DeviceTaskPartition getDeviceTaskPartition(int partitionIndex) { + for (DeviceTaskPartition partition : deviceTaskPartitions) { + if (partition.getPartitionIndex() == partitionIndex) { + return partition; + } + } + throw new IllegalArgumentException( + "Unknown external TsFile device task partition: " + partitionIndex); + } + + @Override + public synchronized void close() { + if (closed) { + return; + } + closed = true; + + releaseFileReaderReferences(); + + if (Files.exists(queryTempRoot)) { + FileUtils.deleteFileOrDirectory(queryTempRoot.toFile(), true); + } + } + + private void retainFileReaderReferences() { + if (readersRetained) { + return; + } + for (String tsFilePath : tsFilePaths) { + FileReaderManager.getInstance().increaseExternalFileReaderReference(tsFilePath); + } + readersRetained = true; + } + + private void releaseFileReaderReferences() { + if (!readersRetained) { + return; + } + for (String tsFilePath : tsFilePaths) { + FileReaderManager.getInstance().decreaseExternalFileReaderReference(tsFilePath); + } + readersRetained = false; + } + + private void checkNotClosed() { + if (closed) { + throw new IllegalStateException("External TsFile query resource has been closed: " + queryId); + } + } + + public interface MultiWayMergeReader extends AutoCloseable { + boolean hasNextDevice() throws IOException; + + DeviceEntry nextDevice() throws IOException; + + List getCurrentDeviceOffsets(); + + @Override + void close() throws IOException; + } + + public class DeviceTaskPartition { + + private final int partitionIndex; + private final List pendingDeviceTasks = new ArrayList<>(); + private final List deviceEntryIndexes = new ArrayList<>(); + private final List runFiles = new ArrayList<>(); + private long estimatedSizeInBytes; + + private DeviceTaskPartition(int partitionIndex) { + this.partitionIndex = partitionIndex; + } + + public int getPartitionIndex() { + return partitionIndex; + } + + public List getDeviceEntryIndexes() { + return deviceEntryIndexes; + } + + private void add(DeviceTask deviceTask) { + pendingDeviceTasks.add(deviceTask); + estimatedSizeInBytes += estimateDeviceTaskSize(deviceTask); + } + + private void flush(Comparator comparator) { + if (pendingDeviceTasks.isEmpty()) { + return; + } + sortPendingDeviceTasks(comparator); + try { + runFiles.add( + writeDeviceTaskRun( + queryTempRoot.resolve("child-" + partitionIndex), + runFiles.size(), + pendingDeviceTasks)); + } catch (IOException e) { + throw new RuntimeException("Failed to flush external TsFile device task partition", e); + } + for (DeviceTask deviceTask : pendingDeviceTasks) { + deviceEntryIndexes.add(deviceTask.deviceEntryIndex); + } + pendingDeviceTasks.clear(); + estimatedSizeInBytes = 0; + } + + private void sortPendingDeviceTasks(Comparator comparator) { + if (comparator != null) { + pendingDeviceTasks.sort( + (left, right) -> + comparator.compare( + deviceEntries.get(left.deviceEntryIndex), + deviceEntries.get(right.deviceEntryIndex))); + } else { + pendingDeviceTasks.sort( + (left, right) -> + deviceEntries + .get(left.deviceEntryIndex) + .getDeviceID() + .compareTo(deviceEntries.get(right.deviceEntryIndex).getDeviceID())); + } + } + + private long getEstimatedSizeInBytes() { + return estimatedSizeInBytes; + } + + private boolean hasDeviceTasks() { + return !deviceEntryIndexes.isEmpty(); + } + + private void sortDeviceEntries(Comparator comparator) { + if (comparator != null) { + deviceEntryIndexes.sort( + (left, right) -> comparator.compare(deviceEntries.get(left), deviceEntries.get(right))); + } else { + deviceEntryIndexes.sort( + (left, right) -> + deviceEntries + .get(left) + .getDeviceID() + .compareTo(deviceEntries.get(right).getDeviceID())); + } + } + + private List getRunFiles() { + return runFiles; + } + } + + private List createDeviceTaskPartitions(int partitionCount) { + if (partitionCount <= 0) { + throw new IllegalArgumentException( + "External TsFile device task partition count must be positive"); + } + List partitions = new ArrayList<>(partitionCount); + for (int i = 0; i < partitionCount; i++) { + partitions.add(new DeviceTaskPartition(i)); + } + return partitions; + } + + private void collectDeviceTaskPartitions( + List partitions, Comparator comparator) { + if (partitions.isEmpty()) { + deviceTaskPartitions = Collections.emptyList(); + return; + } + List nonEmptyPartitions = new ArrayList<>(partitions.size()); + for (DeviceTaskPartition partition : partitions) { + partition.flush(comparator); + if (!partition.hasDeviceTasks()) { + continue; + } + partition.sortDeviceEntries(comparator); + nonEmptyPartitions.add(partition); + } + deviceTaskPartitions = nonEmptyPartitions; + } + + private Path writeDeviceTaskRun(Path runRoot, int runIndex, List deviceTasks) + throws IOException { + Files.createDirectories(runRoot); + Path runFile = runRoot.resolve("run-" + runIndex + ".bin"); + try (DataOutputStream outputStream = + new DataOutputStream(new BufferedOutputStream(Files.newOutputStream(runFile)))) { + ReadWriteIOUtils.write(deviceTasks.size(), outputStream); + for (DeviceTask deviceTask : deviceTasks) { + deviceTask.serialize(outputStream); + } + } + return runFile; + } + + private static long estimateDeviceTaskSize(DeviceTask deviceTask) { + long size = 64L; + for (DeviceOffset offset : deviceTask.deviceOffsets) { + size += + DEVICE_OFFSET_INSTANCE_SIZE_IN_BYTES + + (long) Long.BYTES * offset.measurementNodeOffset.length; + } + return size; + } + + private static class DeviceTaskRunReader implements MultiWayMergeReader { + + private final List deviceEntries; + private final PriorityQueue runCursors; + private DeviceTask nextDeviceTask; + private List currentDeviceOffsets = Collections.emptyList(); + + private DeviceTaskRunReader( + List runFiles, List deviceEntries, Comparator comparator) + throws IOException { + this.deviceEntries = deviceEntries; + Comparator cursorComparator = + (left, right) -> + comparator == null + ? left.getCurrentDeviceEntry() + .getDeviceID() + .compareTo(right.getCurrentDeviceEntry().getDeviceID()) + : comparator.compare(left.getCurrentDeviceEntry(), right.getCurrentDeviceEntry()); + this.runCursors = new PriorityQueue<>(cursorComparator); + for (Path runFile : runFiles) { + DeviceTaskRunCursor cursor = new DeviceTaskRunCursor(runFile, deviceEntries); + if (cursor.hasCurrentDeviceTask()) { + runCursors.add(cursor); + } else { + cursor.close(); + } + } + } + + @Override + public boolean hasNextDevice() throws IOException { + if (nextDeviceTask != null) { + return true; + } + nextDeviceTask = readNextDeviceTask(); + return nextDeviceTask != null; + } + + @Override + public DeviceEntry nextDevice() throws IOException { + if (!hasNextDevice()) { + throw new EOFException("No more external TsFile device task"); + } + DeviceTask deviceTask = nextDeviceTask; + nextDeviceTask = null; + currentDeviceOffsets = deviceTask.deviceOffsets; + return deviceEntries.get(deviceTask.deviceEntryIndex); + } + + @Override + public List getCurrentDeviceOffsets() { + return currentDeviceOffsets; + } + + @Override + public void close() throws IOException { + IOException exception = null; + while (!runCursors.isEmpty()) { + try { + runCursors.poll().close(); + } catch (IOException e) { + if (exception == null) { + exception = e; + } else { + exception.addSuppressed(e); + } + } + } + if (exception != null) { + throw exception; + } + } + + private DeviceTask readNextDeviceTask() throws IOException { + if (runCursors.isEmpty()) { + return null; + } + DeviceTaskRunCursor cursor = runCursors.poll(); + DeviceTask result = cursor.getCurrentDeviceTask(); + cursor.advance(); + if (cursor.hasCurrentDeviceTask()) { + runCursors.add(cursor); + } else { + cursor.close(); + } + return result; + } + } + + private static class DeviceTaskRunCursor implements Closeable { + + private final List deviceEntries; + private final DataInputStream inputStream; + private int remainingDeviceTasks; + private DeviceTask currentDeviceTask; + + private DeviceTaskRunCursor(Path runFile, List deviceEntries) throws IOException { + this.deviceEntries = deviceEntries; + this.inputStream = + new DataInputStream(new BufferedInputStream(Files.newInputStream(runFile))); + this.remainingDeviceTasks = ReadWriteIOUtils.readInt(inputStream); + advance(); + } + + private void advance() throws IOException { + if (remainingDeviceTasks <= 0) { + currentDeviceTask = null; + return; + } + remainingDeviceTasks--; + currentDeviceTask = DeviceTask.deserialize(inputStream); + } + + private boolean hasCurrentDeviceTask() { + return currentDeviceTask != null; + } + + private DeviceTask getCurrentDeviceTask() { + return currentDeviceTask; + } + + private DeviceEntry getCurrentDeviceEntry() { + return deviceEntries.get(currentDeviceTask.deviceEntryIndex); + } + + @Override + public void close() throws IOException { + inputStream.close(); + } + } + + private class DeviceCollector implements Closeable { + + private final Map deviceIteratorMap = new HashMap<>(); + + private IDeviceID currentDevice; + private List currentDeviceOffsets = Collections.emptyList(); + + private DeviceCollector() { + try { + for (int fileIndex = 0; fileIndex < tsFilePaths.size(); fileIndex++) { + TsFileSequenceReader reader = + FileReaderManager.getInstance() + .get(tsFilePaths.get(fileIndex), null, true, ioSizeRecorder, true); + deviceIteratorMap.put( + fileIndex, new LazyTsFileDeviceIterator(reader, tableName, ioSizeRecorder)); + } + } catch (IOException e) { + close(); + throw new RuntimeException("Failed to create external TsFile device collector", e); + } + } + + private boolean hasNextDevice() { + for (LazyTsFileDeviceIterator deviceIterator : deviceIteratorMap.values()) { + if (deviceIterator.hasNext() + || (deviceIterator.hasCurrent() + && !deviceIterator.getCurrentDeviceID().equals(currentDevice))) { + return true; + } + } + return false; + } + + private IDeviceID nextDevice() { + IDeviceID minDevice = null; + Iterator> iterator = + deviceIteratorMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + LazyTsFileDeviceIterator deviceIterator = entry.getValue(); + IDeviceID currentFileDevice = null; + if (deviceIterator.hasCurrent()) { + currentFileDevice = deviceIterator.getCurrentDeviceID(); + } + if (currentFileDevice == null || currentFileDevice.equals(currentDevice)) { + if (deviceIterator.hasNext()) { + currentFileDevice = deviceIterator.next(); + } else { + iterator.remove(); + continue; + } + } + if (minDevice == null || minDevice.compareTo(currentFileDevice) > 0) { + minDevice = currentFileDevice; + } + } + currentDevice = minDevice; + collectCurrentDeviceOffsets(); + return currentDevice; + } + + private void collectCurrentDeviceOffsets() { + List deviceOffsets = new ArrayList<>(); + for (Map.Entry entry : deviceIteratorMap.entrySet()) { + LazyTsFileDeviceIterator deviceIterator = entry.getValue(); + if (currentDevice != null + && deviceIterator.hasCurrent() + && currentDevice.equals(deviceIterator.getCurrentDeviceID())) { + deviceOffsets.add( + new DeviceOffset( + entry.getKey(), deviceIterator.getCurrentDeviceMeasurementNodeOffset())); + } + } + currentDeviceOffsets = deviceOffsets; + } + + private List getCurrentDeviceOffsets() { + return currentDeviceOffsets; + } + + @Override + public void close() { + deviceIteratorMap.clear(); + currentDeviceOffsets = Collections.emptyList(); + } + } + + private static class DeviceTask { + + private final int deviceEntryIndex; + private final List deviceOffsets; + + private DeviceTask(int deviceEntryIndex, List deviceOffsets) { + this.deviceEntryIndex = deviceEntryIndex; + this.deviceOffsets = deviceOffsets; + } + + private void serialize(DataOutputStream outputStream) throws IOException { + ReadWriteIOUtils.write(deviceEntryIndex, outputStream); + ReadWriteIOUtils.write(deviceOffsets.size(), outputStream); + for (DeviceOffset offset : deviceOffsets) { + ReadWriteIOUtils.write(offset.fileIndex, outputStream); + ReadWriteIOUtils.write(offset.measurementNodeOffset.length, outputStream); + for (long measurementNodeOffset : offset.measurementNodeOffset) { + ReadWriteIOUtils.write(measurementNodeOffset, outputStream); + } + } + } + + private static DeviceTask deserialize(DataInputStream inputStream) throws IOException { + int deviceEntryIndex = ReadWriteIOUtils.readInt(inputStream); + int offsetSize = ReadWriteIOUtils.readInt(inputStream); + List offsets = new ArrayList<>(offsetSize); + for (int i = 0; i < offsetSize; i++) { + int fileIndex = ReadWriteIOUtils.readInt(inputStream); + int measurementNodeOffsetLength = ReadWriteIOUtils.readInt(inputStream); + long[] measurementNodeOffset = new long[measurementNodeOffsetLength]; + for (int j = 0; j < measurementNodeOffsetLength; j++) { + measurementNodeOffset[j] = inputStream.readLong(); + } + offsets.add(new DeviceOffset(fileIndex, measurementNodeOffset)); + } + return new DeviceTask(deviceEntryIndex, offsets); + } + } + + public static class DeviceOffset { + + private final int fileIndex; + private final long[] measurementNodeOffset; + + private DeviceOffset(int fileIndex, long[] measurementNodeOffset) { + this.fileIndex = fileIndex; + this.measurementNodeOffset = measurementNodeOffset; + } + + public int getFileIndex() { + return fileIndex; + } + + public long[] getMeasurementNodeOffset() { + return measurementNodeOffset; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ReadTsFileTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java similarity index 95% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ReadTsFileTableFunction.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java index 71ed1b3670fc3..26f7143595db5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/ReadTsFileTableFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.queryengine.plan.relational.function.tvf; +package org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; @@ -118,7 +118,7 @@ public TableFunctionHandle createTableFunctionHandle() { public TableFunctionProcessorProvider getProcessorProvider( TableFunctionHandle tableFunctionHandle) { throw new UnsupportedOperationException( - "read_tsfile must be planned as an ExternalTsFileScanNode"); + "readTsFile must be planned as an ExternalTsFileScanNode"); } private static String getRequiredStringArgument(Map arguments, String name) { @@ -172,7 +172,7 @@ private static void checkTsFilePathsAreOutsideDataDirs(List tsFilePaths) if (normalizedTsFilePath.startsWith(dataDir) || dataDir.startsWith(normalizedTsFilePath)) { throw new UDFArgumentNotValidException( String.format( - "read_tsfile path %s is not allowed because it may access IoTDB data directory %s", + "readTsFile path %s is not allowed because it may access IoTDB data directory %s", tsFilePath, dataDir)); } } @@ -402,30 +402,6 @@ private static DescribedSchema convertToDescribedSchema(TableSchema tableSchema) return builder.build(); } - public static class ExternalTsFileDeviceOffset { - - private final String tsFilePath; - private final long[] deviceMeasurementNodeOffset; - - public ExternalTsFileDeviceOffset(String tsFilePath, long[] deviceMeasurementNodeOffset) { - this.tsFilePath = tsFilePath; - this.deviceMeasurementNodeOffset = - deviceMeasurementNodeOffset == null - ? null - : Arrays.copyOf(deviceMeasurementNodeOffset, deviceMeasurementNodeOffset.length); - } - - public String getTsFilePath() { - return tsFilePath; - } - - public long[] getDeviceMeasurementNodeOffset() { - return deviceMeasurementNodeOffset == null - ? null - : Arrays.copyOf(deviceMeasurementNodeOffset, deviceMeasurementNodeOffset.length); - } - } - public static class ReadTsFileTableFunctionHandle implements TableFunctionHandle { private String tableName; private List tsFilePaths; 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 5855b15922f2b..8fdf2c1aa1b4c 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 @@ -115,7 +115,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableArgumentAnalysis; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableFunctionInvocationAnalysis; import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; -import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.ReadTsFileTableFunction; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ReadTsFileTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TreeDeviceViewSchema; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.IrUtils; @@ -1596,7 +1596,7 @@ private RelationPlan planExternalTsFileScan( TableFunctionInvocation node, TableFunctionInvocationAnalysis functionAnalysis) { if (!(functionAnalysis.getTableFunctionHandle() instanceof ReadTsFileTableFunction.ReadTsFileTableFunctionHandle)) { - throw new IllegalStateException("read_tsfile table function handle is invalid"); + throw new IllegalStateException("readTsFile table function handle is invalid"); } ReadTsFileTableFunction.ReadTsFileTableFunctionHandle handle = @@ -1607,17 +1607,20 @@ private RelationPlan planExternalTsFileScan( ImmutableList.Builder outputSymbolsBuilder = ImmutableList.builder(); ImmutableMap.Builder assignmentsBuilder = ImmutableMap.builder(); + Map tagAndAttributeIndexMap = new HashMap<>(); + int tagIndex = 0; for (int i = 0; i < relationType.getAllFieldCount(); i++) { Field field = relationType.getFieldByIndex(i); Symbol symbol = symbolAllocator.newSymbol(field); + TsTableColumnCategory columnCategory = handle.getOutputColumnCategories().get(i); outputSymbolsBuilder.add(symbol); assignmentsBuilder.put( symbol, new ColumnSchema( - field.getName().orElse(null), - field.getType(), - field.isHidden(), - handle.getOutputColumnCategories().get(i))); + field.getName().orElse(null), field.getType(), field.isHidden(), columnCategory)); + if (columnCategory == TsTableColumnCategory.TAG) { + tagAndAttributeIndexMap.put(symbol, tagIndex++); + } } List outputSymbols = outputSymbolsBuilder.build(); @@ -1626,16 +1629,17 @@ private RelationPlan planExternalTsFileScan( createExternalTsFileQualifiedObjectName(handle.getTableName()); analysis.addTableSchema(qualifiedObjectName, assignments); - return new RelationPlan( + ExternalTsFileScanNode scanNode = new ExternalTsFileScanNode( idAllocator.genPlanNodeId(), qualifiedObjectName, outputSymbols, assignments, - handle.getTsFilePaths()), - scope, - outputSymbols, - outerContext); + tagAndAttributeIndexMap, + queryContext.createExternalTsFileQueryResource( + handle.getTableName(), handle.getTsFilePaths())); + + return new RelationPlan(scanNode, scope, outputSymbols, outerContext); } private QualifiedObjectName createExternalTsFileQualifiedObjectName(String tableName) { 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 105dc1c1935d2..0b54b2395a7b1 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 @@ -73,7 +73,6 @@ import org.apache.iotdb.commons.schema.table.InformationSchema; import org.apache.iotdb.commons.schema.table.TsTable; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -88,6 +87,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.relational.analyzer.Analysis; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceTaskPartition; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolAllocator; @@ -737,46 +737,22 @@ public List visitExternalTsFileScan( new TRegionReplicaSet(null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())); node.setRegionReplicaSet(localRegionReplicaSet); context.mostUsedRegion = node.getRegionReplicaSet(); - List resultNodes = - splitExternalTsFileScanByDeviceEntries(node, localRegionReplicaSet); - if (context.hasSortProperty) { - processSortProperty(node, resultNodes, context); - } - return resultNodes; - } - - private List splitExternalTsFileScanByDeviceEntries( - final ExternalTsFileScanNode node, final TRegionReplicaSet localRegionReplicaSet) { - List deviceEntries = node.getDeviceEntries(); - if (deviceEntries.size() <= 1) { - return Collections.singletonList(node); - } - - int splitCount = - Math.min( - deviceEntries.size(), - IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism()); - if (splitCount <= 1) { + Optional sortPropertyContext = + context.hasSortProperty ? analyzeSortProperty(node, context) : Optional.empty(); + int partitionCount = IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism(); + node.getExternalTsFileQueryResource() + .collectDeviceEntries( + node.getSchemaFilter(), + sortPropertyContext.map(propertyContext -> propertyContext.comparator).orElse(null), + partitionCount); + + List partitions = + node.getExternalTsFileQueryResource().getDeviceTaskPartitions(); + if (partitions.isEmpty()) { return Collections.singletonList(node); } - - List> splitDeviceEntries = new ArrayList<>(splitCount); - List>> splitDeviceOffsets = new ArrayList<>(splitCount); - for (int i = 0; i < splitCount; i++) { - splitDeviceEntries.add(new ArrayList<>()); - splitDeviceOffsets.add(new ArrayList<>()); - } - for (int i = 0; i < deviceEntries.size(); i++) { - splitDeviceEntries.get(i % splitCount).add(deviceEntries.get(i)); - splitDeviceOffsets.get(i % splitCount).add(node.getDeviceOffsets().get(i)); - } - - List result = new ArrayList<>(splitCount); - for (int i = 0; i < splitDeviceEntries.size(); i++) { - List entries = splitDeviceEntries.get(i); - if (entries.isEmpty()) { - continue; - } + List result = new ArrayList<>(partitions.size()); + for (DeviceTaskPartition partition : partitions) { ExternalTsFileScanNode splitNode = new ExternalTsFileScanNode( queryId.genPlanNodeId(), @@ -789,12 +765,16 @@ private List splitExternalTsFileScanByDeviceEntries( node.getTimePredicate().orElse(null), node.getScanOrder(), node.isPushLimitToEachDevice(), - node.getTsFilePaths(), - entries, - splitDeviceOffsets.get(i)); + node.getTagAndAttributeIndexMap(), + node.getExternalTsFileQueryResource(), + partition.getDeviceEntryIndexes(), + partition.getPartitionIndex(), + node.getSchemaFilter()); splitNode.setRegionReplicaSet(localRegionReplicaSet); result.add(splitNode); } + sortPropertyContext.ifPresent( + propertyContext -> applySortProperty(node, result, propertyContext, false)); return result; } @@ -805,53 +785,28 @@ public List visitExternalTsFileAggregationScan( new TRegionReplicaSet(null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())); node.setRegionReplicaSet(localRegionReplicaSet); context.mostUsedRegion = node.getRegionReplicaSet(); - List resultNodes = - splitExternalTsFileAggregationScanByDeviceEntries(node, localRegionReplicaSet); - if (context.hasSortProperty) { - processSortProperty(node, resultNodes, context); - } - return resultNodes; - } - - private List splitExternalTsFileAggregationScanByDeviceEntries( - final ExternalTsFileAggregationScanNode node, final TRegionReplicaSet localRegionReplicaSet) { - List deviceEntries = node.getDeviceEntries(); - if (deviceEntries.size() <= 1) { - return Collections.singletonList(node); - } - - int splitCount = - Math.min( - deviceEntries.size(), - IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism()); - if (splitCount <= 1) { + Optional sortPropertyContext = + context.hasSortProperty ? analyzeSortProperty(node, context) : Optional.empty(); + int partitionCount = IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism(); + node.getExternalTsFileQueryResource() + .collectDeviceEntries( + node.getSchemaFilter(), + sortPropertyContext.map(propertyContext -> propertyContext.comparator).orElse(null), + partitionCount); + + List partitions = + node.getExternalTsFileQueryResource().getDeviceTaskPartitions(); + if (partitions.isEmpty()) { return Collections.singletonList(node); } - - List> splitDeviceEntries = new ArrayList<>(splitCount); - List>> splitDeviceOffsets = new ArrayList<>(splitCount); - for (int i = 0; i < splitCount; i++) { - splitDeviceEntries.add(new ArrayList<>()); - splitDeviceOffsets.add(new ArrayList<>()); - } - for (int i = 0; i < deviceEntries.size(); i++) { - splitDeviceEntries.get(i % splitCount).add(deviceEntries.get(i)); - splitDeviceOffsets.get(i % splitCount).add(node.getDeviceOffsets().get(i)); - } - - List result = new ArrayList<>(splitCount); - for (int i = 0; i < splitDeviceEntries.size(); i++) { - List entries = splitDeviceEntries.get(i); - if (entries.isEmpty()) { - continue; - } + List result = new ArrayList<>(partitions.size()); + for (DeviceTaskPartition partition : partitions) { ExternalTsFileAggregationScanNode splitNode = new ExternalTsFileAggregationScanNode( queryId.genPlanNodeId(), node.getQualifiedObjectName(), node.getOutputSymbols(), node.getAssignments(), - entries, node.getTagAndAttributeIndexMap(), node.getScanOrder(), node.getTimePredicate().orElse(null), @@ -866,11 +821,15 @@ private List splitExternalTsFileAggregationScanByDeviceEntries( node.getPreGroupedSymbols(), node.getStep(), node.getGroupIdSymbol(), - node.getTsFilePaths(), - splitDeviceOffsets.get(i)); + node.getExternalTsFileQueryResource(), + partition.getDeviceEntryIndexes(), + partition.getPartitionIndex(), + node.getSchemaFilter()); splitNode.setRegionReplicaSet(localRegionReplicaSet); result.add(splitNode); } + sortPropertyContext.ifPresent( + propertyContext -> applySortProperty(node, result, propertyContext, false)); return result; } @@ -1949,18 +1908,26 @@ private void processSortProperty( final DeviceTableScanNode deviceTableScanNode, final List resultTableScanNodeList, final PlanContext context) { + Optional sortPropertyContext = + analyzeSortProperty(deviceTableScanNode, context); + if (!sortPropertyContext.isPresent()) { + return; + } + applySortProperty( + deviceTableScanNode, resultTableScanNodeList, sortPropertyContext.get(), true); + } + + private Optional analyzeSortProperty( + DeviceTableScanNode deviceTableScanNode, PlanContext context) { final List newOrderingSymbols = new ArrayList<>(); final List newSortOrders = new ArrayList<>(); final OrderingScheme expectedOrderingScheme = context.expectedOrderingScheme; boolean lastIsTimeRelated = false; + boolean scanOrderDesc = false; for (final Symbol symbol : expectedOrderingScheme.getOrderBy()) { if (timeRelatedSymbol(symbol, deviceTableScanNode)) { - if (!expectedOrderingScheme.getOrderings().get(symbol).isAscending()) { - // TODO(beyyes) move scan order judgement into logical plan optimizer - resultTableScanNodeList.forEach( - node -> ((DeviceTableScanNode) node).setScanOrder(Ordering.DESC)); - } + scanOrderDesc = !expectedOrderingScheme.getOrderings().get(symbol).isAscending(); newOrderingSymbols.add(symbol); newSortOrders.add(expectedOrderingScheme.getOrdering(symbol)); lastIsTimeRelated = true; @@ -1975,9 +1942,22 @@ private void processSortProperty( // no sort property can be pushed down into DeviceTableScanNode if (newOrderingSymbols.isEmpty()) { - return; + return Optional.empty(); } + return Optional.of( + new SortPropertyContext( + newOrderingSymbols, + newSortOrders, + createDeviceEntryComparator(deviceTableScanNode, newOrderingSymbols, newSortOrders), + lastIsTimeRelated, + scanOrderDesc)); + } + + private Comparator createDeviceEntryComparator( + DeviceTableScanNode deviceTableScanNode, + List newOrderingSymbols, + List newSortOrders) { Optional extractor = createTreeDeviceIdColumnValueExtractor(deviceTableScanNode); final List> orderingRules = new ArrayList<>(); @@ -2052,21 +2032,34 @@ private void processSortProperty( comparator = comparator.thenComparing(thenComparator); } } + return comparator; + } + private void applySortProperty( + final DeviceTableScanNode deviceTableScanNode, + final List resultTableScanNodeList, + final SortPropertyContext sortPropertyContext, + final boolean sortDeviceEntries) { final Optional newOrderingScheme = tableScanOrderingSchema( analysis.getTableColumnSchema(deviceTableScanNode.getQualifiedObjectName()), deviceTableScanNode.getAssignments(), - newOrderingSymbols, - newSortOrders, - lastIsTimeRelated, - deviceTableScanNode.getDeviceEntries().size() == 1); + sortPropertyContext.orderingSymbols, + sortPropertyContext.sortOrders, + sortPropertyContext.lastIsTimeRelated, + resultTableScanNodeList.size() == 1 + && ((DeviceTableScanNode) resultTableScanNodeList.get(0)).getDeviceEntries().size() + == 1); for (final PlanNode planNode : resultTableScanNodeList) { final DeviceTableScanNode scanNode = (DeviceTableScanNode) planNode; + if (sortPropertyContext.scanOrderDesc) { + // TODO(beyyes) move scan order judgement into logical plan optimizer + scanNode.setScanOrder(Ordering.DESC); + } newOrderingScheme.ifPresent( orderingScheme -> nodeOrderingMap.put(scanNode.getPlanNodeId(), orderingScheme)); - if (comparator != null) { - scanNode.sortDeviceEntries(comparator); + if (sortDeviceEntries && sortPropertyContext.comparator != null) { + scanNode.getDeviceEntries().sort(sortPropertyContext.comparator); } } } @@ -2386,6 +2379,27 @@ public List visitUnion(UnionNode node, PlanContext context) { node.getOutputSymbols())); } + private static class SortPropertyContext { + final List orderingSymbols; + final List sortOrders; + final Comparator comparator; + final boolean lastIsTimeRelated; + final boolean scanOrderDesc; + + private SortPropertyContext( + List orderingSymbols, + List sortOrders, + Comparator comparator, + boolean lastIsTimeRelated, + boolean scanOrderDesc) { + this.orderingSymbols = orderingSymbols; + this.sortOrders = sortOrders; + this.comparator = comparator; + this.lastIsTimeRelated = lastIsTimeRelated; + this.scanOrderDesc = scanOrderDesc; + } + } + public static class PlanContext { final Map nodeDistributionMap; boolean hasExchangeNode = false; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java index a8a36c6328e9d..95a833074c1cc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java @@ -103,9 +103,12 @@ public static Optional pruneColumns(TableScanNode node, Set re externalTsFileScanNode.getPushDownOffset(), externalTsFileScanNode.getTimePredicate().orElse(null), externalTsFileScanNode.getScanOrder(), - externalTsFileScanNode.getTsFilePaths(), - externalTsFileScanNode.getDeviceEntries(), - externalTsFileScanNode.getDeviceOffsets()); + externalTsFileScanNode.isPushLimitToEachDevice(), + externalTsFileScanNode.getTagAndAttributeIndexMap(), + externalTsFileScanNode.getExternalTsFileQueryResource(), + externalTsFileScanNode.getDeviceEntryIndexes(), + externalTsFileScanNode.getDeviceTaskPartitionIndex(), + externalTsFileScanNode.getSchemaFilter()); prunedNode.setRegionReplicaSet(externalTsFileScanNode.getRegionReplicaSet()); return Optional.of(prunedNode); } else if (node instanceof DeviceTableScanNode) { 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 19057bc71461e..9402e837575fd 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 @@ -297,28 +297,31 @@ public static AggregationTableScanNode combineAggregationAndTableScan( DeviceTableScanNode tableScanNode) { if (tableScanNode instanceof ExternalTsFileScanNode) { ExternalTsFileScanNode externalTsFileScanNode = (ExternalTsFileScanNode) tableScanNode; - return new ExternalTsFileAggregationScanNode( - id, - tableScanNode.getQualifiedObjectName(), - tableScanNode.getOutputSymbols(), - tableScanNode.getAssignments(), - tableScanNode.getDeviceEntries(), - tableScanNode.getTagAndAttributeIndexMap(), - tableScanNode.getScanOrder(), - tableScanNode.getTimePredicate().orElse(null), - tableScanNode.getPushDownPredicate(), - tableScanNode.getPushDownLimit(), - tableScanNode.getPushDownOffset(), - tableScanNode.isPushLimitToEachDevice(), - tableScanNode.containsNonAlignedDevice(), - projectNode == null ? null : projectNode.getAssignments(), - aggregationNode.getAggregations(), - aggregationNode.getGroupingSets(), - aggregationNode.getPreGroupedSymbols(), - aggregationNode.getStep(), - aggregationNode.getGroupIdSymbol(), - externalTsFileScanNode.getTsFilePaths(), - externalTsFileScanNode.getDeviceOffsets()); + ExternalTsFileAggregationScanNode scanNode = + new ExternalTsFileAggregationScanNode( + id, + tableScanNode.getQualifiedObjectName(), + tableScanNode.getOutputSymbols(), + tableScanNode.getAssignments(), + tableScanNode.getTagAndAttributeIndexMap(), + tableScanNode.getScanOrder(), + tableScanNode.getTimePredicate().orElse(null), + tableScanNode.getPushDownPredicate(), + tableScanNode.getPushDownLimit(), + tableScanNode.getPushDownOffset(), + tableScanNode.isPushLimitToEachDevice(), + tableScanNode.containsNonAlignedDevice(), + projectNode == null ? null : projectNode.getAssignments(), + aggregationNode.getAggregations(), + aggregationNode.getGroupingSets(), + aggregationNode.getPreGroupedSymbols(), + aggregationNode.getStep(), + aggregationNode.getGroupIdSymbol(), + externalTsFileScanNode.getExternalTsFileQueryResource(), + externalTsFileScanNode.getDeviceEntryIndexes(), + externalTsFileScanNode.getDeviceTaskPartitionIndex(), + externalTsFileScanNode.getSchemaFilter()); + return scanNode; } if (tableScanNode instanceof TreeDeviceViewScanNode) { TreeDeviceViewScanNode treeDeviceViewScanNode = (TreeDeviceViewScanNode) tableScanNode; @@ -381,7 +384,6 @@ public static AggregationTableScanNode combineAggregationAndTableScan( tableScanNode.getQualifiedObjectName(), tableScanNode.getOutputSymbols(), tableScanNode.getAssignments(), - tableScanNode.getDeviceEntries(), tableScanNode.getTagAndAttributeIndexMap(), tableScanNode.getScanOrder(), tableScanNode.getTimePredicate().orElse(null), @@ -396,8 +398,10 @@ public static AggregationTableScanNode combineAggregationAndTableScan( aggregationNode.getPreGroupedSymbols(), step, aggregationNode.getGroupIdSymbol(), - externalTsFileScanNode.getTsFilePaths(), - externalTsFileScanNode.getDeviceOffsets()); + externalTsFileScanNode.getExternalTsFileQueryResource(), + externalTsFileScanNode.getDeviceEntryIndexes(), + externalTsFileScanNode.getDeviceTaskPartitionIndex(), + externalTsFileScanNode.getSchemaFilter()); } if (tableScanNode instanceof TreeDeviceViewScanNode) { TreeDeviceViewScanNode treeDeviceViewScanNode = (TreeDeviceViewScanNode) tableScanNode; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/DeviceTableScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/DeviceTableScanNode.java index e4fceb9818b9c..c02c74058afee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/DeviceTableScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/DeviceTableScanNode.java @@ -41,7 +41,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -276,10 +275,6 @@ public void appendDeviceEntry(DeviceEntry deviceEntry) { this.deviceEntries.add(deviceEntry); } - public void sortDeviceEntries(Comparator comparator) { - this.deviceEntries.sort(comparator); - } - public void setPushLimitToEachDevice(boolean pushLimitToEachDevice) { this.pushLimitToEachDevice = pushLimitToEachDevice; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java index 04434ad498ef8..dd338dc2d90dd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java @@ -27,32 +27,32 @@ import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.commons.queryengine.plan.relational.planner.node.AggregationNode; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; +import com.google.common.collect.Lists; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.stream.IntStream; public class ExternalTsFileAggregationScanNode extends AggregationTableScanNode { - private List tsFilePaths; - private List> deviceOffsets = Collections.emptyList(); + private final ExternalTsFileQueryResource externalTsFileQueryResource; + private List deviceEntryIndexes; + private int deviceTaskPartitionIndex = -1; + private SchemaFilter schemaFilter; public ExternalTsFileAggregationScanNode( PlanNodeId id, QualifiedObjectName qualifiedObjectName, List outputSymbols, Map assignments, - List deviceEntries, Map tagAndAttributeIndexMap, Ordering scanOrder, Expression timePredicate, @@ -67,14 +67,16 @@ public ExternalTsFileAggregationScanNode( List preGroupedSymbols, AggregationNode.Step step, Optional groupIdSymbol, - List tsFilePaths, - List> deviceOffsets) { + ExternalTsFileQueryResource externalTsFileQueryResource, + List deviceEntryIndexes, + int deviceTaskPartitionIndex, + SchemaFilter schemaFilter) { super( id, qualifiedObjectName, outputSymbols, assignments, - deviceEntries, + Lists.transform(deviceEntryIndexes, externalTsFileQueryResource.getDeviceEntries()::get), tagAndAttributeIndexMap, scanOrder, timePredicate, @@ -89,12 +91,12 @@ public ExternalTsFileAggregationScanNode( preGroupedSymbols, step, groupIdSymbol); - this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); - this.deviceOffsets = copyDeviceOffsets(deviceOffsets); + this.externalTsFileQueryResource = externalTsFileQueryResource; + this.deviceEntryIndexes = deviceEntryIndexes; + this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; + this.schemaFilter = schemaFilter; } - protected ExternalTsFileAggregationScanNode() {} - @Override public R accept(IPlanVisitor visitor, C context) { return ((PlanVisitor) visitor).visitExternalTsFileAggregationScan(this, context); @@ -107,7 +109,6 @@ public ExternalTsFileAggregationScanNode clone() { qualifiedObjectName, outputSymbols, assignments, - deviceEntries, tagAndAttributeIndexMap, scanOrder, timePredicate, @@ -122,47 +123,40 @@ public ExternalTsFileAggregationScanNode clone() { preGroupedSymbols, step, groupIdSymbol, - tsFilePaths, - deviceOffsets); + externalTsFileQueryResource, + deviceEntryIndexes, + deviceTaskPartitionIndex, + schemaFilter); } public List getTsFilePaths() { - return tsFilePaths; + return externalTsFileQueryResource.getTsFilePaths(); + } + + public ExternalTsFileQueryResource getExternalTsFileQueryResource() { + return externalTsFileQueryResource; + } + + public List getDeviceEntryIndexes() { + return deviceEntryIndexes; } - public List> getDeviceOffsets() { - return deviceOffsets; + public int getDeviceTaskPartitionIndex() { + return deviceTaskPartitionIndex; } @Override - public void sortDeviceEntries(Comparator comparator) { - int[] indexes = - IntStream.range(0, deviceEntries.size()) - .boxed() - .sorted( - (left, right) -> - comparator.compare(deviceEntries.get(left), deviceEntries.get(right))) - .mapToInt(Integer::intValue) - .toArray(); - List sortedDeviceEntries = new ArrayList<>(deviceEntries.size()); - List> sortedDeviceOffsets = - new ArrayList<>(deviceOffsets.size()); - for (int index : indexes) { - sortedDeviceEntries.add(deviceEntries.get(index)); - sortedDeviceOffsets.add(deviceOffsets.get(index)); - } - this.deviceEntries = sortedDeviceEntries; - this.deviceOffsets = sortedDeviceOffsets; + public void setDeviceEntries(List deviceEntries) { + throw new UnsupportedOperationException( + "ExternalTsFileAggregationScanNode device entries must be set by device entry indexes"); + } + + public SchemaFilter getSchemaFilter() { + return schemaFilter; } - private static List> copyDeviceOffsets( - List> deviceOffsets) { - List> copiedDeviceOffsets = - new ArrayList<>(deviceOffsets.size()); - for (List offsets : deviceOffsets) { - copiedDeviceOffsets.add(new ArrayList<>(offsets)); - } - return copiedDeviceOffsets; + public void setSchemaFilter(SchemaFilter schemaFilter) { + this.schemaFilter = schemaFilter; } @Override @@ -177,11 +171,6 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { "ExternalTsFileAggregationScanNode cannot be serialized because it reads local external TsFiles"); } - public static ExternalTsFileAggregationScanNode deserialize(ByteBuffer byteBuffer) { - throw new UnsupportedOperationException( - "ExternalTsFileAggregationScanNode cannot be deserialized because it reads local external TsFiles"); - } - @Override public String toString() { return "ExternalTsFileAggregationScanNode-" + this.getPlanNodeId(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java index f21a7a755c0bb..be83256c404b7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -25,26 +25,26 @@ import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; +import com.google.common.collect.Lists; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.stream.IntStream; - -import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TAG; public class ExternalTsFileScanNode extends DeviceTableScanNode { - private List tsFilePaths; - private List> deviceOffsets = Collections.emptyList(); + private ExternalTsFileQueryResource externalTsFileQueryResource; + private List deviceEntryIndexes = Collections.emptyList(); + private int deviceTaskPartitionIndex = -1; + private SchemaFilter schemaFilter; protected ExternalTsFileScanNode() {} @@ -53,133 +53,10 @@ public ExternalTsFileScanNode( QualifiedObjectName qualifiedObjectName, List outputSymbols, Map assignments, - List tsFilePaths) { - this( - id, - qualifiedObjectName, - outputSymbols, - assignments, - tsFilePaths, - Collections.emptyList(), - Collections.emptyList()); - } - - public ExternalTsFileScanNode( - PlanNodeId id, - QualifiedObjectName qualifiedObjectName, - List outputSymbols, - Map assignments, - List tsFilePaths, - List deviceEntries, - List> deviceOffsets) { - super(id, qualifiedObjectName, outputSymbols, assignments, buildTagIndexMap(assignments)); - this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); - this.deviceEntries = new ArrayList<>(deviceEntries); - this.deviceOffsets = copyDeviceOffsets(deviceOffsets); - } - - public ExternalTsFileScanNode( - PlanNodeId id, - QualifiedObjectName qualifiedObjectName, - List outputSymbols, - Map assignments, - Expression pushDownPredicate, - long pushDownLimit, - long pushDownOffset, - Ordering scanOrder, - List tsFilePaths) { - this( - id, - qualifiedObjectName, - outputSymbols, - assignments, - pushDownPredicate, - pushDownLimit, - pushDownOffset, - scanOrder, - tsFilePaths, - Collections.emptyList()); - } - - public ExternalTsFileScanNode( - PlanNodeId id, - QualifiedObjectName qualifiedObjectName, - List outputSymbols, - Map assignments, - Expression pushDownPredicate, - long pushDownLimit, - long pushDownOffset, - Ordering scanOrder, - List tsFilePaths, - List deviceEntries) { - this( - id, - qualifiedObjectName, - outputSymbols, - assignments, - pushDownPredicate, - pushDownLimit, - pushDownOffset, - null, - scanOrder, - tsFilePaths, - deviceEntries, - Collections.emptyList()); - } - - public ExternalTsFileScanNode( - PlanNodeId id, - QualifiedObjectName qualifiedObjectName, - List outputSymbols, - Map assignments, - Expression pushDownPredicate, - long pushDownLimit, - long pushDownOffset, - Expression timePredicate, - Ordering scanOrder, - List tsFilePaths) { - this( - id, - qualifiedObjectName, - outputSymbols, - assignments, - pushDownPredicate, - pushDownLimit, - pushDownOffset, - timePredicate, - scanOrder, - tsFilePaths, - Collections.emptyList(), - Collections.emptyList()); - } - - public ExternalTsFileScanNode( - PlanNodeId id, - QualifiedObjectName qualifiedObjectName, - List outputSymbols, - Map assignments, - Expression pushDownPredicate, - long pushDownLimit, - long pushDownOffset, - Expression timePredicate, - Ordering scanOrder, - List tsFilePaths, - List deviceEntries, - List> deviceOffsets) { - this( - id, - qualifiedObjectName, - outputSymbols, - assignments, - pushDownPredicate, - pushDownLimit, - pushDownOffset, - timePredicate, - scanOrder, - false, - tsFilePaths, - deviceEntries, - deviceOffsets); + Map tagAndAttributeIndexMap, + ExternalTsFileQueryResource externalTsFileQueryResource) { + super(id, qualifiedObjectName, outputSymbols, assignments, tagAndAttributeIndexMap); + this.externalTsFileQueryResource = externalTsFileQueryResource; } public ExternalTsFileScanNode( @@ -193,16 +70,18 @@ public ExternalTsFileScanNode( Expression timePredicate, Ordering scanOrder, boolean pushLimitToEachDevice, - List tsFilePaths, - List deviceEntries, - List> deviceOffsets) { + Map tagAndAttributeIndexMap, + ExternalTsFileQueryResource externalTsFileQueryResource, + List deviceEntryIndexes, + int deviceTaskPartitionIndex, + SchemaFilter schemaFilter) { super( id, qualifiedObjectName, outputSymbols, assignments, - new ArrayList<>(deviceEntries), - buildTagIndexMap(assignments), + Lists.transform(deviceEntryIndexes, externalTsFileQueryResource.getDeviceEntries()::get), + tagAndAttributeIndexMap, scanOrder, timePredicate, pushDownPredicate, @@ -210,8 +89,10 @@ public ExternalTsFileScanNode( pushDownOffset, pushLimitToEachDevice, false); - this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); - this.deviceOffsets = copyDeviceOffsets(deviceOffsets); + this.externalTsFileQueryResource = externalTsFileQueryResource; + this.deviceEntryIndexes = deviceEntryIndexes; + this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; + this.schemaFilter = schemaFilter; } @Override @@ -232,52 +113,41 @@ public ExternalTsFileScanNode clone() { timePredicate, scanOrder, pushLimitToEachDevice, - tsFilePaths, - deviceEntries, - deviceOffsets); + tagAndAttributeIndexMap, + externalTsFileQueryResource, + deviceEntryIndexes, + deviceTaskPartitionIndex, + schemaFilter); } public List getTsFilePaths() { - return tsFilePaths; + return externalTsFileQueryResource.getTsFilePaths(); + } + + public ExternalTsFileQueryResource getExternalTsFileQueryResource() { + return externalTsFileQueryResource; } - public List> getDeviceOffsets() { - return deviceOffsets; + public List getDeviceEntryIndexes() { + return deviceEntryIndexes; } - public void setDeviceOffsets(List> deviceOffsets) { - this.deviceOffsets = copyDeviceOffsets(deviceOffsets); + public int getDeviceTaskPartitionIndex() { + return deviceTaskPartitionIndex; } @Override - public void sortDeviceEntries(Comparator comparator) { - int[] indexes = - IntStream.range(0, deviceEntries.size()) - .boxed() - .sorted( - (left, right) -> - comparator.compare(deviceEntries.get(left), deviceEntries.get(right))) - .mapToInt(Integer::intValue) - .toArray(); - List sortedDeviceEntries = new ArrayList<>(deviceEntries.size()); - List> sortedDeviceOffsets = - new ArrayList<>(deviceOffsets.size()); - for (int index : indexes) { - sortedDeviceEntries.add(deviceEntries.get(index)); - sortedDeviceOffsets.add(deviceOffsets.get(index)); - } - this.deviceEntries = sortedDeviceEntries; - this.deviceOffsets = sortedDeviceOffsets; + public void setDeviceEntries(List deviceEntries) { + throw new UnsupportedOperationException( + "ExternalTsFileScanNode device entries must be set by device entry indexes"); + } + + public SchemaFilter getSchemaFilter() { + return schemaFilter; } - private static List> copyDeviceOffsets( - List> deviceOffsets) { - List> copiedDeviceOffsets = - new ArrayList<>(deviceOffsets.size()); - for (List offsets : deviceOffsets) { - copiedDeviceOffsets.add(new ArrayList<>(offsets)); - } - return copiedDeviceOffsets; + public void setSchemaFilter(SchemaFilter schemaFilter) { + this.schemaFilter = schemaFilter; } @Override @@ -292,24 +162,8 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { "ExternalTsFileScanNode cannot be serialized because it reads local external TsFiles"); } - public static ExternalTsFileScanNode deserialize(ByteBuffer byteBuffer) { - throw new UnsupportedOperationException( - "ExternalTsFileScanNode cannot be deserialized because it reads local external TsFiles"); - } - @Override public String toString() { return "ExternalTsFileScanNode-" + this.getPlanNodeId(); } - - private static Map buildTagIndexMap(Map assignments) { - Map tagIndexMap = new java.util.HashMap<>(); - int tagIndex = 0; - for (Map.Entry entry : assignments.entrySet()) { - if (TAG.equals(entry.getValue().getColumnCategory())) { - tagIndexMap.put(entry.getKey(), tagIndex++); - } - } - return tagIndexMap; - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index af6ddc5db1cc1..5ffac0473bafb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -53,12 +53,10 @@ import org.apache.iotdb.commons.schema.table.InformationSchema; import org.apache.iotdb.commons.schema.table.TsTable; import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; -import org.apache.iotdb.commons.udf.builtin.relational.tvf.ReadTsFileTableFunction.ExternalTsFileDeviceOffset; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; -import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileDeviceFilterVisitor; import org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; @@ -68,7 +66,6 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicateCombineIntoTableScanChecker; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicatePushIntoMetadataChecker; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.schema.ConvertSchemaPredicateToFilterVisitor; -import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.metadata.NonAlignedDeviceEntry; @@ -87,24 +84,18 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.LazyTsFileDeviceIterator; -import org.apache.tsfile.read.TsFileSequenceReader; import org.apache.tsfile.read.common.type.Type; import org.apache.tsfile.read.filter.basic.Filter; -import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.Pair; import javax.annotation.Nullable; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -560,43 +551,8 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre private void collectExternalTsFileDeviceTasks( ExternalTsFileScanNode tableScanNode, List metadataExpressions) { - SchemaFilter deviceFilter = - constructExternalTsFileDeviceFilter(tableScanNode, metadataExpressions); - ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = - new ExternalTsFileDeviceFilterVisitor(); - Map> deviceOffsetMap = new LinkedHashMap<>(); - for (String tsFilePath : tableScanNode.getTsFilePaths()) { - try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFilePath)) { - LazyTsFileDeviceIterator deviceIterator = - new LazyTsFileDeviceIterator( - reader, tableScanNode.getQualifiedObjectName().getObjectName(), ignored -> {}); - while (deviceIterator.hasNext()) { - IDeviceID deviceID = deviceIterator.next(); - if (deviceFilter != null - && !Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID))) { - continue; - } - deviceOffsetMap - .computeIfAbsent(deviceID, ignored -> new ArrayList<>()) - .add( - new ExternalTsFileDeviceOffset( - tsFilePath, deviceIterator.getCurrentDeviceMeasurementNodeOffset())); - } - } catch (IOException e) { - throw new SemanticException( - "Failed to collect devices from external TsFile: " + tsFilePath); - } - } - List deviceEntries = new ArrayList<>(deviceOffsetMap.size()); - List> deviceOffsets = - new ArrayList<>(deviceOffsetMap.size()); - for (Map.Entry> entry : - deviceOffsetMap.entrySet()) { - deviceEntries.add(new AlignedDeviceEntry(entry.getKey(), new Binary[0])); - deviceOffsets.add(entry.getValue()); - } - tableScanNode.setDeviceEntries(deviceEntries); - tableScanNode.setDeviceOffsets(deviceOffsets); + tableScanNode.setSchemaFilter( + constructExternalTsFileDeviceFilter(tableScanNode, metadataExpressions)); } private SchemaFilter constructExternalTsFileDeviceFilter( 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 b72b05fbff52a..d11a012a7d58b 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 @@ -298,9 +298,12 @@ public PlanAndMappings visitExternalTsFileScan( node.getPushDownOffset(), node.getTimePredicate().map(mapper::map).orElse(null), node.getScanOrder(), - node.getTsFilePaths(), - node.getDeviceEntries(), - node.getDeviceOffsets()); + node.isPushLimitToEachDevice(), + node.getTagAndAttributeIndexMap(), + node.getExternalTsFileQueryResource(), + node.getDeviceEntryIndexes(), + node.getDeviceTaskPartitionIndex(), + node.getSchemaFilter()); rewrittenNode.setRegionReplicaSet(node.getRegionReplicaSet()); return new PlanAndMappings(rewrittenNode, mapping); } From 194cd3076122a636aecb1e4c34bb81e5bde52745 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Wed, 10 Jun 2026 17:59:34 +0800 Subject: [PATCH 14/32] fix bug --- .../org/apache/iotdb/it/env/EnvFactory.java | 2 +- .../IoTDBReadTsFileTableFunctionIT.java | 388 ++++++++++++++++++ .../fragment/FragmentInstanceContext.java | 79 ++-- .../AbstractAggTableScanOperator.java | 42 +- .../AbstractDeviceTableScanOperator.java | 78 ---- .../relational/AbstractTableScanOperator.java | 50 ++- .../ExternalTsFileAggTableScanOperator.java | 112 ++--- .../ExternalTsFileSeriesScanUtil.java | 21 +- .../ExternalTsFileTableScanOperator.java | 104 ++--- .../source/relational/TableScanOperator.java | 2 +- .../TreeAlignedDeviceViewScanOperator.java | 2 +- .../DataNodeTableOperatorGenerator.java | 27 +- .../ExternalTsFileQueryDataSource.java | 67 +++ .../ExternalTsFileQueryResource.java | 278 ++++++++----- .../TableDistributedPlanGenerator.java | 27 +- 15 files changed, 864 insertions(+), 415 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDeviceTableScanOperator.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/EnvFactory.java b/integration-test/src/main/java/org/apache/iotdb/it/env/EnvFactory.java index 2fbc9672837c8..2855bec5b7bb9 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/EnvFactory.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/EnvFactory.java @@ -43,7 +43,7 @@ public static BaseEnv getEnv() { try { Class.forName(Config.JDBC_DRIVER_NAME); logger.info(">>>>>>>{}", System.getProperty("TestEnv")); - EnvType envType = EnvType.getSystemEnvType(); + EnvType envType = EnvType.Remote; switch (envType) { case Simple: case TABLE_SIMPLE: diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java new file mode 100644 index 0000000000000..ed236b901ac52 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java @@ -0,0 +1,388 @@ +/* + * 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.env.cluster.node.DataNodeWrapper; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.TableClusterIT; +import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; + +import org.apache.tsfile.enums.ColumnCategory; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.metadata.TableSchema; +import org.apache.tsfile.write.TsFileWriter; +import org.apache.tsfile.write.record.Tablet; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.sql.Connection; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +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 IoTDBReadTsFileTableFunctionIT { + private static final String DATABASE_NAME = "test_read_tsfile"; + + private static File tmpDir; + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv().initClusterEnvironment(); + try (Connection connection = EnvFactory.getEnv().getTableConnection(); + Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE " + DATABASE_NAME); + } + } + + @Before + public void setUpBeforeTest() throws IOException { + tmpDir = new File(Files.createTempDirectory("read-tsfile").toUri()); + } + + @After + public void tearDownAfterTest() { + deleteTmpDir(); + } + + @AfterClass + public static void tearDown() { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Test + public void testReadSingleTsFile() throws Exception { + File tsFile = new File(tmpDir, "single.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + generateTable(writer, "table2", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + + String[] expectedHeader = new String[] {"time", "tag1", "tag2", "s1", "s2"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,tag2_1,1,1,", + "1970-01-01T00:00:00.002Z,tag1_1,tag2_1,2,2,", + "1970-01-01T00:00:00.001Z,tag1_2,tag2_2,1,1,", + "1970-01-01T00:00:00.002Z,tag1_2,tag2_2,2,2,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, tag2, s1, s2 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile) + + "', TABLE_NAME => 'table1') ORDER BY tag1, tag2, time", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testReadTsFileWithNoMatchedDevice() throws Exception { + File tsFile = new File(tmpDir, "empty-device.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + } + + tableResultSetEqualTest( + "SELECT time, tag1, tag2, s1, s2 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile) + + "', TABLE_NAME => 'table1') WHERE tag1 = 'not_exists'", + new String[] {"time", "tag1", "tag2", "s1", "s2"}, + new String[] {}, + DATABASE_NAME); + } + + @Test + public void testReadTsFileAggregationWithNoMatchedDevice() throws Exception { + File tsFile = new File(tmpDir, "empty-device-aggregation.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + } + + tableResultSetEqualTest( + "SELECT count(*) AS count_star, count(s1) AS count_s1, sum(s1) AS sum_s1" + + " FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile) + + "', TABLE_NAME => 'table1') WHERE tag1 = 'not_exists'", + new String[] {"count_star", "count_s1", "sum_s1"}, + new String[] {"0,0,null,"}, + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithDeviceFilter() throws Exception { + File tsFile1 = new File(tmpDir, "multi-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + } + File tsFile2 = new File(tmpDir, "multi-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 3, 4); + } + + String[] expectedHeader = new String[] {"time", "tag1", "tag2", "s1", "s2"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,tag2_1,1,1,", + "1970-01-01T00:00:00.002Z,tag1_1,tag2_1,2,2,", + "1970-01-01T00:00:00.003Z,tag1_1,tag2_1,3,3,", + "1970-01-01T00:00:00.004Z,tag1_1,tag2_1,4,4,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, tag2, s1, s2 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1') WHERE tag1 = 'tag1_1' ORDER BY time", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithSchemaMerge() throws Exception { + File tsFile1 = new File(tmpDir, "schema-merge-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "schema-merge-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1", "s2"), 3, 4); + } + + String[] expectedHeader = new String[] {"time", "tag1", "s1", "s2"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,1,null,", + "1970-01-01T00:00:00.002Z,tag1_1,2,null,", + "1970-01-01T00:00:00.003Z,tag1_1,3,3,", + "1970-01-01T00:00:00.004Z,tag1_1,4,4,", + "1970-01-01T00:00:00.001Z,tag1_2,1,null,", + "1970-01-01T00:00:00.002Z,tag1_2,2,null,", + "1970-01-01T00:00:00.003Z,tag1_2,3,3,", + "1970-01-01T00:00:00.004Z,tag1_2,4,4,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, s1, s2 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1') ORDER BY tag1, time", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithConflictingFieldType() throws Exception { + File tsFile1 = new File(tmpDir, "conflict-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "conflict-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable( + writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), TSDataType.DOUBLE, 3, 4); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1')", + "has conflicting data types when merging table schema", + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithConflictingTagColumns() throws Exception { + File tsFile1 = new File(tmpDir, "tag-conflict-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "tag-conflict-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable(writer, "table1", Arrays.asList("tag2"), Arrays.asList("s1"), 3, 4); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1')", + "Tag columns conflict when merging table schema", + DATABASE_NAME); + } + + @Test + public void testReadTsFileWithoutTableNameWhenMultipleTablesExist() throws Exception { + File tsFile = new File(tmpDir, "multiple-tables.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + generateTable(writer, "table2", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(tsFile) + "')", + "Cannot infer table name from TsFile because multiple tables are found", + DATABASE_NAME); + } + + @Test + public void testReadTsFileWithInvalidPaths() throws IOException { + File missingFile = new File(tmpDir, "missing.tsfile"); + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(missingFile) + "')", + "TsFile path does not exist", + DATABASE_NAME); + + DataNodeWrapper dataNodeWrapper = EnvFactory.getEnv().getDataNodeWrapper(0); + File dataDir = + new File(dataNodeWrapper.getDataNodeDir() + File.separator + "data", "forbidden.tsfile"); + Files.createDirectories(dataDir.getParentFile().toPath()); + Files.write(dataDir.toPath(), new byte[0]); + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(dataDir) + "')", + "is not allowed because it may access IoTDB data directory", + DATABASE_NAME); + } + + private static void generateTable( + TsFileWriter writer, + String tableName, + List tagColumns, + List fieldColumns, + long startTime, + long endTime) + throws IOException, WriteProcessException { + generateTable( + writer, tableName, tagColumns, fieldColumns, TSDataType.INT64, startTime, endTime); + } + + private static void generateTable( + TsFileWriter writer, + String tableName, + List tagColumns, + List fieldColumns, + TSDataType fieldType, + long startTime, + long endTime) + throws IOException, WriteProcessException { + List columnNames = new ArrayList<>(tagColumns.size() + fieldColumns.size()); + List columnTypes = new ArrayList<>(tagColumns.size() + fieldColumns.size()); + List columnCategories = + new ArrayList<>(tagColumns.size() + fieldColumns.size()); + for (String tagColumn : tagColumns) { + columnNames.add(tagColumn); + columnTypes.add(TSDataType.STRING); + columnCategories.add(ColumnCategory.TAG); + } + for (String fieldColumn : fieldColumns) { + columnNames.add(fieldColumn); + columnTypes.add(fieldType); + columnCategories.add(ColumnCategory.FIELD); + } + + writer.registerTableSchema( + new TableSchema(tableName, columnNames, columnTypes, columnCategories)); + Tablet tablet = new Tablet(tableName, columnNames, columnTypes, columnCategories); + for (int deviceIndex = 1; deviceIndex <= 2; deviceIndex++) { + for (long time = startTime; time <= endTime; time++) { + int row = tablet.getRowSize(); + tablet.addTimestamp(row, time); + for (int i = 0; i < tagColumns.size(); i++) { + tablet.addValue(row, i, tagColumns.get(i) + "_" + deviceIndex); + } + for (int i = 0; i < fieldColumns.size(); i++) { + addFieldValue(tablet, row, tagColumns.size() + i, fieldType, time); + } + if (tablet.getRowSize() == tablet.getMaxRowNumber()) { + writer.writeTable(tablet); + tablet.reset(); + } + } + } + if (tablet.getRowSize() != 0) { + writer.writeTable(tablet); + } + } + + private static void addFieldValue( + Tablet tablet, int row, int column, TSDataType fieldType, long time) { + if (fieldType == TSDataType.DOUBLE) { + tablet.addValue(row, column, (double) time); + return; + } + tablet.addValue(row, column, time); + } + + private static String toSqlPath(File file) { + return file.getAbsolutePath().replace("\\", "\\\\").replace("'", "''"); + } + + private static void clearTmpDir() { + if (tmpDir == null || !tmpDir.exists()) { + return; + } + File[] files = tmpDir.listFiles(); + if (files != null) { + for (File file : files) { + try { + Files.delete(file.toPath()); + } catch (IOException ignored) { + // ignore + } + } + } + } + + private static void deleteTmpDir() { + clearTmpDir(); + if (tmpDir == null || !tmpDir.exists()) { + return; + } + try { + Files.delete(tmpDir.toPath()); + } catch (IOException ignored) { + // ignore + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 8ae3e885f5e6b..24c28e668b159 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -46,6 +46,8 @@ import org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.planner.memory.ThreadSafeMemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.TimePredicate; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryDataSource; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.IDataRegionForQuery; @@ -56,8 +58,6 @@ import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceType; import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; 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.tsfile.timeindex.FileTimeIndex; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsResp; @@ -71,8 +71,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; import java.time.ZoneId; import java.time.format.DateTimeParseException; import java.util.ArrayList; @@ -121,8 +119,7 @@ public class FragmentInstanceContext extends QueryContext { // Used for region scan, relating methods are to be added. private Map devicePathsToContext; - private List externalTsFilePaths; - private List externalTsFileResources; + private ExternalTsFileQueryResource externalTsFileQueryResource; // Shared by all scan operators in this fragment instance to avoid memory problem protected IQueryDataSource sharedQueryDataSource; @@ -624,12 +621,26 @@ public void setDevicePathsToContext(Map devicePathsToC this.devicePathsToContext = devicePathsToContext; } - public void addExternalTsFilePaths(List externalTsFilePaths) { - if (this.externalTsFilePaths == null) { - this.externalTsFilePaths = new ArrayList<>(externalTsFilePaths); - return; + public void addExternalTsFileQueryResource( + ExternalTsFileQueryResource externalTsFileQueryResource) { + this.externalTsFileQueryResource = externalTsFileQueryResource; + } + + public boolean initExternalTsFileQueryDataSource( + ExternalTsFileQueryResource externalTsFileQueryResource) { + long startTime = System.nanoTime(); + try { + if (externalTsFileQueryResource == null) { + this.sharedQueryDataSource = EMPTY_QUERY_DATA_SOURCE; + return true; + } + + this.sharedQueryDataSource = new ExternalTsFileQueryDataSource(externalTsFileQueryResource); + closedUnseqFileNum = externalTsFileQueryResource.getTsFileResources().size(); + return true; + } finally { + addInitQueryDataSourceCost(System.nanoTime() - startTime); } - this.externalTsFilePaths.addAll(externalTsFilePaths); } public MemoryReservationManager getMemoryReservationContext() { @@ -799,44 +810,6 @@ public boolean initRegionScanQueryDataSource(List pathList) { } } - public boolean initExternalTsFileQueryDataSource(List externalTsFilePaths) - throws QueryProcessException { - long startTime = System.nanoTime(); - try { - if (externalTsFilePaths == null || externalTsFilePaths.isEmpty()) { - this.sharedQueryDataSource = EMPTY_QUERY_DATA_SOURCE; - return true; - } - - externalTsFileResources = new ArrayList<>(externalTsFilePaths.size()); - for (String externalTsFilePath : externalTsFilePaths) { - TsFileResource resource = - new TsFileResource(new File(externalTsFilePath), TsFileResourceStatus.NORMAL); - if (resource.resourceFileExists()) { - try { - resource.deserialize(); - } catch (IOException e) { - throw new QueryProcessException( - "Failed to deserialize external TsFile resource: " - + externalTsFilePath - + ", " - + e.getMessage()); - } - } else { - resource.setTimeIndex(new FileTimeIndex(Long.MIN_VALUE, Long.MAX_VALUE)); - } - externalTsFileResources.add(resource); - } - - this.sharedQueryDataSource = - new QueryDataSource(Collections.emptyList(), externalTsFileResources); - closedUnseqFileNum = externalTsFileResources.size(); - return true; - } finally { - addInitQueryDataSourceCost(System.nanoTime() - startTime); - } - } - public synchronized IQueryDataSource getSharedQueryDataSource() throws QueryProcessException { if (sharedQueryDataSource == null) { switch (queryDataSourceType) { @@ -863,8 +836,8 @@ public synchronized IQueryDataSource getSharedQueryDataSource() throws QueryProc } break; case EXTERNAL_TSFILE_SCAN: - if (initExternalTsFileQueryDataSource(externalTsFilePaths)) { - externalTsFilePaths = null; + if (initExternalTsFileQueryDataSource(externalTsFileQueryResource)) { + externalTsFileQueryResource = null; } else { return getUnfinishedQueryDataSource(); } @@ -1081,9 +1054,7 @@ public synchronized void releaseResource() { unClosedFilePaths = null; } - if (externalTsFileResources != null) { - externalTsFileResources = null; - } + externalTsFileQueryResource = null; // release TVList/AlignedTVList owned by current query releaseTVListOwnedByQuery(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java index e3030c0752cef..031c10dc8a4db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java @@ -251,18 +251,9 @@ protected Optional calculateAggregationResultForCurrentTimeRange() thro // all data of current device has been consumed updateResultTsBlock(); timeIterator.resetCurTimeRange(); - nextDevice(); - - if (currentDeviceIndex < deviceCount) { - // construct AlignedSeriesScanUtil for next device - constructAlignedSeriesScanUtil(); - queryDataSource.reset(); - this.seriesScanUtil.initQueryDataSource(queryDataSource); - } + moveToNextDevice(); if (currentDeviceIndex >= deviceCount) { - // all devices have been consumed - timeIterator.setFinished(); return Optional.of(true); } else { return Optional.of(false); @@ -715,22 +706,25 @@ protected void checkIfAllAggregatorHasFinalResult() throws Exception { if (allAggregatorsHasFinalResult && (timeIterator.getType() == ITableTimeRangeIterator.TimeIteratorType.SINGLE_TIME_ITERATOR || tableAggregators.isEmpty())) { - nextDevice(); - inputTsBlock = null; - - if (currentDeviceIndex < deviceCount) { - // construct AlignedSeriesScanUtil for next device - constructAlignedSeriesScanUtil(); - queryDataSource.reset(); - this.seriesScanUtil.initQueryDataSource(queryDataSource); - } + moveToNextDevice(); + allAggregatorsHasFinalResult = false; + } + } - if (currentDeviceIndex >= deviceCount) { - // all devices have been consumed - timeIterator.setFinished(); - } + protected void moveToNextDevice() throws Exception { + nextDevice(); + inputTsBlock = null; - allAggregatorsHasFinalResult = false; + if (currentDeviceIndex < deviceCount) { + // construct AlignedSeriesScanUtil for next device + constructAlignedSeriesScanUtil(); + queryDataSource.reset(); + this.seriesScanUtil.initQueryDataSource(queryDataSource); + } + + if (currentDeviceIndex >= deviceCount) { + // all devices have been consumed + timeIterator.setFinished(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDeviceTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDeviceTableScanOperator.java deleted file mode 100644 index e4b119291aed1..0000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDeviceTableScanOperator.java +++ /dev/null @@ -1,78 +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.queryengine.execution.operator.source.relational; - -import org.apache.iotdb.calc.plan.planner.CommonOperatorUtils; -import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; - -import org.apache.tsfile.utils.RamUsageEstimator; - -import java.util.List; - -import static org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter.DEVICE_NUMBER; - -public abstract class AbstractDeviceTableScanOperator extends AbstractTableScanOperator { - private static final long INSTANCE_SIZE = - RamUsageEstimator.shallowSizeOfInstance(AbstractDeviceTableScanOperator.class); - - protected List deviceEntries; - protected int deviceCount; - protected int currentDeviceIndex; - - protected AbstractDeviceTableScanOperator(AbstractTableScanOperatorParameter parameter) { - super(parameter); - this.deviceEntries = parameter.deviceEntries; - this.deviceCount = parameter.deviceEntries.size(); - this.currentDeviceIndex = 0; - this.operatorContext.recordSpecifiedInfo(DEVICE_NUMBER, Integer.toString(this.deviceCount)); - recordCurrentDeviceIndex(); - constructAlignedSeriesScanUtil(); - } - - @Override - protected boolean hasCurrentDeviceEntry() { - return currentDeviceIndex < deviceCount; - } - - @Override - protected DeviceEntry getCurrentDeviceEntry() { - return deviceEntries.get(currentDeviceIndex); - } - - @Override - protected boolean advanceDeviceEntry() { - currentDeviceIndex++; - return hasCurrentDeviceEntry(); - } - - @Override - protected void recordCurrentDeviceIndex() { - this.operatorContext.recordSpecifiedInfo( - CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); - } - - @Override - public long ramBytesUsed() { - return super.ramBytesUsed() - + INSTANCE_SIZE - - AbstractTableScanOperator.INSTANCE_SIZE - + RamUsageEstimator.sizeOfCollection(deviceEntries); - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java index 45481e95f11d4..523f0d1367a67 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; +import org.apache.iotdb.calc.plan.planner.CommonOperatorUtils; import org.apache.iotdb.commons.path.AlignedFullPath; import org.apache.iotdb.commons.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; @@ -50,15 +51,20 @@ import static org.apache.iotdb.commons.queryengine.plan.relational.type.InternalTypeManager.getTSDataType; import static org.apache.iotdb.db.queryengine.execution.operator.source.AlignedSeriesScanOperator.appendDataIntoBuilder; +import static org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter.DEVICE_NUMBER; public abstract class AbstractTableScanOperator extends AbstractSeriesScanOperator { protected static final long INSTANCE_SIZE = - RamUsageEstimator.shallowSizeOfInstance(AbstractTableScanOperator.class); + RamUsageEstimator.shallowSizeOfInstance(TableScanOperator.class); private final List columnSchemas; private final int[] columnsIndexArray; + protected final List deviceEntries; + + protected final int deviceCount; + protected final Ordering scanOrder; protected final SeriesScanOptions seriesScanOptions; @@ -78,11 +84,16 @@ public abstract class AbstractTableScanOperator extends AbstractSeriesScanOperat private QueryDataSource queryDataSource; + protected int currentDeviceIndex; + public AbstractTableScanOperator(AbstractTableScanOperatorParameter parameter) { this.sourceId = parameter.sourceId; this.operatorContext = parameter.context; this.columnSchemas = parameter.columnSchemas; this.columnsIndexArray = parameter.columnsIndexArray; + this.deviceEntries = parameter.deviceEntries; + this.deviceCount = parameter.deviceEntries.size(); + this.operatorContext.recordSpecifiedInfo(DEVICE_NUMBER, Integer.toString(this.deviceCount)); this.scanOrder = parameter.scanOrder; this.seriesScanOptions = parameter.seriesScanOptions; this.measurementColumnNames = parameter.measurementColumnNames; @@ -92,12 +103,17 @@ public AbstractTableScanOperator(AbstractTableScanOperatorParameter parameter) { parameter.measurementSchemas.stream() .map(IMeasurementSchema::getType) .collect(Collectors.toList()); + this.currentDeviceIndex = 0; + this.operatorContext.recordSpecifiedInfo( + CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING, Integer.toString(0)); + // allSensors include time and all field columns this.maxReturnSize = Math.min( maxReturnSize, allSensors.size() * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte()); this.maxTsBlockLineNum = parameter.maxTsBlockLineNum; + constructAlignedSeriesScanUtil(); } @Override @@ -181,7 +197,7 @@ protected void buildResult(TsBlock tsBlock) { } private void constructResultTsBlock() { - DeviceEntry currentDeviceEntry = getCurrentDeviceEntry(); + DeviceEntry currentDeviceEntry = deviceEntries.get(currentDeviceIndex); this.resultTsBlock = MeasurementToTableViewAdaptorUtils.toTableBlock( measurementDataBlock, @@ -201,7 +217,7 @@ public boolean hasNext() throws Exception { @Override public boolean isFinished() throws Exception { return (retainedTsBlock == null) - && (!hasCurrentDeviceEntry() || seriesScanOptions.limitConsumedUp()); + && (currentDeviceIndex >= deviceCount || seriesScanOptions.limitConsumedUp()); } @Override @@ -233,37 +249,32 @@ public void initQueryDataSource(IQueryDataSource dataSource) { this.measurementDataBuilder.setMaxTsBlockLineNumber(this.maxTsBlockLineNum); } - private void moveToNextDevice() { - if (advanceDeviceEntry()) { + protected void moveToNextDevice() { + currentDeviceIndex++; + if (currentDeviceIndex < deviceCount) { // construct AlignedSeriesScanUtil for next device constructAlignedSeriesScanUtil(); // reset QueryDataSource queryDataSource.reset(); this.seriesScanUtil.initQueryDataSource(queryDataSource); - recordCurrentDeviceIndex(); + this.operatorContext.recordSpecifiedInfo( + CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); } } - protected abstract boolean hasCurrentDeviceEntry(); - - protected abstract DeviceEntry getCurrentDeviceEntry(); - - protected abstract boolean advanceDeviceEntry(); - - protected abstract void recordCurrentDeviceIndex(); - protected void constructAlignedSeriesScanUtil() { - if (!hasCurrentDeviceEntry()) { + if (this.deviceEntries.isEmpty() || currentDeviceIndex >= deviceCount) { // no need to construct SeriesScanUtil, hasNext will return false return; } - if (getCurrentDeviceEntry() == null) { - throw new IllegalStateException("Current device entry in TableScanOperator is empty"); + if (this.deviceEntries.get(this.currentDeviceIndex) == null) { + throw new IllegalStateException( + "Device entries of index " + this.currentDeviceIndex + " in TableScanOperator is empty"); } - DeviceEntry deviceEntry = getCurrentDeviceEntry(); + DeviceEntry deviceEntry = this.deviceEntries.get(this.currentDeviceIndex); AlignedFullPath alignedPath = constructAlignedPath(deviceEntry, measurementColumnNames, measurementSchemas, allSensors); this.seriesScanUtil = @@ -291,7 +302,8 @@ public long ramBytesUsed() { + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(seriesScanUtil) + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext) + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(sourceId) - + (resultTsBlockBuilder == null ? 0 : resultTsBlockBuilder.getRetainedSizeInBytes()); + + (resultTsBlockBuilder == null ? 0 : resultTsBlockBuilder.getRetainedSizeInBytes()) + + RamUsageEstimator.sizeOfCollection(deviceEntries); } public static class AbstractTableScanOperatorParameter { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java index 3b4b27041032c..f6e4a84fc7515 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java @@ -22,11 +22,12 @@ import org.apache.iotdb.commons.path.AlignedFullPath; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.SeriesScanUtil; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryDataSource; import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; -import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceOffset; -import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.MultiWayMergeReader; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceTaskRunReader; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; @@ -34,8 +35,6 @@ import org.apache.tsfile.utils.RamUsageEstimator; import java.io.IOException; -import java.util.Collections; -import java.util.List; import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.constructAlignedPath; @@ -43,39 +42,19 @@ public class ExternalTsFileAggTableScanOperator extends DefaultAggTableScanOpera private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(ExternalTsFileAggTableScanOperator.class); - private final String tableName; - private final ExternalTsFileQueryResource externalTsFileQueryResource; private final int deviceTaskPartitionIndex; - private MultiWayMergeReader deviceTaskReader; - private int loadedDeviceOffsetIndex = -1; - private List currentDeviceOffsets = Collections.emptyList(); + private DeviceTaskRunReader deviceTaskReader; public ExternalTsFileAggTableScanOperator( - AbstractAggTableScanOperatorParameter parameter, - String tableName, - ExternalTsFileQueryResource externalTsFileQueryResource, - int deviceTaskPartitionIndex) { + AbstractAggTableScanOperatorParameter parameter, int deviceTaskPartitionIndex) { super(parameter); - this.tableName = tableName; - this.externalTsFileQueryResource = externalTsFileQueryResource; this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; } - @Override - String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { - int segmentOffset = - deviceEntry.getDeviceID().segmentNum() > 0 - && tableName.equalsIgnoreCase((String) deviceEntry.getNthSegment(0)) - ? 1 - : 0; - Object segment = deviceEntry.getNthSegment(idColumnIndex + segmentOffset); - return segment == null ? null : (String) segment; - } - @Override protected void constructAlignedSeriesScanUtil() { DeviceEntry deviceEntry = - deviceEntries.isEmpty() || deviceEntries.get(currentDeviceIndex) == null + !hasCurrentRealDeviceEntry() ? new AlignedDeviceEntry(SeriesScanUtil.EMPTY_DEVICE_ID, new Binary[0]) : deviceEntries.get(currentDeviceIndex); this.seriesScanUtil = @@ -92,39 +71,75 @@ protected void constructAlignedSeriesScanUtil() { private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( TsFileResource resource, AlignedFullPath alignedPath) throws IOException { - if (deviceEntries.isEmpty() || currentDeviceIndex >= deviceEntries.size()) { + if (!hasCurrentRealDeviceEntry()) { return null; } return ExternalTsFileSeriesScanUtil.loadTimeSeriesMetadata( resource, alignedPath, deviceEntries.get(currentDeviceIndex).getDeviceID(), - getCurrentDeviceOffsets(), - externalTsFileQueryResource.getTsFilePaths(), + deviceTaskReader.getCurrentDeviceOffsetMap().get(resource), ((OperatorContext) operatorContext).getInstanceContext(), seriesScanOptions.getGlobalTimeFilter()); } - private List getCurrentDeviceOffsets() throws IOException { - if (loadedDeviceOffsetIndex == currentDeviceIndex) { - return currentDeviceOffsets; - } - if (deviceTaskReader == null) { + @Override + public void initQueryDataSource(IQueryDataSource dataSource) { + ExternalTsFileQueryResource externalTsFileQueryResource = + ((ExternalTsFileQueryDataSource) dataSource).getExternalTsFileQueryResource(); + if (hasCurrentRealDeviceEntry() && deviceTaskReader == null) { deviceTaskReader = - externalTsFileQueryResource.getMultiWayMergeReader(deviceTaskPartitionIndex); + externalTsFileQueryResource.getDeviceTaskRunReader(deviceTaskPartitionIndex); } - DeviceEntry currentDeviceEntry = deviceEntries.get(currentDeviceIndex); - while (deviceTaskReader.hasNextDevice()) { - DeviceEntry deviceEntry = deviceTaskReader.nextDevice(); - if (deviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { - currentDeviceOffsets = deviceTaskReader.getCurrentDeviceOffsets(); - loadedDeviceOffsetIndex = currentDeviceIndex; - return currentDeviceOffsets; + super.initQueryDataSource( + hasCurrentRealDeviceEntry() ? updateCurrentDeviceQueryDataSource() : dataSource); + } + + @Override + protected void moveToNextDevice() throws Exception { + nextDevice(); + inputTsBlock = null; + + if (currentDeviceIndex < deviceCount) { + constructAlignedSeriesScanUtil(); + queryDataSource = + hasCurrentRealDeviceEntry() ? updateCurrentDeviceQueryDataSource() : queryDataSource; + seriesScanUtil.initQueryDataSource(queryDataSource); + } + + if (currentDeviceIndex >= deviceCount) { + timeIterator.setFinished(); + } + } + + private boolean hasCurrentRealDeviceEntry() { + return !deviceEntries.isEmpty() + && currentDeviceIndex < deviceEntries.size() + && deviceEntries.get(currentDeviceIndex) != null; + } + + private ExternalTsFileQueryDataSource updateCurrentDeviceQueryDataSource() { + try { + if (!deviceTaskReader.nextDevice()) { + throw new IllegalStateException( + "Unexpected end of external TsFile device task reader at device index " + + currentDeviceIndex); + } + DeviceEntry expectedDeviceEntry = deviceEntries.get(currentDeviceIndex); + DeviceEntry currentDeviceEntry = deviceTaskReader.getCurrentDevice(); + if (!expectedDeviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { + throw new IllegalStateException( + String.format( + "External TsFile device task reader is not aligned with device entries at index %d:" + + " expected %s but got %s", + currentDeviceIndex, + expectedDeviceEntry.getDeviceID(), + currentDeviceEntry.getDeviceID())); } + return deviceTaskReader.getCurrentDeviceQueryDataSource(); + } catch (IOException e) { + throw new RuntimeException("Failed to update external TsFile device resources", e); } - currentDeviceOffsets = Collections.emptyList(); - loadedDeviceOffsetIndex = currentDeviceIndex; - return currentDeviceOffsets; } @Override @@ -138,9 +153,6 @@ public void close() throws Exception { @Override public long ramBytesUsed() { - return super.ramBytesUsed() - + INSTANCE_SIZE - - AbstractDefaultAggTableScanOperator.INSTANCE_SIZE - + RamUsageEstimator.sizeOfCollection(currentDeviceOffsets); + return super.ramBytesUsed() + INSTANCE_SIZE - AbstractDefaultAggTableScanOperator.INSTANCE_SIZE; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java index 6ad6aa2af3f78..d4d327be703cf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java @@ -86,20 +86,15 @@ static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( TsFileResource resource, AlignedFullPath alignedPath, IDeviceID currentDeviceID, - List currentDeviceOffsets, - List tsFilePaths, + DeviceOffset currentDeviceOffset, FragmentInstanceContext context, Filter globalTimeFilter) throws IOException { - if (currentDeviceOffsets == null || !currentDeviceID.equals(alignedPath.getDeviceId())) { + if (currentDeviceOffset == null || !currentDeviceID.equals(alignedPath.getDeviceId())) { return null; } - long[] deviceMeasurementNodeOffset = - getDeviceMeasurementNodeOffset(currentDeviceOffsets, tsFilePaths, resource.getTsFilePath()); - if (deviceMeasurementNodeOffset == null) { - return null; - } + long[] deviceMeasurementNodeOffset = currentDeviceOffset.getMeasurementNodeOffset(); // TODO: Use deviceMeasurementNodeOffset after FileLoaderUtils supports offset-based metadata // loading in this branch. return FileLoaderUtils.loadAlignedTimeSeriesMetadata( @@ -111,16 +106,6 @@ static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( context.isIgnoreAllNullRows()); } - private static long[] getDeviceMeasurementNodeOffset( - List currentDeviceOffsets, List tsFilePaths, String tsFilePath) { - for (DeviceOffset offset : currentDeviceOffsets) { - if (tsFilePath.equals(tsFilePaths.get(offset.getFileIndex()))) { - return offset.getMeasurementNodeOffset(); - } - } - return null; - } - @FunctionalInterface public interface ExternalTsFileMetadataLoader { AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java index ad14540e29173..73a9cb11ad58b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java @@ -19,63 +19,41 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; +import org.apache.iotdb.calc.plan.planner.CommonOperatorUtils; import org.apache.iotdb.commons.path.AlignedFullPath; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryDataSource; import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; -import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceOffset; -import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.MultiWayMergeReader; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceTaskRunReader; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; import org.apache.tsfile.utils.RamUsageEstimator; import java.io.IOException; -import java.util.Collections; -import java.util.List; public class ExternalTsFileTableScanOperator extends TableScanOperator { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(ExternalTsFileTableScanOperator.class); - private static final long ABSTRACT_DEVICE_TABLE_SCAN_OPERATOR_INSTANCE_SIZE = - RamUsageEstimator.shallowSizeOfInstance(AbstractDeviceTableScanOperator.class); - private final String tableName; - private final ExternalTsFileQueryResource externalTsFileQueryResource; private final int deviceTaskPartitionIndex; - private MultiWayMergeReader deviceTaskReader; - private int loadedDeviceOffsetIndex = -1; - private List currentDeviceOffsets = Collections.emptyList(); + private DeviceTaskRunReader deviceTaskReader; public ExternalTsFileTableScanOperator( - AbstractTableScanOperatorParameter parameter, - String tableName, - ExternalTsFileQueryResource externalTsFileQueryResource, - int deviceTaskPartitionIndex) { + AbstractTableScanOperatorParameter parameter, int deviceTaskPartitionIndex) { super(parameter); - this.tableName = tableName; - this.externalTsFileQueryResource = externalTsFileQueryResource; this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; } - @Override - String getNthIdColumnValue(DeviceEntry deviceEntry, int idColumnIndex) { - int segmentOffset = - deviceEntry.getDeviceID().segmentNum() > 0 - && tableName.equalsIgnoreCase((String) deviceEntry.getNthSegment(0)) - ? 1 - : 0; - Object segment = deviceEntry.getNthSegment(idColumnIndex + segmentOffset); - return segment == null ? null : (String) segment; - } - @Override protected void constructAlignedSeriesScanUtil() { - if (!hasCurrentDeviceEntry()) { + if (currentDeviceIndex >= deviceCount) { return; } - DeviceEntry deviceEntry = getCurrentDeviceEntry(); + DeviceEntry deviceEntry = deviceEntries.get(currentDeviceIndex); if (deviceEntry == null) { throw new IllegalStateException("Current device entry in TableScanOperator is empty"); } @@ -97,33 +75,58 @@ private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( return ExternalTsFileSeriesScanUtil.loadTimeSeriesMetadata( resource, alignedPath, - getCurrentDeviceEntry().getDeviceID(), - getCurrentDeviceOffsets(), - externalTsFileQueryResource.getTsFilePaths(), + deviceEntries.get(currentDeviceIndex).getDeviceID(), + deviceTaskReader.getCurrentDeviceOffsetMap().get(resource), ((OperatorContext) operatorContext).getInstanceContext(), seriesScanOptions.getGlobalTimeFilter()); } - private List getCurrentDeviceOffsets() throws IOException { - if (loadedDeviceOffsetIndex == currentDeviceIndex) { - return currentDeviceOffsets; - } + @Override + public void initQueryDataSource(IQueryDataSource dataSource) { + ExternalTsFileQueryResource externalTsFileQueryResource = + ((ExternalTsFileQueryDataSource) dataSource).getExternalTsFileQueryResource(); if (deviceTaskReader == null) { deviceTaskReader = - externalTsFileQueryResource.getMultiWayMergeReader(deviceTaskPartitionIndex); + externalTsFileQueryResource.getDeviceTaskRunReader(deviceTaskPartitionIndex); + } + IQueryDataSource currentDataSource = + currentDeviceIndex < deviceCount ? updateCurrentDeviceQueryDataSource() : dataSource; + super.initQueryDataSource(currentDataSource); + } + + @Override + protected void moveToNextDevice() { + currentDeviceIndex++; + if (currentDeviceIndex < deviceCount) { + constructAlignedSeriesScanUtil(); + seriesScanUtil.initQueryDataSource(updateCurrentDeviceQueryDataSource()); + this.operatorContext.recordSpecifiedInfo( + CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); } - DeviceEntry currentDeviceEntry = getCurrentDeviceEntry(); - while (deviceTaskReader.hasNextDevice()) { - DeviceEntry deviceEntry = deviceTaskReader.nextDevice(); - if (deviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { - currentDeviceOffsets = deviceTaskReader.getCurrentDeviceOffsets(); - loadedDeviceOffsetIndex = currentDeviceIndex; - return currentDeviceOffsets; + } + + private ExternalTsFileQueryDataSource updateCurrentDeviceQueryDataSource() { + try { + if (!deviceTaskReader.nextDevice()) { + throw new IllegalStateException( + "Unexpected end of external TsFile device task reader at device index " + + currentDeviceIndex); + } + DeviceEntry expectedDeviceEntry = deviceEntries.get(currentDeviceIndex); + DeviceEntry currentDeviceEntry = deviceTaskReader.getCurrentDevice(); + if (!expectedDeviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { + throw new IllegalStateException( + String.format( + "External TsFile device task reader is not aligned with device entries at index %d:" + + " expected %s but got %s", + currentDeviceIndex, + expectedDeviceEntry.getDeviceID(), + currentDeviceEntry.getDeviceID())); } + return deviceTaskReader.getCurrentDeviceQueryDataSource(); + } catch (IOException e) { + throw new RuntimeException("Failed to update external TsFile device resources", e); } - currentDeviceOffsets = Collections.emptyList(); - loadedDeviceOffsetIndex = currentDeviceIndex; - return currentDeviceOffsets; } @Override @@ -137,9 +140,6 @@ public void close() throws Exception { @Override public long ramBytesUsed() { - return super.ramBytesUsed() - + INSTANCE_SIZE - - ABSTRACT_DEVICE_TABLE_SCAN_OPERATOR_INSTANCE_SIZE - + RamUsageEstimator.sizeOfCollection(currentDeviceOffsets); + return super.ramBytesUsed() + INSTANCE_SIZE - AbstractTableScanOperator.INSTANCE_SIZE; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java index a50e978b377d0..11f10ef3dae94 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TableScanOperator.java @@ -21,7 +21,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; -public class TableScanOperator extends AbstractDeviceTableScanOperator { +public class TableScanOperator extends AbstractTableScanOperator { public TableScanOperator(AbstractTableScanOperatorParameter parameter) { super(parameter); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TreeAlignedDeviceViewScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TreeAlignedDeviceViewScanOperator.java index 12feaccc42f20..03d0e998e7193 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TreeAlignedDeviceViewScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/TreeAlignedDeviceViewScanOperator.java @@ -23,7 +23,7 @@ import org.apache.tsfile.file.metadata.IDeviceID; -public class TreeAlignedDeviceViewScanOperator extends AbstractDeviceTableScanOperator { +public class TreeAlignedDeviceViewScanOperator extends AbstractTableScanOperator { private final IDeviceID.TreeDeviceIdColumnValueExtractor extractor; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index b0fc9e1d5546c..1bc40a2896b16 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -1128,6 +1128,13 @@ public Operator visitDeviceTableScan( @Override public Operator visitExternalTsFileScan( ExternalTsFileScanNode node, LocalExecutionPlanContext context) { + if (node.getDeviceEntries().isEmpty()) { + OperatorContext operatorContext = + addOperatorContext( + context, node.getPlanNodeId(), EmptyDataOperator.class.getSimpleName()); + return new EmptyDataOperator(operatorContext); + } + AbstractTableScanOperator.AbstractTableScanOperatorParameter parameter = constructAbstractTableScanOperatorParameter( node, @@ -1137,11 +1144,7 @@ public Operator visitExternalTsFileScan( Long.MAX_VALUE); AbstractTableScanOperator externalTsFileTableScanOperator = - new ExternalTsFileTableScanOperator( - parameter, - node.getQualifiedObjectName().getObjectName(), - node.getExternalTsFileQueryResource(), - node.getDeviceTaskPartitionIndex()); + new ExternalTsFileTableScanOperator(parameter, node.getDeviceTaskPartitionIndex()); context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); @@ -1149,7 +1152,9 @@ public Operator visitExternalTsFileScan( dataDriverContext.addSourceOperator(externalTsFileTableScanOperator); dataDriverContext.setQueryDataSourceType(QueryDataSourceType.EXTERNAL_TSFILE_SCAN); dataDriverContext.setInputDriver(true); - context.getInstanceContext().addExternalTsFilePaths(node.getTsFilePaths()); + context + .getInstanceContext() + .addExternalTsFileQueryResource(node.getExternalTsFileQueryResource()); return externalTsFileTableScanOperator; } @@ -1640,11 +1645,7 @@ public Operator visitExternalTsFileAggregationScan( constructAbstractAggTableScanOperatorParameter(node, context); ExternalTsFileAggTableScanOperator aggTableScanOperator = - new ExternalTsFileAggTableScanOperator( - parameter, - node.getQualifiedObjectName().getObjectName(), - node.getExternalTsFileQueryResource(), - node.getDeviceTaskPartitionIndex()); + new ExternalTsFileAggTableScanOperator(parameter, node.getDeviceTaskPartitionIndex()); context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); addSource( @@ -1658,7 +1659,9 @@ public Operator visitExternalTsFileAggregationScan( DataDriverContext dataDriverContext = (DataDriverContext) context.getDriverContext(); dataDriverContext.setQueryDataSourceType(QueryDataSourceType.EXTERNAL_TSFILE_SCAN); - context.getInstanceContext().addExternalTsFilePaths(node.getTsFilePaths()); + context + .getInstanceContext() + .addExternalTsFileQueryResource(node.getExternalTsFileQueryResource()); return aggTableScanOperator; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java new file mode 100644 index 0000000000000..ac55a8cbfe256 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java @@ -0,0 +1,67 @@ +/* + * 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.function.tvf.readTsFile; + +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.filter.basic.Filter; + +import java.util.Collections; +import java.util.List; + +public class ExternalTsFileQueryDataSource extends QueryDataSource { + + private final ExternalTsFileQueryResource externalTsFileQueryResource; + + public ExternalTsFileQueryDataSource(ExternalTsFileQueryResource externalTsFileQueryResource) { + this(externalTsFileQueryResource, Collections.emptyList()); + } + + ExternalTsFileQueryDataSource( + ExternalTsFileQueryResource externalTsFileQueryResource, + List unseqResources) { + super(Collections.emptyList(), unseqResources); + this.externalTsFileQueryResource = externalTsFileQueryResource; + } + + @Override + public IQueryDataSource clone() { + return new ExternalTsFileQueryDataSource(externalTsFileQueryResource, getUnseqResources()); + } + + public ExternalTsFileQueryResource getExternalTsFileQueryResource() { + return externalTsFileQueryResource; + } + + @Override + public boolean isSeqSatisfied( + IDeviceID deviceID, int curIndex, Filter timeFilter, boolean debug) { + return true; + } + + @Override + public boolean isUnSeqSatisfied( + IDeviceID deviceID, int curIndex, Filter timeFilter, boolean debug) { + return true; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 8371c6a16873f..bd45d94e18815 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -26,6 +26,9 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; +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.tsfile.timeindex.FileTimeIndex; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.LazyTsFileDeviceIterator; @@ -38,7 +41,7 @@ import java.io.Closeable; import java.io.DataInputStream; import java.io.DataOutputStream; -import java.io.EOFException; +import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; @@ -64,19 +67,20 @@ public class ExternalTsFileQueryResource implements AutoCloseable { public static final String EXTERNAL_TSFILE_TMP_DIR = "external-tsfile"; - private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 8L * 1024 * 1024; + private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 1; + // private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 8L * 1024 * 1024; private static final long DEVICE_OFFSET_INSTANCE_SIZE_IN_BYTES = 32L; private final QueryId queryId; private final Path queryTempRoot; private final String tableName; private final List tsFilePaths; + private final List tsFileResources; private final LongConsumer ioSizeRecorder; private final List deviceEntries = new ArrayList<>(); private List deviceTaskPartitions = Collections.emptyList(); private Comparator deviceEntryComparator; - private boolean readersRetained; private boolean closed; public ExternalTsFileQueryResource( @@ -95,16 +99,19 @@ public ExternalTsFileQueryResource( this.tableName = tableName; this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(requireNonNull(tsFilePaths, "tsFilePaths"))); + this.tsFileResources = createTsFileResources(this.tsFilePaths); this.ioSizeRecorder = requireNonNull(ioSizeRecorder, "ioSizeRecorder is null"); + for (String tsFilePath : tsFilePaths) { + FileReaderManager.getInstance().increaseExternalFileReaderReference(tsFilePath); + } } - public synchronized void collectDeviceEntries( + public void collectDeviceEntries( SchemaFilter schemaFilter, Comparator comparator, int partitionCount) { checkNotClosed(); - retainFileReaderReferences(); ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = new ExternalTsFileDeviceFilterVisitor(); try (DeviceCollector deviceCollector = new DeviceCollector()) { - List partitions = createDeviceTaskPartitions(partitionCount); + createDeviceTaskPartitions(partitionCount); while (deviceCollector.hasNextDevice()) { IDeviceID deviceID = deviceCollector.nextDevice(); if (schemaFilter != null @@ -118,22 +125,23 @@ public synchronized void collectDeviceEntries( new DeviceTask( deviceEntryIndex, new ArrayList<>(deviceCollector.getCurrentDeviceOffsets())); DeviceTaskPartition partition = - partitions.get(Math.floorMod(deviceID.hashCode(), partitions.size())); + deviceTaskPartitions.get( + Math.floorMod(deviceID.hashCode(), deviceTaskPartitions.size())); partition.add(deviceTask); if (partition.getEstimatedSizeInBytes() >= DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES) { partition.flush(comparator); } } deviceEntryComparator = comparator; - collectDeviceTaskPartitions(partitions, comparator); + collectDeviceTaskPartitions(comparator); } } - public synchronized MultiWayMergeReader getMultiWayMergeReader(int partitionIndex) { + public DeviceTaskRunReader getDeviceTaskRunReader(int partitionIndex) { checkNotClosed(); DeviceTaskPartition partition = getDeviceTaskPartition(partitionIndex); try { - return new DeviceTaskRunReader(partition.getRunFiles(), deviceEntries, deviceEntryComparator); + return new DeviceTaskRunReader(partition); } catch (IOException e) { throw new RuntimeException("Failed to create external TsFile device task run reader", e); } @@ -143,6 +151,10 @@ public List getTsFilePaths() { return tsFilePaths; } + public List getTsFileResources() { + return tsFileResources; + } + public List getDeviceEntries() { return deviceEntries; } @@ -162,7 +174,7 @@ private DeviceTaskPartition getDeviceTaskPartition(int partitionIndex) { } @Override - public synchronized void close() { + public void close() { if (closed) { return; } @@ -175,24 +187,10 @@ public synchronized void close() { } } - private void retainFileReaderReferences() { - if (readersRetained) { - return; - } - for (String tsFilePath : tsFilePaths) { - FileReaderManager.getInstance().increaseExternalFileReaderReference(tsFilePath); - } - readersRetained = true; - } - private void releaseFileReaderReferences() { - if (!readersRetained) { - return; - } for (String tsFilePath : tsFilePaths) { FileReaderManager.getInstance().decreaseExternalFileReaderReference(tsFilePath); } - readersRetained = false; } private void checkNotClosed() { @@ -201,15 +199,28 @@ private void checkNotClosed() { } } - public interface MultiWayMergeReader extends AutoCloseable { - boolean hasNextDevice() throws IOException; - - DeviceEntry nextDevice() throws IOException; - - List getCurrentDeviceOffsets(); - - @Override - void close() throws IOException; + private static List createTsFileResources(List tsFilePaths) { + List tsFileResources = new ArrayList<>(tsFilePaths.size()); + for (String tsFilePath : tsFilePaths) { + TsFileResource resource = + new TsFileResource(new File(tsFilePath), TsFileResourceStatus.NORMAL); + if (resource.resourceFileExists()) { + try { + resource.deserialize(); + } catch (IOException e) { + throw new RuntimeException( + "Failed to deserialize external TsFile resource: " + + tsFilePath + + ", " + + e.getMessage(), + e); + } + } else { + resource.setTimeIndex(new FileTimeIndex(Long.MIN_VALUE, Long.MAX_VALUE)); + } + tsFileResources.add(resource); + } + return Collections.unmodifiableList(tsFileResources); } public class DeviceTaskPartition { @@ -283,6 +294,16 @@ private boolean hasDeviceTasks() { return !deviceEntryIndexes.isEmpty(); } + private void finish(Comparator comparator) { + if (pendingDeviceTasks.isEmpty()) { + return; + } + sortPendingDeviceTasks(comparator); + for (DeviceTask deviceTask : pendingDeviceTasks) { + deviceEntryIndexes.add(deviceTask.deviceEntryIndex); + } + } + private void sortDeviceEntries(Comparator comparator) { if (comparator != null) { deviceEntryIndexes.sort( @@ -300,36 +321,34 @@ private void sortDeviceEntries(Comparator comparator) { private List getRunFiles() { return runFiles; } + + private List getPendingDeviceTasks() { + return pendingDeviceTasks; + } } - private List createDeviceTaskPartitions(int partitionCount) { + private void createDeviceTaskPartitions(int partitionCount) { if (partitionCount <= 0) { throw new IllegalArgumentException( "External TsFile device task partition count must be positive"); } - List partitions = new ArrayList<>(partitionCount); + deviceTaskPartitions = new ArrayList<>(partitionCount); for (int i = 0; i < partitionCount; i++) { - partitions.add(new DeviceTaskPartition(i)); + deviceTaskPartitions.add(new DeviceTaskPartition(i)); } - return partitions; } - private void collectDeviceTaskPartitions( - List partitions, Comparator comparator) { - if (partitions.isEmpty()) { - deviceTaskPartitions = Collections.emptyList(); - return; - } - List nonEmptyPartitions = new ArrayList<>(partitions.size()); - for (DeviceTaskPartition partition : partitions) { - partition.flush(comparator); + private void collectDeviceTaskPartitions(Comparator comparator) { + Iterator iterator = deviceTaskPartitions.iterator(); + while (iterator.hasNext()) { + DeviceTaskPartition partition = iterator.next(); + partition.finish(comparator); if (!partition.hasDeviceTasks()) { + iterator.remove(); continue; } partition.sortDeviceEntries(comparator); - nonEmptyPartitions.add(partition); } - deviceTaskPartitions = nonEmptyPartitions; } private Path writeDeviceTaskRun(Path runRoot, int runIndex, List deviceTasks) @@ -356,58 +375,74 @@ private static long estimateDeviceTaskSize(DeviceTask deviceTask) { return size; } - private static class DeviceTaskRunReader implements MultiWayMergeReader { + public class DeviceTaskRunReader implements AutoCloseable { - private final List deviceEntries; private final PriorityQueue runCursors; - private DeviceTask nextDeviceTask; - private List currentDeviceOffsets = Collections.emptyList(); + private DeviceEntry currentDevice; + private ExternalTsFileQueryDataSource currentDeviceQueryDataSource; + private Map currentDeviceOffsetMap; - private DeviceTaskRunReader( - List runFiles, List deviceEntries, Comparator comparator) - throws IOException { - this.deviceEntries = deviceEntries; + private DeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { Comparator cursorComparator = (left, right) -> - comparator == null + deviceEntryComparator == null ? left.getCurrentDeviceEntry() .getDeviceID() .compareTo(right.getCurrentDeviceEntry().getDeviceID()) - : comparator.compare(left.getCurrentDeviceEntry(), right.getCurrentDeviceEntry()); + : deviceEntryComparator.compare( + left.getCurrentDeviceEntry(), right.getCurrentDeviceEntry()); this.runCursors = new PriorityQueue<>(cursorComparator); - for (Path runFile : runFiles) { - DeviceTaskRunCursor cursor = new DeviceTaskRunCursor(runFile, deviceEntries); + for (Path runFile : partition.getRunFiles()) { + DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFile, deviceEntries); if (cursor.hasCurrentDeviceTask()) { runCursors.add(cursor); } else { cursor.close(); } } + DeviceTaskRunCursor memoryCursor = + new MemoryDeviceTaskRunCursor(partition.getPendingDeviceTasks(), deviceEntries); + if (memoryCursor.hasCurrentDeviceTask()) { + runCursors.add(memoryCursor); + } } - @Override - public boolean hasNextDevice() throws IOException { - if (nextDeviceTask != null) { - return true; + public boolean nextDevice() throws IOException { + if (runCursors.isEmpty()) { + return false; + } + DeviceTaskRunCursor cursor = runCursors.poll(); + DeviceTask result = cursor.getCurrentDeviceTask(); + cursor.advance(); + if (cursor.hasCurrentDeviceTask()) { + runCursors.add(cursor); + } else { + cursor.close(); } - nextDeviceTask = readNextDeviceTask(); - return nextDeviceTask != null; - } - @Override - public DeviceEntry nextDevice() throws IOException { - if (!hasNextDevice()) { - throw new EOFException("No more external TsFile device task"); + currentDevice = deviceEntries.get(result.deviceEntryIndex); + List unseqResources = new ArrayList<>(result.deviceOffsets.size()); + currentDeviceOffsetMap = new HashMap<>(result.deviceOffsets.size()); + for (DeviceOffset deviceOffset : result.deviceOffsets) { + TsFileResource tsFileResource = tsFileResources.get(deviceOffset.getFileIndex()); + unseqResources.add(tsFileResource); + currentDeviceOffsetMap.put(tsFileResource, deviceOffset); } - DeviceTask deviceTask = nextDeviceTask; - nextDeviceTask = null; - currentDeviceOffsets = deviceTask.deviceOffsets; - return deviceEntries.get(deviceTask.deviceEntryIndex); + currentDeviceQueryDataSource = + new ExternalTsFileQueryDataSource(ExternalTsFileQueryResource.this, unseqResources); + return true; } - @Override - public List getCurrentDeviceOffsets() { - return currentDeviceOffsets; + public DeviceEntry getCurrentDevice() { + return currentDevice; + } + + public ExternalTsFileQueryDataSource getCurrentDeviceQueryDataSource() { + return currentDeviceQueryDataSource; + } + + public Map getCurrentDeviceOffsetMap() { + return currentDeviceOffsetMap; } @Override @@ -428,31 +463,28 @@ public void close() throws IOException { throw exception; } } + } - private DeviceTask readNextDeviceTask() throws IOException { - if (runCursors.isEmpty()) { - return null; - } - DeviceTaskRunCursor cursor = runCursors.poll(); - DeviceTask result = cursor.getCurrentDeviceTask(); - cursor.advance(); - if (cursor.hasCurrentDeviceTask()) { - runCursors.add(cursor); - } else { - cursor.close(); - } - return result; - } + private interface DeviceTaskRunCursor extends Closeable { + + boolean hasCurrentDeviceTask(); + + DeviceTask getCurrentDeviceTask(); + + DeviceEntry getCurrentDeviceEntry(); + + void advance() throws IOException; } - private static class DeviceTaskRunCursor implements Closeable { + private static class DiskDeviceTaskRunCursor implements DeviceTaskRunCursor { private final List deviceEntries; private final DataInputStream inputStream; private int remainingDeviceTasks; private DeviceTask currentDeviceTask; - private DeviceTaskRunCursor(Path runFile, List deviceEntries) throws IOException { + private DiskDeviceTaskRunCursor(Path runFile, List deviceEntries) + throws IOException { this.deviceEntries = deviceEntries; this.inputStream = new DataInputStream(new BufferedInputStream(Files.newInputStream(runFile))); @@ -460,7 +492,8 @@ private DeviceTaskRunCursor(Path runFile, List deviceEntries) throw advance(); } - private void advance() throws IOException { + @Override + public void advance() throws IOException { if (remainingDeviceTasks <= 0) { currentDeviceTask = null; return; @@ -469,15 +502,18 @@ private void advance() throws IOException { currentDeviceTask = DeviceTask.deserialize(inputStream); } - private boolean hasCurrentDeviceTask() { + @Override + public boolean hasCurrentDeviceTask() { return currentDeviceTask != null; } - private DeviceTask getCurrentDeviceTask() { + @Override + public DeviceTask getCurrentDeviceTask() { return currentDeviceTask; } - private DeviceEntry getCurrentDeviceEntry() { + @Override + public DeviceEntry getCurrentDeviceEntry() { return deviceEntries.get(currentDeviceTask.deviceEntryIndex); } @@ -487,6 +523,50 @@ public void close() throws IOException { } } + private static class MemoryDeviceTaskRunCursor implements DeviceTaskRunCursor { + + private final List deviceTasks; + private final List deviceEntries; + private int nextIndex; + private DeviceTask currentDeviceTask; + + private MemoryDeviceTaskRunCursor( + List deviceTasks, List deviceEntries) { + this.deviceTasks = deviceTasks; + this.deviceEntries = deviceEntries; + advance(); + } + + @Override + public void advance() { + if (nextIndex >= deviceTasks.size()) { + currentDeviceTask = null; + return; + } + currentDeviceTask = deviceTasks.get(nextIndex++); + } + + @Override + public boolean hasCurrentDeviceTask() { + return currentDeviceTask != null; + } + + @Override + public DeviceTask getCurrentDeviceTask() { + return currentDeviceTask; + } + + @Override + public DeviceEntry getCurrentDeviceEntry() { + return deviceEntries.get(currentDeviceTask.deviceEntryIndex); + } + + @Override + public void close() { + currentDeviceTask = null; + } + } + private class DeviceCollector implements Closeable { private final Map deviceIteratorMap = new HashMap<>(); 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 0b54b2395a7b1..92d30d4f6497e 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 @@ -799,13 +799,23 @@ public List visitExternalTsFileAggregationScan( if (partitions.isEmpty()) { return Collections.singletonList(node); } + boolean needFinalAggregation = + node.getStep() == SINGLE && node.getGroupingKeys().isEmpty() && partitions.size() > 1; + AggregationNode finalAggregation = null; + AggregationTableScanNode partialTemplateNode = node; + if (needFinalAggregation) { + Pair splitResult = + split(node, symbolAllocator, queryId); + finalAggregation = splitResult.left; + partialTemplateNode = splitResult.right; + } List result = new ArrayList<>(partitions.size()); for (DeviceTaskPartition partition : partitions) { ExternalTsFileAggregationScanNode splitNode = new ExternalTsFileAggregationScanNode( queryId.genPlanNodeId(), node.getQualifiedObjectName(), - node.getOutputSymbols(), + partialTemplateNode.getOutputSymbols(), node.getAssignments(), node.getTagAndAttributeIndexMap(), node.getScanOrder(), @@ -816,11 +826,11 @@ public List visitExternalTsFileAggregationScan( node.isPushLimitToEachDevice(), node.containsNonAlignedDevice(), node.getProjection(), - node.getAggregations(), - node.getGroupingSets(), - node.getPreGroupedSymbols(), - node.getStep(), - node.getGroupIdSymbol(), + partialTemplateNode.getAggregations(), + partialTemplateNode.getGroupingSets(), + partialTemplateNode.getPreGroupedSymbols(), + partialTemplateNode.getStep(), + partialTemplateNode.getGroupIdSymbol(), node.getExternalTsFileQueryResource(), partition.getDeviceEntryIndexes(), partition.getPartitionIndex(), @@ -830,6 +840,11 @@ public List visitExternalTsFileAggregationScan( } sortPropertyContext.ifPresent( propertyContext -> applySortProperty(node, result, propertyContext, false)); + if (needFinalAggregation) { + OrderingScheme childOrdering = nodeOrderingMap.get(result.get(0).getPlanNodeId()); + finalAggregation.setChild(mergeChildrenViaCollectOrMergeSort(childOrdering, result)); + return Collections.singletonList(finalAggregation); + } return result; } From 5dd9a42ec301bd863c5a93e0d5a952e1ad844e58 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Wed, 10 Jun 2026 18:01:00 +0800 Subject: [PATCH 15/32] fix --- .../src/main/java/org/apache/iotdb/it/env/EnvFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/EnvFactory.java b/integration-test/src/main/java/org/apache/iotdb/it/env/EnvFactory.java index 2855bec5b7bb9..2fbc9672837c8 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/EnvFactory.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/EnvFactory.java @@ -43,7 +43,7 @@ public static BaseEnv getEnv() { try { Class.forName(Config.JDBC_DRIVER_NAME); logger.info(">>>>>>>{}", System.getProperty("TestEnv")); - EnvType envType = EnvType.Remote; + EnvType envType = EnvType.getSystemEnvType(); switch (envType) { case Simple: case TABLE_SIMPLE: From 9a63ebf29181b706dc638343134840419cb1f61c Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Thu, 11 Jun 2026 10:36:19 +0800 Subject: [PATCH 16/32] fix --- .../source/AlignedSeriesScanUtil.java | 4 +- .../operator/source/FileLoaderUtils.java | 22 +- .../ExternalTsFileSeriesScanUtil.java | 26 +-- .../MultiTsFileResourceIterator.java | 210 ------------------ .../ExternalTsFileQueryResource.java | 81 ++++--- .../buffer/TimeSeriesMetadataCache.java | 81 ++++--- .../execution/MemoryEstimationHelper.java | 2 +- 7 files changed, 110 insertions(+), 316 deletions(-) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java index f7ddaee472ea9..e80f305e73b92 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java @@ -38,6 +38,7 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; public class AlignedSeriesScanUtil extends SeriesScanUtil { @@ -100,7 +101,8 @@ protected AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( context, scanOptions.getGlobalTimeFilter(), isSeq, - ignoreAllNullRows); + ignoreAllNullRows, + Optional.empty()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java index dcc38e49a61e7..79235360cdf38 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java @@ -57,6 +57,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -110,8 +111,7 @@ public static TimeseriesMetadata loadTimeSeriesMetadata( context.ignoreNotExistsDevice() || resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE, context.isDebug(), - context, - context.isExternalTsFileScan()); + context); if (timeSeriesMetadata != null) { long t2 = System.nanoTime(); List pathModifications = @@ -199,7 +199,8 @@ public static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadata( FragmentInstanceContext context, Filter globalTimeFilter, boolean isSeq, - boolean ignoreAllNullRows) + boolean ignoreAllNullRows, + Optional rootMeasurementMetadataIndexNodeOffset) throws IOException { final long t1 = System.nanoTime(); boolean loadFromMem = false; @@ -213,7 +214,12 @@ public static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadata( if (resource.isClosed()) { alignedTimeSeriesMetadata = loadAlignedTimeSeriesMetadataFromDisk( - resource, alignedPath, context, globalTimeFilter, ignoreAllNullRows); + resource, + alignedPath, + context, + globalTimeFilter, + ignoreAllNullRows, + rootMeasurementMetadataIndexNodeOffset); } else { // if the tsfile is unclosed, we just get it directly from TsFileResource loadFromMem = true; alignedTimeSeriesMetadata = @@ -286,7 +292,8 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr AlignedFullPath alignedPath, FragmentInstanceContext context, Filter globalTimeFilter, - boolean ignoreAllNullRows) + boolean ignoreAllNullRows, + Optional rootMeasurementMetadataIndexNodeOffset) throws IOException { AbstractAlignedTimeSeriesMetadata alignedTimeSeriesMetadata = null; // load all the TimeseriesMetadata of vector, the first one is for time column and the @@ -298,7 +305,6 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr boolean isDebug = context.isDebug(); String filePath = resource.getTsFilePath(); IDeviceID deviceId = alignedPath.getDeviceId(); - boolean isExternalTsFile = context.isExternalTsFileScan(); // when resource.getTimeIndexType() == 1, TsFileResource.timeIndexType is deviceTimeIndex // we should not ignore the non-exist of device in TsFileMetadata @@ -311,7 +317,7 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr || resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE, isDebug, context, - isExternalTsFile); + rootMeasurementMetadataIndexNodeOffset); if (timeColumn != null) { // only need time column, like count_time aggregation if (valueMeasurementList.isEmpty()) { @@ -341,7 +347,7 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr || resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE, isDebug, context, - isExternalTsFile); + rootMeasurementMetadataIndexNodeOffset); exist = (exist || (valueColumn != null)); valueTimeSeriesMetadataList.add(valueColumn); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java index d4d327be703cf..c22384160ae4d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java @@ -36,6 +36,7 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; public class ExternalTsFileSeriesScanUtil extends AlignedSeriesScanUtil { @@ -53,24 +54,6 @@ public ExternalTsFileSeriesScanUtil( this.metadataLoader = metadataLoader; } - public ExternalTsFileSeriesScanUtil( - AlignedFullPath seriesPath, - Ordering scanOrder, - SeriesScanOptions scanOptions, - FragmentInstanceContext context, - boolean queryAllSensors, - List givenDataTypes, - MultiTsFileResourceIterator resourceIterator) { - this( - seriesPath, - scanOrder, - scanOptions, - context, - queryAllSensors, - givenDataTypes, - resourceIterator::loadTimeSeriesMetadata); - } - @Override protected AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( TsFileResource resource, boolean isSeq) throws IOException { @@ -94,16 +77,15 @@ static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( return null; } - long[] deviceMeasurementNodeOffset = currentDeviceOffset.getMeasurementNodeOffset(); - // TODO: Use deviceMeasurementNodeOffset after FileLoaderUtils supports offset-based metadata - // loading in this branch. return FileLoaderUtils.loadAlignedTimeSeriesMetadata( resource, alignedPath, context, globalTimeFilter, resource.isSeq(), - context.isIgnoreAllNullRows()); + context.isIgnoreAllNullRows(), + Optional.of( + new long[] {currentDeviceOffset.getStartOffset(), currentDeviceOffset.getEndOffset()})); } @FunctionalInterface diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java deleted file mode 100644 index c55d336a2246b..0000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/MultiTsFileResourceIterator.java +++ /dev/null @@ -1,210 +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.queryengine.execution.operator.source.relational; - -import org.apache.iotdb.commons.path.AlignedFullPath; -import org.apache.iotdb.commons.schema.filter.SchemaFilter; -import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; -import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; -import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; - -import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; -import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.LazyTsFileDeviceIterator; -import org.apache.tsfile.read.TsFileSequenceReader; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.function.LongConsumer; - -public class MultiTsFileResourceIterator { - - private final String tableName; - private final FragmentInstanceContext fragmentInstanceContext; - private final SeriesScanOptions seriesScanOptions; - private final SchemaFilter deviceFilter; - private final ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = - new ExternalTsFileDeviceFilterVisitor(); - private final Map deviceIteratorMap = - new HashMap<>(); - - private IDeviceID currentDevice; - - public MultiTsFileResourceIterator( - String tableName, - List seqResources, - List unseqResources, - Map resourceReaderMap, - FragmentInstanceContext fragmentInstanceContext, - SeriesScanOptions seriesScanOptions, - SchemaFilter deviceFilter) { - this.tableName = tableName; - this.fragmentInstanceContext = fragmentInstanceContext; - this.seriesScanOptions = seriesScanOptions; - this.deviceFilter = deviceFilter; - initDeviceIterators(seqResources, resourceReaderMap); - initDeviceIterators(unseqResources, resourceReaderMap); - } - - private void initDeviceIterators( - List resources, Map resourceReaderMap) { - for (TsFileResource resource : resources) { - try { - TsFileSequenceReader reader = resourceReaderMap.get(resource); - if (reader == null) { - throw new IllegalArgumentException( - "Missing external TsFile reader: " + resource.getTsFilePath()); - } - deviceIteratorMap.put(resource, new TsFileResourceDeviceIterator(resource, reader)); - } catch (IOException e) { - throw new RuntimeException( - "Failed to create device iterator for external TsFile: " + resource.getTsFilePath(), e); - } - } - } - - public boolean hasNextDevice() { - for (TsFileResourceDeviceIterator iterator : deviceIteratorMap.values()) { - if (iterator.hasNextDevice() - || (iterator.getCurrentDevice() != null - && !iterator.getCurrentDevice().equals(currentDevice))) { - return true; - } - } - return false; - } - - public IDeviceID nextDevice() { - IDeviceID nextDevice = null; - List exhaustedResources = new ArrayList<>(); - for (Map.Entry entry : - deviceIteratorMap.entrySet()) { - TsFileResource resource = entry.getKey(); - TsFileResourceDeviceIterator iterator = entry.getValue(); - if (iterator.getCurrentDevice() == null - || iterator.getCurrentDevice().equals(currentDevice)) { - if (iterator.hasNextDevice()) { - if (iterator.nextDevice() == null) { - exhaustedResources.add(resource); - continue; - } - } else { - exhaustedResources.add(resource); - continue; - } - } - if (nextDevice == null || nextDevice.compareTo(iterator.getCurrentDevice()) > 0) { - nextDevice = iterator.getCurrentDevice(); - } - } - for (TsFileResource resource : exhaustedResources) { - deviceIteratorMap.remove(resource); - } - currentDevice = nextDevice; - return currentDevice; - } - - public IDeviceID getCurrentDevice() { - return currentDevice; - } - - public AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( - TsFileResource resource, AlignedFullPath alignedPath) throws IOException { - TsFileResourceDeviceIterator iterator = deviceIteratorMap.get(resource); - if (iterator == null - || currentDevice == null - || !currentDevice.equals(iterator.getCurrentDevice())) { - return null; - } - return iterator.loadTimeSeriesMetadata(alignedPath); - } - - public long[] getCurrentDeviceMeasurementNodeOffset(TsFileResource resource) { - TsFileResourceDeviceIterator iterator = deviceIteratorMap.get(resource); - if (iterator == null - || currentDevice == null - || !currentDevice.equals(iterator.getCurrentDevice())) { - return null; - } - return iterator.getCurrentDeviceMeasurementNodeOffset(); - } - - private boolean isDeviceMatched(IDeviceID deviceID) { - return deviceFilter == null - || Boolean.TRUE.equals(deviceFilter.accept(deviceFilterVisitor, deviceID)); - } - - private class TsFileResourceDeviceIterator { - - private final TsFileResource resource; - private final LazyTsFileDeviceIterator deviceIterator; - private IDeviceID currentDevice; - - private TsFileResourceDeviceIterator(TsFileResource resource, TsFileSequenceReader reader) - throws IOException { - this.resource = resource; - LongConsumer ioSizeRecorder = - fragmentInstanceContext.getQueryStatistics().getLoadTimeSeriesMetadataActualIOSize() - ::addAndGet; - this.deviceIterator = new LazyTsFileDeviceIterator(reader, tableName, ioSizeRecorder); - } - - private boolean hasNextDevice() { - return deviceIterator.hasNext(); - } - - private IDeviceID nextDevice() { - while (deviceIterator.hasNext()) { - IDeviceID nextDevice = deviceIterator.next(); - if (isDeviceMatched(nextDevice)) { - currentDevice = nextDevice; - return currentDevice; - } - } - currentDevice = null; - return null; - } - - private IDeviceID getCurrentDevice() { - return currentDevice; - } - - private long[] getCurrentDeviceMeasurementNodeOffset() { - return deviceIterator.getCurrentDeviceMeasurementNodeOffset(); - } - - private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata(AlignedFullPath alignedPath) - throws IOException { - // TODO: Pass getCurrentDeviceMeasurementNodeOffset() to FileLoaderUtils after this branch - // supports offset-based metadata loading. - return FileLoaderUtils.loadAlignedTimeSeriesMetadata( - resource, - alignedPath, - fragmentInstanceContext, - seriesScanOptions.getGlobalTimeFilter(), - resource.isSeq(), - fragmentInstanceContext.isIgnoreAllNullRows()); - } - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index bd45d94e18815..4d85a11dbc6bd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile; +import org.apache.iotdb.commons.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.queryengine.common.QueryId; @@ -33,7 +34,9 @@ import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.LazyTsFileDeviceIterator; import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Accountable; import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.RamUsageEstimator; import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.BufferedInputStream; @@ -67,9 +70,7 @@ public class ExternalTsFileQueryResource implements AutoCloseable { public static final String EXTERNAL_TSFILE_TMP_DIR = "external-tsfile"; - private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 1; - // private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 8L * 1024 * 1024; - private static final long DEVICE_OFFSET_INSTANCE_SIZE_IN_BYTES = 32L; + private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 8L * 1024 * 1024; private final QueryId queryId; private final Path queryTempRoot; @@ -78,10 +79,10 @@ public class ExternalTsFileQueryResource implements AutoCloseable { private final List tsFileResources; private final LongConsumer ioSizeRecorder; private final List deviceEntries = new ArrayList<>(); - private List deviceTaskPartitions = Collections.emptyList(); + private final List deviceTaskPartitions = new ArrayList<>(); private Comparator deviceEntryComparator; - private boolean closed; + private volatile boolean closed; public ExternalTsFileQueryResource( QueryId queryId, @@ -122,8 +123,7 @@ public void collectDeviceEntries( int deviceEntryIndex = deviceEntries.size(); deviceEntries.add(deviceEntry); DeviceTask deviceTask = - new DeviceTask( - deviceEntryIndex, new ArrayList<>(deviceCollector.getCurrentDeviceOffsets())); + new DeviceTask(deviceEntryIndex, deviceCollector.getCurrentDeviceOffsets()); DeviceTaskPartition partition = deviceTaskPartitions.get( Math.floorMod(deviceID.hashCode(), deviceTaskPartitions.size())); @@ -245,7 +245,7 @@ public List getDeviceEntryIndexes() { private void add(DeviceTask deviceTask) { pendingDeviceTasks.add(deviceTask); - estimatedSizeInBytes += estimateDeviceTaskSize(deviceTask); + estimatedSizeInBytes += deviceTask.ramBytesUsed(); } private void flush(Comparator comparator) { @@ -332,7 +332,6 @@ private void createDeviceTaskPartitions(int partitionCount) { throw new IllegalArgumentException( "External TsFile device task partition count must be positive"); } - deviceTaskPartitions = new ArrayList<>(partitionCount); for (int i = 0; i < partitionCount; i++) { deviceTaskPartitions.add(new DeviceTaskPartition(i)); } @@ -365,16 +364,6 @@ private Path writeDeviceTaskRun(Path runRoot, int runIndex, List dev return runFile; } - private static long estimateDeviceTaskSize(DeviceTask deviceTask) { - long size = 64L; - for (DeviceOffset offset : deviceTask.deviceOffsets) { - size += - DEVICE_OFFSET_INSTANCE_SIZE_IN_BYTES - + (long) Long.BYTES * offset.measurementNodeOffset.length; - } - return size; - } - public class DeviceTaskRunReader implements AutoCloseable { private final PriorityQueue runCursors; @@ -572,7 +561,7 @@ private class DeviceCollector implements Closeable { private final Map deviceIteratorMap = new HashMap<>(); private IDeviceID currentDevice; - private List currentDeviceOffsets = Collections.emptyList(); + private List currentDeviceOffsets; private DeviceCollector() { try { @@ -637,7 +626,9 @@ private void collectCurrentDeviceOffsets() { && currentDevice.equals(deviceIterator.getCurrentDeviceID())) { deviceOffsets.add( new DeviceOffset( - entry.getKey(), deviceIterator.getCurrentDeviceMeasurementNodeOffset())); + entry.getKey(), + deviceIterator.getCurrentDeviceMeasurementNodeOffset()[0], + deviceIterator.getCurrentDeviceMeasurementNodeOffset()[1])); } } currentDeviceOffsets = deviceOffsets; @@ -654,7 +645,10 @@ public void close() { } } - private static class DeviceTask { + private static class DeviceTask implements Accountable { + + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(DeviceTask.class); private final int deviceEntryIndex; private final List deviceOffsets; @@ -669,10 +663,8 @@ private void serialize(DataOutputStream outputStream) throws IOException { ReadWriteIOUtils.write(deviceOffsets.size(), outputStream); for (DeviceOffset offset : deviceOffsets) { ReadWriteIOUtils.write(offset.fileIndex, outputStream); - ReadWriteIOUtils.write(offset.measurementNodeOffset.length, outputStream); - for (long measurementNodeOffset : offset.measurementNodeOffset) { - ReadWriteIOUtils.write(measurementNodeOffset, outputStream); - } + ReadWriteIOUtils.write(offset.startOffset, outputStream); + ReadWriteIOUtils.write(offset.endOffset, outputStream); } } @@ -682,33 +674,48 @@ private static DeviceTask deserialize(DataInputStream inputStream) throws IOExce List offsets = new ArrayList<>(offsetSize); for (int i = 0; i < offsetSize; i++) { int fileIndex = ReadWriteIOUtils.readInt(inputStream); - int measurementNodeOffsetLength = ReadWriteIOUtils.readInt(inputStream); - long[] measurementNodeOffset = new long[measurementNodeOffsetLength]; - for (int j = 0; j < measurementNodeOffsetLength; j++) { - measurementNodeOffset[j] = inputStream.readLong(); - } - offsets.add(new DeviceOffset(fileIndex, measurementNodeOffset)); + long startOffset = inputStream.readLong(); + long endOffset = inputStream.readLong(); + offsets.add(new DeviceOffset(fileIndex, startOffset, endOffset)); } return new DeviceTask(deviceEntryIndex, offsets); } + + @Override + public long ramBytesUsed() { + return INSTANCE_SIZE + + MemoryEstimationHelper.ARRAY_LIST_INSTANCE_SIZE + + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * deviceOffsets.size() + + deviceOffsets.size() * DeviceOffset.INSTANCE_SIZE; + } } public static class DeviceOffset { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(DeviceOffset.class); + private final int fileIndex; - private final long[] measurementNodeOffset; + private final long startOffset; + private final long endOffset; - private DeviceOffset(int fileIndex, long[] measurementNodeOffset) { + private DeviceOffset(int fileIndex, long startOffset, long endOffset) { this.fileIndex = fileIndex; - this.measurementNodeOffset = measurementNodeOffset; + this.startOffset = startOffset; + this.endOffset = endOffset; } public int getFileIndex() { return fileIndex; } - public long[] getMeasurementNodeOffset() { - return measurementNodeOffset; + public long getStartOffset() { + return startOffset; + } + + public long getEndOffset() { + return endOffset; } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java index ba26db0f1d711..7eacf5928f152 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java @@ -50,6 +50,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.WeakHashMap; import java.util.concurrent.atomic.AtomicLong; @@ -119,7 +120,7 @@ public TimeseriesMetadata get( boolean debug, QueryContext queryContext) throws IOException { - return get(filePath, key, allSensors, ignoreNotExists, debug, queryContext, false); + return get(filePath, key, allSensors, ignoreNotExists, debug, queryContext, Optional.empty()); } @SuppressWarnings({"squid:S1860", "squid:S6541", "squid:S3776"}) // Suppress synchronize warning @@ -130,7 +131,7 @@ public TimeseriesMetadata get( boolean ignoreNotExists, boolean debug, QueryContext queryContext, - boolean externalTsFile) + Optional deviceMetadataIndexNodeOffset) throws IOException { long startTime = System.nanoTime(); long loadBloomFilterTime = 0; @@ -139,6 +140,7 @@ public TimeseriesMetadata get( LongConsumer bloomFilterIoSizeRecorder = queryContext.getQueryStatistics().getLoadBloomFilterActualIOSize()::addAndGet; boolean cacheHit = true; + boolean externalTsFile = queryContext.isExternalTsFileScan(); try { if (!CACHE_ENABLE || externalTsFile) { String deviceStringFormat = key.device.toString(); @@ -148,19 +150,25 @@ public TimeseriesMetadata get( TsFileSequenceReader reader = FileReaderManager.getInstance() .get(filePath, key.tsFileID, true, bloomFilterIoSizeRecorder, externalTsFile); - BloomFilter bloomFilter = reader.readBloomFilter(bloomFilterIoSizeRecorder); - queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount().incrementAndGet(); - if (bloomFilter != null - && !bloomFilter.contains( - deviceStringFormat + IoTDBConstant.PATH_SEPARATOR + key.measurement)) { + if (!deviceMetadataIndexNodeOffset.isPresent()) { + BloomFilter bloomFilter = reader.readBloomFilter(bloomFilterIoSizeRecorder); + queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount().incrementAndGet(); + if (bloomFilter != null + && !bloomFilter.contains( + deviceStringFormat + IoTDBConstant.PATH_SEPARATOR + key.measurement)) { + loadBloomFilterTime = System.nanoTime() - startTime; + return null; + } loadBloomFilterTime = System.nanoTime() - startTime; - return null; } - loadBloomFilterTime = System.nanoTime() - startTime; TimeseriesMetadata timeseriesMetadata = reader.readTimeseriesMetadata( - key.device, key.measurement, ignoreNotExists, timeSeriesMetadataIoSizeRecorder); + key.device, + deviceMetadataIndexNodeOffset, + key.measurement, + ignoreNotExists, + timeSeriesMetadataIoSizeRecorder); return (timeseriesMetadata == null || timeseriesMetadata.getStatistics().getCount() == 0) ? null : timeseriesMetadata; @@ -184,41 +192,40 @@ public TimeseriesMetadata get( if (timeseriesMetadata == null) { cacheHit = false; - long loadBloomFilterStartTime = System.nanoTime(); - // bloom filter part - BloomFilter bloomFilter = - BloomFilterCache.getInstance() - .get( - new BloomFilterCache.BloomFilterCacheKey(filePath, key.tsFileID), - debug, - bloomFilterIoSizeRecorder, - queryContext.getQueryStatistics().getLoadBloomFilterFromCacheCount() - ::addAndGet, - queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount() - ::addAndGet, - externalTsFile); - if (bloomFilter != null - && !bloomFilter.contains( - deviceStringFormat + TsFileConstant.PATH_SEPARATOR + key.measurement)) { - if (debug) { - DEBUG_LOGGER.info(StorageEngineMessages.TS_METADATA_FILTERED_BY_BLOOM_FILTER, key); + if (!deviceMetadataIndexNodeOffset.isPresent()) { + long loadBloomFilterStartTime = System.nanoTime(); + // bloom filter part + BloomFilter bloomFilter = + BloomFilterCache.getInstance() + .get( + new BloomFilterCache.BloomFilterCacheKey(filePath, key.tsFileID), + debug, + bloomFilterIoSizeRecorder, + queryContext.getQueryStatistics().getLoadBloomFilterFromCacheCount() + ::addAndGet, + queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount() + ::addAndGet, + false); + if (bloomFilter != null + && !bloomFilter.contains( + deviceStringFormat + TsFileConstant.PATH_SEPARATOR + key.measurement)) { + if (debug) { + DEBUG_LOGGER.info( + StorageEngineMessages.TS_METADATA_FILTERED_BY_BLOOM_FILTER, key); + } + loadBloomFilterTime = System.nanoTime() - loadBloomFilterStartTime; + return null; } + loadBloomFilterTime = System.nanoTime() - loadBloomFilterStartTime; - return null; } - - loadBloomFilterTime = System.nanoTime() - loadBloomFilterStartTime; TsFileSequenceReader reader = FileReaderManager.getInstance() - .get( - filePath, - key.tsFileID, - true, - timeSeriesMetadataIoSizeRecorder, - externalTsFile); + .get(filePath, key.tsFileID, true, timeSeriesMetadataIoSizeRecorder, false); List timeSeriesMetadataList = reader.readTimeseriesMetadata( key.device, + deviceMetadataIndexNodeOffset, key.measurement, allSensors, ignoreNotExists, diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/execution/MemoryEstimationHelper.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/execution/MemoryEstimationHelper.java index 95796129011c6..e1436a1153816 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/execution/MemoryEstimationHelper.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/execution/MemoryEstimationHelper.java @@ -44,7 +44,7 @@ public class MemoryEstimationHelper { private static final long MEASUREMENT_PATH_INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(AlignedPath.class); - private static final long ARRAY_LIST_INSTANCE_SIZE = + public static final long ARRAY_LIST_INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(ArrayList.class); private static final long INTEGER_INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(Integer.class); From 80fce1a741ddb740ceb9e7c43817bddb0aae7896 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Thu, 11 Jun 2026 15:34:59 +0800 Subject: [PATCH 17/32] add memory control --- .../queryengine/common/MPPQueryContext.java | 6 +- .../ExternalTsFileQueryResource.java | 57 +++++++++++++++---- 2 files changed, 51 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 8b1303ef44ea3..68d2de1c2a70b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -261,7 +261,7 @@ public ExternalTsFileQueryResource createExternalTsFileQueryResource( } ExternalTsFileQueryResource externalTsFileQueryResource = new ExternalTsFileQueryResource( - queryId, + this, Paths.get(IoTDBDescriptor.getInstance().getConfig().getSortTmpDir()) .resolve(ExternalTsFileQueryResource.EXTERNAL_TSFILE_TMP_DIR) .resolve(queryId.getId()) @@ -517,6 +517,10 @@ public void reserveMemoryForFrontEndImmediately() { this.memoryReservationManager.reserveMemoryImmediately(); } + public void reserveMemoryForFrontEndImmediately(final long bytes) { + this.memoryReservationManager.reserveMemoryImmediately(bytes); + } + public void releaseAllMemoryReservedForFrontEnd() { this.memoryReservationManager.releaseAllReservedMemory(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 4d85a11dbc6bd..74f39f736139a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -19,9 +19,11 @@ package org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile; +import org.apache.iotdb.calc.exception.MemoryNotEnoughException; import org.apache.iotdb.commons.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileDeviceFilterVisitor; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; @@ -70,9 +72,9 @@ public class ExternalTsFileQueryResource implements AutoCloseable { public static final String EXTERNAL_TSFILE_TMP_DIR = "external-tsfile"; - private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 8L * 1024 * 1024; private final QueryId queryId; + private final MPPQueryContext queryContext; private final Path queryTempRoot; private final String tableName; private final List tsFilePaths; @@ -85,18 +87,18 @@ public class ExternalTsFileQueryResource implements AutoCloseable { private volatile boolean closed; public ExternalTsFileQueryResource( - QueryId queryId, + MPPQueryContext queryContext, Path tempRoot, String tableName, List tsFilePaths, LongConsumer ioSizeRecorder, boolean useExactTempRoot) { - this.queryId = queryId; + this.queryContext = requireNonNull(queryContext, "queryContext is null"); + this.queryId = queryContext.getQueryId(); this.queryTempRoot = useExactTempRoot ? requireNonNull(tempRoot, "tempRoot is null") - : requireNonNull(tempRoot, "tempRoot is null") - .resolve(requireNonNull(queryId, "queryId is null").getId()); + : requireNonNull(tempRoot, "tempRoot is null").resolve(this.queryId.getId()); this.tableName = tableName; this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(requireNonNull(tsFilePaths, "tsFilePaths"))); @@ -128,7 +130,7 @@ public void collectDeviceEntries( deviceTaskPartitions.get( Math.floorMod(deviceID.hashCode(), deviceTaskPartitions.size())); partition.add(deviceTask); - if (partition.getEstimatedSizeInBytes() >= DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES) { + if (partition.shouldFlush()) { partition.flush(comparator); } } @@ -225,11 +227,15 @@ private static List createTsFileResources(List tsFilePat public class DeviceTaskPartition { + private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 8L * 1024 * 1024; + private static final long MEMORY_RESERVE_BATCH_SIZE_IN_BYTES = 1024 * 1024; + private final int partitionIndex; private final List pendingDeviceTasks = new ArrayList<>(); private final List deviceEntryIndexes = new ArrayList<>(); private final List runFiles = new ArrayList<>(); - private long estimatedSizeInBytes; + private long reservedBytes; + private long unreservedBytes; private DeviceTaskPartition(int partitionIndex) { this.partitionIndex = partitionIndex; @@ -245,7 +251,7 @@ public List getDeviceEntryIndexes() { private void add(DeviceTask deviceTask) { pendingDeviceTasks.add(deviceTask); - estimatedSizeInBytes += deviceTask.ramBytesUsed(); + unreservedBytes += deviceTask.ramBytesUsed(); } private void flush(Comparator comparator) { @@ -266,7 +272,7 @@ private void flush(Comparator comparator) { deviceEntryIndexes.add(deviceTask.deviceEntryIndex); } pendingDeviceTasks.clear(); - estimatedSizeInBytes = 0; + releaseDeviceTaskMemory(); } private void sortPendingDeviceTasks(Comparator comparator) { @@ -286,8 +292,37 @@ private void sortPendingDeviceTasks(Comparator comparator) { } } - private long getEstimatedSizeInBytes() { - return estimatedSizeInBytes; + private boolean shouldFlush() { + if (getPendingMemoryBytes() >= DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES) { + return true; + } + if (unreservedBytes < MEMORY_RESERVE_BATCH_SIZE_IN_BYTES) { + return false; + } + return !reserveUnreservedMemory(); + } + + private boolean reserveUnreservedMemory() { + try { + queryContext.reserveMemoryForFrontEndImmediately(unreservedBytes); + } catch (MemoryNotEnoughException e) { + return false; + } + reservedBytes += unreservedBytes; + unreservedBytes = 0; + return true; + } + + private long getPendingMemoryBytes() { + return reservedBytes + unreservedBytes; + } + + private void releaseDeviceTaskMemory() { + if (reservedBytes != 0) { + queryContext.releaseMemoryReservedForFrontEnd(reservedBytes); + reservedBytes = 0; + } + unreservedBytes = 0; } private boolean hasDeviceTasks() { From 96b1077b2b7305c81b285a4ddbbf41823f3b6ef9 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Thu, 11 Jun 2026 16:11:27 +0800 Subject: [PATCH 18/32] remove seralize/deserialize from ReadTsFileTableFunction --- .../readTsFile/ReadTsFileTableFunction.java | 62 ++----------------- 1 file changed, 4 insertions(+), 58 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java index 26f7143595db5..bb8e8a8148347 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java @@ -45,8 +45,6 @@ import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -477,41 +475,14 @@ public List getOutputColumnCategories() { @Override public byte[] serialize() { - ByteBuffer buffer = ByteBuffer.allocate(calculateSerializeSize()); - writeString(buffer, tableName); - buffer.putInt(tsFilePaths.size()); - tsFilePaths.forEach(path -> writeString(buffer, path)); - buffer.putInt(outputColumnNames.size()); - for (int i = 0; i < outputColumnNames.size(); i++) { - writeString(buffer, outputColumnNames.get(i)); - buffer.put(outputColumnTypes.get(i).getType()); - buffer.put(outputColumnCategories.get(i).getCategory()); - } - return buffer.array(); + throw new UnsupportedOperationException( + "ReadTsFileTableFunctionHandle does not support serialization"); } @Override public void deserialize(byte[] bytes) { - ByteBuffer buffer = ByteBuffer.wrap(bytes); - tableName = readString(buffer); - int size = buffer.getInt(); - List paths = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - paths.add(readString(buffer)); - } - tsFilePaths = Collections.unmodifiableList(paths); - size = buffer.getInt(); - List columnNames = new ArrayList<>(size); - List columnTypes = new ArrayList<>(size); - List columnCategories = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - columnNames.add(readString(buffer)); - columnTypes.add(Type.valueOf(buffer.get())); - columnCategories.add(TsTableColumnCategory.deserialize(buffer.get())); - } - outputColumnNames = Collections.unmodifiableList(columnNames); - outputColumnTypes = Collections.unmodifiableList(columnTypes); - outputColumnCategories = Collections.unmodifiableList(columnCategories); + throw new UnsupportedOperationException( + "ReadTsFileTableFunctionHandle does not support deserialization"); } @Override @@ -530,30 +501,5 @@ public String toString() { + outputColumnCategories + '}'; } - - private int calculateSerializeSize() { - int size = Integer.BYTES + tableName.getBytes(StandardCharsets.UTF_8).length; - size += Integer.BYTES; - for (String path : tsFilePaths) { - size += Integer.BYTES + path.getBytes(StandardCharsets.UTF_8).length; - } - size += Integer.BYTES; - for (String columnName : outputColumnNames) { - size += Integer.BYTES + columnName.getBytes(StandardCharsets.UTF_8).length + 2 * Byte.BYTES; - } - return size; - } - - private static void writeString(ByteBuffer buffer, String value) { - byte[] bytes = value.getBytes(StandardCharsets.UTF_8); - buffer.putInt(bytes.length); - buffer.put(bytes); - } - - private static String readString(ByteBuffer buffer) { - byte[] bytes = new byte[buffer.getInt()]; - buffer.get(bytes); - return new String(bytes, StandardCharsets.UTF_8); - } } } From 5e7bc0af91559f7c67f2672c8b52399aae4da48c Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Fri, 12 Jun 2026 11:10:56 +0800 Subject: [PATCH 19/32] fix mem control --- .../ExternalTsFileAggTableScanOperator.java | 3 +- .../ExternalTsFileTableScanOperator.java | 3 +- .../ExternalTsFileQueryDataSource.java | 28 ++++--------------- .../ExternalTsFileQueryResource.java | 13 ++++++--- 4 files changed, 18 insertions(+), 29 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java index f6e4a84fc7515..a3e7566fc9374 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; @@ -118,7 +119,7 @@ private boolean hasCurrentRealDeviceEntry() { && deviceEntries.get(currentDeviceIndex) != null; } - private ExternalTsFileQueryDataSource updateCurrentDeviceQueryDataSource() { + private QueryDataSource updateCurrentDeviceQueryDataSource() { try { if (!deviceTaskReader.nextDevice()) { throw new IllegalStateException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java index 73a9cb11ad58b..59d072b1362a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceTaskRunReader; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; @@ -105,7 +106,7 @@ protected void moveToNextDevice() { } } - private ExternalTsFileQueryDataSource updateCurrentDeviceQueryDataSource() { + private QueryDataSource updateCurrentDeviceQueryDataSource() { try { if (!deviceTaskReader.nextDevice()) { throw new IllegalStateException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java index ac55a8cbfe256..fd18da081c70d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java @@ -21,47 +21,29 @@ import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; - -import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.filter.basic.Filter; import java.util.Collections; -import java.util.List; public class ExternalTsFileQueryDataSource extends QueryDataSource { private final ExternalTsFileQueryResource externalTsFileQueryResource; public ExternalTsFileQueryDataSource(ExternalTsFileQueryResource externalTsFileQueryResource) { - this(externalTsFileQueryResource, Collections.emptyList()); - } - - ExternalTsFileQueryDataSource( - ExternalTsFileQueryResource externalTsFileQueryResource, - List unseqResources) { - super(Collections.emptyList(), unseqResources); + super(Collections.emptyList(), Collections.emptyList()); this.externalTsFileQueryResource = externalTsFileQueryResource; } - @Override - public IQueryDataSource clone() { - return new ExternalTsFileQueryDataSource(externalTsFileQueryResource, getUnseqResources()); - } - public ExternalTsFileQueryResource getExternalTsFileQueryResource() { return externalTsFileQueryResource; } @Override - public boolean isSeqSatisfied( - IDeviceID deviceID, int curIndex, Filter timeFilter, boolean debug) { - return true; + public IQueryDataSource clone() { + return new ExternalTsFileQueryDataSource(externalTsFileQueryResource); } @Override - public boolean isUnSeqSatisfied( - IDeviceID deviceID, int curIndex, Filter timeFilter, boolean debug) { - return true; + public boolean isEmpty() { + return externalTsFileQueryResource.getTsFileResources().isEmpty(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 74f39f736139a..9e3208a725132 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileDeviceFilterVisitor; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; @@ -112,6 +113,7 @@ public ExternalTsFileQueryResource( public void collectDeviceEntries( SchemaFilter schemaFilter, Comparator comparator, int partitionCount) { checkNotClosed(); + acquireMemoryForTsFileReaders(); ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = new ExternalTsFileDeviceFilterVisitor(); try (DeviceCollector deviceCollector = new DeviceCollector()) { createDeviceTaskPartitions(partitionCount); @@ -139,6 +141,10 @@ public void collectDeviceEntries( } } + private void acquireMemoryForTsFileReaders() { + queryContext.reserveMemoryForFrontEndImmediately((long) tsFilePaths.size() * 4 * 1024); + } + public DeviceTaskRunReader getDeviceTaskRunReader(int partitionIndex) { checkNotClosed(); DeviceTaskPartition partition = getDeviceTaskPartition(partitionIndex); @@ -403,7 +409,7 @@ public class DeviceTaskRunReader implements AutoCloseable { private final PriorityQueue runCursors; private DeviceEntry currentDevice; - private ExternalTsFileQueryDataSource currentDeviceQueryDataSource; + private QueryDataSource currentDeviceQueryDataSource; private Map currentDeviceOffsetMap; private DeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { @@ -452,8 +458,7 @@ public boolean nextDevice() throws IOException { unseqResources.add(tsFileResource); currentDeviceOffsetMap.put(tsFileResource, deviceOffset); } - currentDeviceQueryDataSource = - new ExternalTsFileQueryDataSource(ExternalTsFileQueryResource.this, unseqResources); + currentDeviceQueryDataSource = new QueryDataSource(Collections.emptyList(), unseqResources); return true; } @@ -461,7 +466,7 @@ public DeviceEntry getCurrentDevice() { return currentDevice; } - public ExternalTsFileQueryDataSource getCurrentDeviceQueryDataSource() { + public QueryDataSource getCurrentDeviceQueryDataSource() { return currentDeviceQueryDataSource; } From f06a3648d01112e2cb8b418b87b30d0589057150 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Mon, 15 Jun 2026 10:48:04 +0800 Subject: [PATCH 20/32] refactor --- .../ExternalTsFileQueryResource.java | 1 + .../readTsFile/ReadTsFileTableFunction.java | 232 +------------- .../tvf/readTsFile/TsFileSchemaCollector.java | 287 ++++++++++++++++++ 3 files changed, 297 insertions(+), 223 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 9e3208a725132..49de26e2dffc5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -459,6 +459,7 @@ public boolean nextDevice() throws IOException { currentDeviceOffsetMap.put(tsFileResource, deviceOffset); } currentDeviceQueryDataSource = new QueryDataSource(Collections.emptyList(), unseqResources); + currentDeviceQueryDataSource.setSingleDevice(true); return true; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java index bb8e8a8148347..e027ffd3cf177 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java @@ -35,31 +35,19 @@ import org.apache.iotdb.udf.api.relational.table.specification.ScalarParameterSpecification; import org.apache.iotdb.udf.api.type.Type; -import org.apache.tsfile.common.conf.TSFileConfig; -import org.apache.tsfile.enums.ColumnCategory; -import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.TableSchema; -import org.apache.tsfile.read.TsFileSequenceReader; import org.apache.tsfile.write.schema.IMeasurementSchema; -import org.apache.tsfile.write.schema.MeasurementSchema; import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; /** Reads one or more TsFiles as a table function source. */ public class ReadTsFileTableFunction implements TableFunction { @@ -83,23 +71,25 @@ public TableFunctionAnalysis analyze(Map arguments) throws UDF List tsFilePaths = parseTsFilePaths(getRequiredStringArgument(arguments, PATHS_PARAMETER_NAME)); checkTsFilePathsAreOutsideDataDirs(tsFilePaths); - TsFileSchemaCollection schemaCollection = - collectTsFilesAndResolveSchema(tableName.isEmpty() ? null : tableName, tsFilePaths); - if (schemaCollection.mergedTableSchema == null) { + TsFileSchemaCollector schemaCollector = + new TsFileSchemaCollector(tableName.isEmpty() ? null : tableName); + schemaCollector.collect(tsFilePaths); + TableSchema mergedTableSchema = schemaCollector.getMergedTableSchema(); + if (mergedTableSchema == null) { throw new UDFArgumentNotValidException( tableName.isEmpty() ? "No table schema found in TsFiles" : "No table schema found for table " + tableName + " in TsFiles"); } - DescribedSchema outputSchema = convertToDescribedSchema(schemaCollection.mergedTableSchema); + DescribedSchema outputSchema = convertToDescribedSchema(mergedTableSchema); ReadTsFileTableFunctionHandle handle = new ReadTsFileTableFunctionHandle( - schemaCollection.tableName, - schemaCollection.tsFiles.stream() + schemaCollector.getTableName(), + schemaCollector.getTsFiles().stream() .map(File::getAbsolutePath) .collect(Collectors.toList()), - schemaCollection.mergedTableSchema.getColumnTypes().stream() + mergedTableSchema.getColumnTypes().stream() .map(TsTableColumnCategory::fromTsFileColumnCategory) .collect(Collectors.toList()), outputSchema); @@ -186,210 +176,6 @@ private static Path normalizePath(String path) { } } - private static TsFileSchemaCollection collectTsFilesAndResolveSchema( - String specifiedTableName, List tsFilePaths) { - List tsFiles = new ArrayList<>(); - MergedTableSchemaBuilder schemaBuilder = null; - String resolvedTableName = - specifiedTableName == null ? null : specifiedTableName.toLowerCase(Locale.ENGLISH); - for (String tsFilePath : tsFilePaths) { - Path path = new File(tsFilePath).toPath(); - if (!Files.exists(path)) { - throw new UDFArgumentNotValidException("TsFile path does not exist: " + tsFilePath); - } - try (Stream walkedPaths = Files.walk(path)) { - Iterator iterator = walkedPaths.filter(Files::isRegularFile).iterator(); - while (iterator.hasNext()) { - Path filePath = iterator.next(); - TableSchema tableSchema = - specifiedTableName == null - ? tryInferTableSchema(filePath.toFile()) - : tryReadTableSchema(specifiedTableName, filePath.toFile()); - if (tableSchema == null) { - continue; - } - String currentTableName = tableSchema.getTableName().toLowerCase(Locale.ENGLISH); - if (resolvedTableName == null) { - resolvedTableName = currentTableName; - } else if (!resolvedTableName.equals(currentTableName)) { - throw new UDFArgumentNotValidException( - String.format( - "Cannot infer table name from TsFiles because multiple tables are found: %s and %s", - resolvedTableName, currentTableName)); - } - tsFiles.add(filePath.toFile()); - if (schemaBuilder == null) { - schemaBuilder = new MergedTableSchemaBuilder(resolvedTableName, tableSchema); - } else { - schemaBuilder.merge(tableSchema); - } - } - } catch (IOException e) { - throw new UDFArgumentNotValidException("Failed to scan TsFile path: " + tsFilePath); - } - } - if (tsFiles.isEmpty()) { - throw new UDFArgumentNotValidException("No valid TsFiles found"); - } - return new TsFileSchemaCollection(resolvedTableName, tsFiles, schemaBuilder.build()); - } - - private static TableSchema tryReadTableSchema(String tableName, File tsFile) { - if (!tsFile.canRead()) { - return null; - } - try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { - if (!reader.isComplete() || reader.readVersionNumber() != TSFileConfig.VERSION_NUMBER) { - return null; - } - Map tableSchemaMap = reader.getTableSchemaMap(); - return tableSchemaMap.get(tableName.toLowerCase(Locale.ENGLISH)); - } catch (Exception e) { - return null; - } - } - - private static TableSchema tryInferTableSchema(File tsFile) { - if (!tsFile.canRead()) { - return null; - } - try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { - if (!reader.isComplete() || reader.readVersionNumber() != TSFileConfig.VERSION_NUMBER) { - return null; - } - Map tableSchemaMap = reader.getTableSchemaMap(); - if (tableSchemaMap.isEmpty()) { - throw new UDFArgumentNotValidException( - "Cannot infer table name from TsFile because no table schema is found in " - + tsFile.getAbsolutePath()); - } - if (tableSchemaMap.size() > 1) { - throw new UDFArgumentNotValidException( - "Cannot infer table name from TsFile because multiple tables are found in " - + tsFile.getAbsolutePath()); - } - return tableSchemaMap.values().iterator().next(); - } catch (UDFArgumentNotValidException e) { - throw e; - } catch (Exception e) { - return null; - } - } - - private static class TsFileSchemaCollection { - private final String tableName; - private final List tsFiles; - private final TableSchema mergedTableSchema; - - private TsFileSchemaCollection( - String tableName, List tsFiles, TableSchema mergedTableSchema) { - this.tableName = tableName; - this.tsFiles = tsFiles; - this.mergedTableSchema = mergedTableSchema; - } - } - - private static class MergedTableSchemaBuilder { - private final String tableName; - private IMeasurementSchema timeColumnSchema; - private final List tagColumnSchemas = new ArrayList<>(); - private final Map fieldColumnSchemaMap = new LinkedHashMap<>(); - - private MergedTableSchemaBuilder(String tableName, TableSchema tableSchema) { - this.tableName = tableName.toLowerCase(Locale.ENGLISH); - merge(tableSchema); - } - - private void merge(TableSchema tableSchema) { - IMeasurementSchema currentTimeColumn = null; - List currentTagColumns = new ArrayList<>(); - List currentFieldColumns = new ArrayList<>(); - List columnSchemas = tableSchema.getColumnSchemas(); - List columnCategories = tableSchema.getColumnTypes(); - - for (int i = 0; i < columnCategories.size(); i++) { - if (columnCategories.get(i) == ColumnCategory.TIME) { - if (currentTimeColumn != null) { - throw new UDFArgumentNotValidException( - "Multiple time columns found when merging table schema for table " + tableName); - } - currentTimeColumn = columnSchemas.get(i); - } else if (columnCategories.get(i) == ColumnCategory.TAG) { - currentTagColumns.add(columnSchemas.get(i)); - } else if (columnCategories.get(i) == ColumnCategory.FIELD) { - currentFieldColumns.add(columnSchemas.get(i)); - } - } - - mergeTimeColumn(currentTimeColumn); - mergeTagColumns(currentTagColumns); - mergeFieldColumns(currentFieldColumns); - } - - private void mergeTimeColumn(IMeasurementSchema currentTimeColumn) { - if (currentTimeColumn == null) { - return; - } - if (timeColumnSchema == null) { - timeColumnSchema = currentTimeColumn; - return; - } - if (!timeColumnSchema.getMeasurementName().equals(currentTimeColumn.getMeasurementName()) - || currentTimeColumn.getType() != TSDataType.TIMESTAMP) { - throw new UDFArgumentNotValidException( - "Time column conflicts when merging table schema for table " + tableName); - } - } - - private void mergeTagColumns(List currentTagColumns) { - int prefixLength = Math.min(tagColumnSchemas.size(), currentTagColumns.size()); - for (int i = 0; i < prefixLength; i++) { - if (!tagColumnSchemas - .get(i) - .getMeasurementName() - .equals(currentTagColumns.get(i).getMeasurementName())) { - throw new UDFArgumentNotValidException( - "Tag columns conflict when merging table schema for table " + tableName); - } - } - tagColumnSchemas.addAll(currentTagColumns.subList(prefixLength, currentTagColumns.size())); - } - - private void mergeFieldColumns(List currentFieldColumns) { - for (IMeasurementSchema fieldColumn : currentFieldColumns) { - String fieldName = fieldColumn.getMeasurementName().toLowerCase(Locale.ENGLISH); - IMeasurementSchema existingColumn = fieldColumnSchemaMap.get(fieldName); - if (existingColumn != null && existingColumn.getType() != fieldColumn.getType()) { - throw new UDFArgumentNotValidException( - "Field column " - + fieldColumn.getMeasurementName() - + " has conflicting data types when merging table schema for table " - + tableName); - } - fieldColumnSchemaMap.putIfAbsent(fieldName, fieldColumn); - } - } - - private TableSchema build() { - List columnSchemas = new ArrayList<>(); - List columnCategories = new ArrayList<>(); - columnSchemas.add( - timeColumnSchema == null - ? new MeasurementSchema(TIME_COLUMN_NAME, TSDataType.TIMESTAMP) - : timeColumnSchema); - columnCategories.add(ColumnCategory.TIME); - for (IMeasurementSchema tagColumnSchema : tagColumnSchemas) { - columnSchemas.add(tagColumnSchema); - columnCategories.add(ColumnCategory.TAG); - } - for (IMeasurementSchema fieldColumnSchema : fieldColumnSchemaMap.values()) { - columnSchemas.add(fieldColumnSchema); - columnCategories.add(ColumnCategory.FIELD); - } - return new TableSchema(tableName, columnSchemas, columnCategories); - } - } - private static DescribedSchema convertToDescribedSchema(TableSchema tableSchema) { DescribedSchema.Builder builder = DescribedSchema.builder(); for (IMeasurementSchema columnSchema : tableSchema.getColumnSchemas()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java new file mode 100644 index 0000000000000..aa2ccdeab8bc1 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java @@ -0,0 +1,287 @@ +/* + * 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.function.tvf.readTsFile; + +import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.common.constant.TsFileConstant; +import org.apache.tsfile.enums.ColumnCategory; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.TableSchema; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; + +final class TsFileSchemaCollector { + private final String specifiedTableName; + private String tableName; + private final List tsFiles = new ArrayList<>(); + private MergedTableSchemaBuilder schemaBuilder; + private TableSchema mergedTableSchema; + + TsFileSchemaCollector(String specifiedTableName) { + this.specifiedTableName = + specifiedTableName == null ? null : specifiedTableName.toLowerCase(Locale.ENGLISH); + this.tableName = this.specifiedTableName; + } + + void collect(List tsFilePaths) { + for (String tsFilePath : tsFilePaths) { + Path path = new File(tsFilePath).toPath(); + if (!Files.exists(path)) { + throw new UDFArgumentNotValidException("TsFile path does not exist: " + tsFilePath); + } + if (Files.isRegularFile(path)) { + TableSchema tableSchema = readTableSchema(specifiedTableName, path.toFile(), true); + collect(path.toFile(), tableSchema); + continue; + } + if (!Files.isDirectory(path)) { + throw new UDFArgumentNotValidException( + "TsFile path is neither a file nor a directory: " + tsFilePath); + } + collectFromDirectory(tsFilePath, path); + } + if (tsFiles.isEmpty()) { + throw new UDFArgumentNotValidException("No valid TsFiles found"); + } + } + + String getTableName() { + return tableName; + } + + List getTsFiles() { + return tsFiles; + } + + TableSchema getMergedTableSchema() { + return mergedTableSchema; + } + + private void collectFromDirectory(String tsFilePath, Path path) { + try (Stream walkedPaths = Files.walk(path)) { + Iterator iterator = + walkedPaths.filter(Files::isRegularFile).filter(this::hasTsFileSuffix).iterator(); + while (iterator.hasNext()) { + Path filePath = iterator.next(); + TableSchema tableSchema = readTableSchema(specifiedTableName, filePath.toFile(), false); + collect(filePath.toFile(), tableSchema); + } + } catch (IOException e) { + throw new UDFArgumentNotValidException("Failed to scan TsFile path: " + tsFilePath); + } + } + + private boolean hasTsFileSuffix(Path filePath) { + Path fileName = filePath.getFileName(); + return fileName != null && fileName.toString().endsWith(TsFileConstant.TSFILE_SUFFIX); + } + + private void collect(File tsFile, TableSchema tableSchema) { + if (tableSchema == null) { + return; + } + String currentTableName = tableSchema.getTableName().toLowerCase(Locale.ENGLISH); + if (tableName == null) { + tableName = currentTableName; + } else if (!tableName.equals(currentTableName)) { + throw new UDFArgumentNotValidException( + String.format( + "Cannot infer table name from TsFiles because multiple tables are found: %s and %s", + tableName, currentTableName)); + } + tsFiles.add(tsFile); + if (schemaBuilder == null) { + schemaBuilder = new MergedTableSchemaBuilder(tableName, tableSchema); + } else { + schemaBuilder.merge(tableSchema); + } + mergedTableSchema = schemaBuilder.build(); + } + + private TableSchema readTableSchema( + String specifiedTableName, File tsFile, boolean failOnInvalidTsFile) { + if (!tsFile.canRead()) { + if (failOnInvalidTsFile) { + throw invalidTsFileException(tsFile); + } + return null; + } + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + if (!isValidTsFile(reader)) { + if (failOnInvalidTsFile) { + throw invalidTsFileException(tsFile); + } + return null; + } + Map tableSchemaMap = reader.getTableSchemaMap(); + if (specifiedTableName != null) { + return tableSchemaMap.get(specifiedTableName.toLowerCase(Locale.ENGLISH)); + } + if (tableSchemaMap.isEmpty()) { + throw new UDFArgumentNotValidException( + "Cannot infer table name from TsFile because no table schema is found in " + + tsFile.getAbsolutePath()); + } + if (tableSchemaMap.size() > 1) { + throw new UDFArgumentNotValidException( + "Cannot infer table name from TsFile because multiple tables are found in " + + tsFile.getAbsolutePath()); + } + return tableSchemaMap.values().iterator().next(); + } catch (UDFArgumentNotValidException e) { + throw e; + } catch (Exception e) { + if (failOnInvalidTsFile) { + throw invalidTsFileException(tsFile); + } + return null; + } + } + + private boolean isValidTsFile(TsFileSequenceReader reader) throws IOException { + return TSFileConfig.MAGIC_STRING.equals(reader.readHeadMagic()) + && reader.readVersionNumber() == TSFileConfig.VERSION_NUMBER + && TSFileConfig.MAGIC_STRING.equals(reader.readTailMagic()); + } + + private UDFArgumentNotValidException invalidTsFileException(File tsFile) { + return new UDFArgumentNotValidException( + "File is not a valid TsFile: " + tsFile.getAbsolutePath()); + } + + private static class MergedTableSchemaBuilder { + private final String tableName; + private IMeasurementSchema timeColumnSchema; + private final List tagColumnSchemas = new ArrayList<>(); + private final Map fieldColumnSchemaMap = new LinkedHashMap<>(); + + private MergedTableSchemaBuilder(String tableName, TableSchema tableSchema) { + this.tableName = tableName.toLowerCase(Locale.ENGLISH); + merge(tableSchema); + } + + private void merge(TableSchema tableSchema) { + IMeasurementSchema currentTimeColumn = null; + List currentTagColumns = new ArrayList<>(); + List currentFieldColumns = new ArrayList<>(); + List columnSchemas = tableSchema.getColumnSchemas(); + List columnCategories = tableSchema.getColumnTypes(); + + for (int i = 0; i < columnCategories.size(); i++) { + if (columnCategories.get(i) == ColumnCategory.TIME) { + if (currentTimeColumn != null) { + throw new UDFArgumentNotValidException( + "Multiple time columns found when merging table schema for table " + tableName); + } + currentTimeColumn = columnSchemas.get(i); + } else if (columnCategories.get(i) == ColumnCategory.TAG) { + currentTagColumns.add(columnSchemas.get(i)); + } else if (columnCategories.get(i) == ColumnCategory.FIELD) { + currentFieldColumns.add(columnSchemas.get(i)); + } + } + + mergeTimeColumn(currentTimeColumn); + mergeTagColumns(currentTagColumns); + mergeFieldColumns(currentFieldColumns); + } + + private void mergeTimeColumn(IMeasurementSchema currentTimeColumn) { + if (currentTimeColumn == null) { + return; + } + if (timeColumnSchema == null) { + timeColumnSchema = currentTimeColumn; + return; + } + if (!timeColumnSchema.getMeasurementName().equals(currentTimeColumn.getMeasurementName()) + || currentTimeColumn.getType() != TSDataType.TIMESTAMP) { + throw new UDFArgumentNotValidException( + "Time column conflicts when merging table schema for table " + tableName); + } + } + + private void mergeTagColumns(List currentTagColumns) { + int prefixLength = Math.min(tagColumnSchemas.size(), currentTagColumns.size()); + for (int i = 0; i < prefixLength; i++) { + if (!tagColumnSchemas + .get(i) + .getMeasurementName() + .equals(currentTagColumns.get(i).getMeasurementName())) { + throw new UDFArgumentNotValidException( + "Tag columns conflict when merging table schema for table " + tableName); + } + } + tagColumnSchemas.addAll(currentTagColumns.subList(prefixLength, currentTagColumns.size())); + } + + private void mergeFieldColumns(List currentFieldColumns) { + for (IMeasurementSchema fieldColumn : currentFieldColumns) { + String fieldName = fieldColumn.getMeasurementName().toLowerCase(Locale.ENGLISH); + IMeasurementSchema existingColumn = fieldColumnSchemaMap.get(fieldName); + if (existingColumn != null + && !existingColumn.getType().isCompatible(fieldColumn.getType())) { + throw new UDFArgumentNotValidException( + "Field column " + + fieldColumn.getMeasurementName() + + " has conflicting data types when merging table schema for table " + + tableName); + } + fieldColumnSchemaMap.putIfAbsent(fieldName, fieldColumn); + } + } + + private TableSchema build() { + List columnSchemas = new ArrayList<>(); + List columnCategories = new ArrayList<>(); + columnSchemas.add( + timeColumnSchema == null + ? new MeasurementSchema(TIME_COLUMN_NAME, TSDataType.TIMESTAMP) + : timeColumnSchema); + columnCategories.add(ColumnCategory.TIME); + for (IMeasurementSchema tagColumnSchema : tagColumnSchemas) { + columnSchemas.add(tagColumnSchema); + columnCategories.add(ColumnCategory.TAG); + } + for (IMeasurementSchema fieldColumnSchema : fieldColumnSchemaMap.values()) { + columnSchemas.add(fieldColumnSchema); + columnCategories.add(ColumnCategory.FIELD); + } + return new TableSchema(tableName, columnSchemas, columnCategories); + } + } +} From ec2f68ede93e4489baf94f40d064f2c9322e412b Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Mon, 15 Jun 2026 15:30:33 +0800 Subject: [PATCH 21/32] refactor & add it --- .../IoTDBReadTsFileTableFunctionIT.java | 101 +++++++++++++++- .../fragment/FragmentInstanceContext.java | 2 +- .../ExternalTsFileQueryDataSource.java | 2 +- .../ExternalTsFileQueryResource.java | 109 ++++++++++-------- .../ExternalTsFileAggregationScanNode.java | 3 +- .../planner/node/ExternalTsFileScanNode.java | 3 +- 6 files changed, 164 insertions(+), 56 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java index ed236b901ac52..c033491e95b0c 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java @@ -209,6 +209,40 @@ public void testReadMultipleTsFilesWithSchemaMerge() throws Exception { DATABASE_NAME); } + @Test + public void testReadMultipleTsFilesWithTagSchemaMerge() throws Exception { + File tsFile1 = new File(tmpDir, "tag-schema-merge-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "tag-schema-merge-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable(writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1"), 3, 4); + } + + String[] expectedHeader = new String[] {"time", "tag1", "tag2", "s1"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,null,1,", + "1970-01-01T00:00:00.002Z,tag1_1,null,2,", + "1970-01-01T00:00:00.003Z,tag1_1,tag2_1,3,", + "1970-01-01T00:00:00.004Z,tag1_1,tag2_1,4,", + "1970-01-01T00:00:00.001Z,tag1_2,null,1,", + "1970-01-01T00:00:00.002Z,tag1_2,null,2,", + "1970-01-01T00:00:00.003Z,tag1_2,tag2_2,3,", + "1970-01-01T00:00:00.004Z,tag1_2,tag2_2,4,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, tag2, s1 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1') ORDER BY tag1, time", + expectedHeader, + retArray, + DATABASE_NAME); + } + @Test public void testReadMultipleTsFilesWithConflictingFieldType() throws Exception { File tsFile1 = new File(tmpDir, "conflict-1.tsfile"); @@ -266,6 +300,53 @@ public void testReadTsFileWithoutTableNameWhenMultipleTablesExist() throws Excep DATABASE_NAME); } + @Test + public void testReadSpecifiedInvalidFileFails() throws IOException { + File invalidFile = new File(tmpDir, "invalid-file.txt"); + Files.write(invalidFile.toPath(), new byte[] {1, 2, 3}); + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(invalidFile) + "')", + "not a valid TsFile", + DATABASE_NAME); + } + + @Test + public void testReadDirectoryOnlyScansValidTsFileSuffixFiles() throws Exception { + File scanDir = new File(tmpDir, "scan-dir"); + File nestedDir = new File(scanDir, "nested"); + Files.createDirectories(nestedDir.toPath()); + + File validTsFile = new File(nestedDir, "valid.tsfile"); + try (TsFileWriter writer = new TsFileWriter(validTsFile)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + + File invalidTsFile = new File(scanDir, "invalid.tsfile"); + Files.write(invalidTsFile.toPath(), new byte[] {1, 2, 3}); + + File validFileWithoutTsFileSuffix = new File(scanDir, "valid.data"); + try (TsFileWriter writer = new TsFileWriter(validFileWithoutTsFileSuffix)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 3, 4); + } + + String[] expectedHeader = new String[] {"time", "tag1", "s1"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,1,", + "1970-01-01T00:00:00.002Z,tag1_1,2,", + "1970-01-01T00:00:00.001Z,tag1_2,1,", + "1970-01-01T00:00:00.002Z,tag1_2,2,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, s1 FROM read_tsfile(PATHS => '" + + toSqlPath(scanDir) + + "', TABLE_NAME => 'table1') ORDER BY tag1, time", + expectedHeader, + retArray, + DATABASE_NAME); + } + @Test public void testReadTsFileWithInvalidPaths() throws IOException { File missingFile = new File(tmpDir, "missing.tsfile"); @@ -365,13 +446,25 @@ private static void clearTmpDir() { File[] files = tmpDir.listFiles(); if (files != null) { for (File file : files) { - try { - Files.delete(file.toPath()); - } catch (IOException ignored) { - // ignore + deleteRecursively(file); + } + } + } + + private static void deleteRecursively(File file) { + if (file.isDirectory()) { + File[] children = file.listFiles(); + if (children != null) { + for (File child : children) { + deleteRecursively(child); } } } + try { + Files.delete(file.toPath()); + } catch (IOException ignored) { + // ignore + } } private static void deleteTmpDir() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index ac6969d45534f..c5df6082ab5f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -656,7 +656,7 @@ public boolean initExternalTsFileQueryDataSource( } this.sharedQueryDataSource = new ExternalTsFileQueryDataSource(externalTsFileQueryResource); - closedUnseqFileNum = externalTsFileQueryResource.getTsFileResources().size(); + closedUnseqFileNum = externalTsFileQueryResource.getSharedTsFileResources().size(); return true; } finally { addInitQueryDataSourceCost(System.nanoTime() - startTime); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java index fd18da081c70d..c9b8a08ea0f59 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryDataSource.java @@ -44,6 +44,6 @@ public IQueryDataSource clone() { @Override public boolean isEmpty() { - return externalTsFileQueryResource.getTsFileResources().isEmpty(); + return externalTsFileQueryResource.getSharedTsFileResources().isEmpty(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 49de26e2dffc5..4a7ce9f89b38e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -51,6 +51,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -59,6 +60,7 @@ import java.util.List; import java.util.Map; import java.util.PriorityQueue; +import java.util.Queue; import java.util.function.LongConsumer; import static java.util.Objects.requireNonNull; @@ -74,14 +76,16 @@ public class ExternalTsFileQueryResource implements AutoCloseable { public static final String EXTERNAL_TSFILE_TMP_DIR = "external-tsfile"; + private static final long TSFILE_READER_MEMORY_RESERVE_SIZE_IN_BYTES = 4L * 1024; + private final QueryId queryId; private final MPPQueryContext queryContext; private final Path queryTempRoot; private final String tableName; private final List tsFilePaths; - private final List tsFileResources; + private final List sharedTsFileResources; private final LongConsumer ioSizeRecorder; - private final List deviceEntries = new ArrayList<>(); + private final List sharedDeviceEntries = new ArrayList<>(); private final List deviceTaskPartitions = new ArrayList<>(); private Comparator deviceEntryComparator; @@ -103,7 +107,7 @@ public ExternalTsFileQueryResource( this.tableName = tableName; this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(requireNonNull(tsFilePaths, "tsFilePaths"))); - this.tsFileResources = createTsFileResources(this.tsFilePaths); + this.sharedTsFileResources = createTsFileResources(this.tsFilePaths); this.ioSizeRecorder = requireNonNull(ioSizeRecorder, "ioSizeRecorder is null"); for (String tsFilePath : tsFilePaths) { FileReaderManager.getInstance().increaseExternalFileReaderReference(tsFilePath); @@ -113,6 +117,7 @@ public ExternalTsFileQueryResource( public void collectDeviceEntries( SchemaFilter schemaFilter, Comparator comparator, int partitionCount) { checkNotClosed(); + deviceEntryComparator = comparator; acquireMemoryForTsFileReaders(); ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = new ExternalTsFileDeviceFilterVisitor(); try (DeviceCollector deviceCollector = new DeviceCollector()) { @@ -124,8 +129,8 @@ public void collectDeviceEntries( continue; } DeviceEntry deviceEntry = new AlignedDeviceEntry(deviceID, new Binary[0]); - int deviceEntryIndex = deviceEntries.size(); - deviceEntries.add(deviceEntry); + int deviceEntryIndex = sharedDeviceEntries.size(); + sharedDeviceEntries.add(deviceEntry); DeviceTask deviceTask = new DeviceTask(deviceEntryIndex, deviceCollector.getCurrentDeviceOffsets()); DeviceTaskPartition partition = @@ -133,16 +138,16 @@ public void collectDeviceEntries( Math.floorMod(deviceID.hashCode(), deviceTaskPartitions.size())); partition.add(deviceTask); if (partition.shouldFlush()) { - partition.flush(comparator); + partition.flush(); } } - deviceEntryComparator = comparator; - collectDeviceTaskPartitions(comparator); + collectDeviceTaskPartitions(); } } private void acquireMemoryForTsFileReaders() { - queryContext.reserveMemoryForFrontEndImmediately((long) tsFilePaths.size() * 4 * 1024); + queryContext.reserveMemoryForFrontEndImmediately( + tsFilePaths.size() * TSFILE_READER_MEMORY_RESERVE_SIZE_IN_BYTES); } public DeviceTaskRunReader getDeviceTaskRunReader(int partitionIndex) { @@ -159,12 +164,12 @@ public List getTsFilePaths() { return tsFilePaths; } - public List getTsFileResources() { - return tsFileResources; + public List getSharedTsFileResources() { + return sharedTsFileResources; } - public List getDeviceEntries() { - return deviceEntries; + public List getSharedDeviceEntries() { + return sharedDeviceEntries; } public List getDeviceTaskPartitions() { @@ -260,11 +265,11 @@ private void add(DeviceTask deviceTask) { unreservedBytes += deviceTask.ramBytesUsed(); } - private void flush(Comparator comparator) { + private void flush() { if (pendingDeviceTasks.isEmpty()) { return; } - sortPendingDeviceTasks(comparator); + sortPendingDeviceTasks(); try { runFiles.add( writeDeviceTaskRun( @@ -281,20 +286,20 @@ private void flush(Comparator comparator) { releaseDeviceTaskMemory(); } - private void sortPendingDeviceTasks(Comparator comparator) { - if (comparator != null) { + private void sortPendingDeviceTasks() { + if (deviceEntryComparator != null) { pendingDeviceTasks.sort( (left, right) -> - comparator.compare( - deviceEntries.get(left.deviceEntryIndex), - deviceEntries.get(right.deviceEntryIndex))); + deviceEntryComparator.compare( + sharedDeviceEntries.get(left.deviceEntryIndex), + sharedDeviceEntries.get(right.deviceEntryIndex))); } else { pendingDeviceTasks.sort( (left, right) -> - deviceEntries + sharedDeviceEntries .get(left.deviceEntryIndex) .getDeviceID() - .compareTo(deviceEntries.get(right.deviceEntryIndex).getDeviceID())); + .compareTo(sharedDeviceEntries.get(right.deviceEntryIndex).getDeviceID())); } } @@ -335,27 +340,29 @@ private boolean hasDeviceTasks() { return !deviceEntryIndexes.isEmpty(); } - private void finish(Comparator comparator) { + private void finish() { if (pendingDeviceTasks.isEmpty()) { return; } - sortPendingDeviceTasks(comparator); + sortPendingDeviceTasks(); for (DeviceTask deviceTask : pendingDeviceTasks) { deviceEntryIndexes.add(deviceTask.deviceEntryIndex); } } - private void sortDeviceEntries(Comparator comparator) { - if (comparator != null) { + private void sortDeviceEntries() { + if (deviceEntryComparator != null) { deviceEntryIndexes.sort( - (left, right) -> comparator.compare(deviceEntries.get(left), deviceEntries.get(right))); + (left, right) -> + deviceEntryComparator.compare( + sharedDeviceEntries.get(left), sharedDeviceEntries.get(right))); } else { deviceEntryIndexes.sort( (left, right) -> - deviceEntries + sharedDeviceEntries .get(left) .getDeviceID() - .compareTo(deviceEntries.get(right).getDeviceID())); + .compareTo(sharedDeviceEntries.get(right).getDeviceID())); } } @@ -378,16 +385,16 @@ private void createDeviceTaskPartitions(int partitionCount) { } } - private void collectDeviceTaskPartitions(Comparator comparator) { + private void collectDeviceTaskPartitions() { Iterator iterator = deviceTaskPartitions.iterator(); while (iterator.hasNext()) { DeviceTaskPartition partition = iterator.next(); - partition.finish(comparator); + partition.finish(); if (!partition.hasDeviceTasks()) { iterator.remove(); continue; } - partition.sortDeviceEntries(comparator); + partition.sortDeviceEntries(); } } @@ -407,23 +414,24 @@ private Path writeDeviceTaskRun(Path runRoot, int runIndex, List dev public class DeviceTaskRunReader implements AutoCloseable { - private final PriorityQueue runCursors; + private final Queue runCursors; + private final boolean usePriorityQueue; private DeviceEntry currentDevice; private QueryDataSource currentDeviceQueryDataSource; private Map currentDeviceOffsetMap; private DeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { - Comparator cursorComparator = - (left, right) -> - deviceEntryComparator == null - ? left.getCurrentDeviceEntry() - .getDeviceID() - .compareTo(right.getCurrentDeviceEntry().getDeviceID()) - : deviceEntryComparator.compare( - left.getCurrentDeviceEntry(), right.getCurrentDeviceEntry()); - this.runCursors = new PriorityQueue<>(cursorComparator); + Comparator comparator = deviceEntryComparator; + this.usePriorityQueue = comparator != null; + this.runCursors = + usePriorityQueue + ? new PriorityQueue<>( + (left, right) -> + comparator.compare( + left.getCurrentDeviceEntry(), right.getCurrentDeviceEntry())) + : new ArrayDeque<>(); for (Path runFile : partition.getRunFiles()) { - DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFile, deviceEntries); + DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFile, sharedDeviceEntries); if (cursor.hasCurrentDeviceTask()) { runCursors.add(cursor); } else { @@ -431,7 +439,7 @@ private DeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { } } DeviceTaskRunCursor memoryCursor = - new MemoryDeviceTaskRunCursor(partition.getPendingDeviceTasks(), deviceEntries); + new MemoryDeviceTaskRunCursor(partition.getPendingDeviceTasks(), sharedDeviceEntries); if (memoryCursor.hasCurrentDeviceTask()) { runCursors.add(memoryCursor); } @@ -441,20 +449,25 @@ public boolean nextDevice() throws IOException { if (runCursors.isEmpty()) { return false; } - DeviceTaskRunCursor cursor = runCursors.poll(); + DeviceTaskRunCursor cursor = usePriorityQueue ? runCursors.poll() : runCursors.peek(); DeviceTask result = cursor.getCurrentDeviceTask(); cursor.advance(); if (cursor.hasCurrentDeviceTask()) { - runCursors.add(cursor); + if (usePriorityQueue) { + runCursors.add(cursor); + } } else { + if (!usePriorityQueue) { + runCursors.poll(); + } cursor.close(); } - currentDevice = deviceEntries.get(result.deviceEntryIndex); + currentDevice = sharedDeviceEntries.get(result.deviceEntryIndex); List unseqResources = new ArrayList<>(result.deviceOffsets.size()); currentDeviceOffsetMap = new HashMap<>(result.deviceOffsets.size()); for (DeviceOffset deviceOffset : result.deviceOffsets) { - TsFileResource tsFileResource = tsFileResources.get(deviceOffset.getFileIndex()); + TsFileResource tsFileResource = sharedTsFileResources.get(deviceOffset.getFileIndex()); unseqResources.add(tsFileResource); currentDeviceOffsetMap.put(tsFileResource, deviceOffset); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java index dd338dc2d90dd..d3200ff987c2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java @@ -76,7 +76,8 @@ public ExternalTsFileAggregationScanNode( qualifiedObjectName, outputSymbols, assignments, - Lists.transform(deviceEntryIndexes, externalTsFileQueryResource.getDeviceEntries()::get), + Lists.transform( + deviceEntryIndexes, externalTsFileQueryResource.getSharedDeviceEntries()::get), tagAndAttributeIndexMap, scanOrder, timePredicate, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java index be83256c404b7..d9f66d4aa9371 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -80,7 +80,8 @@ public ExternalTsFileScanNode( qualifiedObjectName, outputSymbols, assignments, - Lists.transform(deviceEntryIndexes, externalTsFileQueryResource.getDeviceEntries()::get), + Lists.transform( + deviceEntryIndexes, externalTsFileQueryResource.getSharedDeviceEntries()::get), tagAndAttributeIndexMap, scanOrder, timePredicate, From 8587cbd3366987271a61d5ff240b28b3fdc4fe8b Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 12:04:34 +0800 Subject: [PATCH 22/32] add tests --- .../IoTDBReadTsFileTableFunctionIT.java | 42 ++++ .../source/AlignedSeriesScanUtil.java | 3 +- .../operator/source/FileLoaderUtils.java | 5 +- .../ExternalTsFileSeriesScanUtil.java | 4 +- .../ExternalTsFileQueryResource.java | 53 +++-- .../tvf/readTsFile/TsFileSchemaCollector.java | 24 ++- .../buffer/TimeSeriesMetadataCache.java | 9 +- .../ExternalTsFileQueryResourceTest.java | 193 ++++++++++++++++++ 8 files changed, 298 insertions(+), 35 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java index c033491e95b0c..870a37aedafda 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java @@ -286,6 +286,48 @@ public void testReadMultipleTsFilesWithConflictingTagColumns() throws Exception DATABASE_NAME); } + @Test + public void testReadMultipleTsFilesWithConflictingTagAndFieldColumns() throws Exception { + File tsFile1 = new File(tmpDir, "tag-field-conflict-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("shared"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "tag-field-conflict-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable(writer, "table1", new ArrayList<>(), Arrays.asList("shared"), 3, 4); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1')", + "conflicting categories when merging table schema", + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithConflictingFieldAndTagColumns() throws Exception { + File tsFile1 = new File(tmpDir, "field-tag-conflict-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", new ArrayList<>(), Arrays.asList("shared"), 1, 2); + } + File tsFile2 = new File(tmpDir, "field-tag-conflict-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable(writer, "table1", Arrays.asList("shared"), Arrays.asList("s1"), 3, 4); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1')", + "conflicting categories when merging table schema", + DATABASE_NAME); + } + @Test public void testReadTsFileWithoutTableNameWhenMultipleTablesExist() throws Exception { File tsFile = new File(tmpDir, "multiple-tables.tsfile"); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java index e80f305e73b92..331a5a5b267d1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java @@ -38,7 +38,6 @@ import java.io.IOException; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; public class AlignedSeriesScanUtil extends SeriesScanUtil { @@ -102,7 +101,7 @@ protected AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( scanOptions.getGlobalTimeFilter(), isSeq, ignoreAllNullRows, - Optional.empty()); + null); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java index 79235360cdf38..e5761050509ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java @@ -57,7 +57,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -200,7 +199,7 @@ public static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadata( Filter globalTimeFilter, boolean isSeq, boolean ignoreAllNullRows, - Optional rootMeasurementMetadataIndexNodeOffset) + long[] rootMeasurementMetadataIndexNodeOffset) throws IOException { final long t1 = System.nanoTime(); boolean loadFromMem = false; @@ -293,7 +292,7 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr FragmentInstanceContext context, Filter globalTimeFilter, boolean ignoreAllNullRows, - Optional rootMeasurementMetadataIndexNodeOffset) + long[] rootMeasurementMetadataIndexNodeOffset) throws IOException { AbstractAlignedTimeSeriesMetadata alignedTimeSeriesMetadata = null; // load all the TimeseriesMetadata of vector, the first one is for time column and the diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java index c22384160ae4d..7726a510a14ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java @@ -36,7 +36,6 @@ import java.io.IOException; import java.util.List; -import java.util.Optional; public class ExternalTsFileSeriesScanUtil extends AlignedSeriesScanUtil { @@ -84,8 +83,7 @@ static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( globalTimeFilter, resource.isSeq(), context.isIgnoreAllNullRows(), - Optional.of( - new long[] {currentDeviceOffset.getStartOffset(), currentDeviceOffset.getEndOffset()})); + new long[] {currentDeviceOffset.getStartOffset(), currentDeviceOffset.getEndOffset()}); } @FunctionalInterface diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 4a7ce9f89b38e..dfb1fd8b6259d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -20,12 +20,14 @@ package org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile; import org.apache.iotdb.calc.exception.MemoryNotEnoughException; -import org.apache.iotdb.commons.queryengine.execution.MemoryEstimationHelper; +import org.apache.iotdb.calc.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileDeviceFilterVisitor; +import org.apache.iotdb.db.queryengine.plan.planner.memory.ThreadSafeMemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; @@ -79,7 +81,9 @@ public class ExternalTsFileQueryResource implements AutoCloseable { private static final long TSFILE_READER_MEMORY_RESERVE_SIZE_IN_BYTES = 4L * 1024; private final QueryId queryId; - private final MPPQueryContext queryContext; + // This resource outlives the frontend planning phase, whose MPPQueryContext memory manager is + // released after dispatch. Keep a dedicated manager and release it when this resource closes. + private final MemoryReservationManager externalTsFileResourceMemoryReservationManager; private final Path queryTempRoot; private final String tableName; private final List tsFilePaths; @@ -98,8 +102,10 @@ public ExternalTsFileQueryResource( List tsFilePaths, LongConsumer ioSizeRecorder, boolean useExactTempRoot) { - this.queryContext = requireNonNull(queryContext, "queryContext is null"); - this.queryId = queryContext.getQueryId(); + this.queryId = requireNonNull(queryContext, "queryContext is null").getQueryId(); + this.externalTsFileResourceMemoryReservationManager = + new ThreadSafeMemoryReservationManager( + queryId, ExternalTsFileQueryResource.class.getName()); this.queryTempRoot = useExactTempRoot ? requireNonNull(tempRoot, "tempRoot is null") @@ -146,7 +152,7 @@ public void collectDeviceEntries( } private void acquireMemoryForTsFileReaders() { - queryContext.reserveMemoryForFrontEndImmediately( + externalTsFileResourceMemoryReservationManager.reserveMemoryImmediately( tsFilePaths.size() * TSFILE_READER_MEMORY_RESERVE_SIZE_IN_BYTES); } @@ -160,6 +166,11 @@ public DeviceTaskRunReader getDeviceTaskRunReader(int partitionIndex) { } } + @TestOnly + public void setDeviceEntryComparator(Comparator deviceEntryComparator) { + this.deviceEntryComparator = deviceEntryComparator; + } + public List getTsFilePaths() { return tsFilePaths; } @@ -193,10 +204,14 @@ public void close() { } closed = true; - releaseFileReaderReferences(); + try { + releaseFileReaderReferences(); - if (Files.exists(queryTempRoot)) { - FileUtils.deleteFileOrDirectory(queryTempRoot.toFile(), true); + if (Files.exists(queryTempRoot)) { + FileUtils.deleteFileOrDirectory(queryTempRoot.toFile(), true); + } + } finally { + externalTsFileResourceMemoryReservationManager.releaseAllReservedMemory(); } } @@ -248,7 +263,7 @@ public class DeviceTaskPartition { private long reservedBytes; private long unreservedBytes; - private DeviceTaskPartition(int partitionIndex) { + DeviceTaskPartition(int partitionIndex) { this.partitionIndex = partitionIndex; } @@ -260,12 +275,12 @@ public List getDeviceEntryIndexes() { return deviceEntryIndexes; } - private void add(DeviceTask deviceTask) { + void add(DeviceTask deviceTask) { pendingDeviceTasks.add(deviceTask); unreservedBytes += deviceTask.ramBytesUsed(); } - private void flush() { + void flush() { if (pendingDeviceTasks.isEmpty()) { return; } @@ -315,7 +330,7 @@ private boolean shouldFlush() { private boolean reserveUnreservedMemory() { try { - queryContext.reserveMemoryForFrontEndImmediately(unreservedBytes); + externalTsFileResourceMemoryReservationManager.reserveMemoryImmediately(unreservedBytes); } catch (MemoryNotEnoughException e) { return false; } @@ -330,7 +345,7 @@ private long getPendingMemoryBytes() { private void releaseDeviceTaskMemory() { if (reservedBytes != 0) { - queryContext.releaseMemoryReservedForFrontEnd(reservedBytes); + externalTsFileResourceMemoryReservationManager.releaseMemoryCumulatively(reservedBytes); reservedBytes = 0; } unreservedBytes = 0; @@ -340,7 +355,7 @@ private boolean hasDeviceTasks() { return !deviceEntryIndexes.isEmpty(); } - private void finish() { + void finish() { if (pendingDeviceTasks.isEmpty()) { return; } @@ -420,7 +435,7 @@ public class DeviceTaskRunReader implements AutoCloseable { private QueryDataSource currentDeviceQueryDataSource; private Map currentDeviceOffsetMap; - private DeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { + DeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { Comparator comparator = deviceEntryComparator; this.usePriorityQueue = comparator != null; this.runCursors = @@ -699,7 +714,7 @@ public void close() { } } - private static class DeviceTask implements Accountable { + static class DeviceTask implements Accountable { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(DeviceTask.class); @@ -707,7 +722,7 @@ private static class DeviceTask implements Accountable { private final int deviceEntryIndex; private final List deviceOffsets; - private DeviceTask(int deviceEntryIndex, List deviceOffsets) { + DeviceTask(int deviceEntryIndex, List deviceOffsets) { this.deviceEntryIndex = deviceEntryIndex; this.deviceOffsets = deviceOffsets; } @@ -738,7 +753,7 @@ private static DeviceTask deserialize(DataInputStream inputStream) throws IOExce @Override public long ramBytesUsed() { return INSTANCE_SIZE - + MemoryEstimationHelper.ARRAY_LIST_INSTANCE_SIZE + + RamUsageEstimator.shallowSizeOfInstance(ArrayList.class) + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * deviceOffsets.size() + deviceOffsets.size() * DeviceOffset.INSTANCE_SIZE; @@ -754,7 +769,7 @@ public static class DeviceOffset { private final long startOffset; private final long endOffset; - private DeviceOffset(int fileIndex, long startOffset, long endOffset) { + DeviceOffset(int fileIndex, long startOffset, long endOffset) { this.fileIndex = fileIndex; this.startOffset = startOffset; this.endOffset = endOffset; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java index aa2ccdeab8bc1..395b6c9d6ca61 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java @@ -188,6 +188,7 @@ private static class MergedTableSchemaBuilder { private IMeasurementSchema timeColumnSchema; private final List tagColumnSchemas = new ArrayList<>(); private final Map fieldColumnSchemaMap = new LinkedHashMap<>(); + private final Map columnCategoryMap = new LinkedHashMap<>(); private MergedTableSchemaBuilder(String tableName, TableSchema tableSchema) { this.tableName = tableName.toLowerCase(Locale.ENGLISH); @@ -202,15 +203,18 @@ private void merge(TableSchema tableSchema) { List columnCategories = tableSchema.getColumnTypes(); for (int i = 0; i < columnCategories.size(); i++) { - if (columnCategories.get(i) == ColumnCategory.TIME) { + ColumnCategory currentCategory = columnCategories.get(i); + if (currentCategory == ColumnCategory.TIME) { if (currentTimeColumn != null) { throw new UDFArgumentNotValidException( "Multiple time columns found when merging table schema for table " + tableName); } currentTimeColumn = columnSchemas.get(i); - } else if (columnCategories.get(i) == ColumnCategory.TAG) { + } else if (currentCategory == ColumnCategory.TAG) { + checkAndRecordColumnCategory(columnSchemas.get(i), currentCategory); currentTagColumns.add(columnSchemas.get(i)); - } else if (columnCategories.get(i) == ColumnCategory.FIELD) { + } else if (currentCategory == ColumnCategory.FIELD) { + checkAndRecordColumnCategory(columnSchemas.get(i), currentCategory); currentFieldColumns.add(columnSchemas.get(i)); } } @@ -265,6 +269,20 @@ private void mergeFieldColumns(List currentFieldColumns) { } } + private void checkAndRecordColumnCategory( + IMeasurementSchema columnSchema, ColumnCategory currentCategory) { + String columnName = columnSchema.getMeasurementName().toLowerCase(Locale.ENGLISH); + ColumnCategory existingCategory = columnCategoryMap.get(columnName); + if (existingCategory != null && existingCategory != currentCategory) { + throw new UDFArgumentNotValidException( + "Column " + + columnSchema.getMeasurementName() + + " has conflicting categories when merging table schema for table " + + tableName); + } + columnCategoryMap.putIfAbsent(columnName, currentCategory); + } + private TableSchema build() { List columnSchemas = new ArrayList<>(); List columnCategories = new ArrayList<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java index 1e1df3e3c5744..255cce7c738d2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java @@ -53,7 +53,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.WeakHashMap; import java.util.concurrent.atomic.AtomicLong; @@ -146,7 +145,7 @@ public TimeseriesMetadata get( boolean debug, QueryContext queryContext) throws IOException { - return get(filePath, key, allSensors, ignoreNotExists, debug, queryContext, Optional.empty()); + return get(filePath, key, allSensors, ignoreNotExists, debug, queryContext, null); } @SuppressWarnings({"squid:S1860", "squid:S6541", "squid:S3776"}) // Suppress synchronize warning @@ -157,7 +156,7 @@ public TimeseriesMetadata get( boolean ignoreNotExists, boolean debug, QueryContext queryContext, - Optional deviceMetadataIndexNodeOffset) + long[] deviceMetadataIndexNodeOffset) throws IOException { long startTime = System.nanoTime(); long loadBloomFilterTime = 0; @@ -176,7 +175,7 @@ public TimeseriesMetadata get( TsFileSequenceReader reader = FileReaderManager.getInstance() .get(filePath, key.tsFileID, true, bloomFilterIoSizeRecorder, externalTsFile); - if (!deviceMetadataIndexNodeOffset.isPresent()) { + if (deviceMetadataIndexNodeOffset == null) { BloomFilter bloomFilter = reader.readBloomFilter(bloomFilterIoSizeRecorder); queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount().incrementAndGet(); if (bloomFilter != null @@ -218,7 +217,7 @@ public TimeseriesMetadata get( if (timeseriesMetadata == null) { cacheHit = false; - if (!deviceMetadataIndexNodeOffset.isPresent()) { + if (deviceMetadataIndexNodeOffset == null) { long loadBloomFilterStartTime = System.nanoTime(); // bloom filter part BloomFilter bloomFilter = diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java new file mode 100644 index 0000000000000..c1db965364da6 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java @@ -0,0 +1,193 @@ +/* + * 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.function.tvf.readTsFile; + +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceOffset; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceTask; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceTaskPartition; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource.DeviceTaskRunReader; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.utils.Binary; +import org.junit.After; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ExternalTsFileQueryResourceTest { + + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private ExternalTsFileQueryResource resource; + + @After + public void tearDown() { + if (resource != null) { + resource.close(); + } + } + + @Test + public void testDeviceTaskRunReaderMergesMultipleRunsWithComparator() throws Exception { + resource = newResource("merge_comparator", Arrays.asList("file-0.tsfile", "file-1.tsfile")); + addDevices("d1", "d2", "d3", "d4", "d5"); + resource.setDeviceEntryComparator(Comparator.comparing(DeviceEntry::getDeviceID)); + DeviceTaskPartition partition = resource.new DeviceTaskPartition(0); + + partition.add(task(2, offset(0, 30, 39))); + partition.add(task(4, offset(1, 50, 59))); + partition.flush(); + partition.add(task(0, offset(0, 10, 19))); + partition.add(task(3, offset(1, 40, 49))); + partition.flush(); + partition.add(task(1, offset(0, 20, 29))); + partition.flush(); + + try (DeviceTaskRunReader reader = resource.new DeviceTaskRunReader(partition)) { + assertDeviceOrder(reader, "d1", "d2", "d3", "d4", "d5"); + } + } + + @Test + public void testDeviceTaskRunReaderReadsRunsInFifoOrderWithoutComparator() throws Exception { + resource = newResource("fifo", Collections.singletonList("file-0.tsfile")); + addDevices("d1", "d2", "d3", "d4"); + DeviceTaskPartition partition = resource.new DeviceTaskPartition(0); + + partition.add(task(2, offset(0, 30, 39))); + partition.flush(); + partition.add(task(0, offset(0, 10, 19))); + partition.add(task(1, offset(0, 20, 29))); + partition.flush(); + partition.add(task(3, offset(0, 40, 49))); + partition.flush(); + + try (DeviceTaskRunReader reader = resource.new DeviceTaskRunReader(partition)) { + assertDeviceOrder(reader, "d3", "d1", "d2", "d4"); + } + } + + @Test + public void testDeviceTaskRunReaderReadsDiskRunBeforePendingMemoryTasksWithoutComparator() + throws Exception { + resource = newResource("disk_memory", Collections.singletonList("file-0.tsfile")); + addDevices("d1", "d2", "d3"); + DeviceTaskPartition partition = resource.new DeviceTaskPartition(0); + + partition.add(task(1, offset(0, 20, 29))); + partition.flush(); + partition.add(task(0, offset(0, 10, 19))); + partition.add(task(2, offset(0, 30, 39))); + partition.finish(); + + try (DeviceTaskRunReader reader = resource.new DeviceTaskRunReader(partition)) { + assertDeviceOrder(reader, "d2", "d1", "d3"); + } + } + + @Test + public void testDeviceTaskRunReaderUsesSharedTsFileResourceAsOffsetMapKey() throws Exception { + resource = newResource("offset_map", Arrays.asList("file-0.tsfile", "file-1.tsfile")); + addDevices("d1"); + DeviceTaskPartition partition = resource.new DeviceTaskPartition(0); + partition.add(task(0, offset(0, 11, 22), offset(1, 33, 44))); + partition.finish(); + + try (DeviceTaskRunReader reader = resource.new DeviceTaskRunReader(partition)) { + assertTrue(reader.nextDevice()); + Map offsetMap = reader.getCurrentDeviceOffsetMap(); + List sharedResources = resource.getSharedTsFileResources(); + + assertEquals(2, offsetMap.size()); + assertTrue(offsetMap.containsKey(sharedResources.get(0))); + assertTrue(offsetMap.containsKey(sharedResources.get(1))); + assertOffset(offsetMap.get(sharedResources.get(0)), 0, 11, 22); + assertOffset(offsetMap.get(sharedResources.get(1)), 1, 33, 44); + assertEquals(sharedResources, reader.getCurrentDeviceQueryDataSource().getUnseqResources()); + assertFalse(reader.nextDevice()); + } + } + + private ExternalTsFileQueryResource newResource(String queryId, List fileNames) + throws Exception { + File root = temporaryFolder.newFolder(queryId); + List tsFilePaths = new ArrayList<>(fileNames.size()); + for (String fileName : fileNames) { + tsFilePaths.add(new File(root, fileName).getAbsolutePath()); + } + MPPQueryContext queryContext = new MPPQueryContext(new QueryId(queryId)); + return new ExternalTsFileQueryResource( + queryContext, root.toPath().resolve("tmp"), "table1", tsFilePaths, ignored -> {}, true); + } + + private void addDevices(String... deviceNames) { + for (String deviceName : deviceNames) { + resource + .getSharedDeviceEntries() + .add( + new AlignedDeviceEntry( + IDeviceID.Factory.DEFAULT_FACTORY.create(deviceName), new Binary[0])); + } + } + + private DeviceTask task(int deviceEntryIndex, DeviceOffset... offsets) { + return new DeviceTask(deviceEntryIndex, Arrays.asList(offsets)); + } + + private DeviceOffset offset(int fileIndex, long startOffset, long endOffset) { + return new DeviceOffset(fileIndex, startOffset, endOffset); + } + + private void assertDeviceOrder(DeviceTaskRunReader reader, String... expectedDeviceNames) + throws Exception { + for (String expectedDeviceName : expectedDeviceNames) { + assertTrue(reader.nextDevice()); + assertEquals(expectedDeviceName, reader.getCurrentDevice().getDeviceID().toString()); + } + assertFalse(reader.nextDevice()); + } + + private void assertOffset( + DeviceOffset offset, + int expectedFileIndex, + long expectedStartOffset, + long expectedEndOffset) { + assertEquals(expectedFileIndex, offset.getFileIndex()); + assertEquals(expectedStartOffset, offset.getStartOffset()); + assertEquals(expectedEndOffset, offset.getEndOffset()); + } +} From 3106fa17d1c2de8776c7a7774f3a92c4ca070151 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 14:06:50 +0800 Subject: [PATCH 23/32] fix i18n --- .../iotdb/db/i18n/DataNodeQueryMessages.java | 98 +++++++++++++++++++ .../iotdb/db/i18n/DataNodeQueryMessages.java | 97 ++++++++++++++++++ .../ExternalTsFileAggTableScanOperator.java | 11 ++- .../ExternalTsFileDeviceFilterVisitor.java | 8 +- .../ExternalTsFileTableScanOperator.java | 14 ++- .../ExternalTsFileQueryResource.java | 29 +++--- .../readTsFile/ReadTsFileTableFunction.java | 35 ++++--- .../tvf/readTsFile/TsFileSchemaCollector.java | 46 +++++---- .../relational/planner/RelationPlanner.java | 3 +- .../ExternalTsFileAggregationScanNode.java | 8 +- .../planner/node/ExternalTsFileScanNode.java | 8 +- .../PushPredicateIntoTableScan.java | 3 +- 12 files changed, 295 insertions(+), 65 deletions(-) diff --git a/iotdb-core/datanode/src/main/i18n/en/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java b/iotdb-core/datanode/src/main/i18n/en/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java index b63739fef9b06..252e415972be9 100644 --- a/iotdb-core/datanode/src/main/i18n/en/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java +++ b/iotdb-core/datanode/src/main/i18n/en/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java @@ -197,6 +197,21 @@ public final class DataNodeQueryMessages { "Failed to close reader in TableDiskUsageSupplier"; public static final String UNSUPPORTED_CATEGORY = "Unsupported category: "; + public static final String CURRENT_DEVICE_ENTRY_IN_TABLESCANOPERATOR_IS_EMPTY = + "Current device entry in TableScanOperator is empty"; + public static final String UNEXPECTED_END_OF_EXTERNAL_TSFILE_DEVICE_TASK_READER_AT_DEVICE_INDEX = + "Unexpected end of external TsFile device task reader at device index "; + public static final String + EXTERNAL_TSFILE_DEVICE_TASK_READER_IS_NOT_ALIGNED_WITH_DEVICE_ENTRIES = + "External TsFile device task reader is not aligned with device entries at index %d:" + + " expected %s but got %s"; + public static final String FAILED_TO_UPDATE_EXTERNAL_TSFILE_DEVICE_RESOURCES = + "Failed to update external TsFile device resources"; + public static final String SCHEMA_FILTER_TYPE_IS_NOT_SUPPORTED = + "The schema filter type %s is not supported"; + public static final String + ATTRIBUTE_FILTER_IS_NOT_SUPPORTED_FOR_EXTERNAL_TSFILE_DEVICE_FILTERING = + "Attribute filter is not supported for external TsFile device filtering"; // --- Execution / Operator / Window --- @@ -761,6 +776,77 @@ public final class DataNodeQueryMessages { "Tag column only support data type STRING."; public static final String ATTRIBUTE_COLUMN_ONLY_SUPPORT_DATA_TYPE_STRING = "Attribute column only support data type STRING."; + public static final String UNSUPPORTED_EXTERNAL_TSFILE_DEVICE_FILTER = + "Unsupported external TsFile device filter: "; + + // --- Plan / Relational / Table Function --- + + public static final String NO_TABLE_SCHEMA_FOUND_IN_TSFILES = + "No table schema found in TsFiles"; + public static final String NO_TABLE_SCHEMA_FOUND_FOR_TABLE_IN_TSFILES = + "No table schema found for table %s in TsFiles"; + public static final String READ_TSFILE_MUST_BE_PLANNED_AS_EXTERNAL_TSFILE_SCAN_NODE = + "readTsFile must be planned as an ExternalTsFileScanNode"; + public static final String MISSING_SCALAR_ARGUMENT = + "Missing scalar argument: "; + public static final String ARGUMENT_SHOULD_NOT_BE_EMPTY = + "Argument %s should not be empty"; + public static final String INVALID_SCALAR_ARGUMENT = + "Invalid scalar argument: "; + public static final String ARGUMENT_SHOULD_BE_A_STRING = + "Argument %s should be a string"; + public static final String ARGUMENT_SHOULD_CONTAIN_AT_LEAST_ONE_PATH = + "Argument %s should contain at least one path"; + public static final String READ_TSFILE_PATH_IS_NOT_ALLOWED = + "readTsFile path %s is not allowed because it may access IoTDB data directory %s"; + public static final String OUTPUT_COLUMN_NAMES_AND_TYPES_SIZE_MISMATCH = + "Output column names and types size mismatch"; + public static final String OUTPUT_COLUMN_NAMES_AND_CATEGORIES_SIZE_MISMATCH = + "Output column names and categories size mismatch"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_SERIALIZATION = + "ReadTsFileTableFunctionHandle does not support serialization"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_DESERIALIZATION = + "ReadTsFileTableFunctionHandle does not support deserialization"; + public static final String TSFILE_PATH_DOES_NOT_EXIST = + "TsFile path does not exist: "; + public static final String TSFILE_PATH_IS_NEITHER_A_FILE_NOR_A_DIRECTORY = + "TsFile path is neither a file nor a directory: "; + public static final String NO_VALID_TSFILES_FOUND = + "No valid TsFiles found"; + public static final String FAILED_TO_SCAN_TSFILE_PATH = + "Failed to scan TsFile path: "; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILES_MULTIPLE_TABLES = + "Cannot infer table name from TsFiles because multiple tables are found: %s and %s"; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILE_NO_TABLE_SCHEMA = + "Cannot infer table name from TsFile because no table schema is found in "; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILE_MULTIPLE_TABLES = + "Cannot infer table name from TsFile because multiple tables are found in "; + public static final String FILE_IS_NOT_A_VALID_TSFILE = + "File is not a valid TsFile: "; + public static final String MULTIPLE_TIME_COLUMNS_FOUND_WHEN_MERGING_TABLE_SCHEMA = + "Multiple time columns found when merging table schema for table "; + public static final String TIME_COLUMN_CONFLICTS_WHEN_MERGING_TABLE_SCHEMA = + "Time column conflicts when merging table schema for table "; + public static final String TAG_COLUMNS_CONFLICT_WHEN_MERGING_TABLE_SCHEMA = + "Tag columns conflict when merging table schema for table "; + public static final String FIELD_COLUMN_HAS_CONFLICTING_DATA_TYPES_WHEN_MERGING_TABLE_SCHEMA = + "Field column %s has conflicting data types when merging table schema for table %s"; + public static final String COLUMN_HAS_CONFLICTING_CATEGORIES_WHEN_MERGING_TABLE_SCHEMA = + "Column %s has conflicting categories when merging table schema for table %s"; + public static final String FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_TASK_RUN_READER = + "Failed to create external TsFile device task run reader"; + public static final String UNKNOWN_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION = + "Unknown external TsFile device task partition: "; + public static final String EXTERNAL_TSFILE_QUERY_RESOURCE_HAS_BEEN_CLOSED = + "External TsFile query resource has been closed: "; + public static final String FAILED_TO_DESERIALIZE_EXTERNAL_TSFILE_RESOURCE = + "Failed to deserialize external TsFile resource: %s, %s"; + public static final String FAILED_TO_FLUSH_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION = + "Failed to flush external TsFile device task partition"; + public static final String EXTERNAL_TSFILE_DEVICE_TASK_PARTITION_COUNT_MUST_BE_POSITIVE = + "External TsFile device task partition count must be positive"; + public static final String FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_COLLECTOR = + "Failed to create external TsFile device collector"; // --- Plan / Relational / Planner --- @@ -804,6 +890,8 @@ public final class DataNodeQueryMessages { "Values is not supported in current version."; public static final String SUBSCRIPT_IS_NOT_SUPPORTED_IN_CURRENT_VERSION = "Subscript is not supported in current version"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_IS_INVALID = + "readTsFile table function handle is invalid"; // --- Plan / Relational / Planner / IR --- @@ -833,6 +921,16 @@ public final class DataNodeQueryMessages { "Not supported yet."; public static final String COPYTONODE_SHOULD_NOT_BE_SERIALIZED = "CopyToNode should not be serialized"; + public static final String + EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES = + "ExternalTsFileAggregationScanNode device entries must be set by device entry indexes"; + public static final String EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_CANNOT_BE_SERIALIZED = + "ExternalTsFileAggregationScanNode cannot be serialized because it reads local external TsFiles"; + public static final String + EXTERNAL_TSFILE_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES = + "ExternalTsFileScanNode device entries must be set by device entry indexes"; + public static final String EXTERNAL_TSFILE_SCAN_NODE_CANNOT_BE_SERIALIZED = + "ExternalTsFileScanNode cannot be serialized because it reads local external TsFiles"; // --- Plan / Relational / Planner / Optimizations --- diff --git a/iotdb-core/datanode/src/main/i18n/zh/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java b/iotdb-core/datanode/src/main/i18n/zh/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java index e70f00f2db909..9f6e0ba30d49a 100644 --- a/iotdb-core/datanode/src/main/i18n/zh/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java +++ b/iotdb-core/datanode/src/main/i18n/zh/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java @@ -197,6 +197,20 @@ public final class DataNodeQueryMessages { "在 TableDiskUsageSupplier 中关闭 reader 失败"; public static final String UNSUPPORTED_CATEGORY = "不支持的列类别:"; + public static final String CURRENT_DEVICE_ENTRY_IN_TABLESCANOPERATOR_IS_EMPTY = + "TableScanOperator 中当前设备条目为空"; + public static final String UNEXPECTED_END_OF_EXTERNAL_TSFILE_DEVICE_TASK_READER_AT_DEVICE_INDEX = + "外部 TsFile 设备任务读取器在设备索引处意外结束:"; + public static final String + EXTERNAL_TSFILE_DEVICE_TASK_READER_IS_NOT_ALIGNED_WITH_DEVICE_ENTRIES = + "外部 TsFile 设备任务读取器与设备条目不匹配,索引 %d:期望 %s,实际 %s"; + public static final String FAILED_TO_UPDATE_EXTERNAL_TSFILE_DEVICE_RESOURCES = + "更新外部 TsFile 设备资源失败"; + public static final String SCHEMA_FILTER_TYPE_IS_NOT_SUPPORTED = + "不支持 SchemaFilter 类型 %s"; + public static final String + ATTRIBUTE_FILTER_IS_NOT_SUPPORTED_FOR_EXTERNAL_TSFILE_DEVICE_FILTERING = + "外部 TsFile 设备过滤暂不支持属性过滤"; // --- Execution / Operator / Window --- @@ -760,6 +774,77 @@ public final class DataNodeQueryMessages { "标签列仅支持 STRING 数据类型。"; public static final String ATTRIBUTE_COLUMN_ONLY_SUPPORT_DATA_TYPE_STRING = "属性列仅支持 STRING 数据类型。"; + public static final String UNSUPPORTED_EXTERNAL_TSFILE_DEVICE_FILTER = + "不支持的外部 TsFile 设备过滤器:"; + + // --- Plan / Relational / Table Function --- + + public static final String NO_TABLE_SCHEMA_FOUND_IN_TSFILES = + "TsFile 中未找到表结构"; + public static final String NO_TABLE_SCHEMA_FOUND_FOR_TABLE_IN_TSFILES = + "TsFile 中未找到表 %s 的表结构"; + public static final String READ_TSFILE_MUST_BE_PLANNED_AS_EXTERNAL_TSFILE_SCAN_NODE = + "readTsFile 必须规划为 ExternalTsFileScanNode"; + public static final String MISSING_SCALAR_ARGUMENT = + "缺少标量参数:"; + public static final String ARGUMENT_SHOULD_NOT_BE_EMPTY = + "参数 %s 不应为空"; + public static final String INVALID_SCALAR_ARGUMENT = + "无效的标量参数:"; + public static final String ARGUMENT_SHOULD_BE_A_STRING = + "参数 %s 应为字符串"; + public static final String ARGUMENT_SHOULD_CONTAIN_AT_LEAST_ONE_PATH = + "参数 %s 应至少包含一个路径"; + public static final String READ_TSFILE_PATH_IS_NOT_ALLOWED = + "不允许 readTsFile 路径 %s,因为它可能访问 IoTDB 数据目录 %s"; + public static final String OUTPUT_COLUMN_NAMES_AND_TYPES_SIZE_MISMATCH = + "输出列名和类型数量不匹配"; + public static final String OUTPUT_COLUMN_NAMES_AND_CATEGORIES_SIZE_MISMATCH = + "输出列名和类别数量不匹配"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_SERIALIZATION = + "ReadTsFileTableFunctionHandle 不支持序列化"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_DESERIALIZATION = + "ReadTsFileTableFunctionHandle 不支持反序列化"; + public static final String TSFILE_PATH_DOES_NOT_EXIST = + "TsFile 路径不存在:"; + public static final String TSFILE_PATH_IS_NEITHER_A_FILE_NOR_A_DIRECTORY = + "TsFile 路径既不是文件也不是目录:"; + public static final String NO_VALID_TSFILES_FOUND = + "未找到有效的 TsFile"; + public static final String FAILED_TO_SCAN_TSFILE_PATH = + "扫描 TsFile 路径失败:"; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILES_MULTIPLE_TABLES = + "无法从 TsFile 推断表名,因为发现了多个表:%s 和 %s"; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILE_NO_TABLE_SCHEMA = + "无法从 TsFile 推断表名,因为未找到表结构,文件:"; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILE_MULTIPLE_TABLES = + "无法从 TsFile 推断表名,因为发现了多个表,文件:"; + public static final String FILE_IS_NOT_A_VALID_TSFILE = + "文件不是有效的 TsFile:"; + public static final String MULTIPLE_TIME_COLUMNS_FOUND_WHEN_MERGING_TABLE_SCHEMA = + "合并表结构时发现多个时间列,表:"; + public static final String TIME_COLUMN_CONFLICTS_WHEN_MERGING_TABLE_SCHEMA = + "合并表结构时时间列冲突,表:"; + public static final String TAG_COLUMNS_CONFLICT_WHEN_MERGING_TABLE_SCHEMA = + "合并表结构时标签列冲突,表:"; + public static final String FIELD_COLUMN_HAS_CONFLICTING_DATA_TYPES_WHEN_MERGING_TABLE_SCHEMA = + "字段列 %s 在合并表 %s 的结构时存在冲突的数据类型"; + public static final String COLUMN_HAS_CONFLICTING_CATEGORIES_WHEN_MERGING_TABLE_SCHEMA = + "列 %s 在合并表 %s 的结构时存在冲突的类别"; + public static final String FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_TASK_RUN_READER = + "创建外部 TsFile 设备任务运行读取器失败"; + public static final String UNKNOWN_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION = + "未知的外部 TsFile 设备任务分区:"; + public static final String EXTERNAL_TSFILE_QUERY_RESOURCE_HAS_BEEN_CLOSED = + "外部 TsFile 查询资源已关闭:"; + public static final String FAILED_TO_DESERIALIZE_EXTERNAL_TSFILE_RESOURCE = + "反序列化外部 TsFile 资源失败:%s,%s"; + public static final String FAILED_TO_FLUSH_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION = + "刷写外部 TsFile 设备任务分区失败"; + public static final String EXTERNAL_TSFILE_DEVICE_TASK_PARTITION_COUNT_MUST_BE_POSITIVE = + "外部 TsFile 设备任务分区数量必须为正数"; + public static final String FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_COLLECTOR = + "创建外部 TsFile 设备收集器失败"; // --- Plan / Relational / Planner --- @@ -803,6 +888,8 @@ public final class DataNodeQueryMessages { "当前版本不支持 Values。"; public static final String SUBSCRIPT_IS_NOT_SUPPORTED_IN_CURRENT_VERSION = "当前版本不支持下标操作"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_IS_INVALID = + "readTsFile 表函数句柄无效"; // --- Plan / Relational / Planner / IR --- @@ -832,6 +919,16 @@ public final class DataNodeQueryMessages { "暂不支持。"; public static final String COPYTONODE_SHOULD_NOT_BE_SERIALIZED = "CopyToNode 不应被序列化"; + public static final String + EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES = + "ExternalTsFileAggregationScanNode 的设备条目必须通过设备条目索引设置"; + public static final String EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_CANNOT_BE_SERIALIZED = + "ExternalTsFileAggregationScanNode 读取本地外部 TsFile,因此不能被序列化"; + public static final String + EXTERNAL_TSFILE_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES = + "ExternalTsFileScanNode 的设备条目必须通过设备条目索引设置"; + public static final String EXTERNAL_TSFILE_SCAN_NODE_CANNOT_BE_SERIALIZED = + "ExternalTsFileScanNode 读取本地外部 TsFile,因此不能被序列化"; // --- Plan / Relational / Planner / Optimizations --- diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java index a3e7566fc9374..fe2cdfab6a861 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.SeriesScanUtil; import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryDataSource; @@ -123,7 +124,8 @@ private QueryDataSource updateCurrentDeviceQueryDataSource() { try { if (!deviceTaskReader.nextDevice()) { throw new IllegalStateException( - "Unexpected end of external TsFile device task reader at device index " + DataNodeQueryMessages + .UNEXPECTED_END_OF_EXTERNAL_TSFILE_DEVICE_TASK_READER_AT_DEVICE_INDEX + currentDeviceIndex); } DeviceEntry expectedDeviceEntry = deviceEntries.get(currentDeviceIndex); @@ -131,15 +133,16 @@ private QueryDataSource updateCurrentDeviceQueryDataSource() { if (!expectedDeviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { throw new IllegalStateException( String.format( - "External TsFile device task reader is not aligned with device entries at index %d:" - + " expected %s but got %s", + DataNodeQueryMessages + .EXTERNAL_TSFILE_DEVICE_TASK_READER_IS_NOT_ALIGNED_WITH_DEVICE_ENTRIES, currentDeviceIndex, expectedDeviceEntry.getDeviceID(), currentDeviceEntry.getDeviceID())); } return deviceTaskReader.getCurrentDeviceQueryDataSource(); } catch (IOException e) { - throw new RuntimeException("Failed to update external TsFile device resources", e); + throw new RuntimeException( + DataNodeQueryMessages.FAILED_TO_UPDATE_EXTERNAL_TSFILE_DEVICE_RESOURCES, e); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java index 005491d2b6ef7..d26713ae7dbed 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.schema.filter.impl.StringValueFilterVisitor; import org.apache.iotdb.commons.schema.filter.impl.singlechild.AttributeFilter; import org.apache.iotdb.commons.schema.filter.impl.singlechild.TagFilter; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.tsfile.file.metadata.IDeviceID; @@ -32,7 +33,9 @@ public class ExternalTsFileDeviceFilterVisitor extends SchemaFilterVisitor createTsFileResources(List tsFilePat resource.deserialize(); } catch (IOException e) { throw new RuntimeException( - "Failed to deserialize external TsFile resource: " - + tsFilePath - + ", " - + e.getMessage(), + String.format( + DataNodeQueryMessages.FAILED_TO_DESERIALIZE_EXTERNAL_TSFILE_RESOURCE, + tsFilePath, + e.getMessage()), e); } } else { @@ -292,7 +295,8 @@ void flush() { runFiles.size(), pendingDeviceTasks)); } catch (IOException e) { - throw new RuntimeException("Failed to flush external TsFile device task partition", e); + throw new RuntimeException( + DataNodeQueryMessages.FAILED_TO_FLUSH_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION, e); } for (DeviceTask deviceTask : pendingDeviceTasks) { deviceEntryIndexes.add(deviceTask.deviceEntryIndex); @@ -393,7 +397,7 @@ private List getPendingDeviceTasks() { private void createDeviceTaskPartitions(int partitionCount) { if (partitionCount <= 0) { throw new IllegalArgumentException( - "External TsFile device task partition count must be positive"); + DataNodeQueryMessages.EXTERNAL_TSFILE_DEVICE_TASK_PARTITION_COUNT_MUST_BE_POSITIVE); } for (int i = 0; i < partitionCount; i++) { deviceTaskPartitions.add(new DeviceTaskPartition(i)); @@ -643,7 +647,8 @@ private DeviceCollector() { } } catch (IOException e) { close(); - throw new RuntimeException("Failed to create external TsFile device collector", e); + throw new RuntimeException( + DataNodeQueryMessages.FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_COLLECTOR, e); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java index e027ffd3cf177..c078115908f49 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; import org.apache.iotdb.udf.api.exception.UDFException; import org.apache.iotdb.udf.api.relational.TableFunction; @@ -78,8 +79,9 @@ public TableFunctionAnalysis analyze(Map arguments) throws UDF if (mergedTableSchema == null) { throw new UDFArgumentNotValidException( tableName.isEmpty() - ? "No table schema found in TsFiles" - : "No table schema found for table " + tableName + " in TsFiles"); + ? DataNodeQueryMessages.NO_TABLE_SCHEMA_FOUND_IN_TSFILES + : String.format( + DataNodeQueryMessages.NO_TABLE_SCHEMA_FOUND_FOR_TABLE_IN_TSFILES, tableName)); } DescribedSchema outputSchema = convertToDescribedSchema(mergedTableSchema); @@ -106,17 +108,18 @@ public TableFunctionHandle createTableFunctionHandle() { public TableFunctionProcessorProvider getProcessorProvider( TableFunctionHandle tableFunctionHandle) { throw new UnsupportedOperationException( - "readTsFile must be planned as an ExternalTsFileScanNode"); + DataNodeQueryMessages.READ_TSFILE_MUST_BE_PLANNED_AS_EXTERNAL_TSFILE_SCAN_NODE); } private static String getRequiredStringArgument(Map arguments, String name) { Argument argument = arguments.get(name); if (!(argument instanceof ScalarArgument)) { - throw new UDFArgumentNotValidException("Missing scalar argument: " + name); + throw new UDFArgumentNotValidException(DataNodeQueryMessages.MISSING_SCALAR_ARGUMENT + name); } Object value = ((ScalarArgument) argument).getValue(); if (!(value instanceof String) || ((String) value).trim().isEmpty()) { - throw new UDFArgumentNotValidException("Argument " + name + " should not be empty"); + throw new UDFArgumentNotValidException( + String.format(DataNodeQueryMessages.ARGUMENT_SHOULD_NOT_BE_EMPTY, name)); } return ((String) value).trim(); } @@ -127,11 +130,12 @@ private static String getOptionalStringArgument(Map arguments, return ""; } if (!(argument instanceof ScalarArgument)) { - throw new UDFArgumentNotValidException("Invalid scalar argument: " + name); + throw new UDFArgumentNotValidException(DataNodeQueryMessages.INVALID_SCALAR_ARGUMENT + name); } Object value = ((ScalarArgument) argument).getValue(); if (!(value instanceof String)) { - throw new UDFArgumentNotValidException("Argument " + name + " should be a string"); + throw new UDFArgumentNotValidException( + String.format(DataNodeQueryMessages.ARGUMENT_SHOULD_BE_A_STRING, name)); } return ((String) value).trim(); } @@ -144,7 +148,9 @@ private static List parseTsFilePaths(String tsFilePaths) { .collect(Collectors.toList()); if (paths.isEmpty()) { throw new UDFArgumentNotValidException( - "Argument " + PATHS_PARAMETER_NAME + " should contain at least one path"); + String.format( + DataNodeQueryMessages.ARGUMENT_SHOULD_CONTAIN_AT_LEAST_ONE_PATH, + PATHS_PARAMETER_NAME)); } return paths; } @@ -160,8 +166,7 @@ private static void checkTsFilePathsAreOutsideDataDirs(List tsFilePaths) if (normalizedTsFilePath.startsWith(dataDir) || dataDir.startsWith(normalizedTsFilePath)) { throw new UDFArgumentNotValidException( String.format( - "readTsFile path %s is not allowed because it may access IoTDB data directory %s", - tsFilePath, dataDir)); + DataNodeQueryMessages.READ_TSFILE_PATH_IS_NOT_ALLOWED, tsFilePath, dataDir)); } } } @@ -226,10 +231,12 @@ private ReadTsFileTableFunctionHandle( List outputColumnTypes, List outputColumnCategories) { if (outputColumnNames.size() != outputColumnTypes.size()) { - throw new IllegalArgumentException("Output column names and types size mismatch"); + throw new IllegalArgumentException( + DataNodeQueryMessages.OUTPUT_COLUMN_NAMES_AND_TYPES_SIZE_MISMATCH); } if (outputColumnNames.size() != outputColumnCategories.size()) { - throw new IllegalArgumentException("Output column names and categories size mismatch"); + throw new IllegalArgumentException( + DataNodeQueryMessages.OUTPUT_COLUMN_NAMES_AND_CATEGORIES_SIZE_MISMATCH); } this.tableName = tableName; this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); @@ -262,13 +269,13 @@ public List getOutputColumnCategories() { @Override public byte[] serialize() { throw new UnsupportedOperationException( - "ReadTsFileTableFunctionHandle does not support serialization"); + DataNodeQueryMessages.READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_SERIALIZATION); } @Override public void deserialize(byte[] bytes) { throw new UnsupportedOperationException( - "ReadTsFileTableFunctionHandle does not support deserialization"); + DataNodeQueryMessages.READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_DESERIALIZATION); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java index 395b6c9d6ca61..090eda825efd4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; import org.apache.tsfile.common.conf.TSFileConfig; @@ -61,7 +62,8 @@ void collect(List tsFilePaths) { for (String tsFilePath : tsFilePaths) { Path path = new File(tsFilePath).toPath(); if (!Files.exists(path)) { - throw new UDFArgumentNotValidException("TsFile path does not exist: " + tsFilePath); + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.TSFILE_PATH_DOES_NOT_EXIST + tsFilePath); } if (Files.isRegularFile(path)) { TableSchema tableSchema = readTableSchema(specifiedTableName, path.toFile(), true); @@ -70,12 +72,12 @@ void collect(List tsFilePaths) { } if (!Files.isDirectory(path)) { throw new UDFArgumentNotValidException( - "TsFile path is neither a file nor a directory: " + tsFilePath); + DataNodeQueryMessages.TSFILE_PATH_IS_NEITHER_A_FILE_NOR_A_DIRECTORY + tsFilePath); } collectFromDirectory(tsFilePath, path); } if (tsFiles.isEmpty()) { - throw new UDFArgumentNotValidException("No valid TsFiles found"); + throw new UDFArgumentNotValidException(DataNodeQueryMessages.NO_VALID_TSFILES_FOUND); } } @@ -101,7 +103,8 @@ private void collectFromDirectory(String tsFilePath, Path path) { collect(filePath.toFile(), tableSchema); } } catch (IOException e) { - throw new UDFArgumentNotValidException("Failed to scan TsFile path: " + tsFilePath); + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.FAILED_TO_SCAN_TSFILE_PATH + tsFilePath); } } @@ -120,8 +123,9 @@ private void collect(File tsFile, TableSchema tableSchema) { } else if (!tableName.equals(currentTableName)) { throw new UDFArgumentNotValidException( String.format( - "Cannot infer table name from TsFiles because multiple tables are found: %s and %s", - tableName, currentTableName)); + DataNodeQueryMessages.CANNOT_INFER_TABLE_NAME_FROM_TSFILES_MULTIPLE_TABLES, + tableName, + currentTableName)); } tsFiles.add(tsFile); if (schemaBuilder == null) { @@ -153,12 +157,12 @@ private TableSchema readTableSchema( } if (tableSchemaMap.isEmpty()) { throw new UDFArgumentNotValidException( - "Cannot infer table name from TsFile because no table schema is found in " + DataNodeQueryMessages.CANNOT_INFER_TABLE_NAME_FROM_TSFILE_NO_TABLE_SCHEMA + tsFile.getAbsolutePath()); } if (tableSchemaMap.size() > 1) { throw new UDFArgumentNotValidException( - "Cannot infer table name from TsFile because multiple tables are found in " + DataNodeQueryMessages.CANNOT_INFER_TABLE_NAME_FROM_TSFILE_MULTIPLE_TABLES + tsFile.getAbsolutePath()); } return tableSchemaMap.values().iterator().next(); @@ -180,7 +184,7 @@ private boolean isValidTsFile(TsFileSequenceReader reader) throws IOException { private UDFArgumentNotValidException invalidTsFileException(File tsFile) { return new UDFArgumentNotValidException( - "File is not a valid TsFile: " + tsFile.getAbsolutePath()); + DataNodeQueryMessages.FILE_IS_NOT_A_VALID_TSFILE + tsFile.getAbsolutePath()); } private static class MergedTableSchemaBuilder { @@ -207,7 +211,8 @@ private void merge(TableSchema tableSchema) { if (currentCategory == ColumnCategory.TIME) { if (currentTimeColumn != null) { throw new UDFArgumentNotValidException( - "Multiple time columns found when merging table schema for table " + tableName); + DataNodeQueryMessages.MULTIPLE_TIME_COLUMNS_FOUND_WHEN_MERGING_TABLE_SCHEMA + + tableName); } currentTimeColumn = columnSchemas.get(i); } else if (currentCategory == ColumnCategory.TAG) { @@ -235,7 +240,7 @@ private void mergeTimeColumn(IMeasurementSchema currentTimeColumn) { if (!timeColumnSchema.getMeasurementName().equals(currentTimeColumn.getMeasurementName()) || currentTimeColumn.getType() != TSDataType.TIMESTAMP) { throw new UDFArgumentNotValidException( - "Time column conflicts when merging table schema for table " + tableName); + DataNodeQueryMessages.TIME_COLUMN_CONFLICTS_WHEN_MERGING_TABLE_SCHEMA + tableName); } } @@ -247,7 +252,7 @@ private void mergeTagColumns(List currentTagColumns) { .getMeasurementName() .equals(currentTagColumns.get(i).getMeasurementName())) { throw new UDFArgumentNotValidException( - "Tag columns conflict when merging table schema for table " + tableName); + DataNodeQueryMessages.TAG_COLUMNS_CONFLICT_WHEN_MERGING_TABLE_SCHEMA + tableName); } } tagColumnSchemas.addAll(currentTagColumns.subList(prefixLength, currentTagColumns.size())); @@ -260,10 +265,11 @@ private void mergeFieldColumns(List currentFieldColumns) { if (existingColumn != null && !existingColumn.getType().isCompatible(fieldColumn.getType())) { throw new UDFArgumentNotValidException( - "Field column " - + fieldColumn.getMeasurementName() - + " has conflicting data types when merging table schema for table " - + tableName); + String.format( + DataNodeQueryMessages + .FIELD_COLUMN_HAS_CONFLICTING_DATA_TYPES_WHEN_MERGING_TABLE_SCHEMA, + fieldColumn.getMeasurementName(), + tableName)); } fieldColumnSchemaMap.putIfAbsent(fieldName, fieldColumn); } @@ -275,10 +281,10 @@ private void checkAndRecordColumnCategory( ColumnCategory existingCategory = columnCategoryMap.get(columnName); if (existingCategory != null && existingCategory != currentCategory) { throw new UDFArgumentNotValidException( - "Column " - + columnSchema.getMeasurementName() - + " has conflicting categories when merging table schema for table " - + tableName); + String.format( + DataNodeQueryMessages.COLUMN_HAS_CONFLICTING_CATEGORIES_WHEN_MERGING_TABLE_SCHEMA, + columnSchema.getMeasurementName(), + tableName)); } columnCategoryMap.putIfAbsent(columnName, currentCategory); } 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 54c239c0fef9e..5578b2bc7dd7a 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 @@ -1603,7 +1603,8 @@ private RelationPlan planExternalTsFileScan( TableFunctionInvocation node, TableFunctionInvocationAnalysis functionAnalysis) { if (!(functionAnalysis.getTableFunctionHandle() instanceof ReadTsFileTableFunction.ReadTsFileTableFunctionHandle)) { - throw new IllegalStateException("readTsFile table function handle is invalid"); + throw new IllegalStateException( + DataNodeQueryMessages.READ_TSFILE_TABLE_FUNCTION_HANDLE_IS_INVALID); } ReadTsFileTableFunction.ReadTsFileTableFunctionHandle handle = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java index d3200ff987c2d..7996248d4ea8b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java @@ -28,6 +28,7 @@ import org.apache.iotdb.commons.queryengine.plan.relational.planner.node.AggregationNode; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; @@ -149,7 +150,8 @@ public int getDeviceTaskPartitionIndex() { @Override public void setDeviceEntries(List deviceEntries) { throw new UnsupportedOperationException( - "ExternalTsFileAggregationScanNode device entries must be set by device entry indexes"); + DataNodeQueryMessages + .EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES); } public SchemaFilter getSchemaFilter() { @@ -163,13 +165,13 @@ public void setSchemaFilter(SchemaFilter schemaFilter) { @Override protected void serializeAttributes(ByteBuffer byteBuffer) { throw new UnsupportedOperationException( - "ExternalTsFileAggregationScanNode cannot be serialized because it reads local external TsFiles"); + DataNodeQueryMessages.EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_CANNOT_BE_SERIALIZED); } @Override protected void serializeAttributes(DataOutputStream stream) throws IOException { throw new UnsupportedOperationException( - "ExternalTsFileAggregationScanNode cannot be serialized because it reads local external TsFiles"); + DataNodeQueryMessages.EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_CANNOT_BE_SERIALIZED); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java index d9f66d4aa9371..fddcd3b66fa3c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -26,6 +26,7 @@ import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ExternalTsFileQueryResource; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; @@ -140,7 +141,8 @@ public int getDeviceTaskPartitionIndex() { @Override public void setDeviceEntries(List deviceEntries) { throw new UnsupportedOperationException( - "ExternalTsFileScanNode device entries must be set by device entry indexes"); + DataNodeQueryMessages + .EXTERNAL_TSFILE_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES); } public SchemaFilter getSchemaFilter() { @@ -154,13 +156,13 @@ public void setSchemaFilter(SchemaFilter schemaFilter) { @Override protected void serializeAttributes(ByteBuffer byteBuffer) { throw new UnsupportedOperationException( - "ExternalTsFileScanNode cannot be serialized because it reads local external TsFiles"); + DataNodeQueryMessages.EXTERNAL_TSFILE_SCAN_NODE_CANNOT_BE_SERIALIZED); } @Override protected void serializeAttributes(DataOutputStream stream) throws IOException { throw new UnsupportedOperationException( - "ExternalTsFileScanNode cannot be serialized because it reads local external TsFiles"); + DataNodeQueryMessages.EXTERNAL_TSFILE_SCAN_NODE_CANNOT_BE_SERIALIZED); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 5ffac0473bafb..3f5927e9bb8c4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -55,6 +55,7 @@ import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet; @@ -580,7 +581,7 @@ private SchemaFilter constructExternalTsFileDeviceFilter( new ConvertSchemaPredicateToFilterVisitor.Context(table)); if (deviceFilter == null) { throw new UnsupportedOperationException( - "Unsupported external TsFile device filter: " + predicate); + DataNodeQueryMessages.UNSUPPORTED_EXTERNAL_TSFILE_DEVICE_FILTER + predicate); } return deviceFilter; } From 2be51e712a019b00efb5702bf339ce3a50ca1cfc Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 15:26:10 +0800 Subject: [PATCH 24/32] fix --- .../queryengine/common/MPPQueryContext.java | 4 +- .../ExternalTsFileQueryResource.java | 40 ++++--------------- .../ExternalTsFileQueryResourceTest.java | 2 +- pom.xml | 2 +- 4 files changed, 10 insertions(+), 38 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index a19e72f41ddbd..6ec9fca5946ec 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -269,9 +269,7 @@ public ExternalTsFileQueryResource createExternalTsFileQueryResource( .resolve(queryId.getId()) .resolve(String.valueOf(externalTsFileQueryResources.size())), tableName, - tsFilePaths, - ignored -> {}, - true); + tsFilePaths); externalTsFileQueryResources.add(externalTsFileQueryResource); return externalTsFileQueryResource; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 0d81d36ae3760..56eaed315ac53 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -64,7 +64,6 @@ import java.util.Map; import java.util.PriorityQueue; import java.util.Queue; -import java.util.function.LongConsumer; import static java.util.Objects.requireNonNull; @@ -89,7 +88,6 @@ public class ExternalTsFileQueryResource implements AutoCloseable { private final String tableName; private final List tsFilePaths; private final List sharedTsFileResources; - private final LongConsumer ioSizeRecorder; private final List sharedDeviceEntries = new ArrayList<>(); private final List deviceTaskPartitions = new ArrayList<>(); private Comparator deviceEntryComparator; @@ -97,25 +95,15 @@ public class ExternalTsFileQueryResource implements AutoCloseable { private volatile boolean closed; public ExternalTsFileQueryResource( - MPPQueryContext queryContext, - Path tempRoot, - String tableName, - List tsFilePaths, - LongConsumer ioSizeRecorder, - boolean useExactTempRoot) { + MPPQueryContext queryContext, Path tempRoot, String tableName, List tsFilePaths) { this.queryId = requireNonNull(queryContext, "queryContext is null").getQueryId(); this.externalTsFileResourceMemoryReservationManager = new NotThreadSafeMemoryReservationManager( queryId, ExternalTsFileQueryResource.class.getName()); - this.queryTempRoot = - useExactTempRoot - ? requireNonNull(tempRoot, "tempRoot is null") - : requireNonNull(tempRoot, "tempRoot is null").resolve(this.queryId.getId()); + this.queryTempRoot = requireNonNull(tempRoot, "tempRoot is null"); this.tableName = tableName; - this.tsFilePaths = - Collections.unmodifiableList(new ArrayList<>(requireNonNull(tsFilePaths, "tsFilePaths"))); + this.tsFilePaths = requireNonNull(tsFilePaths, "tsFilePaths"); this.sharedTsFileResources = createTsFileResources(this.tsFilePaths); - this.ioSizeRecorder = requireNonNull(ioSizeRecorder, "ioSizeRecorder is null"); for (String tsFilePath : tsFilePaths) { FileReaderManager.getInstance().increaseExternalFileReaderReference(tsFilePath); } @@ -124,7 +112,7 @@ public ExternalTsFileQueryResource( public void collectDeviceEntries( SchemaFilter schemaFilter, Comparator comparator, int partitionCount) { checkNotClosed(); - deviceEntryComparator = comparator; + this.deviceEntryComparator = comparator; acquireMemoryForTsFileReaders(); ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = new ExternalTsFileDeviceFilterVisitor(); try (DeviceCollector deviceCollector = new DeviceCollector()) { @@ -235,20 +223,7 @@ private static List createTsFileResources(List tsFilePat for (String tsFilePath : tsFilePaths) { TsFileResource resource = new TsFileResource(new File(tsFilePath), TsFileResourceStatus.NORMAL); - if (resource.resourceFileExists()) { - try { - resource.deserialize(); - } catch (IOException e) { - throw new RuntimeException( - String.format( - DataNodeQueryMessages.FAILED_TO_DESERIALIZE_EXTERNAL_TSFILE_RESOURCE, - tsFilePath, - e.getMessage()), - e); - } - } else { - resource.setTimeIndex(new FileTimeIndex(Long.MIN_VALUE, Long.MAX_VALUE)); - } + resource.setTimeIndex(new FileTimeIndex(Long.MIN_VALUE, Long.MAX_VALUE)); tsFileResources.add(resource); } return Collections.unmodifiableList(tsFileResources); @@ -641,9 +616,8 @@ private DeviceCollector() { for (int fileIndex = 0; fileIndex < tsFilePaths.size(); fileIndex++) { TsFileSequenceReader reader = FileReaderManager.getInstance() - .get(tsFilePaths.get(fileIndex), null, true, ioSizeRecorder, true); - deviceIteratorMap.put( - fileIndex, new LazyTsFileDeviceIterator(reader, tableName, ioSizeRecorder)); + .get(tsFilePaths.get(fileIndex), null, true, null, true); + deviceIteratorMap.put(fileIndex, new LazyTsFileDeviceIterator(reader, tableName, null)); } } catch (IOException e) { close(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java index c1db965364da6..ed9edf146f31a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java @@ -151,7 +151,7 @@ private ExternalTsFileQueryResource newResource(String queryId, List fil } MPPQueryContext queryContext = new MPPQueryContext(new QueryId(queryId)); return new ExternalTsFileQueryResource( - queryContext, root.toPath().resolve("tmp"), "table1", tsFilePaths, ignored -> {}, true); + queryContext, root.toPath().resolve("tmp"), "table1", tsFilePaths); } private void addDevices(String... deviceNames) { diff --git a/pom.xml b/pom.xml index c577243cb0ac2..096536c706a02 100644 --- a/pom.xml +++ b/pom.xml @@ -151,7 +151,7 @@ 0.14.1 1.9 1.5.6-3 - 2.3.2-260608-SNAPSHOT + 2.3.2-260616-SNAPSHOT en From 824f4dd48defc06d109c47b57646f6a8b71ae92c Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 15:59:27 +0800 Subject: [PATCH 25/32] fix --- .../queryengine/common/MPPQueryContext.java | 4 ---- .../relational/AbstractTableScanOperator.java | 5 +++-- .../DataNodeTableOperatorGenerator.java | 14 +++++-------- .../ExternalTsFileDeviceFilterVisitor.java | 2 +- .../ExternalTsFileQueryResource.java | 1 - .../PushPredicateIntoTableScan.java | 14 ++++--------- .../TransformSortToStreamSort.java | 20 +++++++++---------- .../db/storageengine/buffer/ChunkCache.java | 12 +---------- .../reader/chunk/DiskAlignedChunkLoader.java | 6 ++---- .../read/reader/chunk/DiskChunkLoader.java | 6 ++---- 10 files changed, 27 insertions(+), 57 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/{execution/operator/source/relational => plan/relational/function/tvf/readTsFile}/ExternalTsFileDeviceFilterVisitor.java (96%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 6ec9fca5946ec..871e17b261a2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -525,10 +525,6 @@ public void reserveMemoryForFrontEndImmediately() { this.memoryReservationManager.reserveMemoryImmediately(); } - public void reserveMemoryForFrontEndImmediately(final long bytes) { - this.memoryReservationManager.reserveMemoryImmediately(bytes); - } - public void releaseAllMemoryReservedForFrontEnd() { this.memoryReservationManager.releaseAllReservedMemory(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java index 523f0d1367a67..c759330e96261 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java @@ -89,11 +89,12 @@ public abstract class AbstractTableScanOperator extends AbstractSeriesScanOperat public AbstractTableScanOperator(AbstractTableScanOperatorParameter parameter) { this.sourceId = parameter.sourceId; this.operatorContext = parameter.context; + this.operatorContext.recordSpecifiedInfo( + DEVICE_NUMBER, Integer.toString(parameter.deviceEntries.size())); this.columnSchemas = parameter.columnSchemas; this.columnsIndexArray = parameter.columnsIndexArray; this.deviceEntries = parameter.deviceEntries; this.deviceCount = parameter.deviceEntries.size(); - this.operatorContext.recordSpecifiedInfo(DEVICE_NUMBER, Integer.toString(this.deviceCount)); this.scanOrder = parameter.scanOrder; this.seriesScanOptions = parameter.seriesScanOptions; this.measurementColumnNames = parameter.measurementColumnNames; @@ -203,7 +204,7 @@ private void constructResultTsBlock() { measurementDataBlock, columnsIndexArray, columnSchemas, - currentDeviceEntry, + deviceEntries.get(currentDeviceIndex), idColumnIndex -> getNthIdColumnValue(currentDeviceEntry, idColumnIndex)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index 7ff3db01dfec7..426531b2e1623 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -40,7 +40,6 @@ import org.apache.iotdb.commons.queryengine.common.SessionInfo; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.planner.plan.parameter.InputLocation; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; @@ -450,8 +449,7 @@ public Operator visitTreeNonAlignedDeviceViewScan( "PushDownOffset should not be set when isPushLimitToEachDevice is true."); } CommonTableScanOperatorParameters commonParameter = - new CommonTableScanOperatorParameters( - node, fieldColumnsRenameMap, true, node.getTagAndAttributeIndexMap()); + new CommonTableScanOperatorParameters(node, fieldColumnsRenameMap, true); List measurementSchemas = commonParameter.measurementSchemas; List measurementSchemaIndex2Symbols = commonParameter.measurementSchemaIndex2Symbol; List measurementColumnNames = commonParameter.measurementColumnNames; @@ -861,10 +859,9 @@ private static class CommonTableScanOperatorParameters { int idx; private CommonTableScanOperatorParameters( - TableScanNode node, + DeviceTableScanNode node, Map fieldColumnsRenameMap, - boolean keepNonOutputMeasurementColumns, - Map tagAndAttributeColumnsIndexMap) { + boolean keepNonOutputMeasurementColumns) { outputColumnNames = node.getOutputSymbols(); int outputColumnCount = keepNonOutputMeasurementColumns ? node.getAssignments().size() : outputColumnNames.size(); @@ -872,7 +869,7 @@ private CommonTableScanOperatorParameters( symbolInputs = new ArrayList<>(outputColumnCount); columnsIndexArray = new int[outputColumnCount]; columnSchemaMap = node.getAssignments(); - this.tagAndAttributeColumnsIndexMap = tagAndAttributeColumnsIndexMap; + this.tagAndAttributeColumnsIndexMap = node.getTagAndAttributeIndexMap(); measurementColumnNames = new ArrayList<>(); measurementColumnsIndexMap = new HashMap<>(); measurementSchemas = new ArrayList<>(); @@ -1029,8 +1026,7 @@ private void addSource( long viewTTL) { CommonTableScanOperatorParameters commonParameter = - new CommonTableScanOperatorParameters( - node, fieldColumnsRenameMap, false, node.getTagAndAttributeIndexMap()); + new CommonTableScanOperatorParameters(node, fieldColumnsRenameMap, false); List measurementSchemas = commonParameter.measurementSchemas; List measurementColumnNames = commonParameter.measurementColumnNames; List columnSchemas = commonParameter.columnSchemas; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileDeviceFilterVisitor.java similarity index 96% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileDeviceFilterVisitor.java index d26713ae7dbed..e03e3f5e49b44 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileDeviceFilterVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileDeviceFilterVisitor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.queryengine.execution.operator.source.relational; +package org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile; import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.schema.filter.SchemaFilterVisitor; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 56eaed315ac53..6f9c99a5efdb4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -27,7 +27,6 @@ import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; -import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileDeviceFilterVisitor; import org.apache.iotdb.db.queryengine.plan.planner.memory.NotThreadSafeMemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 3f5927e9bb8c4..8440bb24c7b46 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -483,7 +483,6 @@ public PlanNode visitDeviceTableScan( public PlanNode visitExternalTsFileScan( ExternalTsFileScanNode tableScanNode, RewriteContext context) { if (TRUE_LITERAL.equals(context.inheritedPredicate)) { - collectExternalTsFileDeviceTasks(tableScanNode, Collections.emptyList()); return tableScanNode; } @@ -528,9 +527,10 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre } // do index scan after expressionCanPushDown is processed - if (tableScanNode instanceof ExternalTsFileScanNode) { - collectExternalTsFileDeviceTasks( - (ExternalTsFileScanNode) tableScanNode, splitExpression.getMetadataExpressions()); + if (tableScanNode instanceof ExternalTsFileScanNode externalTsFileScanNode) { + externalTsFileScanNode.setSchemaFilter( + constructExternalTsFileDeviceFilter( + externalTsFileScanNode, splitExpression.getMetadataExpressions())); } else if (tableScanNode instanceof DeviceTableScanNode) { getDeviceEntriesWithDataPartitions( (DeviceTableScanNode) tableScanNode, splitExpression.getMetadataExpressions()); @@ -550,12 +550,6 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre return tableScanNode; } - private void collectExternalTsFileDeviceTasks( - ExternalTsFileScanNode tableScanNode, List metadataExpressions) { - tableScanNode.setSchemaFilter( - constructExternalTsFileDeviceFilter(tableScanNode, metadataExpressions)); - } - private SchemaFilter constructExternalTsFileDeviceFilter( ExternalTsFileScanNode tableScanNode, List metadataExpressions) { if (metadataExpressions.isEmpty()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java index e6f5962f2e523..1c1b650e78852 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNode; -import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode; import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.planner.OrderingScheme; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; @@ -93,13 +92,9 @@ public PlanNode visitSort(SortNode node, Context context) { } context.setCanTransform(false); - TableScanNode tableScanNode = context.getTableScanNode(); - if (tableScanNode == null) { - node.setChild(child); - return node; - } + DeviceTableScanNode deviceTableScanNode = context.getTableScanNode(); Map tableColumnSchema = - analysis.getTableColumnSchema(tableScanNode.getQualifiedObjectName()); + analysis.getTableColumnSchema(deviceTableScanNode.getQualifiedObjectName()); OrderingScheme orderingScheme = node.getOrderingScheme(); int streamSortIndex = -1; @@ -116,7 +111,10 @@ public PlanNode visitSort(SortNode node, Context context) { if (streamSortIndex >= 0) { boolean orderByAllIdsAndTime = isOrderByAllIdsAndTime( - tableColumnSchema, tableScanNode.getAssignments(), orderingScheme, streamSortIndex); + tableColumnSchema, + deviceTableScanNode.getAssignments(), + orderingScheme, + streamSortIndex); return new StreamSortNode( queryContext.getQueryId().genPlanNodeId(), @@ -209,15 +207,15 @@ public static boolean isOrderByAllIdsAndTime( } private static class Context { - private TableScanNode tableScanNode; + private DeviceTableScanNode tableScanNode; private boolean canTransform = true; - public TableScanNode getTableScanNode() { + public DeviceTableScanNode getTableScanNode() { return tableScanNode; } - public void setTableScanNode(TableScanNode tableScanNode) { + public void setTableScanNode(DeviceTableScanNode tableScanNode) { this.tableScanNode = tableScanNode; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/ChunkCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/ChunkCache.java index 08d750328d518..90b755103e6cb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/ChunkCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/ChunkCache.java @@ -131,16 +131,6 @@ public Chunk get( Statistics chunkStatistic, QueryContext queryContext) throws IOException { - return get(chunkCacheKey, timeRangeList, chunkStatistic, queryContext, false); - } - - public Chunk get( - ChunkCacheKey chunkCacheKey, - List timeRangeList, - Statistics chunkStatistic, - QueryContext queryContext, - boolean externalTsFile) - throws IOException { LongConsumer ioSizeRecorder = queryContext.getQueryStatistics().getLoadChunkActualIOSize()::addAndGet; LongConsumer cacheHitAdder = @@ -155,7 +145,7 @@ public Chunk get( ioSizeRecorder, cacheHitAdder, cacheMissAdder, - externalTsFile); + queryContext.isExternalTsFileScan()); } private Chunk get( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java index 2ea8ec0c6bab9..7c1e9d1263518 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java @@ -92,8 +92,7 @@ public IChunkReader getChunkReader(IChunkMetadata chunkMetaData, Filter globalTi resource.isClosed()), timeChunkMetadata.getDeleteIntervalList(), timeChunkMetadata.getStatistics(), - context, - context.isExternalTsFileScan()); + context); List valueChunkList = new ArrayList<>(); for (IChunkMetadata valueChunkMetadata : alignedChunkMetadata.getValueChunkMetadataList()) { Chunk chunk = @@ -108,8 +107,7 @@ public IChunkReader getChunkReader(IChunkMetadata chunkMetaData, Filter globalTi resource.isClosed()), valueChunkMetadata.getDeleteIntervalList(), valueChunkMetadata.getStatistics(), - context, - context.isExternalTsFileScan()); + context); final TsFileID tsFileID = getTsFileID(); if (chunk != null && tsFileID.regionId > 0 diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java index 73355bed0c85b..7d259a9e50534 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java @@ -64,8 +64,7 @@ public Chunk loadChunk(ChunkMetadata chunkMetaData) throws IOException { resource.isClosed()), chunkMetaData.getDeleteIntervalList(), chunkMetaData.getStatistics(), - context, - context.isExternalTsFileScan()); + context); } @Override @@ -88,8 +87,7 @@ public IChunkReader getChunkReader(IChunkMetadata chunkMetaData, Filter globalTi resource.isClosed()), chunkMetaData.getDeleteIntervalList(), chunkMetaData.getStatistics(), - context, - context.isExternalTsFileScan()); + context); byte chunkType = chunk.getHeader().getChunkType(); if (chunkType != MetaMarker.CHUNK_HEADER && chunkType != MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) { From 1051e394d7f3b679a6657be72d20fdfb64683083 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 16:43:07 +0800 Subject: [PATCH 26/32] fix --- .../ExternalTsFileQueryResource.java | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 6f9c99a5efdb4..32ab740427c98 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -283,9 +283,7 @@ private void sortPendingDeviceTasks() { if (deviceEntryComparator != null) { pendingDeviceTasks.sort( (left, right) -> - deviceEntryComparator.compare( - sharedDeviceEntries.get(left.deviceEntryIndex), - sharedDeviceEntries.get(right.deviceEntryIndex))); + compareDeviceEntryIndexes(left.deviceEntryIndex, right.deviceEntryIndex)); } else { pendingDeviceTasks.sort( (left, right) -> @@ -345,10 +343,7 @@ void finish() { private void sortDeviceEntries() { if (deviceEntryComparator != null) { - deviceEntryIndexes.sort( - (left, right) -> - deviceEntryComparator.compare( - sharedDeviceEntries.get(left), sharedDeviceEntries.get(right))); + deviceEntryIndexes.sort(ExternalTsFileQueryResource.this::compareDeviceEntryIndexes); } else { deviceEntryIndexes.sort( (left, right) -> @@ -368,6 +363,15 @@ private List getPendingDeviceTasks() { } } + private int compareDeviceEntryIndexes(int leftIndex, int rightIndex) { + int result = + deviceEntryComparator.compare( + sharedDeviceEntries.get(leftIndex), sharedDeviceEntries.get(rightIndex)); + // Use the stable device entry index as a tie-breaker so list sorting and run-file merging keep + // the same deterministic order when the pushed-down comparator is not a total order. + return result != 0 ? result : Integer.compare(leftIndex, rightIndex); + } + private void createDeviceTaskPartitions(int partitionCount) { if (partitionCount <= 0) { throw new IllegalArgumentException( @@ -420,8 +424,9 @@ public class DeviceTaskRunReader implements AutoCloseable { usePriorityQueue ? new PriorityQueue<>( (left, right) -> - comparator.compare( - left.getCurrentDeviceEntry(), right.getCurrentDeviceEntry())) + compareDeviceEntryIndexes( + left.getCurrentDeviceTask().deviceEntryIndex, + right.getCurrentDeviceTask().deviceEntryIndex)) : new ArrayDeque<>(); for (Path runFile : partition.getRunFiles()) { DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFile, sharedDeviceEntries); From c44579f66e97ddc036c1da0ba428c7009572bd8b Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 17:06:07 +0800 Subject: [PATCH 27/32] check query timeout --- .../db/queryengine/common/MPPQueryContext.java | 8 ++++++++ .../queryengine/plan/execution/QueryExecution.java | 7 +------ .../plan/relational/analyzer/StatementAnalyzer.java | 4 ++++ .../tvf/readTsFile/ExternalTsFileQueryResource.java | 5 ++++- .../tvf/readTsFile/ReadTsFileTableFunction.java | 9 ++++++++- .../tvf/readTsFile/TsFileSchemaCollector.java | 13 ++++++++++++- .../readTsFile/ExternalTsFileQueryResourceTest.java | 2 ++ 7 files changed, 39 insertions(+), 9 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 871e17b261a2d..6b73e81b3fa74 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -35,6 +35,7 @@ import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.query.QueryTimeoutRuntimeException; import org.apache.iotdb.db.queryengine.plan.analyze.Analysis; import org.apache.iotdb.db.queryengine.plan.analyze.PredicateUtils; import org.apache.iotdb.db.queryengine.plan.analyze.QueryType; @@ -312,6 +313,13 @@ public void setTimeOut(long timeOut) { this.timeOut = timeOut; } + public void checkTimeOut() { + long currentTime = System.currentTimeMillis(); + if (currentTime - startTime >= timeOut) { + throw new QueryTimeoutRuntimeException(startTime, currentTime, timeOut); + } + } + public void setQueryType(QueryType queryType) { this.queryType = queryType; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java index 0a921dd2dd819..c14293ba380c0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java @@ -29,7 +29,6 @@ import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.query.KilledByOthersException; -import org.apache.iotdb.db.exception.query.QueryTimeoutRuntimeException; import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.common.FragmentInstanceId; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; @@ -223,11 +222,7 @@ private void checkTimeOutForQuery() { // only check query operation's timeout because we will never limit write operation's execution // time if (isQuery()) { - long currentTime = System.currentTimeMillis(); - if (currentTime - context.getStartTime() >= context.getTimeOut()) { - throw new QueryTimeoutRuntimeException( - context.getStartTime(), currentTime, context.getTimeOut()); - } + context.checkTimeOut(); } } 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 a2d606dbd06e6..cc39ce8bd540c 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 @@ -138,6 +138,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.ArgumentsAnalysis; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableArgumentAnalysis; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableFunctionInvocationAnalysis; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile.ReadTsFileTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.planner.PlannerContext; import org.apache.iotdb.db.queryengine.plan.relational.planner.ScopeAware; @@ -5024,6 +5025,9 @@ public Scope visitTableFunctionInvocation(TableFunctionInvocation node, Optional TableFunctionAnalysis functionAnalysis; try { + if (function instanceof ReadTsFileTableFunction) { + ((ReadTsFileTableFunction) function).setMPPQueryContext(queryContext); + } functionAnalysis = function.analyze(argumentsAnalysis.getPassedArguments()); } catch (UDFException e) { throw new SemanticException(e.getMessage()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 32ab740427c98..7beed2c3cfb03 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -79,6 +79,7 @@ public class ExternalTsFileQueryResource implements AutoCloseable { private static final long TSFILE_READER_MEMORY_RESERVE_SIZE_IN_BYTES = 4L * 1024; + private final MPPQueryContext queryContext; private final QueryId queryId; // This resource outlives the frontend planning phase, whose MPPQueryContext memory manager is // released after dispatch. Keep a dedicated manager and release it when this resource closes. @@ -95,7 +96,8 @@ public class ExternalTsFileQueryResource implements AutoCloseable { public ExternalTsFileQueryResource( MPPQueryContext queryContext, Path tempRoot, String tableName, List tsFilePaths) { - this.queryId = requireNonNull(queryContext, "queryContext is null").getQueryId(); + this.queryContext = requireNonNull(queryContext, "queryContext is null"); + this.queryId = queryContext.getQueryId(); this.externalTsFileResourceMemoryReservationManager = new NotThreadSafeMemoryReservationManager( queryId, ExternalTsFileQueryResource.class.getName()); @@ -117,6 +119,7 @@ public void collectDeviceEntries( try (DeviceCollector deviceCollector = new DeviceCollector()) { createDeviceTaskPartitions(partitionCount); while (deviceCollector.hasNextDevice()) { + queryContext.checkTimeOut(); IDeviceID deviceID = deviceCollector.nextDevice(); if (schemaFilter != null && !Boolean.TRUE.equals(schemaFilter.accept(deviceFilterVisitor, deviceID))) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java index c078115908f49..6dde302e59bea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ReadTsFileTableFunction.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; import org.apache.iotdb.udf.api.exception.UDFException; import org.apache.iotdb.udf.api.relational.TableFunction; @@ -55,6 +56,12 @@ public class ReadTsFileTableFunction implements TableFunction { private static final String TABLE_NAME_PARAMETER_NAME = "TABLE_NAME"; private static final String PATHS_PARAMETER_NAME = "PATHS"; + private MPPQueryContext mppQueryContext; + + public void setMPPQueryContext(MPPQueryContext mppQueryContext) { + this.mppQueryContext = mppQueryContext; + } + @Override public List getArgumentsSpecifications() { return Arrays.asList( @@ -73,7 +80,7 @@ public TableFunctionAnalysis analyze(Map arguments) throws UDF parseTsFilePaths(getRequiredStringArgument(arguments, PATHS_PARAMETER_NAME)); checkTsFilePathsAreOutsideDataDirs(tsFilePaths); TsFileSchemaCollector schemaCollector = - new TsFileSchemaCollector(tableName.isEmpty() ? null : tableName); + new TsFileSchemaCollector(tableName.isEmpty() ? null : tableName, mppQueryContext); schemaCollector.collect(tsFilePaths); TableSchema mergedTableSchema = schemaCollector.getMergedTableSchema(); if (mergedTableSchema == null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java index 090eda825efd4..26fc04dd3f168 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/TsFileSchemaCollector.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.function.tvf.readTsFile; import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; import org.apache.tsfile.common.conf.TSFileConfig; @@ -47,14 +48,16 @@ final class TsFileSchemaCollector { private final String specifiedTableName; + private final MPPQueryContext mppQueryContext; private String tableName; private final List tsFiles = new ArrayList<>(); private MergedTableSchemaBuilder schemaBuilder; private TableSchema mergedTableSchema; - TsFileSchemaCollector(String specifiedTableName) { + TsFileSchemaCollector(String specifiedTableName, MPPQueryContext mppQueryContext) { this.specifiedTableName = specifiedTableName == null ? null : specifiedTableName.toLowerCase(Locale.ENGLISH); + this.mppQueryContext = mppQueryContext; this.tableName = this.specifiedTableName; } @@ -66,6 +69,7 @@ void collect(List tsFilePaths) { DataNodeQueryMessages.TSFILE_PATH_DOES_NOT_EXIST + tsFilePath); } if (Files.isRegularFile(path)) { + checkTimeOutIfNeeded(); TableSchema tableSchema = readTableSchema(specifiedTableName, path.toFile(), true); collect(path.toFile(), tableSchema); continue; @@ -99,6 +103,7 @@ private void collectFromDirectory(String tsFilePath, Path path) { walkedPaths.filter(Files::isRegularFile).filter(this::hasTsFileSuffix).iterator(); while (iterator.hasNext()) { Path filePath = iterator.next(); + checkTimeOutIfNeeded(); TableSchema tableSchema = readTableSchema(specifiedTableName, filePath.toFile(), false); collect(filePath.toFile(), tableSchema); } @@ -113,6 +118,12 @@ private boolean hasTsFileSuffix(Path filePath) { return fileName != null && fileName.toString().endsWith(TsFileConstant.TSFILE_SUFFIX); } + private void checkTimeOutIfNeeded() { + if (mppQueryContext != null) { + mppQueryContext.checkTimeOut(); + } + } + private void collect(File tsFile, TableSchema tableSchema) { if (tableSchema == null) { return; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java index ed9edf146f31a..eb34961a6c5b6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java @@ -150,6 +150,8 @@ private ExternalTsFileQueryResource newResource(String queryId, List fil tsFilePaths.add(new File(root, fileName).getAbsolutePath()); } MPPQueryContext queryContext = new MPPQueryContext(new QueryId(queryId)); + queryContext.setStartTime(System.currentTimeMillis()); + queryContext.setTimeOut(Long.MAX_VALUE); return new ExternalTsFileQueryResource( queryContext, root.toPath().resolve("tmp"), "table1", tsFilePaths); } From c6ea31f79cfebbef3a250e761fa27eb44c5c400e Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 18:03:39 +0800 Subject: [PATCH 28/32] refactor --- .../ExternalTsFileQueryResource.java | 177 ++++++++++++------ .../ExternalTsFileQueryResourceTest.java | 13 +- 2 files changed, 132 insertions(+), 58 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 7beed2c3cfb03..a84f42dee1a87 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -53,7 +53,6 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -127,6 +126,8 @@ public void collectDeviceEntries( } DeviceEntry deviceEntry = new AlignedDeviceEntry(deviceID, new Binary[0]); int deviceEntryIndex = sharedDeviceEntries.size(); + externalTsFileResourceMemoryReservationManager.reserveMemoryCumulatively( + deviceEntry.ramBytesUsed()); sharedDeviceEntries.add(deviceEntry); DeviceTask deviceTask = new DeviceTask(deviceEntryIndex, deviceCollector.getCurrentDeviceOffsets()); @@ -151,7 +152,9 @@ public DeviceTaskRunReader getDeviceTaskRunReader(int partitionIndex) { checkNotClosed(); DeviceTaskPartition partition = getDeviceTaskPartition(partitionIndex); try { - return new DeviceTaskRunReader(partition); + return deviceEntryComparator == null + ? new SequentialDeviceTaskRunReader(partition) + : new PriorityDeviceTaskRunReader(partition); } catch (IOException e) { throw new RuntimeException( DataNodeQueryMessages.FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_TASK_RUN_READER, e); @@ -412,57 +415,36 @@ private Path writeDeviceTaskRun(Path runRoot, int runIndex, List dev return runFile; } - public class DeviceTaskRunReader implements AutoCloseable { + public abstract class DeviceTaskRunReader implements AutoCloseable { - private final Queue runCursors; - private final boolean usePriorityQueue; private DeviceEntry currentDevice; private QueryDataSource currentDeviceQueryDataSource; private Map currentDeviceOffsetMap; - DeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { - Comparator comparator = deviceEntryComparator; - this.usePriorityQueue = comparator != null; - this.runCursors = - usePriorityQueue - ? new PriorityQueue<>( - (left, right) -> - compareDeviceEntryIndexes( - left.getCurrentDeviceTask().deviceEntryIndex, - right.getCurrentDeviceTask().deviceEntryIndex)) - : new ArrayDeque<>(); + protected void initialize(DeviceTaskPartition partition) throws IOException { for (Path runFile : partition.getRunFiles()) { - DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFile, sharedDeviceEntries); + DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFile); if (cursor.hasCurrentDeviceTask()) { - runCursors.add(cursor); + addCursor(cursor); } else { cursor.close(); } } DeviceTaskRunCursor memoryCursor = - new MemoryDeviceTaskRunCursor(partition.getPendingDeviceTasks(), sharedDeviceEntries); + new MemoryDeviceTaskRunCursor(partition.getPendingDeviceTasks()); if (memoryCursor.hasCurrentDeviceTask()) { - runCursors.add(memoryCursor); + addCursor(memoryCursor); } } public boolean nextDevice() throws IOException { - if (runCursors.isEmpty()) { + DeviceTaskRunCursor cursor = pollCursor(); + if (cursor == null) { return false; } - DeviceTaskRunCursor cursor = usePriorityQueue ? runCursors.poll() : runCursors.peek(); DeviceTask result = cursor.getCurrentDeviceTask(); cursor.advance(); - if (cursor.hasCurrentDeviceTask()) { - if (usePriorityQueue) { - runCursors.add(cursor); - } - } else { - if (!usePriorityQueue) { - runCursors.poll(); - } - cursor.close(); - } + recycleOrCloseCursor(cursor); currentDevice = sharedDeviceEntries.get(result.deviceEntryIndex); List unseqResources = new ArrayList<>(result.deviceOffsets.size()); @@ -477,6 +459,14 @@ public boolean nextDevice() throws IOException { return true; } + protected abstract DeviceTaskRunCursor pollCursor() throws IOException; + + protected abstract void addCursor(DeviceTaskRunCursor cursor); + + protected abstract void recycleOrCloseCursor(DeviceTaskRunCursor cursor) throws IOException; + + protected abstract DeviceTaskRunCursor pollRemainingCursor(); + public DeviceEntry getCurrentDevice() { return currentDevice; } @@ -492,9 +482,10 @@ public Map getCurrentDeviceOffsetMap() { @Override public void close() throws IOException { IOException exception = null; - while (!runCursors.isEmpty()) { + DeviceTaskRunCursor cursor; + while ((cursor = pollRemainingCursor()) != null) { try { - runCursors.poll().close(); + cursor.close(); } catch (IOException e) { if (exception == null) { exception = e; @@ -509,27 +500,118 @@ public void close() throws IOException { } } + private class PriorityDeviceTaskRunReader extends DeviceTaskRunReader { + + private final Queue runCursors = + new PriorityQueue<>( + (left, right) -> + compareDeviceEntryIndexes( + left.getCurrentDeviceTask().deviceEntryIndex, + right.getCurrentDeviceTask().deviceEntryIndex)); + + private PriorityDeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { + initialize(partition); + } + + @Override + protected DeviceTaskRunCursor pollCursor() { + return runCursors.poll(); + } + + @Override + protected void addCursor(DeviceTaskRunCursor cursor) { + runCursors.add(cursor); + } + + @Override + protected void recycleOrCloseCursor(DeviceTaskRunCursor cursor) throws IOException { + if (cursor.hasCurrentDeviceTask()) { + addCursor(cursor); + } else { + cursor.close(); + } + } + + @Override + protected DeviceTaskRunCursor pollRemainingCursor() { + return runCursors.poll(); + } + } + + private class SequentialDeviceTaskRunReader extends DeviceTaskRunReader { + + private final Iterator runFileIterator; + private final List pendingDeviceTasks; + private DeviceTaskRunCursor currentCursor; + private boolean memoryCursorLoaded; + + private SequentialDeviceTaskRunReader(DeviceTaskPartition partition) { + this.runFileIterator = partition.getRunFiles().iterator(); + this.pendingDeviceTasks = partition.getPendingDeviceTasks(); + } + + @Override + protected DeviceTaskRunCursor pollCursor() throws IOException { + if (currentCursor != null) { + DeviceTaskRunCursor cursor = currentCursor; + currentCursor = null; + return cursor; + } + while (runFileIterator.hasNext()) { + DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFileIterator.next()); + if (cursor.hasCurrentDeviceTask()) { + return cursor; + } + cursor.close(); + } + if (!memoryCursorLoaded) { + memoryCursorLoaded = true; + DeviceTaskRunCursor cursor = new MemoryDeviceTaskRunCursor(pendingDeviceTasks); + if (cursor.hasCurrentDeviceTask()) { + return cursor; + } + } + return null; + } + + @Override + protected void addCursor(DeviceTaskRunCursor cursor) { + currentCursor = cursor; + } + + @Override + protected void recycleOrCloseCursor(DeviceTaskRunCursor cursor) throws IOException { + if (cursor.hasCurrentDeviceTask()) { + currentCursor = cursor; + } else { + cursor.close(); + } + } + + @Override + protected DeviceTaskRunCursor pollRemainingCursor() { + DeviceTaskRunCursor cursor = currentCursor; + currentCursor = null; + return cursor; + } + } + private interface DeviceTaskRunCursor extends Closeable { boolean hasCurrentDeviceTask(); DeviceTask getCurrentDeviceTask(); - DeviceEntry getCurrentDeviceEntry(); - void advance() throws IOException; } private static class DiskDeviceTaskRunCursor implements DeviceTaskRunCursor { - private final List deviceEntries; private final DataInputStream inputStream; private int remainingDeviceTasks; private DeviceTask currentDeviceTask; - private DiskDeviceTaskRunCursor(Path runFile, List deviceEntries) - throws IOException { - this.deviceEntries = deviceEntries; + private DiskDeviceTaskRunCursor(Path runFile) throws IOException { this.inputStream = new DataInputStream(new BufferedInputStream(Files.newInputStream(runFile))); this.remainingDeviceTasks = ReadWriteIOUtils.readInt(inputStream); @@ -556,11 +638,6 @@ public DeviceTask getCurrentDeviceTask() { return currentDeviceTask; } - @Override - public DeviceEntry getCurrentDeviceEntry() { - return deviceEntries.get(currentDeviceTask.deviceEntryIndex); - } - @Override public void close() throws IOException { inputStream.close(); @@ -570,14 +647,11 @@ public void close() throws IOException { private static class MemoryDeviceTaskRunCursor implements DeviceTaskRunCursor { private final List deviceTasks; - private final List deviceEntries; private int nextIndex; private DeviceTask currentDeviceTask; - private MemoryDeviceTaskRunCursor( - List deviceTasks, List deviceEntries) { + private MemoryDeviceTaskRunCursor(List deviceTasks) { this.deviceTasks = deviceTasks; - this.deviceEntries = deviceEntries; advance(); } @@ -600,11 +674,6 @@ public DeviceTask getCurrentDeviceTask() { return currentDeviceTask; } - @Override - public DeviceEntry getCurrentDeviceEntry() { - return deviceEntries.get(currentDeviceTask.deviceEntryIndex); - } - @Override public void close() { currentDeviceTask = null; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java index eb34961a6c5b6..912940d452ae0 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java @@ -77,7 +77,7 @@ public void testDeviceTaskRunReaderMergesMultipleRunsWithComparator() throws Exc partition.add(task(1, offset(0, 20, 29))); partition.flush(); - try (DeviceTaskRunReader reader = resource.new DeviceTaskRunReader(partition)) { + try (DeviceTaskRunReader reader = newReader(partition)) { assertDeviceOrder(reader, "d1", "d2", "d3", "d4", "d5"); } } @@ -96,7 +96,7 @@ public void testDeviceTaskRunReaderReadsRunsInFifoOrderWithoutComparator() throw partition.add(task(3, offset(0, 40, 49))); partition.flush(); - try (DeviceTaskRunReader reader = resource.new DeviceTaskRunReader(partition)) { + try (DeviceTaskRunReader reader = newReader(partition)) { assertDeviceOrder(reader, "d3", "d1", "d2", "d4"); } } @@ -114,7 +114,7 @@ public void testDeviceTaskRunReaderReadsDiskRunBeforePendingMemoryTasksWithoutCo partition.add(task(2, offset(0, 30, 39))); partition.finish(); - try (DeviceTaskRunReader reader = resource.new DeviceTaskRunReader(partition)) { + try (DeviceTaskRunReader reader = newReader(partition)) { assertDeviceOrder(reader, "d2", "d1", "d3"); } } @@ -127,7 +127,7 @@ public void testDeviceTaskRunReaderUsesSharedTsFileResourceAsOffsetMapKey() thro partition.add(task(0, offset(0, 11, 22), offset(1, 33, 44))); partition.finish(); - try (DeviceTaskRunReader reader = resource.new DeviceTaskRunReader(partition)) { + try (DeviceTaskRunReader reader = newReader(partition)) { assertTrue(reader.nextDevice()); Map offsetMap = reader.getCurrentDeviceOffsetMap(); List sharedResources = resource.getSharedTsFileResources(); @@ -156,6 +156,11 @@ private ExternalTsFileQueryResource newResource(String queryId, List fil queryContext, root.toPath().resolve("tmp"), "table1", tsFilePaths); } + private DeviceTaskRunReader newReader(DeviceTaskPartition partition) { + resource.getDeviceTaskPartitions().add(partition); + return resource.getDeviceTaskRunReader(partition.getPartitionIndex()); + } + private void addDevices(String... deviceNames) { for (String deviceName : deviceNames) { resource From 355d8ba2b75790b6703a86ec76d56a58c31a7ed1 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 18:31:28 +0800 Subject: [PATCH 29/32] fix it --- .../it/udf/IoTDBSQLFunctionManagementIT.java | 3 +- .../IoTDBReadTsFileTableFunctionIT.java | 30 ++++++++++++++++--- 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/udf/IoTDBSQLFunctionManagementIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/udf/IoTDBSQLFunctionManagementIT.java index f414171cf469d..f56da20b05176 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/udf/IoTDBSQLFunctionManagementIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/udf/IoTDBSQLFunctionManagementIT.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.queryengine.plan.relational.function.TableBuiltinTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinAggregationFunction; import org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.TableClusterIT; @@ -55,7 +56,7 @@ public class IoTDBSQLFunctionManagementIT { private static final int BUILTIN_AGGREGATE_FUNCTIONS_COUNT = TableBuiltinAggregationFunction.values().length; private static final int BUILTIN_TABLE_FUNCTIONS_COUNT = - TableBuiltinTableFunction.values().length; + TableBuiltinTableFunction.values().length + DataNodeTableBuiltinTableFunction.values().length; private static final int BUILTIN_FUNCTIONS_COUNT = BUILTIN_SCALAR_FUNCTIONS_COUNT + BUILTIN_AGGREGATE_FUNCTIONS_COUNT diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java index 870a37aedafda..cde41419adc5f 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java @@ -290,11 +290,11 @@ public void testReadMultipleTsFilesWithConflictingTagColumns() throws Exception public void testReadMultipleTsFilesWithConflictingTagAndFieldColumns() throws Exception { File tsFile1 = new File(tmpDir, "tag-field-conflict-1.tsfile"); try (TsFileWriter writer = new TsFileWriter(tsFile1)) { - generateTable(writer, "table1", Arrays.asList("shared"), Arrays.asList("s1"), 1, 2); + registerTableSchema(writer, "table1", Arrays.asList("shared"), Arrays.asList("s1")); } File tsFile2 = new File(tmpDir, "tag-field-conflict-2.tsfile"); try (TsFileWriter writer = new TsFileWriter(tsFile2)) { - generateTable(writer, "table1", new ArrayList<>(), Arrays.asList("shared"), 3, 4); + registerTableSchema(writer, "table1", new ArrayList<>(), Arrays.asList("shared")); } tableAssertTestFail( @@ -311,11 +311,11 @@ public void testReadMultipleTsFilesWithConflictingTagAndFieldColumns() throws Ex public void testReadMultipleTsFilesWithConflictingFieldAndTagColumns() throws Exception { File tsFile1 = new File(tmpDir, "field-tag-conflict-1.tsfile"); try (TsFileWriter writer = new TsFileWriter(tsFile1)) { - generateTable(writer, "table1", new ArrayList<>(), Arrays.asList("shared"), 1, 2); + registerTableSchema(writer, "table1", new ArrayList<>(), Arrays.asList("shared")); } File tsFile2 = new File(tmpDir, "field-tag-conflict-2.tsfile"); try (TsFileWriter writer = new TsFileWriter(tsFile2)) { - generateTable(writer, "table1", Arrays.asList("shared"), Arrays.asList("s1"), 3, 4); + registerTableSchema(writer, "table1", Arrays.asList("shared"), Arrays.asList("s1")); } tableAssertTestFail( @@ -468,6 +468,28 @@ private static void generateTable( } } + private static void registerTableSchema( + TsFileWriter writer, String tableName, List tagColumns, List fieldColumns) + throws IOException { + List columnNames = new ArrayList<>(tagColumns.size() + fieldColumns.size()); + List columnTypes = new ArrayList<>(tagColumns.size() + fieldColumns.size()); + List columnCategories = + new ArrayList<>(tagColumns.size() + fieldColumns.size()); + for (String tagColumn : tagColumns) { + columnNames.add(tagColumn); + columnTypes.add(TSDataType.STRING); + columnCategories.add(ColumnCategory.TAG); + } + for (String fieldColumn : fieldColumns) { + columnNames.add(fieldColumn); + columnTypes.add(TSDataType.INT64); + columnCategories.add(ColumnCategory.FIELD); + } + + writer.registerTableSchema( + new TableSchema(tableName, columnNames, columnTypes, columnCategories)); + } + private static void addFieldValue( Tablet tablet, int row, int column, TSDataType fieldType, long time) { if (fieldType == TSDataType.DOUBLE) { From 96ea0eef2a1babe276d1b8a6077af3916035eed7 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 18:35:32 +0800 Subject: [PATCH 30/32] modify name of temp dir --- .../tvf/readTsFile/ExternalTsFileQueryResource.java | 11 ++++++++--- .../distribute/TableDistributedPlanGenerator.java | 5 ++--- .../readTsFile/ExternalTsFileQueryResourceTest.java | 12 ++++++++---- 3 files changed, 18 insertions(+), 10 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index a84f42dee1a87..4394834eaec9b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -21,6 +21,7 @@ import org.apache.iotdb.calc.exception.MemoryNotEnoughException; import org.apache.iotdb.calc.plan.planner.memory.MemoryReservationManager; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.commons.utils.TestOnly; @@ -240,6 +241,7 @@ public class DeviceTaskPartition { private static final long MEMORY_RESERVE_BATCH_SIZE_IN_BYTES = 1024 * 1024; private final int partitionIndex; + private final PlanNodeId planNodeId; private final List pendingDeviceTasks = new ArrayList<>(); private final List deviceEntryIndexes = new ArrayList<>(); private final List runFiles = new ArrayList<>(); @@ -248,12 +250,17 @@ public class DeviceTaskPartition { DeviceTaskPartition(int partitionIndex) { this.partitionIndex = partitionIndex; + this.planNodeId = queryId.genPlanNodeId(); } public int getPartitionIndex() { return partitionIndex; } + public PlanNodeId getPlanNodeId() { + return planNodeId; + } + public List getDeviceEntryIndexes() { return deviceEntryIndexes; } @@ -271,9 +278,7 @@ void flush() { try { runFiles.add( writeDeviceTaskRun( - queryTempRoot.resolve("child-" + partitionIndex), - runFiles.size(), - pendingDeviceTasks)); + queryTempRoot.resolve(planNodeId.getId()), runFiles.size(), pendingDeviceTasks)); } catch (IOException e) { throw new RuntimeException( DataNodeQueryMessages.FAILED_TO_FLUSH_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION, e); 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 92d30d4f6497e..87ed89b57fe84 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 @@ -755,7 +755,7 @@ public List visitExternalTsFileScan( for (DeviceTaskPartition partition : partitions) { ExternalTsFileScanNode splitNode = new ExternalTsFileScanNode( - queryId.genPlanNodeId(), + partition.getPlanNodeId(), node.getQualifiedObjectName(), node.getOutputSymbols(), node.getAssignments(), @@ -813,7 +813,7 @@ public List visitExternalTsFileAggregationScan( for (DeviceTaskPartition partition : partitions) { ExternalTsFileAggregationScanNode splitNode = new ExternalTsFileAggregationScanNode( - queryId.genPlanNodeId(), + partition.getPlanNodeId(), node.getQualifiedObjectName(), partialTemplateNode.getOutputSymbols(), node.getAssignments(), @@ -2068,7 +2068,6 @@ private void applySortProperty( for (final PlanNode planNode : resultTableScanNodeList) { final DeviceTableScanNode scanNode = (DeviceTableScanNode) planNode; if (sortPropertyContext.scanOrderDesc) { - // TODO(beyyes) move scan order judgement into logical plan optimizer scanNode.setScanOrder(Ordering.DESC); } newOrderingScheme.ifPresent( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java index 912940d452ae0..1b6bef79ac2af 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResourceTest.java @@ -66,7 +66,7 @@ public void testDeviceTaskRunReaderMergesMultipleRunsWithComparator() throws Exc resource = newResource("merge_comparator", Arrays.asList("file-0.tsfile", "file-1.tsfile")); addDevices("d1", "d2", "d3", "d4", "d5"); resource.setDeviceEntryComparator(Comparator.comparing(DeviceEntry::getDeviceID)); - DeviceTaskPartition partition = resource.new DeviceTaskPartition(0); + DeviceTaskPartition partition = newPartition(); partition.add(task(2, offset(0, 30, 39))); partition.add(task(4, offset(1, 50, 59))); @@ -86,7 +86,7 @@ public void testDeviceTaskRunReaderMergesMultipleRunsWithComparator() throws Exc public void testDeviceTaskRunReaderReadsRunsInFifoOrderWithoutComparator() throws Exception { resource = newResource("fifo", Collections.singletonList("file-0.tsfile")); addDevices("d1", "d2", "d3", "d4"); - DeviceTaskPartition partition = resource.new DeviceTaskPartition(0); + DeviceTaskPartition partition = newPartition(); partition.add(task(2, offset(0, 30, 39))); partition.flush(); @@ -106,7 +106,7 @@ public void testDeviceTaskRunReaderReadsDiskRunBeforePendingMemoryTasksWithoutCo throws Exception { resource = newResource("disk_memory", Collections.singletonList("file-0.tsfile")); addDevices("d1", "d2", "d3"); - DeviceTaskPartition partition = resource.new DeviceTaskPartition(0); + DeviceTaskPartition partition = newPartition(); partition.add(task(1, offset(0, 20, 29))); partition.flush(); @@ -123,7 +123,7 @@ public void testDeviceTaskRunReaderReadsDiskRunBeforePendingMemoryTasksWithoutCo public void testDeviceTaskRunReaderUsesSharedTsFileResourceAsOffsetMapKey() throws Exception { resource = newResource("offset_map", Arrays.asList("file-0.tsfile", "file-1.tsfile")); addDevices("d1"); - DeviceTaskPartition partition = resource.new DeviceTaskPartition(0); + DeviceTaskPartition partition = newPartition(); partition.add(task(0, offset(0, 11, 22), offset(1, 33, 44))); partition.finish(); @@ -161,6 +161,10 @@ private DeviceTaskRunReader newReader(DeviceTaskPartition partition) { return resource.getDeviceTaskRunReader(partition.getPartitionIndex()); } + private DeviceTaskPartition newPartition() { + return resource.new DeviceTaskPartition(0); + } + private void addDevices(String... deviceNames) { for (String deviceName : deviceNames) { resource From 9c5cc36ced8bdba57f2944f1142120548e01579e Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Tue, 16 Jun 2026 18:51:06 +0800 Subject: [PATCH 31/32] fix it --- .../ExternalTsFileQueryResource.java | 39 +++++++++---------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java index 4394834eaec9b..30b95e3e71c07 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/readTsFile/ExternalTsFileQueryResource.java @@ -91,6 +91,7 @@ public class ExternalTsFileQueryResource implements AutoCloseable { private final List sharedDeviceEntries = new ArrayList<>(); private final List deviceTaskPartitions = new ArrayList<>(); private Comparator deviceEntryComparator; + private int deviceTaskPartitionCount; private volatile boolean closed; @@ -113,11 +114,15 @@ public ExternalTsFileQueryResource( public void collectDeviceEntries( SchemaFilter schemaFilter, Comparator comparator, int partitionCount) { checkNotClosed(); + if (partitionCount <= 0) { + throw new IllegalArgumentException( + DataNodeQueryMessages.EXTERNAL_TSFILE_DEVICE_TASK_PARTITION_COUNT_MUST_BE_POSITIVE); + } + this.deviceTaskPartitionCount = partitionCount; this.deviceEntryComparator = comparator; acquireMemoryForTsFileReaders(); ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = new ExternalTsFileDeviceFilterVisitor(); try (DeviceCollector deviceCollector = new DeviceCollector()) { - createDeviceTaskPartitions(partitionCount); while (deviceCollector.hasNextDevice()) { queryContext.checkTimeOut(); IDeviceID deviceID = deviceCollector.nextDevice(); @@ -133,8 +138,8 @@ public void collectDeviceEntries( DeviceTask deviceTask = new DeviceTask(deviceEntryIndex, deviceCollector.getCurrentDeviceOffsets()); DeviceTaskPartition partition = - deviceTaskPartitions.get( - Math.floorMod(deviceID.hashCode(), deviceTaskPartitions.size())); + getOrCreateDeviceTaskPartition( + Math.floorMod(deviceID.hashCode(), deviceTaskPartitionCount)); partition.add(deviceTask); if (partition.shouldFlush()) { partition.flush(); @@ -338,10 +343,6 @@ private void releaseDeviceTaskMemory() { unreservedBytes = 0; } - private boolean hasDeviceTasks() { - return !deviceEntryIndexes.isEmpty(); - } - void finish() { if (pendingDeviceTasks.isEmpty()) { return; @@ -383,25 +384,21 @@ private int compareDeviceEntryIndexes(int leftIndex, int rightIndex) { return result != 0 ? result : Integer.compare(leftIndex, rightIndex); } - private void createDeviceTaskPartitions(int partitionCount) { - if (partitionCount <= 0) { - throw new IllegalArgumentException( - DataNodeQueryMessages.EXTERNAL_TSFILE_DEVICE_TASK_PARTITION_COUNT_MUST_BE_POSITIVE); - } - for (int i = 0; i < partitionCount; i++) { - deviceTaskPartitions.add(new DeviceTaskPartition(i)); + private DeviceTaskPartition getOrCreateDeviceTaskPartition(int partitionIndex) { + for (DeviceTaskPartition partition : deviceTaskPartitions) { + if (partition.getPartitionIndex() == partitionIndex) { + return partition; + } } + DeviceTaskPartition partition = new DeviceTaskPartition(partitionIndex); + deviceTaskPartitions.add(partition); + return partition; } private void collectDeviceTaskPartitions() { - Iterator iterator = deviceTaskPartitions.iterator(); - while (iterator.hasNext()) { - DeviceTaskPartition partition = iterator.next(); + deviceTaskPartitions.sort(Comparator.comparingInt(DeviceTaskPartition::getPartitionIndex)); + for (DeviceTaskPartition partition : deviceTaskPartitions) { partition.finish(); - if (!partition.hasDeviceTasks()) { - iterator.remove(); - continue; - } partition.sortDeviceEntries(); } } From 81c3327fd326d3fd4ca4b531176e65f6fba187e9 Mon Sep 17 00:00:00 2001 From: shuwenwei Date: Wed, 17 Jun 2026 09:57:29 +0800 Subject: [PATCH 32/32] fix tests --- .../IoTDBReadTsFileTableFunctionIT.java | 21 +++++++++++++++---- .../buffer/TimeSeriesMetadataCacheTest.java | 3 +++ 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java index cde41419adc5f..b79623796afc9 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java @@ -24,6 +24,7 @@ import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.TableClusterIT; import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; +import org.apache.iotdb.itbase.env.BaseEnv; import org.apache.tsfile.enums.ColumnCategory; import org.apache.tsfile.enums.TSDataType; @@ -43,6 +44,7 @@ import java.io.IOException; import java.nio.file.Files; import java.sql.Connection; +import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; @@ -50,6 +52,8 @@ import static org.apache.iotdb.db.it.utils.TestUtils.tableAssertTestFail; import static org.apache.iotdb.db.it.utils.TestUtils.tableResultSetEqualTest; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; @RunWith(IoTDBTestRunner.class) @Category({TableLocalStandaloneIT.class, TableClusterIT.class}) @@ -402,10 +406,19 @@ public void testReadTsFileWithInvalidPaths() throws IOException { new File(dataNodeWrapper.getDataNodeDir() + File.separator + "data", "forbidden.tsfile"); Files.createDirectories(dataDir.getParentFile().toPath()); Files.write(dataDir.toPath(), new byte[0]); - tableAssertTestFail( - "SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(dataDir) + "')", - "is not allowed because it may access IoTDB data directory", - DATABASE_NAME); + try (Connection connection = + EnvFactory.getEnv() + .getWriteOnlyConnectionWithSpecifiedDataNode( + dataNodeWrapper, BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("USE " + DATABASE_NAME); + statement.execute("SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(dataDir) + "')"); + fail("Expected read_tsfile to reject paths under the current DataNode data directory"); + } catch (SQLException e) { + assertTrue( + e.getMessage(), + e.getMessage().contains("is not allowed because it may access IoTDB data directory")); + } } private static void generateTable( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCacheTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCacheTest.java index 0c368b41cca94..751eae090aec6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCacheTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCacheTest.java @@ -243,6 +243,7 @@ public void testIgnoreNotExistsResultIsNotCached() throws Exception { Mockito.when( reader.readTimeseriesMetadata( Mockito.eq(deviceID), + Mockito.isNull(), Mockito.eq(measurement), Mockito.eq(allSensors), Mockito.eq(true), @@ -251,6 +252,7 @@ public void testIgnoreNotExistsResultIsNotCached() throws Exception { Mockito.when( reader.readTimeseriesMetadata( Mockito.eq(deviceID), + Mockito.isNull(), Mockito.eq(measurement), Mockito.eq(allSensors), Mockito.eq(false), @@ -264,6 +266,7 @@ public void testIgnoreNotExistsResultIsNotCached() throws Exception { Mockito.verify(reader) .readTimeseriesMetadata( Mockito.eq(deviceID), + Mockito.isNull(), Mockito.eq(measurement), Mockito.eq(allSensors), Mockito.eq(false),