From 9067b9a25b661a930980852151af25717dc7484b Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 29 Aug 2025 16:38:33 +0800 Subject: [PATCH 01/69] cte materlization feat --- .../source/relational/CteScanOperator.java | 82 ++++++++++++ .../plan/planner/TableOperatorGenerator.java | 14 ++ .../planner/plan/node/PlanGraphPrinter.java | 20 +++ .../plan/planner/plan/node/PlanVisitor.java | 5 + .../plan/relational/analyzer/Analysis.java | 14 ++ .../relational/analyzer/CteDataStore.java | 58 +++++++++ .../analyzer/StatementAnalyzer.java | 1 + .../relational/planner/RelationPlanner.java | 24 +++- .../planner/TableLogicalPlanner.java | 3 + .../planner/ir/CteMaterializer.java | 108 ++++++++++++++++ .../relational/planner/node/CteScanNode.java | 122 ++++++++++++++++++ .../UnaliasSymbolReferences.java | 7 + .../plan/relational/sql/ast/Query.java | 9 ++ .../plan/relational/sql/ast/WithQuery.java | 28 +++- .../relational/sql/parser/AstBuilder.java | 11 +- .../relational/grammar/sql/RelationalSql.g4 | 2 +- 16 files changed, 496 insertions(+), 12 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/CteDataStore.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java new file mode 100644 index 000000000000..890ba87253de --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -0,0 +1,82 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, + * * software distributed under the License is distributed on an + * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * * KIND, either express or implied. See the License for the + * * specific language governing permissions and limitations + * * under the License. + * + */ + +package org.apache.iotdb.db.queryengine.execution.operator.source.relational; + +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.AbstractSourceOperator; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.CteDataStore; + +import org.apache.tsfile.read.common.block.TsBlock; + +public class CteScanOperator extends AbstractSourceOperator { + private final CteDataStore dataStore; + private final int totalTsBlockNum; + private int tsBlockIndex; + + public CteScanOperator( + OperatorContext operatorContext, PlanNodeId sourceId, CteDataStore dataStore) { + this.operatorContext = operatorContext; + this.sourceId = sourceId; + this.dataStore = dataStore; + this.tsBlockIndex = 0; + this.totalTsBlockNum = dataStore.getCachedData().size(); + } + + @Override + public TsBlock next() throws Exception { + return dataStore.getTsBlock(tsBlockIndex++); + } + + @Override + public boolean hasNext() throws Exception { + return tsBlockIndex < totalTsBlockNum; + } + + @Override + public void close() throws Exception {} + + @Override + public boolean isFinished() throws Exception { + return false; + } + + @Override + public long calculateMaxPeekMemory() { + return 0; + } + + @Override + public long calculateMaxReturnSize() { + return 0; + } + + @Override + public long calculateRetainedSizeAfterCallingNext() { + return 0; + } + + @Override + public long ramBytesUsed() { + return 0; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index e699ee417deb..d2da556ee832 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -119,6 +119,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.relational.AbstractTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.AsofMergeSortInnerJoinOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.AsofMergeSortLeftJoinOperator; +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.InformationSchemaTableScanOperator; @@ -183,6 +184,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AssignUniqueId; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CollectNode; +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.EnforceSingleRowNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExchangeNode; @@ -1191,6 +1193,18 @@ private void addSource( Long.MAX_VALUE); } + @Override + public Operator visitCteScan(CteScanNode node, LocalExecutionPlanContext context) { + OperatorContext operatorContext = + context + .getDriverContext() + .addOperatorContext( + context.getNextOperatorId(), + node.getPlanNodeId(), + CteScanOperator.class.getSimpleName()); + return new CteScanOperator(operatorContext, node.getPlanNodeId(), node.getDataStore()); + } + @Override public Operator visitTreeDeviceViewScan( TreeDeviceViewScanNode node, LocalExecutionPlanContext context) { 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 bd4dd912e4c1..61eec1fae4ef 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 @@ -69,6 +69,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AssignUniqueId; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CorrelatedJoinNode; +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.EnforceSingleRowNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExceptNode; @@ -623,6 +625,14 @@ public List visitColumnInject(ColumnInjectNode node, GraphContext contex } // =============== Methods below are used for table model ================ + @Override + public List visitCteScan(CteScanNode node, GraphContext context) { + List boxValue = new ArrayList<>(); + boxValue.add(String.format("CteScan-%s", node.getPlanNodeId().getId())); + boxValue.add(String.format("OutputSymbols: %s", node.getOutputSymbols())); + return render(node, boxValue, context); + } + @Override public List visitTableScan(TableScanNode node, GraphContext context) { DeviceTableScanNode deviceTableScanNode = null; @@ -990,6 +1000,16 @@ public List visitTopK( return render(node, boxValue, context); } + @Override + public List visitCorrelatedJoin(CorrelatedJoinNode node, GraphContext context) { + List boxValue = new ArrayList<>(); + boxValue.add(String.format("CorrelatedJoin-%s", node.getPlanNodeId().getId())); + boxValue.add(String.format("JoinType: %s", node.getJoinType())); + boxValue.add(String.format("Correlation: %s", node.getCorrelation())); + boxValue.add(String.format("Filter: %s", node.getFilter())); + return render(node, boxValue, context); + } + @Override public List visitJoin( org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode node, 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 4bebb692254a..e4ed7e342314 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 @@ -123,6 +123,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertTabletNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.GroupReference; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTreeDeviceViewScanNode; +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.ExceptNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GapFillNode; @@ -215,6 +216,10 @@ public R visitTimeSeriesRegionScan(TimeseriesRegionScanNode node, C context) { return visitRegionScan(node, context); } + public R visitCteScan(CteScanNode node, C context) { + return visitSourceNode(node, context); + } + // single child -------------------------------------------------------------------------------- public R visitSingleChildProcess(SingleChildProcessNode node, C context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java index b42559710d61..2e639efa9e0a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java @@ -121,6 +121,8 @@ public class Analysis implements IAnalysis { private String updateType; + private final Map namedQueriesDataStore = new LinkedHashMap<>(); + private final Map, Query> namedQueries = new LinkedHashMap<>(); // map expandable query to the node being the inner recursive reference @@ -278,6 +280,18 @@ public Query getNamedQuery(Table table) { return namedQueries.get(NodeRef.of(table)); } + public List getNamedQueries() { + return ImmutableList.copyOf(namedQueries.values()); + } + + public void addCteDataStore(Query query, CteDataStore dataStore) { + namedQueriesDataStore.put(query, dataStore); + } + + public CteDataStore getCTEDataStore(Query query) { + return namedQueriesDataStore.get(query); + } + public boolean isAnalyzed(Expression expression) { return expression instanceof DataType || types.containsKey(NodeRef.of(expression)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/CteDataStore.java new file mode 100644 index 000000000000..ee1ebaf4a172 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/CteDataStore.java @@ -0,0 +1,58 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, + * * software distributed under the License is distributed on an + * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * * KIND, either express or implied. See the License for the + * * specific language governing permissions and limitations + * * under the License. + * + */ + +package org.apache.iotdb.db.queryengine.plan.relational.analyzer; + +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.ArrayList; +import java.util.List; + +public class CteDataStore { + private final List cachedData; + private long cachedBytes; + + public CteDataStore() { + this.cachedData = new ArrayList<>(); + this.cachedBytes = 0L; + } + + public void addTsBlock(TsBlock tsBlock) { + cachedData.add(tsBlock); + cachedBytes += tsBlock.getRetainedSizeInBytes(); + } + + public void clear() { + cachedData.clear(); + cachedBytes = 0L; + } + + public List getCachedData() { + return cachedData; + } + + public TsBlock getTsBlock(int index) { + if (cachedData.isEmpty()) { + return null; + } + return cachedData.get(index); + } +} 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 e13c52ba8b16..a83c3581029b 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 @@ -3114,6 +3114,7 @@ protected Scope visitTable(Table table, Optional scope) { private Scope createScopeForCommonTableExpression( Table table, Optional scope, WithQuery withQuery) { Query query = withQuery.getQuery(); + query.setMaterialized(withQuery.isMaterialized()); analysis.registerNamedQuery(table, query); // re-alias the fields with the name assigned to the query in the WITH declaration 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 9732f8221d63..01df52532037 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 @@ -37,6 +37,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.analyzer.Analysis; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.CteDataStore; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Field; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.PatternRecognitionAnalysis; @@ -54,6 +55,7 @@ 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; +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.ExceptNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; @@ -242,14 +244,30 @@ protected RelationPlan visitTable(final Table table, final Void context) { // Common Table Expression final Query namedQuery = analysis.getNamedQuery(table); if (namedQuery != null) { - RelationPlan subPlan; if (analysis.isExpandableQuery(namedQuery)) { // recursive cte throw new SemanticException("unexpected recursive cte"); - } else { - subPlan = process(namedQuery, null); } + if (namedQuery.isMaterialized()) { + CteDataStore dataStore = analysis.getCTEDataStore(namedQuery); + if (dataStore != null) { + List outputSymbols = + analysis.getOutputDescriptor(table).getAllFields().stream() + .map(symbolAllocator::newSymbol) + .collect(toImmutableList()); + + // CTE Scan Node + return new RelationPlan( + new CteScanNode( + idAllocator.genPlanNodeId(), table.getName(), outputSymbols, dataStore), + scope, + outputSymbols, + outerContext); + } + } + + RelationPlan subPlan = process(namedQuery, null); // Add implicit coercions if view query produces types that don't match the declared output // types of the view (e.g., if the underlying tables referenced by the view changed) List types = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index f69d7cd2cebb..30a6cb2abb98 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -50,6 +50,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; +import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.CteMaterializer; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; @@ -372,6 +373,8 @@ private RelationPlan createRelationPlan(Analysis analysis, PipeEnriched pipeEnri } private RelationPlan createRelationPlan(Analysis analysis, Query query) { + // materialize cte if needed + CteMaterializer.materializeCTE(analysis, queryContext); return getRelationPlanner(analysis).process(query, null); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java new file mode 100644 index 000000000000..7b6a3bbe351a --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -0,0 +1,108 @@ +/* + * + * * 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.ir; + +import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.commons.exception.IoTDBRuntimeException; +import org.apache.iotdb.db.protocol.session.SessionManager; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.plan.Coordinator; +import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; +import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.CteDataStore; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; +import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; + +public class CteMaterializer { + private static final SqlParser relationSqlParser = new SqlParser(); + + private static final Coordinator coordinator = Coordinator.getInstance(); + + private CteMaterializer() {} + + public static void materializeCTE(Analysis analysis, MPPQueryContext context) { + Set materializedQueries = new HashSet<>(); + for (Query query : analysis.getNamedQueries()) { + if (query.isMaterialized() && !materializedQueries.contains(query)) { + CteDataStore dataStore = fetchCteQueryResult(query, context); + if (dataStore != null) { + analysis.addCteDataStore(query, dataStore); + materializedQueries.add(query); + } + } + } + } + + public static CteDataStore fetchCteQueryResult(Query query, MPPQueryContext context) { + final long queryId = SessionManager.getInstance().requestQueryId(); + Throwable t = null; + try { + final ExecutionResult executionResult = + coordinator.executeForTableModel( + query, + new SqlParser(), + SessionManager.getInstance().getCurrSession(), + queryId, + SessionManager.getInstance() + .getSessionInfoOfTableModel(SessionManager.getInstance().getCurrSession()), + "Materialize common table expression", + LocalExecutionPlanner.getInstance().metadata, + context.getTimeOut(), + false); + if (executionResult.status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return null; + } + + CteDataStore cteDataStore = new CteDataStore(); + while (coordinator.getQueryExecution(queryId).hasNextResult()) { + final Optional tsBlock; + try { + tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); + } catch (final IoTDBException e) { + t = e; + throw new IoTDBRuntimeException( + String.format("Fail to materialize CTE because %s", e.getMessage()), + e.getErrorCode(), + e.isUserException()); + } + if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { + continue; + } + cteDataStore.addTsBlock(tsBlock.get()); + } + return cteDataStore; + } catch (final Throwable throwable) { + t = throwable; + } finally { + coordinator.cleanupQueryExecution(queryId, null, t); + } + return null; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java new file mode 100644 index 000000000000..286a385a6d20 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java @@ -0,0 +1,122 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, + * * software distributed under the License is distributed on an + * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * * KIND, either express or implied. See the License for the + * * specific language governing permissions and limitations + * * under the License. + * + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.node; + +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SourceNode; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.CteDataStore; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.QualifiedName; + +import com.google.common.collect.ImmutableList; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkArgument; + +public class CteScanNode extends SourceNode { + private final QualifiedName qualifiedName; + // Indicate the column this node need to output + private final List outputSymbols; + private final CteDataStore dataStore; + + public CteScanNode( + PlanNodeId id, + QualifiedName qualifiedName, + List outputSymbols, + CteDataStore dataStore) { + super(id); + this.qualifiedName = qualifiedName; + this.outputSymbols = outputSymbols; + this.dataStore = dataStore; + } + + public CteDataStore getDataStore() { + return dataStore; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCteScan(this, context); + } + + @Override + public void open() throws Exception {} + + @Override + public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {} + + @Override + public void close() throws Exception {} + + @Override + public TRegionReplicaSet getRegionReplicaSet() { + return null; + } + + @Override + public List getChildren() { + return ImmutableList.of(); + } + + @Override + public void addChild(PlanNode child) {} + + @Override + public PlanNode clone() { + return new CteScanNode(getPlanNodeId(), qualifiedName, outputSymbols, dataStore); + } + + @Override + public int allowedChildCount() { + return 0; + } + + @Override + public List getOutputColumnNames() { + return outputSymbols.stream().map(Symbol::getName).collect(Collectors.toList()); + } + + @Override + public List getOutputSymbols() { + return outputSymbols; + } + + @Override + public PlanNode replaceChildren(List newChildren) { + checkArgument(newChildren.isEmpty(), "newChildren is not empty"); + return this; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) {} + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException {} +} 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 3fdbec6b0ccf..1b56b6d1ee41 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 @@ -34,6 +34,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ApplyNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AssignUniqueId; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CorrelatedJoinNode; +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.EnforceSingleRowNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExceptNode; @@ -266,6 +267,12 @@ public PlanAndMappings visitInformationSchemaTableScan( mapping); } + @Override + public PlanAndMappings visitCteScan(CteScanNode node, UnaliasContext context) { + Map mapping = new HashMap<>(context.getCorrelationMapping()); + return new PlanAndMappings(node, mapping); + } + @Override public PlanAndMappings visitGapFill(GapFillNode node, UnaliasContext context) { PlanAndMappings rewrittenSource = node.getChild().accept(this, context); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java index c5df5f846a01..12f0b25718b9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java @@ -40,6 +40,7 @@ public class Query extends Statement { private final Optional orderBy; private final Optional offset; private final Optional limit; + private boolean materialized = false; public Query( Optional with, @@ -102,6 +103,14 @@ public Optional getLimit() { return limit; } + public boolean isMaterialized() { + return materialized; + } + + public void setMaterialized(boolean materialized) { + this.materialized = materialized; + } + @Override public R accept(AstVisitor visitor, C context) { return visitor.visitQuery(this, context); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/WithQuery.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/WithQuery.java index ee215cb1bd98..3d7fb15e4f79 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/WithQuery.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/WithQuery.java @@ -38,36 +38,46 @@ public class WithQuery extends Node { private final Identifier name; private final Query query; + private final boolean materialized; @Nullable private final List columnNames; - public WithQuery(Identifier name, Query query) { + public WithQuery(Identifier name, Query query, boolean materialized) { super(null); this.name = name; this.query = requireNonNull(query, "query is null"); this.columnNames = null; + this.materialized = materialized; } - public WithQuery(Identifier name, Query query, List columnNames) { + public WithQuery( + Identifier name, Query query, List columnNames, boolean materialized) { super(null); this.name = name; this.query = requireNonNull(query, "query is null"); this.columnNames = requireNonNull(columnNames, "columnNames is null"); + this.materialized = materialized; } - public WithQuery(NodeLocation location, Identifier name, Query query) { + public WithQuery(NodeLocation location, Identifier name, Query query, boolean materialized) { super(requireNonNull(location, "location is null")); this.name = name; this.query = requireNonNull(query, "query is null"); this.columnNames = null; + this.materialized = materialized; } public WithQuery( - NodeLocation location, Identifier name, Query query, List columnNames) { + NodeLocation location, + Identifier name, + Query query, + List columnNames, + boolean materialized) { super(requireNonNull(location, "location is null")); this.name = name; this.query = requireNonNull(query, "query is null"); this.columnNames = requireNonNull(columnNames, "columnNames is null"); + this.materialized = materialized; } public Identifier getName() { @@ -82,6 +92,10 @@ public Optional> getColumnNames() { return Optional.ofNullable(columnNames); } + public boolean isMaterialized() { + return materialized; + } + @Override public R accept(AstVisitor visitor, C context) { return visitor.visitWithQuery(this, context); @@ -98,13 +112,14 @@ public String toString() { .add("name", name) .add("query", query) .add("columnNames", columnNames) + .add("materialized", materialized) .omitNullValues() .toString(); } @Override public int hashCode() { - return Objects.hash(name, query, columnNames); + return Objects.hash(name, query, columnNames, materialized); } @Override @@ -118,7 +133,8 @@ public boolean equals(Object obj) { WithQuery o = (WithQuery) obj; return Objects.equals(name, o.name) && Objects.equals(query, o.query) - && Objects.equals(columnNames, o.columnNames); + && Objects.equals(columnNames, o.columnNames) + && Objects.equals(materialized, o.materialized); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java index b041c56b2bc9..937c05b7ac9d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java @@ -2097,10 +2097,17 @@ public Node visitNamedQuery(RelationalSqlParser.NamedQueryContext ctx) { if (ctx.columnAliases() != null) { List columns = visit(ctx.columnAliases().identifier(), Identifier.class); return new WithQuery( - getLocation(ctx), (Identifier) visit(ctx.name), (Query) visit(ctx.query()), columns); + getLocation(ctx), + (Identifier) visit(ctx.name), + (Query) visit(ctx.query()), + columns, + ctx.MATERIALIZED() != null); } else { return new WithQuery( - getLocation(ctx), (Identifier) visit(ctx.name), (Query) visit(ctx.query())); + getLocation(ctx), + (Identifier) visit(ctx.name), + (Query) visit(ctx.query()), + ctx.MATERIALIZED() != null); } } diff --git a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 index 1690ea4c855b..be6844302bb9 100644 --- a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 +++ b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 @@ -1019,7 +1019,7 @@ groupingSet ; namedQuery - : name=identifier (columnAliases)? AS '(' query ')' + : name=identifier (columnAliases)? AS MATERIALIZED? '(' query ')' ; setQuantifier From 5dfd94a24d4fa790521c1ac25d5a48c134dc7300 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 1 Sep 2025 16:43:03 +0800 Subject: [PATCH 02/69] spill data --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 12 ++ .../source/relational/CteScanOperator.java | 60 ++++++-- .../plan/relational/analyzer/Analysis.java | 5 +- .../relational/planner/RelationPlanner.java | 2 +- .../planner/ir/CteMaterializer.java | 38 +++-- .../relational/planner/node/CteScanNode.java | 2 +- .../iotdb/db/utils/cte/CteDataReader.java | 35 +++++ .../analyzer => utils/cte}/CteDataStore.java | 38 +++-- .../iotdb/db/utils/cte/DiskSpiller.java | 125 ++++++++++++++++ .../iotdb/db/utils/cte/FileSpillerReader.java | 134 ++++++++++++++++++ .../iotdb/db/utils/cte/MemoryReader.java | 57 ++++++++ .../iotdb/commons/conf/IoTDBConstant.java | 1 + 12 files changed, 475 insertions(+), 34 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/{queryengine/plan/relational/analyzer => utils/cte}/CteDataStore.java (61%) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index b2c808654677..a8d1fced9309 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -322,6 +322,14 @@ public class IoTDBConfig { private String sortTmpDir = IoTDBConstant.DN_DEFAULT_DATA_DIR + File.separator + IoTDBConstant.TMP_FOLDER_NAME; + /** temp result directory for cte */ + private String cteTmpDir = + IoTDBConstant.DN_DEFAULT_DATA_DIR + + File.separator + + IoTDBConstant.CTE_FOLDER_NAME + + File.separator + + IoTDBConstant.TMP_FOLDER_NAME; + /** Maximum MemTable number. Invalid when enableMemControl is true. */ private int maxMemtableNumber = 0; @@ -4189,6 +4197,10 @@ public String getSortTmpDir() { return sortTmpDir; } + public String getCteTmpDir() { + return cteTmpDir; + } + public String getObjectStorageBucket() { throw new UnsupportedOperationException("object storage is not supported yet"); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 890ba87253de..81d1d491c1b2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -21,43 +21,74 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; +import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.AbstractSourceOperator; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.db.queryengine.plan.relational.analyzer.CteDataStore; +import org.apache.iotdb.db.utils.cte.CteDataReader; +import org.apache.iotdb.db.utils.cte.CteDataStore; +import org.apache.iotdb.db.utils.cte.MemoryReader; import org.apache.tsfile.read.common.block.TsBlock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; public class CteScanOperator extends AbstractSourceOperator { + private static final Logger logger = LoggerFactory.getLogger(CteScanOperator.class); + private final CteDataStore dataStore; - private final int totalTsBlockNum; - private int tsBlockIndex; + private List dataReaders; + private int readerIndex; public CteScanOperator( OperatorContext operatorContext, PlanNodeId sourceId, CteDataStore dataStore) { this.operatorContext = operatorContext; this.sourceId = sourceId; this.dataStore = dataStore; - this.tsBlockIndex = 0; - this.totalTsBlockNum = dataStore.getCachedData().size(); } @Override public TsBlock next() throws Exception { - return dataStore.getTsBlock(tsBlockIndex++); + if (dataReaders == null || readerIndex >= dataReaders.size()) { + return null; + } + return dataReaders.get(readerIndex).next(); } @Override public boolean hasNext() throws Exception { - return tsBlockIndex < totalTsBlockNum; + if (dataReaders == null) { + prepareReaders(); + } + while (readerIndex < dataReaders.size()) { + if (dataReaders.get(readerIndex).hasNext()) { + return true; + } else { + readerIndex++; + } + } + return false; } @Override - public void close() throws Exception {} + public void close() throws Exception { + try { + if (dataReaders != null) { + for (CteDataReader reader : dataReaders) { + reader.close(); + } + } + } catch (Exception e) { + logger.error("Fail to close fileChannel", e); + } + } @Override public boolean isFinished() throws Exception { - return false; + return !hasNextWithTimer(); } @Override @@ -79,4 +110,15 @@ public long calculateRetainedSizeAfterCallingNext() { public long ramBytesUsed() { return 0; } + + private void prepareReaders() throws IoTDBException { + if (dataReaders != null) { + return; + } + dataReaders = new ArrayList<>(); + dataReaders.addAll(dataStore.getDiskSpiller().getReaders()); + if (dataStore.getCachedBytes() != 0) { + dataReaders.add(new MemoryReader(dataStore.getCachedData())); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java index 2e639efa9e0a..c6d30cec7008 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java @@ -70,6 +70,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.TableFunctionInvocation; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WindowFrame; import org.apache.iotdb.db.queryengine.plan.statement.component.FillPolicy; +import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ImmutableList; @@ -280,8 +281,8 @@ public Query getNamedQuery(Table table) { return namedQueries.get(NodeRef.of(table)); } - public List getNamedQueries() { - return ImmutableList.copyOf(namedQueries.values()); + public Map, Query> getNamedQueries() { + return namedQueries; } public void addCteDataStore(Query query, CteDataStore dataStore) { 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 01df52532037..53f4320c91c1 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 @@ -37,7 +37,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.analyzer.Analysis; -import org.apache.iotdb.db.queryengine.plan.relational.analyzer.CteDataStore; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Field; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.PatternRecognitionAnalysis; @@ -127,6 +126,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowsStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; +import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index 7b6a3bbe351a..d9790a52e9a3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -23,19 +23,22 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; -import org.apache.iotdb.db.queryengine.plan.relational.analyzer.CteDataStore; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; +import org.apache.iotdb.db.utils.cte.CteDataStore; +import org.apache.iotdb.db.utils.cte.DiskSpiller; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.read.common.block.TsBlock; +import java.io.File; import java.util.HashSet; import java.util.Optional; import java.util.Set; @@ -49,18 +52,23 @@ private CteMaterializer() {} public static void materializeCTE(Analysis analysis, MPPQueryContext context) { Set materializedQueries = new HashSet<>(); - for (Query query : analysis.getNamedQueries()) { - if (query.isMaterialized() && !materializedQueries.contains(query)) { - CteDataStore dataStore = fetchCteQueryResult(query, context); - if (dataStore != null) { - analysis.addCteDataStore(query, dataStore); - materializedQueries.add(query); - } - } - } + analysis + .getNamedQueries() + .forEach( + (tableRef, query) -> { + if (query.isMaterialized() && !materializedQueries.contains(query)) { + CteDataStore dataStore = + fetchCteQueryResult(tableRef.getNode().getName().toString(), query, context); + if (dataStore != null) { + analysis.addCteDataStore(query, dataStore); + materializedQueries.add(query); + } + } + }); } - public static CteDataStore fetchCteQueryResult(Query query, MPPQueryContext context) { + public static CteDataStore fetchCteQueryResult( + String cteName, Query query, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; try { @@ -80,7 +88,13 @@ public static CteDataStore fetchCteQueryResult(Query query, MPPQueryContext cont return null; } - CteDataStore cteDataStore = new CteDataStore(); + String folderPath = + IoTDBDescriptor.getInstance().getConfig().getCteTmpDir() + + File.separator + + queryId + + File.separator; + CteDataStore cteDataStore = + new CteDataStore(new DiskSpiller(folderPath, folderPath + cteName)); while (coordinator.getQueryExecution(queryId).hasNextResult()) { final Optional tsBlock; try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java index 286a385a6d20..9a0788670f84 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java @@ -26,9 +26,9 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SourceNode; -import org.apache.iotdb.db.queryengine.plan.relational.analyzer.CteDataStore; import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.QualifiedName; +import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.collect.ImmutableList; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java new file mode 100644 index 000000000000..5ee2f88efa8f --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java @@ -0,0 +1,35 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, + * * software distributed under the License is distributed on an + * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * * KIND, either express or implied. See the License for the + * * specific language governing permissions and limitations + * * under the License. + * + */ + +package org.apache.iotdb.db.utils.cte; + +import org.apache.iotdb.commons.exception.IoTDBException; + +import org.apache.tsfile.read.common.block.TsBlock; + +public interface CteDataReader { + + boolean hasNext() throws IoTDBException; + + TsBlock next() throws IoTDBException; + + void close() throws IoTDBException; +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java similarity index 61% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/CteDataStore.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index ee1ebaf4a172..43ca36cd98eb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -19,7 +19,9 @@ * */ -package org.apache.iotdb.db.queryengine.plan.relational.analyzer; +package org.apache.iotdb.db.utils.cte; + +import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.tsfile.read.common.block.TsBlock; @@ -27,17 +29,29 @@ import java.util.List; public class CteDataStore { + // 1MB buffer size + private static final long CTE_BUFFER_SIZE = 1024 * 1024L; + private final List cachedData; private long cachedBytes; + private final DiskSpiller diskSpiller; - public CteDataStore() { + public CteDataStore(DiskSpiller diskSpiller) { this.cachedData = new ArrayList<>(); this.cachedBytes = 0L; + this.diskSpiller = diskSpiller; } - public void addTsBlock(TsBlock tsBlock) { + public void addTsBlock(TsBlock tsBlock) throws IoTDBException { + long bytesSize = tsBlock.getRetainedSizeInBytes(); + if (bytesSize + cachedBytes < CTE_BUFFER_SIZE) { + cachedBytes += bytesSize; + } else { + spill(); + cachedData.clear(); + cachedBytes = bytesSize; + } cachedData.add(tsBlock); - cachedBytes += tsBlock.getRetainedSizeInBytes(); } public void clear() { @@ -45,14 +59,20 @@ public void clear() { cachedBytes = 0L; } + public DiskSpiller getDiskSpiller() { + return diskSpiller; + } + public List getCachedData() { return cachedData; } - public TsBlock getTsBlock(int index) { - if (cachedData.isEmpty()) { - return null; - } - return cachedData.get(index); + public long getCachedBytes() { + return cachedBytes; + } + + private void spill() throws IoTDBException { + // TODO: allocate memory + diskSpiller.spill(cachedData); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java new file mode 100644 index 000000000000..c1d899d07a32 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java @@ -0,0 +1,125 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, + * * software distributed under the License is distributed on an + * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * * KIND, either express or implied. See the License for the + * * specific language governing permissions and limitations + * * under the License. + * + */ + +package org.apache.iotdb.db.utils.cte; + +import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.column.TsBlockSerde; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.List; + +public class DiskSpiller { + private static final String FILE_SUFFIX = ".cteTemp"; + private final String folderPath; + private final String filePrefix; + + private int fileIndex; + private boolean folderCreated = false; + private final TsBlockSerde serde = new TsBlockSerde(); + + public DiskSpiller(String folderPath, String filePrefix) { + this.folderPath = folderPath; + this.filePrefix = filePrefix + "-"; + this.fileIndex = 0; + } + + private void createFolder(String folderPath) throws IoTDBException { + try { + Path path = Paths.get(folderPath); + Files.createDirectories(path); + folderCreated = true; + } catch (IOException e) { + throw new IoTDBException( + "Create folder error: " + folderPath, + e, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + } + + public void spill(List tsBlocks) throws IoTDBException { + if (!folderCreated) { + createFolder(folderPath); + } + String fileName = filePrefix + String.format("%05d", fileIndex) + FILE_SUFFIX; + fileIndex++; + + writeData(tsBlocks, fileName); + } + + public List getFilePaths() { + List filePaths = new ArrayList<>(); + for (int i = 0; i < fileIndex; i++) { + filePaths.add(filePrefix + String.format("%05d", i) + FILE_SUFFIX); + } + return filePaths; + } + + public List getReaders() throws IoTDBException { + List filePaths = getFilePaths(); + List fileReaders = new ArrayList<>(); + try { + for (String filePath : filePaths) { + fileReaders.add(new FileSpillerReader(filePath, serde)); + } + } catch (IOException e) { + throw new IoTDBException( + "Can't get file for FileSpillerReader, check if the file exists: " + filePaths, + e, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + return fileReaders; + } + + private void writeData(List tsBlocks, String fileName) throws IoTDBException { + Path filePath = Paths.get(fileName); + try (FileChannel fileChannel = + FileChannel.open( + filePath, + StandardOpenOption.WRITE, + StandardOpenOption.TRUNCATE_EXISTING, + StandardOpenOption.CREATE)) { + for (TsBlock tsBlock : tsBlocks) { + ByteBuffer tsBlockBuffer = serde.serialize(tsBlock); + ByteBuffer length = ByteBuffer.allocate(4); + length.putInt(tsBlockBuffer.capacity()); + length.flip(); + fileChannel.write(length); + fileChannel.write(tsBlockBuffer); + } + } catch (IOException e) { + throw new IoTDBException( + "Can't write CTE data to file: " + fileName, + e, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java new file mode 100644 index 000000000000..9480f560fa8e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java @@ -0,0 +1,134 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, + * * software distributed under the License is distributed on an + * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * * KIND, either express or implied. See the License for the + * * specific language governing permissions and limitations + * * under the License. + * + */ + +package org.apache.iotdb.db.utils.cte; + +import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.common.conf.TSFileDescriptor; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.column.TsBlockSerde; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.List; + +public class FileSpillerReader implements CteDataReader { + private final FileChannel fileChannel; + private final List cacheBlocks; + private final String fileName; + private final TsBlockSerde serde; + + private int tsBlockIndex; + private boolean isEnd = false; + + private static final int DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES = + TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + + public FileSpillerReader(String fileName, TsBlockSerde serde) throws IOException { + this.fileName = fileName; + this.serde = serde; + this.tsBlockIndex = 0; + this.fileChannel = FileChannel.open(Paths.get(fileName), StandardOpenOption.READ); + this.cacheBlocks = new ArrayList<>(); + } + + @Override + public TsBlock next() throws IoTDBException { + if (cacheBlocks == null || tsBlockIndex >= cacheBlocks.size()) { + return null; + } + return cacheBlocks.get(tsBlockIndex++); + } + + @Override + public boolean hasNext() throws IoTDBException { + if (isEnd) { + return false; + } + + if (cacheBlocks.isEmpty() || tsBlockIndex == cacheBlocks.size() - 1) { + boolean hasData = readTsBlockFromFile(); + if (!hasData) { + isEnd = true; + return false; + } + return true; + } + + return tsBlockIndex < cacheBlocks.size(); + } + + @Override + public void close() throws IoTDBException { + try { + fileChannel.close(); + } catch (IOException e) { + throw new IoTDBException( + "Can't close FileSpillerReader: " + fileName, + e, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + } + + private boolean readTsBlockFromFile() throws IoTDBException { + long bufferSize = Long.MAX_VALUE; + cacheBlocks.clear(); + while (bufferSize >= DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES) { + long size = read(); + if (size == -1) { + break; + } + bufferSize -= size; + } + + tsBlockIndex = 0; + return !cacheBlocks.isEmpty(); + } + + private long read() throws IoTDBException { + try { + ByteBuffer bytes = ByteBuffer.allocate(4); + int readLen = fileChannel.read(bytes); + if (readLen == -1) { + return -1; + } + bytes.flip(); + int capacity = bytes.getInt(); + ByteBuffer tsBlockBytes = ByteBuffer.allocate(capacity); + fileChannel.read(tsBlockBytes); + tsBlockBytes.flip(); + TsBlock cachedTsBlock = serde.deserialize(tsBlockBytes); + cacheBlocks.add(cachedTsBlock); + return cachedTsBlock.getRetainedSizeInBytes(); + } catch (IOException e) { + throw new IoTDBException( + "Can't read a new tsBlock in FileSpillerReader: " + fileName, + e, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java new file mode 100644 index 000000000000..d14cb4605e19 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java @@ -0,0 +1,57 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, + * * software distributed under the License is distributed on an + * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * * KIND, either express or implied. See the License for the + * * specific language governing permissions and limitations + * * under the License. + * + */ + +package org.apache.iotdb.db.utils.cte; + +import org.apache.iotdb.commons.exception.IoTDBException; + +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.List; + +public class MemoryReader implements CteDataReader { + // all the data in MemoryReader lies in memory + private final List cachedData; + private final int size; + private int tsBlockIndex; + + public MemoryReader(List cachedTsBlock) { + this.cachedData = cachedTsBlock; + this.size = cachedTsBlock.size(); + this.tsBlockIndex = 0; + } + + @Override + public boolean hasNext() throws IoTDBException { + return cachedData != null && tsBlockIndex < size; + } + + @Override + public TsBlock next() throws IoTDBException { + if (cachedData == null || tsBlockIndex >= size) { + return null; + } + return cachedData.get(tsBlockIndex++); + } + + @Override + public void close() throws IoTDBException {} +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java index adf72842797e..28c3f8aececd 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java @@ -268,6 +268,7 @@ private IoTDBConstant() {} public static final String UDF_FOLDER_NAME = "udf"; public static final String TRIGGER_FOLDER_NAME = "trigger"; public static final String PIPE_FOLDER_NAME = "pipe"; + public static final String CTE_FOLDER_NAME = "cte"; public static final String TMP_FOLDER_NAME = "tmp"; public static final String DELETION_FOLDER_NAME = "deletion"; From e38e8fccfb03936aaf983a6c998e8723379e030e Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 4 Sep 2025 16:27:53 +0800 Subject: [PATCH 03/69] CTE scan operator and CTE materialization memory control # Conflicts: # iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java # Conflicts: # iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java # Conflicts: # iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 16 +++++++ .../apache/iotdb/db/conf/IoTDBDescriptor.java | 6 +++ .../source/relational/CteScanOperator.java | 21 ++++++--- .../plan/execution/QueryExecution.java | 33 ++++++++++++++ .../plan/planner/plan/node/PlanNodeType.java | 1 + .../plan/relational/analyzer/Analysis.java | 14 +++--- .../relational/planner/RelationPlanner.java | 2 +- .../planner/ir/CteMaterializer.java | 27 +++++++++--- .../relational/planner/node/CteScanNode.java | 4 ++ .../iotdb/db/utils/cte/CteDataReader.java | 25 ++++++++++- .../iotdb/db/utils/cte/CteDataStore.java | 12 ++---- .../iotdb/db/utils/cte/DiskSpiller.java | 43 ++++++++++--------- .../iotdb/db/utils/cte/FileSpillerReader.java | 40 +++++++++++------ .../iotdb/db/utils/cte/MemoryReader.java | 15 +++++-- .../conf/iotdb-system.properties.template | 11 +++++ 15 files changed, 207 insertions(+), 63 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index a8d1fced9309..99f18f6b22d8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -429,6 +429,9 @@ public class IoTDBConfig { /** The buffer for sort operation */ private long sortBufferSize = 32 * 1024 * 1024L; + /** The buffer for cte scan operation */ + private long cteBufferSize = 32 * 1024 * 1024L; + /** Mods cache size limit per fi */ private long modsCacheSizeLimitPerFI = 32 * 1024 * 1024; @@ -1379,6 +1382,7 @@ private void formulateFolders() { extPipeDir = addDataHomeDir(extPipeDir); queryDir = addDataHomeDir(queryDir); sortTmpDir = addDataHomeDir(sortTmpDir); + cteTmpDir = addDataHomeDir(cteTmpDir); formulateDataDirs(tierDataDirs); } @@ -4181,6 +4185,14 @@ public long getSortBufferSize() { return sortBufferSize; } + public void setCteBufferSize(long cteBufferSize) { + this.cteBufferSize = cteBufferSize; + } + + public long getCteBufferSize() { + return cteBufferSize; + } + public void setModsCacheSizeLimitPerFI(long modsCacheSizeLimitPerFI) { this.modsCacheSizeLimitPerFI = modsCacheSizeLimitPerFI; } @@ -4197,6 +4209,10 @@ public String getSortTmpDir() { return sortTmpDir; } + public void setCteTmpDir(String cteTmpDir) { + this.cteTmpDir = cteTmpDir; + } + public String getCteTmpDir() { return cteTmpDir; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 2915365262db..5e20fe805012 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -1057,12 +1057,18 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException // The buffer for sort operator to calculate loadFixedSizeLimitForQuery(properties, "sort_buffer_size_in_bytes", conf::setSortBufferSize); + // The buffer for cte scan operator + loadFixedSizeLimitForQuery(properties, "cte_buffer_size_in_bytes", conf::setCteBufferSize); + loadFixedSizeLimitForQuery( properties, "mods_cache_size_limit_per_fi_in_bytes", conf::setModsCacheSizeLimitPerFI); // tmp filePath for sort operator conf.setSortTmpDir(properties.getProperty("sort_tmp_dir", conf.getSortTmpDir())); + // tmp filePath for cte + conf.setCteTmpDir(properties.getProperty("cte_tmp_dir", conf.getCteTmpDir())); + conf.setRateLimiterType(properties.getProperty("rate_limiter_type", conf.getRateLimiterType())); conf.setDataNodeSchemaCacheEvictionPolicy( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 81d1d491c1b2..323700fa470c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -22,6 +22,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.AbstractSourceOperator; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; @@ -30,6 +31,7 @@ import org.apache.iotdb.db.utils.cte.MemoryReader; import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.utils.RamUsageEstimator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,7 +39,9 @@ import java.util.List; public class CteScanOperator extends AbstractSourceOperator { - private static final Logger logger = LoggerFactory.getLogger(CteScanOperator.class); + private static final Logger LOGGER = LoggerFactory.getLogger(CteScanOperator.class); + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(CteScanOperator.class); private final CteDataStore dataStore; private List dataReaders; @@ -82,7 +86,7 @@ public void close() throws Exception { } } } catch (Exception e) { - logger.error("Fail to close fileChannel", e); + LOGGER.error("Fail to close fileChannel", e); } } @@ -93,22 +97,27 @@ public boolean isFinished() throws Exception { @Override public long calculateMaxPeekMemory() { - return 0; + if (dataReaders == null || readerIndex >= dataReaders.size()) { + return 0; + } + return dataReaders.get(readerIndex).bytesUsed(); } @Override public long calculateMaxReturnSize() { - return 0; + // The returned object is a reference to TsBlock in CteDataReader + return RamUsageEstimator.NUM_BYTES_OBJECT_REF; } @Override public long calculateRetainedSizeAfterCallingNext() { - return 0; + return calculateMaxPeekMemoryWithCounter(); } @Override public long ramBytesUsed() { - return 0; + return INSTANCE_SIZE + + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext); } private void prepareReaders() throws IoTDBException { 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 4734db5850a4..8003f6cf3ef8 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 @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; import org.apache.iotdb.commons.service.metric.PerformanceOverviewMetrics; +import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.query.KilledByOthersException; @@ -47,6 +48,8 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeUtil; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; +import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.CteMaterializer; import org.apache.iotdb.db.queryengine.plan.scheduler.IScheduler; import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId; @@ -59,6 +62,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.nio.ByteBuffer; import java.time.format.DateTimeParseException; import java.util.List; @@ -349,6 +353,9 @@ private void releaseResource() { resultHandle.close(); cleanUpResultHandle(); } + if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE)) { + CteMaterializer.cleanUpCTE((Analysis) analysis, context); + } } private void cleanUpResultHandle() { @@ -373,6 +380,15 @@ private void cleanUpResultHandle() { public void stopAndCleanup(Throwable t) { stop(t); releaseResource(t); + + try { + // delete cte tmp file if exists + deleteTmpFile(); + } catch (Throwable err) { + LOGGER.error( + "Errors occurred while attempting to delete tmp files, potentially leading to resource leakage.", + err); + } } /** Release the resources that current QueryExecution hold with a specified exception */ @@ -392,6 +408,9 @@ private void releaseResource(Throwable t) { } cleanUpResultHandle(); } + if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE)) { + CteMaterializer.cleanUpCTE((Analysis) analysis, context); + } } /** @@ -717,4 +736,18 @@ public String toString() { public ScheduledExecutorService getScheduledExecutor() { return planner.getScheduledExecutorService(); } + + private void deleteTmpFile() { + if (context.mayHaveTmpFile()) { + String tmpFilePath = + IoTDBDescriptor.getInstance().getConfig().getCteTmpDir() + + File.separator + + context.getQueryId() + + File.separator; + File tmpFile = new File(tmpFilePath); + if (tmpFile.exists()) { + FileUtils.deleteFileOrDirectory(tmpFile, true); + } + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java index 6ebfa6dcfdc4..851aacfa8523 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java @@ -119,6 +119,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertTabletNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AssignUniqueId; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.EnforceSingleRowNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExceptNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GapFillNode; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java index c6d30cec7008..fb5426dc277e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java @@ -122,7 +122,7 @@ public class Analysis implements IAnalysis { private String updateType; - private final Map namedQueriesDataStore = new LinkedHashMap<>(); + private final Map cteDataStores = new LinkedHashMap<>(); private final Map, Query> namedQueries = new LinkedHashMap<>(); @@ -285,12 +285,16 @@ public Map, Query> getNamedQueries() { return namedQueries; } - public void addCteDataStore(Query query, CteDataStore dataStore) { - namedQueriesDataStore.put(query, dataStore); + public void addCteDataStore(String cteName, CteDataStore dataStore) { + cteDataStores.put(cteName, dataStore); } - public CteDataStore getCTEDataStore(Query query) { - return namedQueriesDataStore.get(query); + public Map getCteDataStores() { + return cteDataStores; + } + + public CteDataStore getCteDataStore(String cteName) { + return cteDataStores.get(cteName); } public boolean isAnalyzed(Expression expression) { 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 53f4320c91c1..9de76ad0b548 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 @@ -250,7 +250,7 @@ protected RelationPlan visitTable(final Table table, final Void context) { } if (namedQuery.isMaterialized()) { - CteDataStore dataStore = analysis.getCTEDataStore(namedQuery); + CteDataStore dataStore = analysis.getCteDataStore(table.getName().toString()); if (dataStore != null) { List outputSymbols = analysis.getOutputDescriptor(table).getAllFields().stream() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index d9790a52e9a3..d4588f0ea64f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -40,11 +40,11 @@ import java.io.File; import java.util.HashSet; +import java.util.Map; import java.util.Optional; import java.util.Set; public class CteMaterializer { - private static final SqlParser relationSqlParser = new SqlParser(); private static final Coordinator coordinator = Coordinator.getInstance(); @@ -57,16 +57,33 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { .forEach( (tableRef, query) -> { if (query.isMaterialized() && !materializedQueries.contains(query)) { - CteDataStore dataStore = - fetchCteQueryResult(tableRef.getNode().getName().toString(), query, context); + if (!context.mayHaveTmpFile()) { + context.setMayHaveTmpFile(true); + } + + String cteName = tableRef.getNode().getName().toString(); + CteDataStore dataStore = fetchCteQueryResult(cteName, query, context); if (dataStore != null) { - analysis.addCteDataStore(query, dataStore); + analysis.addCteDataStore(cteName, dataStore); + context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); materializedQueries.add(query); } } }); } + public static void cleanUpCTE(Analysis analysis, MPPQueryContext context) { + Map cteDataStores = analysis.getCteDataStores(); + cteDataStores + .values() + .forEach( + dataStore -> { + context.releaseMemoryReservedForFrontEnd(dataStore.getCachedBytes()); + dataStore.clear(); + }); + cteDataStores.clear(); + } + public static CteDataStore fetchCteQueryResult( String cteName, Query query, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); @@ -91,7 +108,7 @@ public static CteDataStore fetchCteQueryResult( String folderPath = IoTDBDescriptor.getInstance().getConfig().getCteTmpDir() + File.separator - + queryId + + context.getQueryId() + File.separator; CteDataStore cteDataStore = new CteDataStore(new DiskSpiller(folderPath, folderPath + cteName)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java index 9a0788670f84..110189e96ac1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java @@ -119,4 +119,8 @@ protected void serializeAttributes(ByteBuffer byteBuffer) {} @Override protected void serializeAttributes(DataOutputStream stream) throws IOException {} + + public static CteScanNode deserialize(ByteBuffer byteBuffer) { + return null; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java index 5ee2f88efa8f..fabb13439aa5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java @@ -26,10 +26,33 @@ import org.apache.tsfile.read.common.block.TsBlock; public interface CteDataReader { - + /** + * Check if there is more data in CteDataReader. DiskSpillerReader may run out of current TsBlocks + * , then it needs to read from file and cache more data. This method should be called before + * next() to ensure that there is data to read. + * + * @throws IoTDBException the error occurs when reading data from fileChannel + */ boolean hasNext() throws IoTDBException; + /** + * output the cached data in CteDataReader, it needs to be called after hasNext() returns true. + * + * @return next TsBlock + */ TsBlock next() throws IoTDBException; + /** + * Close the CteDataReader and release resources. + * + * @throws IoTDBException the error occurs when closing fileChannel + */ void close() throws IoTDBException; + + /** + * Get the bytes used by this CteDataReader. + * + * @return the bytes used by this CteDataReader + */ + long bytesUsed(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index 43ca36cd98eb..c05557885c46 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -22,6 +22,7 @@ package org.apache.iotdb.db.utils.cte; import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.tsfile.read.common.block.TsBlock; @@ -29,9 +30,6 @@ import java.util.List; public class CteDataStore { - // 1MB buffer size - private static final long CTE_BUFFER_SIZE = 1024 * 1024L; - private final List cachedData; private long cachedBytes; private final DiskSpiller diskSpiller; @@ -44,14 +42,13 @@ public CteDataStore(DiskSpiller diskSpiller) { public void addTsBlock(TsBlock tsBlock) throws IoTDBException { long bytesSize = tsBlock.getRetainedSizeInBytes(); - if (bytesSize + cachedBytes < CTE_BUFFER_SIZE) { - cachedBytes += bytesSize; - } else { + if (bytesSize + cachedBytes >= IoTDBDescriptor.getInstance().getConfig().getCteBufferSize()) { spill(); cachedData.clear(); - cachedBytes = bytesSize; + cachedBytes = 0; } cachedData.add(tsBlock); + cachedBytes += bytesSize; } public void clear() { @@ -72,7 +69,6 @@ public long getCachedBytes() { } private void spill() throws IoTDBException { - // TODO: allocate memory diskSpiller.spill(cachedData); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java index c1d899d07a32..c36d2dec94bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java @@ -38,6 +38,7 @@ import java.util.List; public class DiskSpiller { + private static final String FILE_SUFFIX = ".cteTemp"; private final String folderPath; private final String filePrefix; @@ -52,19 +53,6 @@ public DiskSpiller(String folderPath, String filePrefix) { this.fileIndex = 0; } - private void createFolder(String folderPath) throws IoTDBException { - try { - Path path = Paths.get(folderPath); - Files.createDirectories(path); - folderCreated = true; - } catch (IOException e) { - throw new IoTDBException( - "Create folder error: " + folderPath, - e, - TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); - } - } - public void spill(List tsBlocks) throws IoTDBException { if (!folderCreated) { createFolder(folderPath); @@ -75,14 +63,6 @@ public void spill(List tsBlocks) throws IoTDBException { writeData(tsBlocks, fileName); } - public List getFilePaths() { - List filePaths = new ArrayList<>(); - for (int i = 0; i < fileIndex; i++) { - filePaths.add(filePrefix + String.format("%05d", i) + FILE_SUFFIX); - } - return filePaths; - } - public List getReaders() throws IoTDBException { List filePaths = getFilePaths(); List fileReaders = new ArrayList<>(); @@ -122,4 +102,25 @@ private void writeData(List tsBlocks, String fileName) throws IoTDBExce TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); } } + + private void createFolder(String folderPath) throws IoTDBException { + try { + Path path = Paths.get(folderPath); + Files.createDirectories(path); + folderCreated = true; + } catch (IOException e) { + throw new IoTDBException( + "Create folder error: " + folderPath, + e, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + } + + private List getFilePaths() { + List filePaths = new ArrayList<>(); + for (int i = 0; i < fileIndex; i++) { + filePaths.add(filePrefix + String.format("%05d", i) + FILE_SUFFIX); + } + return filePaths; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java index 9480f560fa8e..2334e044b8a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java @@ -22,11 +22,13 @@ package org.apache.iotdb.db.utils.cte; import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.block.column.TsBlockSerde; +import org.apache.tsfile.utils.RamUsageEstimator; import java.io.IOException; import java.nio.ByteBuffer; @@ -37,14 +39,19 @@ import java.util.List; public class FileSpillerReader implements CteDataReader { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(FileSpillerReader.class); + private final FileChannel fileChannel; - private final List cacheBlocks; + private final List cachedData; private final String fileName; private final TsBlockSerde serde; private int tsBlockIndex; private boolean isEnd = false; + private long cachedBytes = 0; + private static final int DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES = TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); @@ -53,15 +60,15 @@ public FileSpillerReader(String fileName, TsBlockSerde serde) throws IOException this.serde = serde; this.tsBlockIndex = 0; this.fileChannel = FileChannel.open(Paths.get(fileName), StandardOpenOption.READ); - this.cacheBlocks = new ArrayList<>(); + this.cachedData = new ArrayList<>(); } @Override public TsBlock next() throws IoTDBException { - if (cacheBlocks == null || tsBlockIndex >= cacheBlocks.size()) { + if (cachedData == null || tsBlockIndex >= cachedData.size()) { return null; } - return cacheBlocks.get(tsBlockIndex++); + return cachedData.get(tsBlockIndex++); } @Override @@ -70,16 +77,18 @@ public boolean hasNext() throws IoTDBException { return false; } - if (cacheBlocks.isEmpty() || tsBlockIndex == cacheBlocks.size() - 1) { + if (cachedData.isEmpty() || tsBlockIndex == cachedData.size() - 1) { boolean hasData = readTsBlockFromFile(); if (!hasData) { isEnd = true; + cachedData.clear(); + cachedBytes = 0L; return false; } return true; } - return tsBlockIndex < cacheBlocks.size(); + return tsBlockIndex < cachedData.size(); } @Override @@ -94,19 +103,26 @@ public void close() throws IoTDBException { } } + @Override + public long bytesUsed() { + return INSTANCE_SIZE + cachedBytes; + } + private boolean readTsBlockFromFile() throws IoTDBException { - long bufferSize = Long.MAX_VALUE; - cacheBlocks.clear(); - while (bufferSize >= DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES) { + long cteBufferSize = IoTDBDescriptor.getInstance().getConfig().getCteBufferSize(); + cachedData.clear(); + cachedBytes = 0L; + while (cteBufferSize >= DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES) { long size = read(); if (size == -1) { break; } - bufferSize -= size; + cteBufferSize -= size; + cachedBytes += size; } tsBlockIndex = 0; - return !cacheBlocks.isEmpty(); + return !cachedData.isEmpty(); } private long read() throws IoTDBException { @@ -122,7 +138,7 @@ private long read() throws IoTDBException { fileChannel.read(tsBlockBytes); tsBlockBytes.flip(); TsBlock cachedTsBlock = serde.deserialize(tsBlockBytes); - cacheBlocks.add(cachedTsBlock); + cachedData.add(cachedTsBlock); return cachedTsBlock.getRetainedSizeInBytes(); } catch (IOException e) { throw new IoTDBException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java index d14cb4605e19..1e0ebd6c2891 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java @@ -24,29 +24,31 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.utils.RamUsageEstimator; import java.util.List; public class MemoryReader implements CteDataReader { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(MemoryReader.class); + // all the data in MemoryReader lies in memory private final List cachedData; - private final int size; private int tsBlockIndex; public MemoryReader(List cachedTsBlock) { this.cachedData = cachedTsBlock; - this.size = cachedTsBlock.size(); this.tsBlockIndex = 0; } @Override public boolean hasNext() throws IoTDBException { - return cachedData != null && tsBlockIndex < size; + return cachedData != null && tsBlockIndex < cachedData.size(); } @Override public TsBlock next() throws IoTDBException { - if (cachedData == null || tsBlockIndex >= size) { + if (cachedData == null || tsBlockIndex >= cachedData.size()) { return null; } return cachedData.get(tsBlockIndex++); @@ -54,4 +56,9 @@ public TsBlock next() throws IoTDBException { @Override public void close() throws IoTDBException {} + + @Override + public long bytesUsed() { + return INSTANCE_SIZE; + } } diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 4330a11b1f60..3817c5de8823 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -326,6 +326,12 @@ dn_sync_dir=data/datanode/sync # If its prefix is "/", then the path is absolute. Otherwise, it is relative. sort_tmp_dir=data/datanode/tmp +# cte_tmp_dir +# This property is used to configure the temporary directory for cte materialization. Naming rules are just like sort_tmp_dir. +# effectiveMode: restart +# Datatype: String +sort_tmp_dir=data/datanode/cte + # dn_pipe_receiver_file_dirs # If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/${dn_system_dir}/pipe/receiver). # If it is absolute, system will save the data in the exact location it points to. @@ -1123,6 +1129,11 @@ batch_size=100000 # Datatype: long sort_buffer_size_in_bytes=0 +# The threshold to spill to disk for CTE query result +# effectiveMode: hot_reload +# Datatype: long +cte_buffer_size_in_bytes=0 + # The maximum mod entries size that each FragmentInstance can cache. # if mods_cache_size_limit_per_fi_in_bytes <= 0, default value will be used, default value = min(32MB, memory for query operators / query_thread_count / 2) # if mods_cache_size_limit_per_fi_in_bytes > 0, the specified value will be used. From 68c7777ec3d189c5043a6c9c36f2ad7ecf14394f Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 12 Sep 2025 11:06:33 +0800 Subject: [PATCH 04/69] move cte data stores from Analysis to MPPQueryContext # Conflicts: # iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java --- .../db/queryengine/common/MPPQueryContext.java | 17 +++++++++++++++++ .../plan/relational/analyzer/Analysis.java | 15 --------------- .../relational/planner/RelationPlanner.java | 2 +- .../relational/planner/ir/CteMaterializer.java | 4 ++-- 4 files changed, 20 insertions(+), 18 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 27a1beba195a..8e5358affa88 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 @@ -34,12 +34,15 @@ import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.planner.memory.NotThreadSafeMemoryReservationManager; import org.apache.iotdb.db.queryengine.statistics.QueryPlanStatistics; +import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.tsfile.read.filter.basic.Filter; import java.time.ZoneId; +import java.util.HashMap; import java.util.Collections; import java.util.HashSet; +import java.util.Map; import java.util.List; import java.util.Optional; import java.util.Set; @@ -103,6 +106,8 @@ public class MPPQueryContext implements IAuditEntity { private boolean userQuery = false; + private final Map cteDataStores = new HashMap<>(); + public MPPQueryContext(QueryId queryId) { this.queryId = queryId; this.endPointBlackList = ConcurrentHashMap.newKeySet(); @@ -435,6 +440,18 @@ public void setUserQuery(boolean userQuery) { this.userQuery = userQuery; } + public void addCteDataStore(String cteName, CteDataStore dataStore) { + cteDataStores.put(cteName, dataStore); + } + + public Map getCteDataStores() { + return cteDataStores; + } + + public CteDataStore getCteDataStore(String cteName) { + return cteDataStores.get(cteName); + } + // ================= Authentication Interfaces ========================= private AuditEventType auditEventType; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java index fb5426dc277e..5d373dd4a06b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java @@ -70,7 +70,6 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.TableFunctionInvocation; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WindowFrame; import org.apache.iotdb.db.queryengine.plan.statement.component.FillPolicy; -import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ImmutableList; @@ -122,8 +121,6 @@ public class Analysis implements IAnalysis { private String updateType; - private final Map cteDataStores = new LinkedHashMap<>(); - private final Map, Query> namedQueries = new LinkedHashMap<>(); // map expandable query to the node being the inner recursive reference @@ -285,18 +282,6 @@ public Map, Query> getNamedQueries() { return namedQueries; } - public void addCteDataStore(String cteName, CteDataStore dataStore) { - cteDataStores.put(cteName, dataStore); - } - - public Map getCteDataStores() { - return cteDataStores; - } - - public CteDataStore getCteDataStore(String cteName) { - return cteDataStores.get(cteName); - } - public boolean isAnalyzed(Expression expression) { return expression instanceof DataType || types.containsKey(NodeRef.of(expression)); } 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 9de76ad0b548..df7b926728dd 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 @@ -250,7 +250,7 @@ protected RelationPlan visitTable(final Table table, final Void context) { } if (namedQuery.isMaterialized()) { - CteDataStore dataStore = analysis.getCteDataStore(table.getName().toString()); + CteDataStore dataStore = queryContext.getCteDataStore(table.getName().toString()); if (dataStore != null) { List outputSymbols = analysis.getOutputDescriptor(table).getAllFields().stream() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index d4588f0ea64f..d70dfa892117 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -64,7 +64,7 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { String cteName = tableRef.getNode().getName().toString(); CteDataStore dataStore = fetchCteQueryResult(cteName, query, context); if (dataStore != null) { - analysis.addCteDataStore(cteName, dataStore); + context.addCteDataStore(cteName, dataStore); context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); materializedQueries.add(query); } @@ -73,7 +73,7 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { } public static void cleanUpCTE(Analysis analysis, MPPQueryContext context) { - Map cteDataStores = analysis.getCteDataStores(); + Map cteDataStores = context.getCteDataStores(); cteDataStores .values() .forEach( From 8c44957c90e4ec9c7e5101240ab674786b3fc5bd Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 12 Sep 2025 13:52:25 +0800 Subject: [PATCH 05/69] remove spill-to-disk & add max_rows_in_cte_buffer config --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 28 ++-- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 11 +- .../queryengine/common/MPPQueryContext.java | 3 + .../source/relational/CteScanOperator.java | 44 ++--- .../plan/execution/QueryExecution.java | 25 --- .../planner/ir/CteMaterializer.java | 30 ++-- .../iotdb/db/utils/cte/CteDataStore.java | 28 ++-- .../iotdb/db/utils/cte/DiskSpiller.java | 126 --------------- .../iotdb/db/utils/cte/FileSpillerReader.java | 150 ------------------ .../conf/iotdb-system.properties.template | 14 +- 10 files changed, 64 insertions(+), 395 deletions(-) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 99f18f6b22d8..4fd31e817806 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -322,14 +322,6 @@ public class IoTDBConfig { private String sortTmpDir = IoTDBConstant.DN_DEFAULT_DATA_DIR + File.separator + IoTDBConstant.TMP_FOLDER_NAME; - /** temp result directory for cte */ - private String cteTmpDir = - IoTDBConstant.DN_DEFAULT_DATA_DIR - + File.separator - + IoTDBConstant.CTE_FOLDER_NAME - + File.separator - + IoTDBConstant.TMP_FOLDER_NAME; - /** Maximum MemTable number. Invalid when enableMemControl is true. */ private int maxMemtableNumber = 0; @@ -432,6 +424,9 @@ public class IoTDBConfig { /** The buffer for cte scan operation */ private long cteBufferSize = 32 * 1024 * 1024L; + /** Max number of rows for cte materialization */ + private int maxRowsInCteBuffer = 10000; + /** Mods cache size limit per fi */ private long modsCacheSizeLimitPerFI = 32 * 1024 * 1024; @@ -1382,7 +1377,6 @@ private void formulateFolders() { extPipeDir = addDataHomeDir(extPipeDir); queryDir = addDataHomeDir(queryDir); sortTmpDir = addDataHomeDir(sortTmpDir); - cteTmpDir = addDataHomeDir(cteTmpDir); formulateDataDirs(tierDataDirs); } @@ -4193,6 +4187,14 @@ public long getCteBufferSize() { return cteBufferSize; } + public void setMaxRowsInCteBuffer(int maxRowsInCteBuffer) { + this.maxRowsInCteBuffer = maxRowsInCteBuffer; + } + + public int getMaxRowsInCteBuffer() { + return maxRowsInCteBuffer; + } + public void setModsCacheSizeLimitPerFI(long modsCacheSizeLimitPerFI) { this.modsCacheSizeLimitPerFI = modsCacheSizeLimitPerFI; } @@ -4209,14 +4211,6 @@ public String getSortTmpDir() { return sortTmpDir; } - public void setCteTmpDir(String cteTmpDir) { - this.cteTmpDir = cteTmpDir; - } - - public String getCteTmpDir() { - return cteTmpDir; - } - public String getObjectStorageBucket() { throw new UnsupportedOperationException("object storage is not supported yet"); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 5e20fe805012..ff48b82562f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -1057,18 +1057,21 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException // The buffer for sort operator to calculate loadFixedSizeLimitForQuery(properties, "sort_buffer_size_in_bytes", conf::setSortBufferSize); - // The buffer for cte scan operator + // The buffer size for cte materialization loadFixedSizeLimitForQuery(properties, "cte_buffer_size_in_bytes", conf::setCteBufferSize); + // max number of rows for cte materialization + conf.setMaxRowsInCteBuffer( + Integer.parseInt( + properties.getProperty( + "max_rows_in_cte_buffer", Integer.toString(conf.getMaxRowsInCteBuffer())))); + loadFixedSizeLimitForQuery( properties, "mods_cache_size_limit_per_fi_in_bytes", conf::setModsCacheSizeLimitPerFI); // tmp filePath for sort operator conf.setSortTmpDir(properties.getProperty("sort_tmp_dir", conf.getSortTmpDir())); - // tmp filePath for cte - conf.setCteTmpDir(properties.getProperty("cte_tmp_dir", conf.getCteTmpDir())); - conf.setRateLimiterType(properties.getProperty("rate_limiter_type", conf.getRateLimiterType())); conf.setDataNodeSchemaCacheEvictionPolicy( 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 8e5358affa88..92a0cea4b955 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 @@ -108,6 +108,9 @@ public class MPPQueryContext implements IAuditEntity { private final Map cteDataStores = new HashMap<>(); + /** check if there is tmp file to be deleted. */ + private boolean mayHaveTmpFile = false; + public MPPQueryContext(QueryId queryId) { this.queryId = queryId; this.endPointBlackList = ConcurrentHashMap.newKeySet(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 323700fa470c..5d516e0d5b38 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -21,7 +21,6 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; -import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.AbstractSourceOperator; @@ -35,55 +34,43 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; - public class CteScanOperator extends AbstractSourceOperator { private static final Logger LOGGER = LoggerFactory.getLogger(CteScanOperator.class); private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(CteScanOperator.class); private final CteDataStore dataStore; - private List dataReaders; - private int readerIndex; + private CteDataReader dataReader; public CteScanOperator( OperatorContext operatorContext, PlanNodeId sourceId, CteDataStore dataStore) { this.operatorContext = operatorContext; this.sourceId = sourceId; this.dataStore = dataStore; + prepareReader(); } @Override public TsBlock next() throws Exception { - if (dataReaders == null || readerIndex >= dataReaders.size()) { + if (dataReader == null) { return null; } - return dataReaders.get(readerIndex).next(); + return dataReader.next(); } @Override public boolean hasNext() throws Exception { - if (dataReaders == null) { - prepareReaders(); - } - while (readerIndex < dataReaders.size()) { - if (dataReaders.get(readerIndex).hasNext()) { - return true; - } else { - readerIndex++; - } + if (dataReader == null) { + return false; } - return false; + return dataReader.hasNext(); } @Override public void close() throws Exception { try { - if (dataReaders != null) { - for (CteDataReader reader : dataReaders) { - reader.close(); - } + if (dataReader != null) { + dataReader.close(); } } catch (Exception e) { LOGGER.error("Fail to close fileChannel", e); @@ -97,10 +84,10 @@ public boolean isFinished() throws Exception { @Override public long calculateMaxPeekMemory() { - if (dataReaders == null || readerIndex >= dataReaders.size()) { + if (dataReader == null) { return 0; } - return dataReaders.get(readerIndex).bytesUsed(); + return dataReader.bytesUsed(); } @Override @@ -120,14 +107,9 @@ public long ramBytesUsed() { + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext); } - private void prepareReaders() throws IoTDBException { - if (dataReaders != null) { - return; - } - dataReaders = new ArrayList<>(); - dataReaders.addAll(dataStore.getDiskSpiller().getReaders()); + private void prepareReader() { if (dataStore.getCachedBytes() != 0) { - dataReaders.add(new MemoryReader(dataStore.getCachedData())); + dataReader = new MemoryReader(dataStore.getCachedData()); } } } 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 8003f6cf3ef8..7b4656e25c8e 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 @@ -24,7 +24,6 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; import org.apache.iotdb.commons.service.metric.PerformanceOverviewMetrics; -import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.query.KilledByOthersException; @@ -62,7 +61,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; import java.nio.ByteBuffer; import java.time.format.DateTimeParseException; import java.util.List; @@ -380,15 +378,6 @@ private void cleanUpResultHandle() { public void stopAndCleanup(Throwable t) { stop(t); releaseResource(t); - - try { - // delete cte tmp file if exists - deleteTmpFile(); - } catch (Throwable err) { - LOGGER.error( - "Errors occurred while attempting to delete tmp files, potentially leading to resource leakage.", - err); - } } /** Release the resources that current QueryExecution hold with a specified exception */ @@ -736,18 +725,4 @@ public String toString() { public ScheduledExecutorService getScheduledExecutor() { return planner.getScheduledExecutorService(); } - - private void deleteTmpFile() { - if (context.mayHaveTmpFile()) { - String tmpFilePath = - IoTDBDescriptor.getInstance().getConfig().getCteTmpDir() - + File.separator - + context.getQueryId() - + File.separator; - File tmpFile = new File(tmpFilePath); - if (tmpFile.exists()) { - FileUtils.deleteFileOrDirectory(tmpFile, true); - } - } - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index d70dfa892117..6b718e4e1f98 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -23,7 +23,6 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.plan.Coordinator; @@ -33,12 +32,10 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; import org.apache.iotdb.db.utils.cte.CteDataStore; -import org.apache.iotdb.db.utils.cte.DiskSpiller; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.read.common.block.TsBlock; -import java.io.File; import java.util.HashSet; import java.util.Map; import java.util.Optional; @@ -57,13 +54,11 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { .forEach( (tableRef, query) -> { if (query.isMaterialized() && !materializedQueries.contains(query)) { - if (!context.mayHaveTmpFile()) { - context.setMayHaveTmpFile(true); - } - - String cteName = tableRef.getNode().getName().toString(); - CteDataStore dataStore = fetchCteQueryResult(cteName, query, context); - if (dataStore != null) { + CteDataStore dataStore = fetchCteQueryResult(query, context); + if (dataStore == null) { + query.setMaterialized(false); + } else { + String cteName = tableRef.getNode().getName().toString(); context.addCteDataStore(cteName, dataStore); context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); materializedQueries.add(query); @@ -84,8 +79,7 @@ public static void cleanUpCTE(Analysis analysis, MPPQueryContext context) { cteDataStores.clear(); } - public static CteDataStore fetchCteQueryResult( - String cteName, Query query, MPPQueryContext context) { + public static CteDataStore fetchCteQueryResult(Query query, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; try { @@ -105,13 +99,7 @@ public static CteDataStore fetchCteQueryResult( return null; } - String folderPath = - IoTDBDescriptor.getInstance().getConfig().getCteTmpDir() - + File.separator - + context.getQueryId() - + File.separator; - CteDataStore cteDataStore = - new CteDataStore(new DiskSpiller(folderPath, folderPath + cteName)); + CteDataStore cteDataStore = new CteDataStore(); while (coordinator.getQueryExecution(queryId).hasNextResult()) { final Optional tsBlock; try { @@ -126,7 +114,9 @@ public static CteDataStore fetchCteQueryResult( if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { continue; } - cteDataStore.addTsBlock(tsBlock.get()); + if (!cteDataStore.addTsBlock(tsBlock.get())) { + return null; + } } return cteDataStore; } catch (final Throwable throwable) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index c05557885c46..92c73409ace4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -21,7 +21,7 @@ package org.apache.iotdb.db.utils.cte; -import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.tsfile.read.common.block.TsBlock; @@ -32,23 +32,29 @@ public class CteDataStore { private final List cachedData; private long cachedBytes; - private final DiskSpiller diskSpiller; + private int cachedRows; - public CteDataStore(DiskSpiller diskSpiller) { + public CteDataStore() { this.cachedData = new ArrayList<>(); this.cachedBytes = 0L; - this.diskSpiller = diskSpiller; + this.cachedRows = 0; } - public void addTsBlock(TsBlock tsBlock) throws IoTDBException { + public boolean addTsBlock(TsBlock tsBlock) { + IoTDBConfig iotConfig = IoTDBDescriptor.getInstance().getConfig(); long bytesSize = tsBlock.getRetainedSizeInBytes(); - if (bytesSize + cachedBytes >= IoTDBDescriptor.getInstance().getConfig().getCteBufferSize()) { - spill(); + int rows = tsBlock.getPositionCount(); + if (bytesSize + cachedBytes >= iotConfig.getCteBufferSize() + || rows + cachedRows >= iotConfig.getMaxRowsInCteBuffer()) { cachedData.clear(); cachedBytes = 0; + cachedRows = 0; + return false; } cachedData.add(tsBlock); cachedBytes += bytesSize; + cachedRows += rows; + return true; } public void clear() { @@ -56,10 +62,6 @@ public void clear() { cachedBytes = 0L; } - public DiskSpiller getDiskSpiller() { - return diskSpiller; - } - public List getCachedData() { return cachedData; } @@ -67,8 +69,4 @@ public List getCachedData() { public long getCachedBytes() { return cachedBytes; } - - private void spill() throws IoTDBException { - diskSpiller.spill(cachedData); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java deleted file mode 100644 index c36d2dec94bf..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/DiskSpiller.java +++ /dev/null @@ -1,126 +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.utils.cte; - -import org.apache.iotdb.commons.exception.IoTDBException; -import org.apache.iotdb.rpc.TSStatusCode; - -import org.apache.tsfile.read.common.block.TsBlock; -import org.apache.tsfile.read.common.block.column.TsBlockSerde; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.List; - -public class DiskSpiller { - - private static final String FILE_SUFFIX = ".cteTemp"; - private final String folderPath; - private final String filePrefix; - - private int fileIndex; - private boolean folderCreated = false; - private final TsBlockSerde serde = new TsBlockSerde(); - - public DiskSpiller(String folderPath, String filePrefix) { - this.folderPath = folderPath; - this.filePrefix = filePrefix + "-"; - this.fileIndex = 0; - } - - public void spill(List tsBlocks) throws IoTDBException { - if (!folderCreated) { - createFolder(folderPath); - } - String fileName = filePrefix + String.format("%05d", fileIndex) + FILE_SUFFIX; - fileIndex++; - - writeData(tsBlocks, fileName); - } - - public List getReaders() throws IoTDBException { - List filePaths = getFilePaths(); - List fileReaders = new ArrayList<>(); - try { - for (String filePath : filePaths) { - fileReaders.add(new FileSpillerReader(filePath, serde)); - } - } catch (IOException e) { - throw new IoTDBException( - "Can't get file for FileSpillerReader, check if the file exists: " + filePaths, - e, - TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); - } - return fileReaders; - } - - private void writeData(List tsBlocks, String fileName) throws IoTDBException { - Path filePath = Paths.get(fileName); - try (FileChannel fileChannel = - FileChannel.open( - filePath, - StandardOpenOption.WRITE, - StandardOpenOption.TRUNCATE_EXISTING, - StandardOpenOption.CREATE)) { - for (TsBlock tsBlock : tsBlocks) { - ByteBuffer tsBlockBuffer = serde.serialize(tsBlock); - ByteBuffer length = ByteBuffer.allocate(4); - length.putInt(tsBlockBuffer.capacity()); - length.flip(); - fileChannel.write(length); - fileChannel.write(tsBlockBuffer); - } - } catch (IOException e) { - throw new IoTDBException( - "Can't write CTE data to file: " + fileName, - e, - TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); - } - } - - private void createFolder(String folderPath) throws IoTDBException { - try { - Path path = Paths.get(folderPath); - Files.createDirectories(path); - folderCreated = true; - } catch (IOException e) { - throw new IoTDBException( - "Create folder error: " + folderPath, - e, - TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); - } - } - - private List getFilePaths() { - List filePaths = new ArrayList<>(); - for (int i = 0; i < fileIndex; i++) { - filePaths.add(filePrefix + String.format("%05d", i) + FILE_SUFFIX); - } - return filePaths; - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java deleted file mode 100644 index 2334e044b8a4..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/FileSpillerReader.java +++ /dev/null @@ -1,150 +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.utils.cte; - -import org.apache.iotdb.commons.exception.IoTDBException; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.rpc.TSStatusCode; - -import org.apache.tsfile.common.conf.TSFileDescriptor; -import org.apache.tsfile.read.common.block.TsBlock; -import org.apache.tsfile.read.common.block.column.TsBlockSerde; -import org.apache.tsfile.utils.RamUsageEstimator; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.nio.file.Paths; -import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.List; - -public class FileSpillerReader implements CteDataReader { - private static final long INSTANCE_SIZE = - RamUsageEstimator.shallowSizeOfInstance(FileSpillerReader.class); - - private final FileChannel fileChannel; - private final List cachedData; - private final String fileName; - private final TsBlockSerde serde; - - private int tsBlockIndex; - private boolean isEnd = false; - - private long cachedBytes = 0; - - private static final int DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES = - TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); - - public FileSpillerReader(String fileName, TsBlockSerde serde) throws IOException { - this.fileName = fileName; - this.serde = serde; - this.tsBlockIndex = 0; - this.fileChannel = FileChannel.open(Paths.get(fileName), StandardOpenOption.READ); - this.cachedData = new ArrayList<>(); - } - - @Override - public TsBlock next() throws IoTDBException { - if (cachedData == null || tsBlockIndex >= cachedData.size()) { - return null; - } - return cachedData.get(tsBlockIndex++); - } - - @Override - public boolean hasNext() throws IoTDBException { - if (isEnd) { - return false; - } - - if (cachedData.isEmpty() || tsBlockIndex == cachedData.size() - 1) { - boolean hasData = readTsBlockFromFile(); - if (!hasData) { - isEnd = true; - cachedData.clear(); - cachedBytes = 0L; - return false; - } - return true; - } - - return tsBlockIndex < cachedData.size(); - } - - @Override - public void close() throws IoTDBException { - try { - fileChannel.close(); - } catch (IOException e) { - throw new IoTDBException( - "Can't close FileSpillerReader: " + fileName, - e, - TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); - } - } - - @Override - public long bytesUsed() { - return INSTANCE_SIZE + cachedBytes; - } - - private boolean readTsBlockFromFile() throws IoTDBException { - long cteBufferSize = IoTDBDescriptor.getInstance().getConfig().getCteBufferSize(); - cachedData.clear(); - cachedBytes = 0L; - while (cteBufferSize >= DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES) { - long size = read(); - if (size == -1) { - break; - } - cteBufferSize -= size; - cachedBytes += size; - } - - tsBlockIndex = 0; - return !cachedData.isEmpty(); - } - - private long read() throws IoTDBException { - try { - ByteBuffer bytes = ByteBuffer.allocate(4); - int readLen = fileChannel.read(bytes); - if (readLen == -1) { - return -1; - } - bytes.flip(); - int capacity = bytes.getInt(); - ByteBuffer tsBlockBytes = ByteBuffer.allocate(capacity); - fileChannel.read(tsBlockBytes); - tsBlockBytes.flip(); - TsBlock cachedTsBlock = serde.deserialize(tsBlockBytes); - cachedData.add(cachedTsBlock); - return cachedTsBlock.getRetainedSizeInBytes(); - } catch (IOException e) { - throw new IoTDBException( - "Can't read a new tsBlock in FileSpillerReader: " + fileName, - e, - TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); - } - } -} diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 3817c5de8823..3a64f80f1e8f 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -326,12 +326,6 @@ dn_sync_dir=data/datanode/sync # If its prefix is "/", then the path is absolute. Otherwise, it is relative. sort_tmp_dir=data/datanode/tmp -# cte_tmp_dir -# This property is used to configure the temporary directory for cte materialization. Naming rules are just like sort_tmp_dir. -# effectiveMode: restart -# Datatype: String -sort_tmp_dir=data/datanode/cte - # dn_pipe_receiver_file_dirs # If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/${dn_system_dir}/pipe/receiver). # If it is absolute, system will save the data in the exact location it points to. @@ -1129,11 +1123,17 @@ batch_size=100000 # Datatype: long sort_buffer_size_in_bytes=0 -# The threshold to spill to disk for CTE query result +# The buffer size for CTE materialization # effectiveMode: hot_reload # Datatype: long cte_buffer_size_in_bytes=0 + +# Max rows for CTE materialization +# effectiveMode: hot_reload +# Datatype: int +max_rows_in_cte_buffer=10000 + # The maximum mod entries size that each FragmentInstance can cache. # if mods_cache_size_limit_per_fi_in_bytes <= 0, default value will be used, default value = min(32MB, memory for query operators / query_thread_count / 2) # if mods_cache_size_limit_per_fi_in_bytes > 0, the specified value will be used. From b314d0c0c360c09480f0b5dece78bc09dbdc1c9e Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 12 Sep 2025 14:03:22 +0800 Subject: [PATCH 06/69] do not support CTE scan node serialization --- .../plan/planner/plan/node/PlanNodeType.java | 1 - .../plan/relational/planner/node/CteScanNode.java | 10 +++++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java index 851aacfa8523..6ebfa6dcfdc4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java @@ -119,7 +119,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.RelationalInsertTabletNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AssignUniqueId; -import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.EnforceSingleRowNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExceptNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GapFillNode; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java index 110189e96ac1..e92c8e56dd63 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java @@ -115,12 +115,12 @@ public PlanNode replaceChildren(List newChildren) { } @Override - protected void serializeAttributes(ByteBuffer byteBuffer) {} + protected void serializeAttributes(ByteBuffer byteBuffer) { + throw new UnsupportedOperationException(); + } @Override - protected void serializeAttributes(DataOutputStream stream) throws IOException {} - - public static CteScanNode deserialize(ByteBuffer byteBuffer) { - return null; + protected void serializeAttributes(DataOutputStream stream) throws IOException { + throw new UnsupportedOperationException(); } } From cb41f41500813b8922a7df7a58c6bbc03f51c2e1 Mon Sep 17 00:00:00 2001 From: lancelly <1435078631@qq.com> Date: Fri, 6 Jun 2025 15:34:44 +0800 Subject: [PATCH 07/69] execute uncorrelated scalar subquery in predicate in advance to utilize predicate pushdown --- .../ConvertPredicateToTimeFilterVisitor.java | 10 +- .../relational/planner/SubqueryPlanner.java | 10 + ...correlatedScalarSubqueryReconstructor.java | 171 ++++++++++++++++++ .../sql/ast/ComparisonExpression.java | 12 +- .../plan/relational/sql/ast/LongLiteral.java | 2 +- .../planner/CorrelatedSubqueryTest.java | 3 - 6 files changed, 201 insertions(+), 7 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/ConvertPredicateToTimeFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/ConvertPredicateToTimeFilterVisitor.java index 7368c5d28c2c..b608a4b674aa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/ConvertPredicateToTimeFilterVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/ConvertPredicateToTimeFilterVisitor.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BetweenPredicate; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DoubleLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Extract; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.IfExpression; @@ -299,6 +300,13 @@ protected Filter visitBetweenPredicate(BetweenPredicate node, Void context) { } public static long getLongValue(Expression expression) { - return ((LongLiteral) expression).getParsedValue(); + if (expression instanceof LongLiteral) { + return ((LongLiteral) expression).getParsedValue(); + } else if (expression instanceof DoubleLiteral) { + return (long) ((DoubleLiteral) expression).getValue(); + } else { + throw new IllegalArgumentException( + "Expression should be LongLiteral or DoubleLiteral, but got: " + expression.getClass()); + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java index ebc32b6a247e..f9e62930f029 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java @@ -29,6 +29,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.RelationType; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Scope; import org.apache.iotdb.db.queryengine.plan.relational.planner.QueryPlanner.PlanAndMappings; +import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.PredicateWithUncorrelatedScalarSubqueryReconstructor; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ApplyNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CorrelatedJoinNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.EnforceSingleRowNode; @@ -125,6 +126,9 @@ public PlanBuilder handleSubqueries( List scalarSubqueries = subqueries.getSubqueries(); if (!scalarSubqueries.isEmpty()) { + // try to execute un-correlated scalar subqueries in the predicate in advance to utilize + // predicate pushdown if possible + tryFoldUncorrelatedScalarSubqueryInPredicate(expression, plannerContext); for (Cluster cluster : cluster(builder.getScope(), selectSubqueries(builder, expression, scalarSubqueries))) { builder = planScalarSubquery(builder, cluster); @@ -151,6 +155,12 @@ public PlanBuilder handleSubqueries( return builder; } + private void tryFoldUncorrelatedScalarSubqueryInPredicate( + Expression expression, MPPQueryContext context) { + PredicateWithUncorrelatedScalarSubqueryReconstructor + .reconstructPredicateWithUncorrelatedScalarSubquery(expression, context); + } + /** * Find subqueries from the candidate set that are children of the given parent and that have not * already been handled in the subplan diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java new file mode 100644 index 000000000000..dc933cd65453 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -0,0 +1,171 @@ +/* + * 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.ir; + +import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.db.protocol.session.SessionManager; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.plan.Coordinator; +import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; +import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BinaryLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DoubleLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NotExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SubqueryExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; + +public class PredicateWithUncorrelatedScalarSubqueryReconstructor { + + private static final SqlParser relationSqlParser = new SqlParser(); + + private static final Coordinator coordinator = Coordinator.getInstance(); + + private PredicateWithUncorrelatedScalarSubqueryReconstructor() { + // utility class + } + + public static void reconstructPredicateWithUncorrelatedScalarSubquery( + Expression expression, MPPQueryContext context) { + if (expression instanceof LogicalExpression) { + LogicalExpression logicalExpression = (LogicalExpression) expression; + for (Expression term : logicalExpression.getTerms()) { + reconstructPredicateWithUncorrelatedScalarSubquery(term, context); + } + } else if (expression instanceof NotExpression) { + NotExpression notExpression = (NotExpression) expression; + reconstructPredicateWithUncorrelatedScalarSubquery(notExpression.getValue(), context); + } else if (expression instanceof ComparisonExpression) { + ComparisonExpression comparisonExpression = (ComparisonExpression) expression; + Expression left = comparisonExpression.getLeft(); + Expression right = comparisonExpression.getRight(); + if (left instanceof Identifier && right instanceof SubqueryExpression) { + Optional result = + fetchUncorrelatedSubqueryResultForPredicate((SubqueryExpression) right, context); + // If the subquery result is not present, we cannot reconstruct the predicate. + if (result.isPresent()) { + right = result.get(); + } + } else if (right instanceof Identifier && left instanceof SubqueryExpression) { + Optional result = + fetchUncorrelatedSubqueryResultForPredicate((SubqueryExpression) left, context); + if (result.isPresent()) { + left = result.get(); + } + } + comparisonExpression.setLeft(left); + comparisonExpression.setRight(right); + } + } + + /** + * @return an Optional containing the result of the uncorrelated scalar subquery. Returns + * Optional.empty() if the subquery cannot be executed in advance or if it does not return a + * valid result. + */ + private static Optional fetchUncorrelatedSubqueryResultForPredicate( + SubqueryExpression subqueryExpression, MPPQueryContext context) { + final long queryId = SessionManager.getInstance().requestQueryId(); + Throwable t = null; + + try { + final ExecutionResult executionResult = + coordinator.executeForTableModel( + subqueryExpression.getQuery(), + relationSqlParser, + SessionManager.getInstance().getCurrSession(), + queryId, + SessionManager.getInstance() + .getSessionInfoOfTableModel(SessionManager.getInstance().getCurrSession()), + "Try to Fetch Uncorrelated Scalar Subquery Result for Predicate", + LocalExecutionPlanner.getInstance().metadata, + context.getTimeOut(), + false); + + // This may occur when the subquery cannot be executed in advance (for example, with + // correlated scalar subqueries). + // Since we cannot determine the subquery's validity beforehand, we must submit the subquery. + // This approach may slow down filter involving correlated scalar subqueries. + if (executionResult.status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return Optional.empty(); + } + + while (coordinator.getQueryExecution(queryId).hasNextResult()) { + final Optional tsBlock; + try { + tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); + } catch (final IoTDBException e) { + t = e; + throw new RuntimeException("Failed to Fetch Subquery Result.", e); + } + if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { + continue; + } + final Column[] columns = tsBlock.get().getValueColumns(); + checkArgument(columns.length == 1, "Scalar Subquery result should only have one column."); + checkArgument( + tsBlock.get().getPositionCount() == 1 && !tsBlock.get().getColumn(0).isNull(0), + "Scalar Subquery result should only have one row."); + switch (columns[0].getDataType()) { + case INT32: + case DATE: + return Optional.of(new LongLiteral(Long.toString(columns[0].getInt(0)))); + case INT64: + case TIMESTAMP: + return Optional.of(new LongLiteral(Long.toString(columns[0].getLong(0)))); + case FLOAT: + return Optional.of(new DoubleLiteral(Double.toString(columns[0].getFloat(0)))); + case DOUBLE: + return Optional.of(new DoubleLiteral(Double.toString(columns[0].getDouble(0)))); + case BOOLEAN: + return Optional.of(new BooleanLiteral(Boolean.toString(columns[0].getBoolean(0)))); + case BLOB: + case TEXT: + case STRING: + return Optional.of(new BinaryLiteral(columns[0].getBinary(0).toString())); + default: + throw new IllegalArgumentException( + String.format( + "Unsupported data type for scalar subquery result: %s", + columns[0].getDataType())); + } + } + } catch (final Throwable throwable) { + t = throwable; + } finally { + coordinator.cleanupQueryExecution(queryId, null, t); + } + return Optional.empty(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java index c968ac7c2854..65168dc7e915 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java @@ -100,8 +100,8 @@ public Operator negate() { } private final Operator operator; - private final Expression left; - private final Expression right; + private Expression left; + private Expression right; public ComparisonExpression(Operator operator, Expression left, Expression right) { super(null); @@ -138,6 +138,14 @@ public Expression getRight() { return right; } + public void setLeft(Expression left) { + this.left = left; + } + + public void setRight(Expression right) { + this.right = right; + } + @Override public R accept(AstVisitor visitor, C context) { return visitor.visitComparisonExpression(this, context); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LongLiteral.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LongLiteral.java index a6597e9269ce..930714f1181e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LongLiteral.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LongLiteral.java @@ -103,7 +103,7 @@ public boolean shallowEquals(Node other) { return parsedValue == ((LongLiteral) other).parsedValue; } - private static long parse(String value) { + public static long parse(String value) { value = value.replace("_", ""); if (value.startsWith("0x") || value.startsWith("0X")) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CorrelatedSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CorrelatedSubqueryTest.java index e7baf8078f11..ccc6db6620bf 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CorrelatedSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CorrelatedSubqueryTest.java @@ -63,9 +63,6 @@ public void testCorrelatedExistsSubquery() { PlanMatchPattern tableScan2 = tableScan("testdb.table2", ImmutableMap.of("s2_7", "s2")); - Expression filterPredicate = - new CoalesceExpression(new BooleanLiteral("true"), new BooleanLiteral("false")); - // Verify full LogicalPlan /* * └──OutputNode From 0515980ac243dcda79898089062ea32c0b28f4d7 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 16 Sep 2025 10:23:04 +0800 Subject: [PATCH 08/69] keep cte information in subquery # Conflicts: # iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java # Conflicts: # iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java --- .../queryengine/common/MPPQueryContext.java | 30 +++++++++---- .../db/queryengine/plan/Coordinator.java | 39 ++++++++++++++++ .../plan/execution/QueryExecution.java | 11 ++--- .../plan/relational/analyzer/Analyzer.java | 8 ++++ .../analyzer/StatementAnalyzer.java | 8 ++++ .../relational/planner/RelationPlanner.java | 2 +- .../planner/ir/CteMaterializer.java | 44 ++++++++++++++++--- ...correlatedScalarSubqueryReconstructor.java | 1 + .../PushLimitOffsetIntoTableScan.java | 7 +++ .../iotdb/db/utils/cte/CteDataStore.java | 17 ++++++- 10 files changed, 144 insertions(+), 23 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 92a0cea4b955..b762b123c446 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 @@ -33,6 +33,8 @@ import org.apache.iotdb.db.queryengine.plan.analyze.lock.SchemaLockType; import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.planner.memory.NotThreadSafeMemoryReservationManager; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.statistics.QueryPlanStatistics; import org.apache.iotdb.db.utils.cte.CteDataStore; @@ -106,10 +108,8 @@ public class MPPQueryContext implements IAuditEntity { private boolean userQuery = false; - private final Map cteDataStores = new HashMap<>(); - - /** check if there is tmp file to be deleted. */ - private boolean mayHaveTmpFile = false; + private Map, CteDataStore> cteDataStores = new HashMap<>(); + private boolean uncorrelatedSubquery = false; public MPPQueryContext(QueryId queryId) { this.queryId = queryId; @@ -443,16 +443,28 @@ public void setUserQuery(boolean userQuery) { this.userQuery = userQuery; } - public void addCteDataStore(String cteName, CteDataStore dataStore) { - cteDataStores.put(cteName, dataStore); + public boolean isUncorrelatedSubquery() { + return uncorrelatedSubquery; + } + + public void setUncorrelatedSubquery(boolean uncorrelatedSubquery) { + this.uncorrelatedSubquery = uncorrelatedSubquery; } - public Map getCteDataStores() { + public void addCteDataStore(Table table, CteDataStore dataStore) { + cteDataStores.put(NodeRef.of(table), dataStore); + } + + public Map, CteDataStore> getCteDataStores() { return cteDataStores; } - public CteDataStore getCteDataStore(String cteName) { - return cteDataStores.get(cteName); + public CteDataStore getCteDataStore(Table table) { + return cteDataStores.get(NodeRef.of(table)); + } + + public void setCteDataStores(Map, CteDataStore> cteDataStores) { + this.cteDataStores = cteDataStores; } // ================= Authentication Interfaces ========================= diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index 3210d277d861..db6adaf9cb81 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -130,6 +130,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StartRepairData; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StopRepairData; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SubscriptionStatement; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.UnloadModel; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Use; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WrappedInsertStatement; @@ -141,6 +142,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement; import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.utils.SetThreadName; +import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.thrift.TBase; import org.apache.tsfile.utils.Accountable; @@ -157,6 +159,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.BlockingDeque; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; @@ -394,6 +397,42 @@ private IQueryExecution createQueryExecutionForTreeModel( return new QueryExecution(treeModelPlanner, queryContext, executor); } + /** + * This method is specifically called in fetchUncorrelatedSubqueryResultForPredicate. When + * uncorrelated scalar subquery is handled in SubqueryPlanner, we try to fold it and get constant + * value. Since CTE could be used in the subquery, we should add CTE materialization result into + * MPPQueryContext. + */ + public ExecutionResult executeForTableModel( + org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement, + SqlParser sqlParser, + IClientSession clientSession, + long queryId, + SessionInfo session, + String sql, + Metadata metadata, + Map, CteDataStore> cteDataStoreMap, + long timeOut, + boolean userQuery) { + return execution( + queryId, + session, + sql, + userQuery, + ((queryContext, startTime) -> { + queryContext.setCteDataStores(cteDataStoreMap); + queryContext.setUncorrelatedSubquery(true); + return createQueryExecutionForTableModel( + statement, + sqlParser, + clientSession, + queryContext, + metadata, + timeOut > 0 ? timeOut : CONFIG.getQueryTimeoutThreshold(), + startTime); + })); + } + public ExecutionResult executeForTableModel( org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement, SqlParser sqlParser, 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 7b4656e25c8e..28b35676cda8 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 @@ -47,7 +47,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeUtil; -import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.CteMaterializer; import org.apache.iotdb.db.queryengine.plan.scheduler.IScheduler; import org.apache.iotdb.db.utils.SetThreadName; @@ -351,8 +350,9 @@ private void releaseResource() { resultHandle.close(); cleanUpResultHandle(); } - if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE)) { - CteMaterializer.cleanUpCTE((Analysis) analysis, context); + if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) + && !context.isUncorrelatedSubquery()) { + CteMaterializer.cleanUpCTE(context); } } @@ -397,8 +397,9 @@ private void releaseResource(Throwable t) { } cleanUpResultHandle(); } - if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE)) { - CteMaterializer.cleanUpCTE((Analysis) analysis, context); + if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) + && !context.isUncorrelatedSubquery()) { + CteMaterializer.cleanUpCTE(context); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java index 3f7c5322c8e6..d8f34b2a3ae0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java @@ -80,6 +80,14 @@ public Analysis analyze(Statement statement) { warningCollector); Analysis analysis = new Analysis(rewrittenStatement, parameterLookup); + // Here we register CTE into analysis. Later in RelationPlanner, we create CteScanNode or + // DeviceTableDeviceScanNode based on analysis.getNamedQuery result. + context + .getCteDataStores() + .forEach( + (tableRef, dataStore) -> + analysis.registerNamedQuery(tableRef.getNode(), dataStore.getQuery())); + Statement innerStatement = rewrittenStatement instanceof PipeEnriched ? ((PipeEnriched) rewrittenStatement).getInnerStatement() 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 a83c3581029b..549e701f7d43 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 @@ -196,6 +196,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertBaseStatement; import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; import org.apache.iotdb.db.schemaengine.table.DataNodeTreeViewSchemaUtils; +import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.udf.api.exception.UDFException; @@ -3093,6 +3094,13 @@ protected Scope visitTable(Table table, Optional scope) { table, QualifiedName.of(name.getDatabaseName(), name.getObjectName())); Optional tableSchema = metadata.getTableSchema(sessionContext, name); + // if table schema is not found in metada, we check if it's a CTE defined in the parent query + if (!tableSchema.isPresent()) { + CteDataStore dataStore = queryContext.getCteDataStore(table); + if (dataStore != null) { + tableSchema = Optional.of(dataStore.getTableSchema()); + } + } // This can only be a table if (!tableSchema.isPresent()) { TableMetadataImpl.throwTableNotExistsException( 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 df7b926728dd..275f5580f1e9 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 @@ -250,7 +250,7 @@ protected RelationPlan visitTable(final Table table, final Void context) { } if (namedQuery.isMaterialized()) { - CteDataStore dataStore = queryContext.getCteDataStore(table.getName().toString()); + CteDataStore dataStore = queryContext.getCteDataStore(table); if (dataStore != null) { List outputSymbols = analysis.getOutputDescriptor(table).getAllFields().stream() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index 6b718e4e1f98..798568736138 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -23,20 +23,30 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; +import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.type.TypeFactory; +import java.util.ArrayList; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -53,13 +63,13 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { .getNamedQueries() .forEach( (tableRef, query) -> { + Table table = tableRef.getNode(); if (query.isMaterialized() && !materializedQueries.contains(query)) { - CteDataStore dataStore = fetchCteQueryResult(query, context); + CteDataStore dataStore = fetchCteQueryResult(table, query, context); if (dataStore == null) { query.setMaterialized(false); } else { - String cteName = tableRef.getNode().getName().toString(); - context.addCteDataStore(cteName, dataStore); + context.addCteDataStore(table, dataStore); context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); materializedQueries.add(query); } @@ -67,8 +77,8 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { }); } - public static void cleanUpCTE(Analysis analysis, MPPQueryContext context) { - Map cteDataStores = context.getCteDataStores(); + public static void cleanUpCTE(MPPQueryContext context) { + Map, CteDataStore> cteDataStores = context.getCteDataStores(); cteDataStores .values() .forEach( @@ -79,7 +89,8 @@ public static void cleanUpCTE(Analysis analysis, MPPQueryContext context) { cteDataStores.clear(); } - public static CteDataStore fetchCteQueryResult(Query query, MPPQueryContext context) { + public static CteDataStore fetchCteQueryResult( + Table table, Query query, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; try { @@ -99,7 +110,11 @@ public static CteDataStore fetchCteQueryResult(Query query, MPPQueryContext cont return null; } - CteDataStore cteDataStore = new CteDataStore(); + // get table schema + DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); + TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); + + CteDataStore cteDataStore = new CteDataStore(query, tableSchema); while (coordinator.getQueryExecution(queryId).hasNextResult()) { final Optional tsBlock; try { @@ -126,4 +141,19 @@ public static CteDataStore fetchCteQueryResult(Query query, MPPQueryContext cont } return null; } + + private static TableSchema getTableSchema(DatasetHeader datasetHeader, String cteName) { + List columnNames = datasetHeader.getRespColumns(); + List columnDataTypes = datasetHeader.getRespDataTypes(); + final List columnSchemaList = new ArrayList<>(); + for (int i = 0; i < columnNames.size(); i++) { + columnSchemaList.add( + new ColumnSchema( + columnNames.get(i), + TypeFactory.getType(columnDataTypes.get(i)), + false, + TsTableColumnCategory.FIELD)); + } + return new TableSchema(cteName, columnSchemaList); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index dc933cd65453..9487edae254a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -110,6 +110,7 @@ private static Optional fetchUncorrelatedSubqueryResultForPredicate( .getSessionInfoOfTableModel(SessionManager.getInstance().getCurrSession()), "Try to Fetch Uncorrelated Scalar Subquery Result for Predicate", LocalExecutionPlanner.getInstance().metadata, + context.getCteDataStores(), context.getTimeOut(), false); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java index 2bd9b778aec2..ce8a002a465d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.OrderingScheme; import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode; +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.FilterNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GapFillNode; @@ -237,6 +238,12 @@ public PlanNode visitDeviceTableScan(DeviceTableScanNode node, Context context) return node; } + @Override + public PlanNode visitCteScan(CteScanNode node, Context context) { + context.enablePushDown = false; + return node; + } + @Override public PlanNode visitInformationSchemaTableScan( InformationSchemaTableScanNode node, Context context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index 92c73409ace4..2f5474d9eb50 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -23,6 +23,8 @@ import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.tsfile.read.common.block.TsBlock; @@ -30,11 +32,16 @@ import java.util.List; public class CteDataStore { + private final Query query; + private final TableSchema tableSchema; + private final List cachedData; private long cachedBytes; private int cachedRows; - public CteDataStore() { + public CteDataStore(Query query, TableSchema tableSchema) { + this.query = query; + this.tableSchema = tableSchema; this.cachedData = new ArrayList<>(); this.cachedBytes = 0L; this.cachedRows = 0; @@ -69,4 +76,12 @@ public List getCachedData() { public long getCachedBytes() { return cachedBytes; } + + public TableSchema getTableSchema() { + return tableSchema; + } + + public Query getQuery() { + return query; + } } From 6f0d5121b71e976a8edce18549fd18b635d9ecce Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 17 Sep 2025 10:02:35 +0800 Subject: [PATCH 09/69] fix bug --- .../relational/planner/RelationPlanner.java | 44 ++++++++++++++----- .../planner/ir/CteMaterializer.java | 38 +++++++++++----- 2 files changed, 62 insertions(+), 20 deletions(-) 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 275f5580f1e9..e6e71869663d 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 @@ -252,18 +252,42 @@ protected RelationPlan visitTable(final Table table, final Void context) { if (namedQuery.isMaterialized()) { CteDataStore dataStore = queryContext.getCteDataStore(table); if (dataStore != null) { - List outputSymbols = - analysis.getOutputDescriptor(table).getAllFields().stream() - .map(symbolAllocator::newSymbol) - .collect(toImmutableList()); + List cteSymbols = new ArrayList<>(); + Map cteSymbolMap = new HashMap<>(); + dataStore + .getTableSchema() + .getColumns() + .forEach( + column -> { + Symbol columnSymbol = + symbolAllocator.newSymbol(column.getName(), column.getType()); + cteSymbols.add(columnSymbol); + cteSymbolMap.put(column.getName(), columnSymbol); + }); // CTE Scan Node - return new RelationPlan( - new CteScanNode( - idAllocator.genPlanNodeId(), table.getName(), outputSymbols, dataStore), - scope, - outputSymbols, - outerContext); + CteScanNode cteScanNode = + new CteScanNode(idAllocator.genPlanNodeId(), table.getName(), cteSymbols, dataStore); + + List outputSymbols = new ArrayList<>(); + Assignments.Builder assignments = Assignments.builder(); + analysis + .getOutputDescriptor(table) + .getVisibleFields() + .forEach( + field -> { + String columnName = field.getName().orElse("field"); + Symbol symbol = cteSymbolMap.get(columnName); + outputSymbols.add(symbol); + assignments.put(symbol, symbol.toSymbolReference()); + }); + + // Project Node + ProjectNode projectNode = + new ProjectNode( + queryContext.getQueryId().genPlanNodeId(), cteScanNode, assignments.build()); + + return new RelationPlan(projectNode, scope, outputSymbols, outerContext); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index 798568736138..a8061447ca2b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -50,6 +50,8 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class CteMaterializer { @@ -143,17 +145,33 @@ public static CteDataStore fetchCteQueryResult( } private static TableSchema getTableSchema(DatasetHeader datasetHeader, String cteName) { - List columnNames = datasetHeader.getRespColumns(); - List columnDataTypes = datasetHeader.getRespDataTypes(); - final List columnSchemaList = new ArrayList<>(); - for (int i = 0; i < columnNames.size(); i++) { - columnSchemaList.add( - new ColumnSchema( - columnNames.get(i), - TypeFactory.getType(columnDataTypes.get(i)), - false, - TsTableColumnCategory.FIELD)); + final List columnNames = datasetHeader.getRespColumns(); + final List columnDataTypes = datasetHeader.getRespDataTypes(); + if (columnNames.size() != columnDataTypes.size()) { + throw new IoTDBRuntimeException( + "Size of column names and column data types do not match", + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); } + final Map columnNameIndexMap = datasetHeader.getColumnNameIndexMap(); + final List columnSchemaList = new ArrayList<>(); + + // build name -> type map + Map columnNameDataTypeMap = + IntStream.range(0, columnNames.size()) + .boxed() + .collect(Collectors.toMap(columnNames::get, columnDataTypes::get)); + + // build column schema list of cte table based on columnNameIndexMap + columnNameIndexMap.entrySet().stream() + .sorted(Map.Entry.comparingByValue()) + .forEach( + entry -> + columnSchemaList.add( + new ColumnSchema( + entry.getKey(), + TypeFactory.getType(columnNameDataTypeMap.get(entry.getKey())), + false, + TsTableColumnCategory.FIELD))); return new TableSchema(cteName, columnSchemaList); } } From ce5175348290933ed016ec7b9d0b37d9b01032b9 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 17 Sep 2025 15:20:45 +0800 Subject: [PATCH 10/69] fix: duplicate CTE materialization process --- .../queryengine/common/MPPQueryContext.java | 11 +++++----- .../db/queryengine/plan/Coordinator.java | 2 +- .../plan/execution/QueryExecution.java | 6 ++--- .../planner/ir/CteMaterializer.java | 22 ++++++++++--------- .../plan/relational/sql/ast/Query.java | 13 +++++++++++ 5 files changed, 34 insertions(+), 20 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 b762b123c446..8ad77a7496a6 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 @@ -109,7 +109,8 @@ public class MPPQueryContext implements IAuditEntity { private boolean userQuery = false; private Map, CteDataStore> cteDataStores = new HashMap<>(); - private boolean uncorrelatedSubquery = false; + // If this is a subquery, we do not release CTE query result + private boolean subquery = false; public MPPQueryContext(QueryId queryId) { this.queryId = queryId; @@ -443,12 +444,12 @@ public void setUserQuery(boolean userQuery) { this.userQuery = userQuery; } - public boolean isUncorrelatedSubquery() { - return uncorrelatedSubquery; + public boolean isSubquery() { + return subquery; } - public void setUncorrelatedSubquery(boolean uncorrelatedSubquery) { - this.uncorrelatedSubquery = uncorrelatedSubquery; + public void setSubquery(boolean subquery) { + this.subquery = subquery; } public void addCteDataStore(Table table, CteDataStore dataStore) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index db6adaf9cb81..30f6bd0cf2be 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -421,7 +421,7 @@ public ExecutionResult executeForTableModel( userQuery, ((queryContext, startTime) -> { queryContext.setCteDataStores(cteDataStoreMap); - queryContext.setUncorrelatedSubquery(true); + queryContext.setSubquery(true); return createQueryExecutionForTableModel( statement, sqlParser, 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 28b35676cda8..377c69ec8730 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 @@ -350,8 +350,7 @@ private void releaseResource() { resultHandle.close(); cleanUpResultHandle(); } - if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) - && !context.isUncorrelatedSubquery()) { + if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) && !context.isSubquery()) { CteMaterializer.cleanUpCTE(context); } } @@ -397,8 +396,7 @@ private void releaseResource(Throwable t) { } cleanUpResultHandle(); } - if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) - && !context.isUncorrelatedSubquery()) { + if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) && !context.isSubquery()) { CteMaterializer.cleanUpCTE(context); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index a8061447ca2b..43737b3cc651 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -45,11 +45,9 @@ import org.apache.tsfile.read.common.type.TypeFactory; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -60,21 +58,25 @@ public class CteMaterializer { private CteMaterializer() {} public static void materializeCTE(Analysis analysis, MPPQueryContext context) { - Set materializedQueries = new HashSet<>(); analysis .getNamedQueries() .forEach( (tableRef, query) -> { Table table = tableRef.getNode(); - if (query.isMaterialized() && !materializedQueries.contains(query)) { - CteDataStore dataStore = fetchCteQueryResult(table, query, context); + if (query.isMaterialized()) { + CteDataStore dataStore = query.getCteDataStore(); if (dataStore == null) { - query.setMaterialized(false); - } else { - context.addCteDataStore(table, dataStore); - context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); - materializedQueries.add(query); + dataStore = fetchCteQueryResult(table, query, context); + if (dataStore == null) { + // CTE query execution failed. Use inline instead of materialization in the + // outer query + query.setMaterialized(false); + } else { + context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); + query.setCteDataStore(dataStore); + } } + context.addCteDataStore(table, dataStore); } }); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java index 12f0b25718b9..ed11f9f5865e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.queryengine.plan.relational.sql.ast; +import org.apache.iotdb.db.utils.cte.CteDataStore; + import com.google.common.collect.ImmutableList; import org.apache.tsfile.utils.RamUsageEstimator; @@ -40,7 +42,10 @@ public class Query extends Statement { private final Optional orderBy; private final Optional offset; private final Optional limit; + // whether this query needs serialization private boolean materialized = false; + // query result for common table expression + private CteDataStore cteDataStore = null; public Query( Optional with, @@ -111,6 +116,14 @@ public void setMaterialized(boolean materialized) { this.materialized = materialized; } + public CteDataStore getCteDataStore() { + return cteDataStore; + } + + public void setCteDataStore(CteDataStore cteDataStore) { + this.cteDataStore = cteDataStore; + } + @Override public R accept(AstVisitor visitor, C context) { return visitor.visitQuery(this, context); From 8489a87019d5b8a0831ba2afe03ff1801bcf17f7 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 22 Sep 2025 14:25:48 +0800 Subject: [PATCH 11/69] fix cteDataStore null issue --- .../planner/ir/CteMaterializer.java | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index 43737b3cc651..b54dccdac548 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -65,18 +65,22 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { Table table = tableRef.getNode(); if (query.isMaterialized()) { CteDataStore dataStore = query.getCteDataStore(); + if (dataStore != null) { + context.addCteDataStore(table, dataStore); + return; + } + + dataStore = fetchCteQueryResult(table, query, context); if (dataStore == null) { - dataStore = fetchCteQueryResult(table, query, context); - if (dataStore == null) { - // CTE query execution failed. Use inline instead of materialization in the - // outer query - query.setMaterialized(false); - } else { - context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); - query.setCteDataStore(dataStore); - } + // CTE query execution failed. Use inline instead of materialization + // in the outer query + query.setMaterialized(false); + return; } + + context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); context.addCteDataStore(table, dataStore); + query.setCteDataStore(dataStore); } }); } From 38b3c860a37103f67a976827f8e7a4e3a028ef0b Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 22 Sep 2025 16:59:05 +0800 Subject: [PATCH 12/69] explain cte --- .../queryengine/common/MPPQueryContext.java | 10 ++ ...ableModelStatementMemorySourceVisitor.java | 20 +++- .../planner/ir/CteMaterializer.java | 113 +++++++++++++----- 3 files changed, 110 insertions(+), 33 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 8ad77a7496a6..d155f47e820f 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 @@ -44,6 +44,7 @@ import java.util.HashMap; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.List; import java.util.Optional; @@ -108,6 +109,7 @@ public class MPPQueryContext implements IAuditEntity { private boolean userQuery = false; + private Map, List> cteDistPlans = new HashMap<>(); private Map, CteDataStore> cteDataStores = new HashMap<>(); // If this is a subquery, we do not release CTE query result private boolean subquery = false; @@ -468,6 +470,14 @@ public void setCteDataStores(Map, CteDataStore> cteDataStores) { this.cteDataStores = cteDataStores; } + public void addCteDistPlan(Table table, List cteDistPlan) { + cteDistPlans.put(NodeRef.of(table), cteDistPlan); + } + + public Map, List> getCteDistPlans() { + return cteDistPlans; + } + // ================= Authentication Interfaces ========================= private AuditEventType auditEventType; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java index 4016bd3885dd..dcf367ce5d52 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java @@ -40,6 +40,7 @@ import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; +import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -82,6 +83,18 @@ public StatementMemorySource visitExplain( return new StatementMemorySource(new TsBlock(0), header); } + final List lines = new ArrayList<>(); + + // CTE materialization plan + context + .getQueryContext() + .getCteDistPlans() + .forEach( + (table, distPlan) -> { + lines.add(String.format("CTE '%s' Query", table.getNode().getName())); + lines.addAll(distPlan); + }); + // Generate table model distributed plan final TableDistributedPlanGenerator.PlanContext planContext = new TableDistributedPlanGenerator.PlanContext(); @@ -95,11 +108,14 @@ public StatementMemorySource visitExplain( Coordinator.getInstance().getDataNodeLocationSupplier()) .generateDistributedPlanWithOptimize(planContext); - final List lines = + if (!lines.isEmpty()) { + lines.add("Main Query"); + } + lines.addAll( outputNodeWithExchange.accept( new PlanGraphPrinter(), new PlanGraphPrinter.GraphContext( - context.getQueryContext().getTypeProvider().getTemplatedInfo())); + context.getQueryContext().getTypeProvider().getTemplatedInfo()))); return getStatementMemorySource(header, lines); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index b54dccdac548..9eee486f2c1f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -34,12 +34,15 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Explain; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.tsfile.block.column.Column; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.type.TypeFactory; @@ -48,6 +51,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -70,7 +74,7 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { return; } - dataStore = fetchCteQueryResult(table, query, context); + dataStore = fetchCteQueryResult(table, query, context.getTimeOut()); if (dataStore == null) { // CTE query execution failed. Use inline instead of materialization // in the outer query @@ -81,6 +85,15 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); context.addCteDataStore(table, dataStore); query.setCteDataStore(dataStore); + + // Explain + if (analysis.canSkipExecute(context)) { + List cteDistPlan = + fetchCteExplainResult(table, new Explain(query), context.getTimeOut()); + if (cteDistPlan != null) { + context.addCteDistPlan(table, cteDistPlan); + } + } } }); } @@ -97,51 +110,27 @@ public static void cleanUpCTE(MPPQueryContext context) { cteDataStores.clear(); } - public static CteDataStore fetchCteQueryResult( - Table table, Query query, MPPQueryContext context) { + private static T execute( + Statement statement, long timeout, String sql, Function func) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; try { final ExecutionResult executionResult = coordinator.executeForTableModel( - query, + statement, new SqlParser(), SessionManager.getInstance().getCurrSession(), queryId, SessionManager.getInstance() .getSessionInfoOfTableModel(SessionManager.getInstance().getCurrSession()), - "Materialize common table expression", + sql, LocalExecutionPlanner.getInstance().metadata, - context.getTimeOut(), + timeout, false); if (executionResult.status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { return null; } - - // get table schema - DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); - TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); - - CteDataStore cteDataStore = new CteDataStore(query, tableSchema); - while (coordinator.getQueryExecution(queryId).hasNextResult()) { - final Optional tsBlock; - try { - tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); - } catch (final IoTDBException e) { - t = e; - throw new IoTDBRuntimeException( - String.format("Fail to materialize CTE because %s", e.getMessage()), - e.getErrorCode(), - e.isUserException()); - } - if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { - continue; - } - if (!cteDataStore.addTsBlock(tsBlock.get())) { - return null; - } - } - return cteDataStore; + return func.apply(queryId); } catch (final Throwable throwable) { t = throwable; } finally { @@ -150,6 +139,68 @@ public static CteDataStore fetchCteQueryResult( return null; } + private static List fetchCteExplainResult(Table table, Explain explain, long timeout) { + return execute( + explain, + timeout, + String.format("Explain query for CTE '%s'", table.getName()), + (queryId) -> { + List lines = new ArrayList<>(); + while (coordinator.getQueryExecution(queryId).hasNextResult()) { + final Optional tsBlock; + try { + tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); + } catch (IoTDBException e) { + throw new IoTDBRuntimeException( + String.format("Fail to explain CTE query because %s", e.getMessage()), + e.getErrorCode(), + e.isUserException()); + } + if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { + continue; + } + + Column valueColumn = tsBlock.get().getColumn(0); + for (int i = 0; i < tsBlock.get().getPositionCount(); i++) { + lines.add(valueColumn.getBinary(i).toString()); + } + } + return lines; + }); + } + + private static CteDataStore fetchCteQueryResult(Table table, Query query, long timeout) { + return execute( + query, + timeout, + String.format("Materialize query for CTE '%s'", table.getName()), + (queryId) -> { + // get table schema + DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); + TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); + + CteDataStore cteDataStore = new CteDataStore(query, tableSchema); + while (coordinator.getQueryExecution(queryId).hasNextResult()) { + final Optional tsBlock; + try { + tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); + } catch (final IoTDBException e) { + throw new IoTDBRuntimeException( + String.format("Fail to materialize CTE because %s", e.getMessage()), + e.getErrorCode(), + e.isUserException()); + } + if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { + continue; + } + if (!cteDataStore.addTsBlock(tsBlock.get())) { + return null; + } + } + return cteDataStore; + }); + } + private static TableSchema getTableSchema(DatasetHeader datasetHeader, String cteName) { final List columnNames = datasetHeader.getRespColumns(); final List columnDataTypes = datasetHeader.getRespDataTypes(); From 2efee5ffcdb48b6125aa1f8fbbc00c8fc1f6a966 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 24 Sep 2025 11:01:37 +0800 Subject: [PATCH 13/69] fix: columnNameIndexMap is null if it is empty table --- .../plan/relational/planner/ir/CteMaterializer.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index 9eee486f2c1f..c94c36e749e4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -212,6 +212,19 @@ private static TableSchema getTableSchema(DatasetHeader datasetHeader, String ct final Map columnNameIndexMap = datasetHeader.getColumnNameIndexMap(); final List columnSchemaList = new ArrayList<>(); + // If CTE query returns empty result set, columnNameIndexMap is null + if (columnNameIndexMap == null) { + for (int i = 0; i < columnNames.size(); i++) { + columnSchemaList.add( + new ColumnSchema( + columnNames.get(i), + TypeFactory.getType(columnDataTypes.get(i)), + false, + TsTableColumnCategory.FIELD)); + } + return new TableSchema(cteName, columnSchemaList); + } + // build name -> type map Map columnNameDataTypeMap = IntStream.range(0, columnNames.size()) From a9f311d5c0755af8e5412677ec3a630a28b73007 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 24 Sep 2025 13:56:34 +0800 Subject: [PATCH 14/69] simple implementation of explain & explain analyze --- .../queryengine/common/MPPQueryContext.java | 10 +++++ .../fragment/FragmentInstanceExecution.java | 8 +--- .../operator/ExplainAnalyzeOperator.java | 16 +++++++- .../analyzer/StatementAnalyzer.java | 1 + .../planner/ir/CteMaterializer.java | 41 +++++++++++++++++++ 5 files changed, 68 insertions(+), 8 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 d155f47e820f..4236c078323a 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 @@ -89,6 +89,8 @@ public class MPPQueryContext implements IAuditEntity { private final Set acquiredLocks = new HashSet<>(); private boolean isExplainAnalyze = false; + // used only in table model for cte query + private boolean isVerbose = false; private QueryPlanStatistics queryPlanStatistics = null; @@ -301,6 +303,14 @@ public boolean isExplainAnalyze() { return isExplainAnalyze; } + public void setVerbose(boolean verbose) { + isVerbose = verbose; + } + + public boolean isVerbose() { + return isVerbose; + } + public long getAnalyzeCost() { return queryPlanStatistics.getAnalyzeCost(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java index 08991ae7f7a3..9985abb70984 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java @@ -31,7 +31,6 @@ import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.schedule.IDriverScheduler; import org.apache.iotdb.db.storageengine.dataregion.IDataRegionForQuery; -import org.apache.iotdb.db.storageengine.dataregion.VirtualDataRegion; import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsResp; import org.apache.iotdb.mpp.rpc.thrift.TOperatorStatistics; @@ -148,11 +147,8 @@ private boolean fillFragmentInstanceStatistics( statistics.setQueryStatistics(context.getQueryStatistics().toThrift()); statistics.setState(getInstanceState().toString()); IDataRegionForQuery dataRegionForQuery = context.getDataRegion(); - if (dataRegionForQuery instanceof VirtualDataRegion) { - // We don't need to output the region having ExplainAnalyzeOperator only. - return false; - } - statistics.setDataRegion(context.getDataRegion().getDataRegionIdString()); + + statistics.setDataRegion(context.getDataRegion().getDataRegionId()); statistics.setIp(CONFIG.getInternalAddress() + ":" + CONFIG.getInternalPort()); statistics.setStartTimeInMS(context.getStartTime()); statistics.setEndTimeInMS( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java index 13adf9b1d65a..234d60a7f38a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java @@ -172,8 +172,20 @@ private void logIntermediateResultIfTimeout() { } private TsBlock buildResult() throws FragmentInstanceFetchException { - - List analyzeResult = buildFragmentInstanceStatistics(instances, verbose); + // CTE materialization + List analyzeResult = new ArrayList<>(); + mppQueryContext + .getCteDistPlans() + .forEach( + (table, distPlan) -> { + analyzeResult.add(String.format("CTE '%s' Query", table.getNode().getName())); + analyzeResult.addAll(distPlan); + }); + + if (!analyzeResult.isEmpty()) { + analyzeResult.add("Main Query"); + } + analyzeResult.addAll(buildFragmentInstanceStatistics(instances, verbose)); TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.TEXT)); TimeColumnBuilder timeColumnBuilder = builder.getTimeColumnBuilder(); 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 549e701f7d43..fd0dd1044567 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 @@ -839,6 +839,7 @@ protected Scope visitExplain(Explain node, Optional context) { @Override protected Scope visitExplainAnalyze(ExplainAnalyze node, Optional context) { queryContext.setExplainAnalyze(true); + queryContext.setVerbose(node.isVerbose()); return visitQuery((Query) node.getStatement(), context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index c94c36e749e4..c605c0400c70 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -35,6 +35,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Explain; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ExplainAnalyze; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; @@ -93,6 +94,15 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { if (cteDistPlan != null) { context.addCteDistPlan(table, cteDistPlan); } + } else if (context.isExplainAnalyze()) { + List cteDistPlan = + fetchCteExplainAnalyzeResult( + table, + new ExplainAnalyze(query, context.isVerbose()), + context.getTimeOut()); + if (cteDistPlan != null) { + context.addCteDistPlan(table, cteDistPlan); + } } } }); @@ -169,6 +179,37 @@ private static List fetchCteExplainResult(Table table, Explain explain, }); } + private static List fetchCteExplainAnalyzeResult( + Table table, ExplainAnalyze explainAnalyze, long timeout) { + return execute( + explainAnalyze, + timeout, + String.format("Explain analyze query for CTE '%s'", table.getName()), + (queryId) -> { + List lines = new ArrayList<>(); + while (coordinator.getQueryExecution(queryId).hasNextResult()) { + final Optional tsBlock; + try { + tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); + } catch (IoTDBException e) { + throw new IoTDBRuntimeException( + String.format("Fail to explain CTE query because %s", e.getMessage()), + e.getErrorCode(), + e.isUserException()); + } + if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { + continue; + } + + Column valueColumn = tsBlock.get().getColumn(0); + for (int i = 0; i < tsBlock.get().getPositionCount(); i++) { + lines.add(valueColumn.getBinary(i).toString()); + } + } + return lines; + }); + } + private static CteDataStore fetchCteQueryResult(Table table, Query query, long timeout) { return execute( query, From 1dad568e8de5790a54e95fba97747124e759e803 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 25 Sep 2025 10:54:43 +0800 Subject: [PATCH 15/69] remove duplicate call in CTE materialization --- .../db/queryengine/common/ExplainType.java | 28 +++ .../queryengine/common/MPPQueryContext.java | 31 ++- .../operator/ExplainAnalyzeOperator.java | 62 +++-- .../db/queryengine/plan/Coordinator.java | 17 +- .../plan/analyze/AnalyzeVisitor.java | 3 +- ...ableModelStatementMemorySourceVisitor.java | 7 +- .../planner/plan/DistributedQueryPlan.java | 11 + .../analyzer/StatementAnalyzer.java | 4 +- .../planner/TableLogicalPlanner.java | 2 +- .../relational/planner/TableModelPlanner.java | 2 +- .../distribute/TableDistributedPlanner.java | 18 +- .../planner/ir/CteMaterializer.java | 213 ++++++++---------- ...correlatedScalarSubqueryReconstructor.java | 2 + 13 files changed, 240 insertions(+), 160 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/ExplainType.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/ExplainType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/ExplainType.java new file mode 100644 index 000000000000..65d04feb8b43 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/ExplainType.java @@ -0,0 +1,28 @@ +/* + * + * * 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.common; + +public enum ExplainType { + NONE, + EXPLAIN, + EXPLAIN_ANALYZE, +} 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 4236c078323a..db45075909ac 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 @@ -39,6 +39,7 @@ import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.tsfile.read.filter.basic.Filter; +import org.apache.tsfile.utils.Pair; import java.time.ZoneId; import java.util.HashMap; @@ -88,8 +89,8 @@ public class MPPQueryContext implements IAuditEntity { private final Set acquiredLocks = new HashSet<>(); - private boolean isExplainAnalyze = false; - // used only in table model for cte query + // explainType & isVerbose are used by cte query in table model + private ExplainType explainType = ExplainType.NONE; private boolean isVerbose = false; private QueryPlanStatistics queryPlanStatistics = null; @@ -111,7 +112,9 @@ public class MPPQueryContext implements IAuditEntity { private boolean userQuery = false; - private Map, List> cteDistPlans = new HashMap<>(); + // table -> (maxLineLength, explain/explain analyze lines) + private final Map, Pair>> cteExplainResults = + new HashMap<>(); private Map, CteDataStore> cteDataStores = new HashMap<>(); // If this is a subquery, we do not release CTE query result private boolean subquery = false; @@ -295,12 +298,20 @@ public ZoneId getZoneId() { return session.getZoneId(); } - public void setExplainAnalyze(boolean explainAnalyze) { - isExplainAnalyze = explainAnalyze; + public void setExplainType(ExplainType explainType) { + this.explainType = explainType; + } + + public ExplainType getExplainType() { + return explainType; } public boolean isExplainAnalyze() { - return isExplainAnalyze; + return explainType == ExplainType.EXPLAIN_ANALYZE; + } + + public boolean isExplain() { + return explainType == ExplainType.EXPLAIN; } public void setVerbose(boolean verbose) { @@ -480,12 +491,12 @@ public void setCteDataStores(Map, CteDataStore> cteDataStores) { this.cteDataStores = cteDataStores; } - public void addCteDistPlan(Table table, List cteDistPlan) { - cteDistPlans.put(NodeRef.of(table), cteDistPlan); + public void addCteExplainResult(Table table, Pair> cteExplainResult) { + cteExplainResults.put(NodeRef.of(table), cteExplainResult); } - public Map, List> getCteDistPlans() { - return cteDistPlans; + public Map, Pair>> getCteExplainResults() { + return cteExplainResults; } // ================= Authentication Interfaces ========================= diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java index 234d60a7f38a..67eafe7085ff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java @@ -32,6 +32,8 @@ import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.QueryExecution; import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.statistics.FragmentInstanceStatisticsDrawer; import org.apache.iotdb.db.queryengine.statistics.QueryStatisticsFetcher; import org.apache.iotdb.db.queryengine.statistics.StatisticLine; @@ -45,6 +47,7 @@ import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.common.block.column.TimeColumnBuilder; import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.RamUsageEstimator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -172,20 +175,10 @@ private void logIntermediateResultIfTimeout() { } private TsBlock buildResult() throws FragmentInstanceFetchException { - // CTE materialization - List analyzeResult = new ArrayList<>(); - mppQueryContext - .getCteDistPlans() - .forEach( - (table, distPlan) -> { - analyzeResult.add(String.format("CTE '%s' Query", table.getNode().getName())); - analyzeResult.addAll(distPlan); - }); - - if (!analyzeResult.isEmpty()) { - analyzeResult.add("Main Query"); - } - analyzeResult.addAll(buildFragmentInstanceStatistics(instances, verbose)); + Map, Pair>> cteAnalyzeResults = + mppQueryContext.getCteExplainResults(); + List mainAnalyzeResult = buildFragmentInstanceStatistics(instances, verbose); + List analyzeResult = mergeAnalyzeResults(cteAnalyzeResults, mainAnalyzeResult); TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.TEXT)); TimeColumnBuilder timeColumnBuilder = builder.getTimeColumnBuilder(); @@ -199,6 +192,47 @@ private TsBlock buildResult() throws FragmentInstanceFetchException { return builder.build(); } + private List mergeAnalyzeResults( + Map, Pair>> cteAnalyzeResults, + List mainAnalyzeResult) { + if (cteAnalyzeResults.isEmpty()) { + return mainAnalyzeResult; + } + + final int maxLineLength = + Math.max( + cteAnalyzeResults.values().stream().mapToInt(p -> p.left).max().orElse(0), + fragmentInstanceStatisticsDrawer.getMaxLineLength()); + + List analyzeResult = new ArrayList<>(); + cteAnalyzeResults.forEach( + (table, pair) -> { + analyzeResult.add(String.format("CTE '%s' Query", table.getNode().getName())); + for (String line : pair.right) { + StringBuilder sb = new StringBuilder(); + sb.append(line); + for (int i = 0; i < maxLineLength - line.length(); i++) { + sb.append(" "); + } + analyzeResult.add(sb.toString()); + } + analyzeResult.add(""); + }); + + analyzeResult.add("Main Query"); + mainAnalyzeResult.forEach( + line -> { + StringBuilder sb = new StringBuilder(); + sb.append(line); + for (int i = 0; i < maxLineLength - line.length(); i++) { + sb.append(" "); + } + analyzeResult.add(sb.toString()); + }); + + return analyzeResult; + } + @Override public boolean hasNext() throws Exception { return child.hasNext() || !outputResult; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index 30f6bd0cf2be..afd719739fd9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -39,6 +39,7 @@ import org.apache.iotdb.db.protocol.session.IClientSession; import org.apache.iotdb.db.protocol.session.PreparedStatementInfo; import org.apache.iotdb.db.queryengine.common.DataNodeEndPoints; +import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.common.SessionInfo; @@ -398,10 +399,16 @@ private IQueryExecution createQueryExecutionForTreeModel( } /** - * This method is specifically called in fetchUncorrelatedSubqueryResultForPredicate. When - * uncorrelated scalar subquery is handled in SubqueryPlanner, we try to fold it and get constant - * value. Since CTE could be used in the subquery, we should add CTE materialization result into + * This method is specifically used following subquery: + * + *

1. When uncorrelated scalar subquery is handled + * (fetchUncorrelatedSubqueryResultForPredicate), we try to fold it and get constant value. Since + * CTE might be referenced, we need to add CTE materialization result into subquery's * MPPQueryContext. + * + *

2. When CTE subquery is handled (fetchCteQueryResult), the main query, however, might be + * 'Explain' or 'Explain Analyze' statement. So we need to keep explain/explain analyze results + * along with CTE query dataset. */ public ExecutionResult executeForTableModel( org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement, @@ -412,6 +419,7 @@ public ExecutionResult executeForTableModel( String sql, Metadata metadata, Map, CteDataStore> cteDataStoreMap, + ExplainType explainType, long timeOut, boolean userQuery) { return execution( @@ -420,8 +428,9 @@ public ExecutionResult executeForTableModel( sql, userQuery, ((queryContext, startTime) -> { - queryContext.setCteDataStores(cteDataStoreMap); queryContext.setSubquery(true); + queryContext.setCteDataStores(cteDataStoreMap); + queryContext.setExplainType(explainType); return createQueryExecutionForTableModel( statement, sqlParser, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index 8abc92e9d4ab..10a2d9383edd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -47,6 +47,7 @@ import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.exception.sql.StatementAnalyzeException; import org.apache.iotdb.db.queryengine.common.DeviceContext; +import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.TimeseriesContext; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; @@ -250,7 +251,7 @@ public Analysis visitExplain(ExplainStatement explainStatement, MPPQueryContext public Analysis visitExplainAnalyze( ExplainAnalyzeStatement explainAnalyzeStatement, MPPQueryContext context) { Analysis analysis = visitQuery(explainAnalyzeStatement.getQueryStatement(), context); - context.setExplainAnalyze(true); + context.setExplainType(ExplainType.EXPLAIN_ANALYZE); analysis.setRealStatement(explainAnalyzeStatement); analysis.setRespDatasetHeader( new DatasetHeader( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java index dcf367ce5d52..abc743f09938 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java @@ -88,11 +88,12 @@ public StatementMemorySource visitExplain( // CTE materialization plan context .getQueryContext() - .getCteDistPlans() + .getCteExplainResults() .forEach( - (table, distPlan) -> { + (table, pair) -> { lines.add(String.format("CTE '%s' Query", table.getNode().getName())); - lines.addAll(distPlan); + lines.addAll(pair.getRight()); + lines.add(""); }); // Generate table model distributed plan diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/DistributedQueryPlan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/DistributedQueryPlan.java index 91cbe9ee97e3..d9970103512a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/DistributedQueryPlan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/DistributedQueryPlan.java @@ -21,16 +21,19 @@ import org.apache.iotdb.commons.utils.TestOnly; +import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; public class DistributedQueryPlan { private final SubPlan rootSubPlan; private final List instances; + private final List planText; public DistributedQueryPlan(SubPlan rootSubPlan, List instances) { this.rootSubPlan = rootSubPlan; this.instances = instances; + this.planText = new ArrayList<>(); } @TestOnly @@ -45,4 +48,12 @@ public SubPlan getRootSubPlan() { public List getInstances() { return instances; } + + public List getPlanText() { + return planText; + } + + public void addPlanText(List plan) { + planText.addAll(plan); + } } 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 fd0dd1044567..f8285ecf2f52 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 @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema; import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; import org.apache.iotdb.db.exception.sql.SemanticException; +import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.SessionInfo; import org.apache.iotdb.db.queryengine.execution.warnings.IoTDBWarning; @@ -832,13 +833,14 @@ protected Scope visitLoadTsFile(final LoadTsFile node, final Optional sco @Override protected Scope visitExplain(Explain node, Optional context) { + queryContext.setExplainType(ExplainType.EXPLAIN); analysis.setFinishQueryAfterAnalyze(); return visitQuery((Query) node.getStatement(), context); } @Override protected Scope visitExplainAnalyze(ExplainAnalyze node, Optional context) { - queryContext.setExplainAnalyze(true); + queryContext.setExplainType(ExplainType.EXPLAIN_ANALYZE); queryContext.setVerbose(node.isVerbose()); return visitQuery((Query) node.getStatement(), context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index 30a6cb2abb98..d027df7e6718 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -317,7 +317,7 @@ private PlanNode createOutputPlan(RelationPlan plan, Analysis analysis) { int columnNumber = 0; // TODO perfect the logic of outputDescriptor - if (queryContext.isExplainAnalyze()) { + if (queryContext.isExplainAnalyze() && !queryContext.isSubquery()) { outputs.add(new Symbol(ColumnHeaderConstant.EXPLAIN_ANALYZE)); names.add(ColumnHeaderConstant.EXPLAIN_ANALYZE); columnHeaders.add(new ColumnHeader(ColumnHeaderConstant.EXPLAIN_ANALYZE, TSDataType.TEXT)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java index 78f9729ece34..3b7981fdbd2c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java @@ -167,7 +167,7 @@ public DistributedQueryPlan doDistributionPlan( metadata, distributionPlanOptimizers, dataNodeLocationSupplier); - return planner.plan(); + return planner.plan(context); } finally { if (analysis.isQuery()) { long distributionPlanCost = System.nanoTime() - startTime; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java index cb474d39470f..5f40435a2d71 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java @@ -29,6 +29,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.SubPlan; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; @@ -97,10 +98,17 @@ public TableDistributedPlanner( this.dataNodeLocationSupplier = dataNodeLocationSupplier; } - public DistributedQueryPlan plan() { + public DistributedQueryPlan plan(MPPQueryContext context) { TableDistributedPlanGenerator.PlanContext planContext = new TableDistributedPlanGenerator.PlanContext(); PlanNode outputNodeWithExchange = generateDistributedPlanWithOptimize(planContext); + List planText = null; + if (context.isExplain() && context.isSubquery()) { + planText = + outputNodeWithExchange.accept( + new PlanGraphPrinter(), + new PlanGraphPrinter.GraphContext(context.getTypeProvider().getTemplatedInfo())); + } if (analysis.isQuery()) { analysis @@ -110,7 +118,13 @@ public DistributedQueryPlan plan() { adjustUpStream(outputNodeWithExchange, planContext); - return generateDistributedPlan(outputNodeWithExchange, planContext.nodeDistributionMap); + DistributedQueryPlan distributedPlan = + generateDistributedPlan(outputNodeWithExchange, planContext.nodeDistributionMap); + if (planText != null) { + distributedPlan.addPlanText(planText); + } + + return distributedPlan; } public PlanNode generateDistributedPlanWithOptimize( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index c605c0400c70..89b645666c6e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -21,38 +21,48 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.ir; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.db.exception.mpp.FragmentInstanceFetchException; import org.apache.iotdb.db.protocol.session.SessionManager; +import org.apache.iotdb.db.queryengine.common.FragmentInstanceId; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; +import org.apache.iotdb.db.queryengine.plan.execution.QueryExecution; import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; +import org.apache.iotdb.db.queryengine.plan.planner.plan.DistributedQueryPlan; +import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Explain; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ExplainAnalyze; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; +import org.apache.iotdb.db.queryengine.statistics.FragmentInstanceStatisticsDrawer; +import org.apache.iotdb.db.queryengine.statistics.QueryStatisticsFetcher; +import org.apache.iotdb.db.queryengine.statistics.StatisticLine; import org.apache.iotdb.db.utils.cte.CteDataStore; +import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsResp; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.tsfile.block.column.Column; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.type.TypeFactory; +import org.apache.tsfile.utils.Pair; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -75,7 +85,7 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { return; } - dataStore = fetchCteQueryResult(table, query, context.getTimeOut()); + dataStore = fetchCteQueryResult(table, query, context); if (dataStore == null) { // CTE query execution failed. Use inline instead of materialization // in the outer query @@ -86,24 +96,6 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); context.addCteDataStore(table, dataStore); query.setCteDataStore(dataStore); - - // Explain - if (analysis.canSkipExecute(context)) { - List cteDistPlan = - fetchCteExplainResult(table, new Explain(query), context.getTimeOut()); - if (cteDistPlan != null) { - context.addCteDistPlan(table, cteDistPlan); - } - } else if (context.isExplainAnalyze()) { - List cteDistPlan = - fetchCteExplainAnalyzeResult( - table, - new ExplainAnalyze(query, context.isVerbose()), - context.getTimeOut()); - if (cteDistPlan != null) { - context.addCteDistPlan(table, cteDistPlan); - } - } } }); } @@ -120,27 +112,77 @@ public static void cleanUpCTE(MPPQueryContext context) { cteDataStores.clear(); } - private static T execute( - Statement statement, long timeout, String sql, Function func) { + private static CteDataStore fetchCteQueryResult( + Table table, Query query, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; try { final ExecutionResult executionResult = coordinator.executeForTableModel( - statement, + query, new SqlParser(), SessionManager.getInstance().getCurrSession(), queryId, SessionManager.getInstance() .getSessionInfoOfTableModel(SessionManager.getInstance().getCurrSession()), - sql, + String.format("Materialize query for CTE '%s'", table.getName()), LocalExecutionPlanner.getInstance().metadata, - timeout, + ImmutableMap.of(), + context.getExplainType(), + context.getTimeOut(), false); if (executionResult.status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { return null; } - return func.apply(queryId); + // query execution + QueryExecution execution = (QueryExecution) coordinator.getQueryExecution(queryId); + + // get table schema + DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); + TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); + + CteDataStore cteDataStore = new CteDataStore(query, tableSchema); + while (execution.hasNextResult()) { + final Optional tsBlock; + try { + tsBlock = execution.getBatchResult(); + } catch (final IoTDBException e) { + throw new IoTDBRuntimeException( + String.format("Fail to materialize CTE because %s", e.getMessage()), + e.getErrorCode(), + e.isUserException()); + } + if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { + continue; + } + if (!cteDataStore.addTsBlock(tsBlock.get())) { + return null; + } + } + + DistributedQueryPlan distributedQueryPlan = execution.getDistributedPlan(); + if (distributedQueryPlan != null) { + if (context.isExplainAnalyze()) { + FragmentInstanceStatisticsDrawer fragmentInstanceStatisticsDrawer = + new FragmentInstanceStatisticsDrawer(); + fragmentInstanceStatisticsDrawer.renderPlanStatistics(context); + fragmentInstanceStatisticsDrawer.renderDispatchCost(context); + + List lines = + getCteExplainAnalyzeLines( + fragmentInstanceStatisticsDrawer, + distributedQueryPlan.getInstances(), + context.isVerbose()); + int maxLineLength = fragmentInstanceStatisticsDrawer.getMaxLineLength(); + context.addCteExplainResult(table, new Pair<>(maxLineLength, lines)); + } else if (context.isExplain()) { + List lines = distributedQueryPlan.getPlanText(); + context.addCteExplainResult(table, new Pair<>(-1, lines)); + } + } else { + context.addCteExplainResult(table, new Pair<>(0, ImmutableList.of())); + } + return cteDataStore; } catch (final Throwable throwable) { t = throwable; } finally { @@ -149,99 +191,6 @@ private static T execute( return null; } - private static List fetchCteExplainResult(Table table, Explain explain, long timeout) { - return execute( - explain, - timeout, - String.format("Explain query for CTE '%s'", table.getName()), - (queryId) -> { - List lines = new ArrayList<>(); - while (coordinator.getQueryExecution(queryId).hasNextResult()) { - final Optional tsBlock; - try { - tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); - } catch (IoTDBException e) { - throw new IoTDBRuntimeException( - String.format("Fail to explain CTE query because %s", e.getMessage()), - e.getErrorCode(), - e.isUserException()); - } - if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { - continue; - } - - Column valueColumn = tsBlock.get().getColumn(0); - for (int i = 0; i < tsBlock.get().getPositionCount(); i++) { - lines.add(valueColumn.getBinary(i).toString()); - } - } - return lines; - }); - } - - private static List fetchCteExplainAnalyzeResult( - Table table, ExplainAnalyze explainAnalyze, long timeout) { - return execute( - explainAnalyze, - timeout, - String.format("Explain analyze query for CTE '%s'", table.getName()), - (queryId) -> { - List lines = new ArrayList<>(); - while (coordinator.getQueryExecution(queryId).hasNextResult()) { - final Optional tsBlock; - try { - tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); - } catch (IoTDBException e) { - throw new IoTDBRuntimeException( - String.format("Fail to explain CTE query because %s", e.getMessage()), - e.getErrorCode(), - e.isUserException()); - } - if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { - continue; - } - - Column valueColumn = tsBlock.get().getColumn(0); - for (int i = 0; i < tsBlock.get().getPositionCount(); i++) { - lines.add(valueColumn.getBinary(i).toString()); - } - } - return lines; - }); - } - - private static CteDataStore fetchCteQueryResult(Table table, Query query, long timeout) { - return execute( - query, - timeout, - String.format("Materialize query for CTE '%s'", table.getName()), - (queryId) -> { - // get table schema - DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); - TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); - - CteDataStore cteDataStore = new CteDataStore(query, tableSchema); - while (coordinator.getQueryExecution(queryId).hasNextResult()) { - final Optional tsBlock; - try { - tsBlock = coordinator.getQueryExecution(queryId).getBatchResult(); - } catch (final IoTDBException e) { - throw new IoTDBRuntimeException( - String.format("Fail to materialize CTE because %s", e.getMessage()), - e.getErrorCode(), - e.isUserException()); - } - if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { - continue; - } - if (!cteDataStore.addTsBlock(tsBlock.get())) { - return null; - } - } - return cteDataStore; - }); - } - private static TableSchema getTableSchema(DatasetHeader datasetHeader, String cteName) { final List columnNames = datasetHeader.getRespColumns(); final List columnDataTypes = datasetHeader.getRespDataTypes(); @@ -285,4 +234,22 @@ private static TableSchema getTableSchema(DatasetHeader datasetHeader, String ct TsTableColumnCategory.FIELD))); return new TableSchema(cteName, columnSchemaList); } + + private static List getCteExplainAnalyzeLines( + FragmentInstanceStatisticsDrawer fragmentInstanceStatisticsDrawer, + List instances, + boolean verbose) + throws FragmentInstanceFetchException { + if (instances == null || instances.isEmpty()) { + return ImmutableList.of(); + } + + IClientManager clientManager = + coordinator.getInternalServiceClientManager(); + Map allStatistics = + QueryStatisticsFetcher.fetchAllStatistics(instances, clientManager); + List statisticLines = + fragmentInstanceStatisticsDrawer.renderFragmentInstances(instances, allStatistics, verbose); + return statisticLines.stream().map(StatisticLine::getValue).collect(Collectors.toList()); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 9487edae254a..2cd07ecd7b2b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.db.protocol.session.SessionManager; +import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; @@ -111,6 +112,7 @@ private static Optional fetchUncorrelatedSubqueryResultForPredicate( "Try to Fetch Uncorrelated Scalar Subquery Result for Predicate", LocalExecutionPlanner.getInstance().metadata, context.getCteDataStores(), + ExplainType.NONE, context.getTimeOut(), false); From da312ffec836bc2cbbd016289cec396c1fd98012 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 25 Sep 2025 16:42:49 +0800 Subject: [PATCH 16/69] code refactor --- .../operator/ExplainAnalyzeOperator.java | 17 ++-- ...ableModelStatementMemorySourceVisitor.java | 51 +++++++----- .../planner/ir/CteMaterializer.java | 78 +++++++++++++------ .../iotdb/commons/conf/IoTDBConstant.java | 4 + 4 files changed, 104 insertions(+), 46 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java index 67eafe7085ff..378e4668071f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java @@ -59,6 +59,11 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import static org.apache.iotdb.commons.conf.IoTDBConstant.BLANK; +import static org.apache.iotdb.commons.conf.IoTDBConstant.CTE_QUERY; +import static org.apache.iotdb.commons.conf.IoTDBConstant.MAIN_QUERY; +import static org.apache.iotdb.commons.conf.IoTDBConstant.SPACE; + public class ExplainAnalyzeOperator implements ProcessOperator { private static final Logger logger = LoggerFactory.getLogger(IoTDBConstant.EXPLAIN_ANALYZE_LOGGER_NAME); @@ -145,7 +150,7 @@ private List buildFragmentInstanceStatistics( for (int i = 0; i < fragmentInstanceStatisticsDrawer.getMaxLineLength() - line.getValue().length(); i++) { - sb.append(" "); + sb.append(SPACE); } analyzeResult.add(sb.toString()); } @@ -207,25 +212,25 @@ private List mergeAnalyzeResults( List analyzeResult = new ArrayList<>(); cteAnalyzeResults.forEach( (table, pair) -> { - analyzeResult.add(String.format("CTE '%s' Query", table.getNode().getName())); + analyzeResult.add(String.format("%s : '%s'", CTE_QUERY, table.getNode().getName())); for (String line : pair.right) { StringBuilder sb = new StringBuilder(); sb.append(line); for (int i = 0; i < maxLineLength - line.length(); i++) { - sb.append(" "); + sb.append(SPACE); } analyzeResult.add(sb.toString()); } - analyzeResult.add(""); + analyzeResult.add(BLANK); }); - analyzeResult.add("Main Query"); + analyzeResult.add(MAIN_QUERY); mainAnalyzeResult.forEach( line -> { StringBuilder sb = new StringBuilder(); sb.append(line); for (int i = 0; i < maxLineLength - line.length(); i++) { - sb.append(" "); + sb.append(SPACE); } analyzeResult.add(sb.toString()); }); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java index abc743f09938..932d94197922 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/memory/TableModelStatementMemorySourceVisitor.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolAllocator; import org.apache.iotdb.db.queryengine.plan.relational.planner.TableLogicalPlanner; import org.apache.iotdb.db.queryengine.plan.relational.planner.distribute.TableDistributedPlanGenerator; @@ -36,14 +37,20 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Explain; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Node; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowDevice; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.utils.Pair; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; +import static org.apache.iotdb.commons.conf.IoTDBConstant.BLANK; +import static org.apache.iotdb.commons.conf.IoTDBConstant.CTE_QUERY; +import static org.apache.iotdb.commons.conf.IoTDBConstant.MAIN_QUERY; import static org.apache.iotdb.db.queryengine.common.header.DatasetHeader.EMPTY_HEADER; import static org.apache.iotdb.db.queryengine.execution.warnings.WarningCollector.NOOP; import static org.apache.iotdb.db.queryengine.plan.execution.memory.StatementMemorySourceVisitor.getStatementMemorySource; @@ -83,19 +90,6 @@ public StatementMemorySource visitExplain( return new StatementMemorySource(new TsBlock(0), header); } - final List lines = new ArrayList<>(); - - // CTE materialization plan - context - .getQueryContext() - .getCteExplainResults() - .forEach( - (table, pair) -> { - lines.add(String.format("CTE '%s' Query", table.getNode().getName())); - lines.addAll(pair.getRight()); - lines.add(""); - }); - // Generate table model distributed plan final TableDistributedPlanGenerator.PlanContext planContext = new TableDistributedPlanGenerator.PlanContext(); @@ -109,14 +103,15 @@ public StatementMemorySource visitExplain( Coordinator.getInstance().getDataNodeLocationSupplier()) .generateDistributedPlanWithOptimize(planContext); - if (!lines.isEmpty()) { - lines.add("Main Query"); - } - lines.addAll( + List mainExplainResult = outputNodeWithExchange.accept( new PlanGraphPrinter(), new PlanGraphPrinter.GraphContext( - context.getQueryContext().getTypeProvider().getTemplatedInfo()))); + context.getQueryContext().getTypeProvider().getTemplatedInfo())); + + Map, Pair>> cteExplainResults = + context.getQueryContext().getCteExplainResults(); + List lines = mergeExplainResults(cteExplainResults, mainExplainResult); return getStatementMemorySource(header, lines); } @@ -134,4 +129,24 @@ public StatementMemorySource visitCountDevice( return new StatementMemorySource( node.getTsBlock(context.getAnalysis()), node.getDataSetHeader()); } + + private List mergeExplainResults( + Map, Pair>> cteExplainResults, + List mainExplainResult) { + if (cteExplainResults.isEmpty()) { + return mainExplainResult; + } + + List analyzeResult = new ArrayList<>(); + cteExplainResults.forEach( + (table, pair) -> { + analyzeResult.add(String.format("%s : '%s'", CTE_QUERY, table.getNode().getName())); + analyzeResult.addAll(pair.getRight()); + analyzeResult.add(BLANK); + }); + analyzeResult.add(MAIN_QUERY); + analyzeResult.addAll(mainExplainResult); + + return analyzeResult; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index 89b645666c6e..c1d79ba9ff0e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -67,6 +67,8 @@ import java.util.stream.IntStream; public class CteMaterializer { + private static String CTE_MATERIALIZATION_FAILURE_WARNING = + "***** CTE MATERIALIZATION failed! INLINE mode is adopted in main query *****"; private static final Coordinator coordinator = Coordinator.getInstance(); @@ -136,7 +138,6 @@ private static CteDataStore fetchCteQueryResult( } // query execution QueryExecution execution = (QueryExecution) coordinator.getQueryExecution(queryId); - // get table schema DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); @@ -156,32 +157,20 @@ private static CteDataStore fetchCteQueryResult( continue; } if (!cteDataStore.addTsBlock(tsBlock.get())) { + if (context.isExplainAnalyze()) { + handleCteExplainAnalyzeResults( + context, queryId, table, CTE_MATERIALIZATION_FAILURE_WARNING); + } return null; } } - DistributedQueryPlan distributedQueryPlan = execution.getDistributedPlan(); - if (distributedQueryPlan != null) { - if (context.isExplainAnalyze()) { - FragmentInstanceStatisticsDrawer fragmentInstanceStatisticsDrawer = - new FragmentInstanceStatisticsDrawer(); - fragmentInstanceStatisticsDrawer.renderPlanStatistics(context); - fragmentInstanceStatisticsDrawer.renderDispatchCost(context); - - List lines = - getCteExplainAnalyzeLines( - fragmentInstanceStatisticsDrawer, - distributedQueryPlan.getInstances(), - context.isVerbose()); - int maxLineLength = fragmentInstanceStatisticsDrawer.getMaxLineLength(); - context.addCteExplainResult(table, new Pair<>(maxLineLength, lines)); - } else if (context.isExplain()) { - List lines = distributedQueryPlan.getPlanText(); - context.addCteExplainResult(table, new Pair<>(-1, lines)); - } - } else { - context.addCteExplainResult(table, new Pair<>(0, ImmutableList.of())); + if (context.isExplainAnalyze()) { + handleCteExplainAnalyzeResults(context, queryId, table, null); + } else if (context.isExplain()) { + handleCteExplainResults(context, queryId, table); } + return cteDataStore; } catch (final Throwable throwable) { t = throwable; @@ -252,4 +241,49 @@ private static List getCteExplainAnalyzeLines( fragmentInstanceStatisticsDrawer.renderFragmentInstances(instances, allStatistics, verbose); return statisticLines.stream().map(StatisticLine::getValue).collect(Collectors.toList()); } + + private static void handleCteExplainAnalyzeResults( + MPPQueryContext context, long queryId, Table table, String warnMessage) { + QueryExecution execution = (QueryExecution) coordinator.getQueryExecution(queryId); + DistributedQueryPlan distributedQueryPlan = execution.getDistributedPlan(); + if (distributedQueryPlan == null) { + context.addCteExplainResult(table, new Pair<>(0, ImmutableList.of())); + return; + } + + FragmentInstanceStatisticsDrawer fragmentInstanceStatisticsDrawer = + new FragmentInstanceStatisticsDrawer(); + fragmentInstanceStatisticsDrawer.renderPlanStatistics(context); + fragmentInstanceStatisticsDrawer.renderDispatchCost(context); + + try { + List lines = + getCteExplainAnalyzeLines( + fragmentInstanceStatisticsDrawer, + distributedQueryPlan.getInstances(), + context.isVerbose()); + int maxLineLength = fragmentInstanceStatisticsDrawer.getMaxLineLength(); + if (warnMessage != null) { + lines.add(warnMessage); + maxLineLength = Math.max(maxLineLength, warnMessage.length()); + } + context.addCteExplainResult(table, new Pair<>(maxLineLength, lines)); + } catch (FragmentInstanceFetchException e) { + throw new IoTDBRuntimeException( + "Failed to fetch fragment instance statistics", + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + } + + private static void handleCteExplainResults(MPPQueryContext context, long queryId, Table table) { + QueryExecution execution = (QueryExecution) coordinator.getQueryExecution(queryId); + DistributedQueryPlan distributedQueryPlan = execution.getDistributedPlan(); + if (distributedQueryPlan == null) { + context.addCteExplainResult(table, new Pair<>(0, ImmutableList.of())); + return; + } + + List lines = distributedQueryPlan.getPlanText(); + context.addCteExplainResult(table, new Pair<>(-1, lines)); + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java index 28c3f8aececd..c7575577ef15 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java @@ -216,6 +216,9 @@ private IoTDBConstant() {} public static final String MULTI_LEVEL_PATH_WILDCARD = "**"; public static final String TIME = "time"; + public static final String CTE_QUERY = "CTE Query"; + public static final String MAIN_QUERY = "Main Query"; + // sdt parameters public static final String LOSS = "loss"; public static final String SDT = "sdt"; @@ -301,6 +304,7 @@ private IoTDBConstant() {} public static final String SETTLE_SUFFIX = ".settle"; public static final String MODS_SETTLE_FILE_SUFFIX = ".mods.settle"; public static final String BLANK = ""; + public static final String SPACE = " "; // write ahead log public static final String WAL_FILE_PREFIX = "_"; From 9fb47e60c0002d7f35efb309858a327c4e443268 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 25 Sep 2025 17:24:12 +0800 Subject: [PATCH 17/69] add warn log for cte materialization failure --- .../plan/relational/planner/ir/CteMaterializer.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index c1d79ba9ff0e..667776de15f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -58,6 +58,8 @@ import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.type.TypeFactory; import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; @@ -67,6 +69,7 @@ import java.util.stream.IntStream; public class CteMaterializer { + private static final Logger LOGGER = LoggerFactory.getLogger(CteMaterializer.class); private static String CTE_MATERIALIZATION_FAILURE_WARNING = "***** CTE MATERIALIZATION failed! INLINE mode is adopted in main query *****"; @@ -148,15 +151,15 @@ private static CteDataStore fetchCteQueryResult( try { tsBlock = execution.getBatchResult(); } catch (final IoTDBException e) { - throw new IoTDBRuntimeException( - String.format("Fail to materialize CTE because %s", e.getMessage()), - e.getErrorCode(), - e.isUserException()); + LOGGER.warn("Fail to materialize CTE because {}", e.getMessage()); + return null; } if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { continue; } if (!cteDataStore.addTsBlock(tsBlock.get())) { + LOGGER.warn( + "Fail to materialize CTE because the data size exceeded memory or the row count threshold"); if (context.isExplainAnalyze()) { handleCteExplainAnalyzeResults( context, queryId, table, CTE_MATERIALIZATION_FAILURE_WARNING); From 68e339f6d27ad480397fc5c3d1f65c4a4e550be5 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 26 Sep 2025 11:44:18 +0800 Subject: [PATCH 18/69] fix: should not output ExplainAnalyzeOperator in explain analyze --- .../fragment/FragmentInstanceExecution.java | 27 +++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java index 9985abb70984..0a08144e7317 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java @@ -28,9 +28,9 @@ import org.apache.iotdb.db.queryengine.execution.driver.IDriver; import org.apache.iotdb.db.queryengine.execution.exchange.MPPDataExchangeManager; import org.apache.iotdb.db.queryengine.execution.exchange.sink.ISink; +import org.apache.iotdb.db.queryengine.execution.operator.ExplainAnalyzeOperator; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.schedule.IDriverScheduler; -import org.apache.iotdb.db.storageengine.dataregion.IDataRegionForQuery; import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsResp; import org.apache.iotdb.mpp.rpc.thrift.TOperatorStatistics; @@ -140,13 +140,36 @@ public FragmentInstanceStateMachine getStateMachine() { return stateMachine; } + // Check if this fragment instance should be ignored for statistics + // (i.e., it contains ExplainAnalyzeOperator only) + private boolean shouldIgnoreForStatistics() { + if (drivers == null || drivers.isEmpty()) { + return false; + } + // Check if any driver contains ExplainAnalyzeOperator + return drivers.stream() + .anyMatch( + driver -> + driver.getDriverContext().getOperatorContexts().stream() + .anyMatch( + operatorContext -> + ExplainAnalyzeOperator.class + .getSimpleName() + .equals(operatorContext.getOperatorType()))); + } + // Fill Fragment-Level info for statistics private boolean fillFragmentInstanceStatistics( FragmentInstanceContext context, TFetchFragmentInstanceStatisticsResp statistics) { statistics.setFragmentInstanceId(context.getId().toThrift()); statistics.setQueryStatistics(context.getQueryStatistics().toThrift()); statistics.setState(getInstanceState().toString()); - IDataRegionForQuery dataRegionForQuery = context.getDataRegion(); + + // Check if this fragment instance should be ignored for statistics + if (shouldIgnoreForStatistics()) { + // We don't need to output the region having ExplainAnalyzeOperator only. + return false; + } statistics.setDataRegion(context.getDataRegion().getDataRegionId()); statistics.setIp(CONFIG.getInternalAddress() + ":" + CONFIG.getInternalPort()); From bae0b181b55539777fbe5bc7ad50ca7547b3aaa7 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 26 Sep 2025 14:29:10 +0800 Subject: [PATCH 19/69] fix: plan method --- .../plan/relational/planner/TableModelPlanner.java | 2 +- .../planner/distribute/TableDistributedPlanner.java | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java index 3b7981fdbd2c..78f9729ece34 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java @@ -167,7 +167,7 @@ public DistributedQueryPlan doDistributionPlan( metadata, distributionPlanOptimizers, dataNodeLocationSupplier); - return planner.plan(context); + return planner.plan(); } finally { if (analysis.isQuery()) { long distributionPlanCost = System.nanoTime() - startTime; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java index 5f40435a2d71..e8ffd74a1568 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java @@ -98,16 +98,17 @@ public TableDistributedPlanner( this.dataNodeLocationSupplier = dataNodeLocationSupplier; } - public DistributedQueryPlan plan(MPPQueryContext context) { + public DistributedQueryPlan plan() { TableDistributedPlanGenerator.PlanContext planContext = new TableDistributedPlanGenerator.PlanContext(); PlanNode outputNodeWithExchange = generateDistributedPlanWithOptimize(planContext); List planText = null; - if (context.isExplain() && context.isSubquery()) { + if (mppQueryContext.isExplain() && mppQueryContext.isSubquery()) { planText = outputNodeWithExchange.accept( new PlanGraphPrinter(), - new PlanGraphPrinter.GraphContext(context.getTypeProvider().getTemplatedInfo())); + new PlanGraphPrinter.GraphContext( + mppQueryContext.getTypeProvider().getTemplatedInfo())); } if (analysis.isQuery()) { From c2932b30e7e739c24d2a7ba9ecc5de5d4c73b553 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 26 Sep 2025 17:04:32 +0800 Subject: [PATCH 20/69] fix: Fragment Instances Count --- .../FragmentInstanceStatisticsDrawer.java | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java index 7a8b7097eb9b..4077552c0499 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class FragmentInstanceStatisticsDrawer { private int maxLineLength = 0; @@ -83,14 +84,21 @@ public List renderFragmentInstances( Map allStatistics, boolean verbose) { List table = new ArrayList<>(planHeader); - addLine( - table, 0, String.format("Fragment Instances Count: %s", instancesToBeRendered.size() - 1)); - for (FragmentInstance instance : instancesToBeRendered) { + List validInstances = + instancesToBeRendered.stream() + .filter( + instance -> { + TFetchFragmentInstanceStatisticsResp statistics = + allStatistics.get(instance.getId()); + return statistics != null && statistics.getDataRegion() != null; + }) + .collect(Collectors.toList()); + + addLine(table, 0, String.format("Fragment Instances Count: %s", validInstances.size())); + for (FragmentInstance instance : validInstances) { List singleFragmentInstanceArea = new ArrayList<>(); TFetchFragmentInstanceStatisticsResp statistics = allStatistics.get(instance.getId()); - if (statistics == null || statistics.getDataRegion() == null) { - continue; - } + addBlankLine(singleFragmentInstanceArea); addLine( singleFragmentInstanceArea, From 50e4d1cc22887aa4ab8714159709c0a7a5939e55 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 28 Sep 2025 14:44:35 +0800 Subject: [PATCH 21/69] fix bugs 1. missing visitCteScan in TransformSortToStreamSort 2. project node assignments --- .../relational/planner/RelationPlanner.java | 37 ++++++-------- .../planner/ir/CteMaterializer.java | 48 +++++++------------ .../TransformSortToStreamSort.java | 7 +++ .../iotdb/db/utils/cte/CteDataStore.java | 9 +++- 4 files changed, 47 insertions(+), 54 deletions(-) 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 e6e71869663d..7a24917f5add 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 @@ -252,35 +252,28 @@ protected RelationPlan visitTable(final Table table, final Void context) { if (namedQuery.isMaterialized()) { CteDataStore dataStore = queryContext.getCteDataStore(table); if (dataStore != null) { - List cteSymbols = new ArrayList<>(); - Map cteSymbolMap = new HashMap<>(); - dataStore - .getTableSchema() - .getColumns() - .forEach( - column -> { - Symbol columnSymbol = - symbolAllocator.newSymbol(column.getName(), column.getType()); - cteSymbols.add(columnSymbol); - cteSymbolMap.put(column.getName(), columnSymbol); - }); + List cteSymbols = + dataStore.getTableSchema().getColumns().stream() + .map(column -> symbolAllocator.newSymbol(column.getName(), column.getType())) + .collect(Collectors.toList()); // CTE Scan Node CteScanNode cteScanNode = new CteScanNode(idAllocator.genPlanNodeId(), table.getName(), cteSymbols, dataStore); + List columnIndex2TsBlockColumnIndexList = + dataStore.getColumnIndex2TsBlockColumnIndexList(); + if (columnIndex2TsBlockColumnIndexList == null) { + return new RelationPlan(cteScanNode, scope, cteSymbols, outerContext); + } + List outputSymbols = new ArrayList<>(); Assignments.Builder assignments = Assignments.builder(); - analysis - .getOutputDescriptor(table) - .getVisibleFields() - .forEach( - field -> { - String columnName = field.getName().orElse("field"); - Symbol symbol = cteSymbolMap.get(columnName); - outputSymbols.add(symbol); - assignments.put(symbol, symbol.toSymbolReference()); - }); + for (int index : columnIndex2TsBlockColumnIndexList) { + Symbol columnSymbol = cteSymbols.get(index); + outputSymbols.add(columnSymbol); + assignments.put(columnSymbol, columnSymbol.toSymbolReference()); + } // Project Node ProjectNode projectNode = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index 667776de15f9..b4a5810a33a1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -61,7 +61,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; @@ -145,7 +144,9 @@ private static CteDataStore fetchCteQueryResult( DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); - CteDataStore cteDataStore = new CteDataStore(query, tableSchema); + CteDataStore cteDataStore = + new CteDataStore( + query, tableSchema, datasetHeader.getColumnIndex2TsBlockColumnIndexList()); while (execution.hasNextResult()) { final Optional tsBlock; try { @@ -191,39 +192,24 @@ private static TableSchema getTableSchema(DatasetHeader datasetHeader, String ct "Size of column names and column data types do not match", TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); } - final Map columnNameIndexMap = datasetHeader.getColumnNameIndexMap(); - final List columnSchemaList = new ArrayList<>(); - // If CTE query returns empty result set, columnNameIndexMap is null - if (columnNameIndexMap == null) { - for (int i = 0; i < columnNames.size(); i++) { - columnSchemaList.add( - new ColumnSchema( - columnNames.get(i), - TypeFactory.getType(columnDataTypes.get(i)), - false, - TsTableColumnCategory.FIELD)); - } - return new TableSchema(cteName, columnSchemaList); + List columnIndex2TsBlockColumnIndexList = + datasetHeader.getColumnIndex2TsBlockColumnIndexList(); + if (columnIndex2TsBlockColumnIndexList == null) { + columnIndex2TsBlockColumnIndexList = + IntStream.range(0, columnNames.size()).boxed().collect(Collectors.toList()); } - - // build name -> type map - Map columnNameDataTypeMap = - IntStream.range(0, columnNames.size()) - .boxed() - .collect(Collectors.toMap(columnNames::get, columnDataTypes::get)); - - // build column schema list of cte table based on columnNameIndexMap - columnNameIndexMap.entrySet().stream() - .sorted(Map.Entry.comparingByValue()) - .forEach( - entry -> - columnSchemaList.add( + // build column schema list of cte table based on column2BlockColumnIndex + final List columnSchemaList = + columnIndex2TsBlockColumnIndexList.stream() + .map( + index -> new ColumnSchema( - entry.getKey(), - TypeFactory.getType(columnNameDataTypeMap.get(entry.getKey())), + columnNames.get(index), + TypeFactory.getType(columnDataTypes.get(index)), false, - TsTableColumnCategory.FIELD))); + TsTableColumnCategory.FIELD)) + .collect(Collectors.toList()); return new TableSchema(cteName, columnSchemaList); } 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 3eb338a650cf..7eb6dfb81c97 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 @@ -29,6 +29,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode; 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.GroupNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; @@ -131,6 +132,12 @@ public PlanNode visitGroup(GroupNode node, Context context) { return visitSingleChildProcess(node, context); } + @Override + public PlanNode visitCteScan(CteScanNode node, Context context) { + context.setCanTransform(false); + return node; + } + @Override public PlanNode visitDeviceTableScan(DeviceTableScanNode node, Context context) { context.setTableScanNode(node); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index 2f5474d9eb50..bdedc496b58f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -34,14 +34,17 @@ public class CteDataStore { private final Query query; private final TableSchema tableSchema; + private final List columnIndex2TsBlockColumnIndexList; private final List cachedData; private long cachedBytes; private int cachedRows; - public CteDataStore(Query query, TableSchema tableSchema) { + public CteDataStore( + Query query, TableSchema tableSchema, List columnIndex2TsBlockColumnIndexList) { this.query = query; this.tableSchema = tableSchema; + this.columnIndex2TsBlockColumnIndexList = columnIndex2TsBlockColumnIndexList; this.cachedData = new ArrayList<>(); this.cachedBytes = 0L; this.cachedRows = 0; @@ -84,4 +87,8 @@ public TableSchema getTableSchema() { public Query getQuery() { return query; } + + public List getColumnIndex2TsBlockColumnIndexList() { + return columnIndex2TsBlockColumnIndexList; + } } From 99c33656422f4126f32b0d34a38b665e608b0a32 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 28 Sep 2025 16:06:19 +0800 Subject: [PATCH 22/69] cte IT cases (inline & materialized) --- .../it/query/recent/IoTDBCteIT.java | 247 +++++++++++------- 1 file changed, 149 insertions(+), 98 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java index e3100e6ee5c4..ec8bcebad44d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java @@ -45,7 +45,9 @@ import java.sql.ResultSetMetaData; import java.sql.SQLException; import java.sql.Statement; +import java.util.Arrays; import java.util.Locale; +import java.util.stream.Collectors; import static org.apache.iotdb.db.it.utils.TestUtils.tableAssertTestFail; import static org.apache.iotdb.db.it.utils.TestUtils.tableResultSetEqualTest; @@ -69,6 +71,8 @@ public class IoTDBCteIT { private static final String dropDbSqls = "DROP DATABASE IF EXISTS testdb"; + private static final String[] cteKeywords = {"", "materialized"}; + @BeforeClass public static void setUpClass() { Locale.setDefault(Locale.ENGLISH); @@ -103,6 +107,9 @@ public void tearDown() { @Test public void testQuery() { + final String mainQuery = "select * from cte order by deviceid"; + + // case 1 String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; String[] retArray = new String[] { @@ -110,43 +117,40 @@ public void testQuery() { "1970-01-01T00:00:02.000Z,d1,200.0,", "1970-01-01T00:00:01.000Z,d2,300.0," }; - tableResultSetEqualTest( - "with cte as (select * from testtb) select * from cte order by deviceid", - expectedHeader, - retArray, - DATABASE_NAME); + String[] cteTemplateQueries = new String[] {"cte as %s (select * from testtb)"}; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + // case 2 expectedHeader = new String[] {"deviceid", "voltage"}; retArray = new String[] {"d1,100.0,", "d1,200.0,", "d2,300.0,"}; - tableResultSetEqualTest( - "with cte as (select deviceid, voltage from testtb) select * from cte order by deviceid", - expectedHeader, - retArray, - DATABASE_NAME); + cteTemplateQueries = new String[] {"cte as %s (select deviceid, voltage from testtb)"}; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + // case 3 expectedHeader = new String[] {"deviceid", "avg_voltage"}; retArray = new String[] {"d1,150.0,", "d2,300.0,"}; - tableResultSetEqualTest( - "with cte as (select deviceid, avg(voltage) as avg_voltage from testtb group by deviceid) select * from cte order by deviceid", - expectedHeader, - retArray, - DATABASE_NAME); + cteTemplateQueries = + new String[] { + "cte as %s (select deviceid, avg(voltage) as avg_voltage from testtb group by deviceid)" + }; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); } @Test public void testPartialColumn() { + final String mainQuery = "select * from cte order by id"; + + // case 1 String[] expectedHeader = new String[] {"id", "v"}; String[] retArray = new String[] {"d1,100.0,", "d1,200.0,", "d2,300.0,"}; - tableResultSetEqualTest( - "with cte(id, v) as (select deviceid, voltage from testtb) select * from cte order by id", - expectedHeader, - retArray, - DATABASE_NAME); - - tableAssertTestFail( - "with cte(v) as (select deviceid, voltage from testtb) select * from cte order by id", - "701: Column alias list has 1 entries but relation has 2 columns", - DATABASE_NAME); + String[] cteTemplateQueries = + new String[] {"cte(id, v) as %s (select deviceid, voltage from testtb)"}; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + + // case 2 + cteTemplateQueries = new String[] {"cte(v) as %s (select deviceid, voltage from testtb)"}; + String errMsg = "701: Column alias list has 1 entries but relation has 2 columns"; + testCteFailureWithVariants(cteTemplateQueries, mainQuery, errMsg); } @Test @@ -154,21 +158,27 @@ public void testExplain() throws SQLException { try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { statement.execute("USE testdb"); - // explain - ResultSet resultSet = - statement.executeQuery( - "explain with cte as (select * from testtb) select * from cte order by deviceid"); - ResultSetMetaData metaData = resultSet.getMetaData(); - assertEquals(metaData.getColumnCount(), 1); - assertEquals(metaData.getColumnName(1), "distribution plan"); - - // explain analyze - resultSet = - statement.executeQuery( - "explain analyze with cte as (select * from testtb) select * from cte order by deviceid"); - metaData = resultSet.getMetaData(); - assertEquals(metaData.getColumnCount(), 1); - assertEquals(metaData.getColumnName(1), "Explain Analyze"); + for (String keyword : cteKeywords) { + // explain + ResultSet resultSet = + statement.executeQuery( + String.format( + "explain with cte as %s (select * from testtb) select * from cte order by deviceid", + keyword)); + ResultSetMetaData metaData = resultSet.getMetaData(); + assertEquals(metaData.getColumnCount(), 1); + assertEquals(metaData.getColumnName(1), "distribution plan"); + + // explain analyze + resultSet = + statement.executeQuery( + String.format( + "explain analyze with cte as %s (select * from testtb) select * from cte order by deviceid", + keyword)); + metaData = resultSet.getMetaData(); + assertEquals(metaData.getColumnCount(), 1); + assertEquals(metaData.getColumnName(1), "Explain Analyze"); + } } } @@ -176,46 +186,47 @@ public void testExplain() throws SQLException { public void testMultiReference() { String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; String[] retArray = new String[] {"1970-01-01T00:00:01.000Z,d2,300.0,"}; - tableResultSetEqualTest( - "with cte as (select * from testtb) select * from cte where voltage > (select avg(voltage) from cte)", - expectedHeader, - retArray, - DATABASE_NAME); + String[] cteTemplateQueries = new String[] {"cte as %s (select * from testtb)"}; + String mainQuery = "select * from cte where voltage > (select avg(voltage) from cte)"; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); } @Test public void testDomain() { + final String mainQuery = "select * from testtb order by deviceid"; + String[] expectedHeader = new String[] {"deviceid", "voltage"}; String[] retArray = new String[] {"d1,100.0,", "d1,200.0,", "d2,300.0,"}; - tableResultSetEqualTest( - "with testtb as (select deviceid, voltage from testtb) select * from testtb order by deviceid", - expectedHeader, - retArray, - DATABASE_NAME); - - tableAssertTestFail( - "with testtb as (select voltage from testtb) select * from testtb order by deviceid", - "616: Column 'deviceid' cannot be resolved", - DATABASE_NAME); + String[] cteTemplateQueries = + new String[] {"testtb as %s (select deviceid, voltage from testtb)"}; + + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + + cteTemplateQueries = new String[] {"testtb as %s (select voltage from testtb)"}; + String errMsg = "616: Column 'deviceid' cannot be resolved"; + testCteFailureWithVariants(cteTemplateQueries, mainQuery, errMsg); } @Test public void testSession() throws IoTDBConnectionException, StatementExecutionException { try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { session.executeNonQueryStatement("use testdb"); - SessionDataSet dataSet = - session.executeQueryStatement("with cte as (select * from testtb) select * from cte"); - - assertEquals(dataSet.getColumnNames().size(), 3); - assertEquals(dataSet.getColumnNames().get(0), "time"); - assertEquals(dataSet.getColumnNames().get(1), "deviceid"); - assertEquals(dataSet.getColumnNames().get(2), "voltage"); - int cnt = 0; - while (dataSet.hasNext()) { - dataSet.next(); - cnt++; + for (String keyword : cteKeywords) { + SessionDataSet dataSet = + session.executeQueryStatement( + String.format("with cte as %s (select * from testtb) select * from cte", keyword)); + + assertEquals(dataSet.getColumnNames().size(), 3); + assertEquals(dataSet.getColumnNames().get(0), "time"); + assertEquals(dataSet.getColumnNames().get(1), "deviceid"); + assertEquals(dataSet.getColumnNames().get(2), "voltage"); + int cnt = 0; + while (dataSet.hasNext()) { + dataSet.next(); + cnt++; + } + Assert.assertEquals(3, cnt); } - Assert.assertEquals(3, cnt); } } @@ -229,54 +240,63 @@ public void testJdbc() throws ClassNotFoundException, SQLException { uri, SessionConfig.DEFAULT_USER, SessionConfig.DEFAULT_PASSWORD); Statement statement = connection.createStatement()) { statement.executeUpdate("use testdb"); - ResultSet resultSet = - statement.executeQuery("with cte as (select * from testtb) select * from cte"); - - final ResultSetMetaData metaData = resultSet.getMetaData(); - assertEquals(metaData.getColumnCount(), 3); - assertEquals(metaData.getColumnLabel(1), "time"); - assertEquals(metaData.getColumnLabel(2), "deviceid"); - assertEquals(metaData.getColumnLabel(3), "voltage"); - - int cnt = 0; - while (resultSet.next()) { - cnt++; + for (String keyword : cteKeywords) { + ResultSet resultSet = + statement.executeQuery( + String.format("with cte as %s (select * from testtb) select * from cte", keyword)); + + final ResultSetMetaData metaData = resultSet.getMetaData(); + assertEquals(metaData.getColumnCount(), 3); + assertEquals(metaData.getColumnLabel(1), "time"); + assertEquals(metaData.getColumnLabel(2), "deviceid"); + assertEquals(metaData.getColumnLabel(3), "voltage"); + + int cnt = 0; + while (resultSet.next()) { + cnt++; + } + Assert.assertEquals(3, cnt); } - Assert.assertEquals(3, cnt); } } @Test public void testNest() { - String sql1 = - "WITH" - + " cte1 AS (select deviceid, voltage from testtb where voltage > 200)," - + " cte2 AS (SELECT voltage FROM cte1)" - + " SELECT * FROM cte2"; - - String sql2 = - "WITH" - + " cte2 AS (SELECT voltage FROM cte1)," - + " cte1 AS (select deviceid, voltage from testtb where voltage > 200)" - + " SELECT * FROM cte2"; + final String mainQuery = "SELECT * FROM cte2"; + String[] cteTemplateQueries = + new String[] { + "cte1 as %s (select deviceid, voltage from testtb where voltage > 200)", + "cte2 as %s (SELECT voltage FROM cte1)" + }; String[] expectedHeader = new String[] {"voltage"}; String[] retArray = new String[] {"300.0,"}; - tableResultSetEqualTest(sql1, expectedHeader, retArray, DATABASE_NAME); + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); - tableAssertTestFail(sql2, "550: Table 'testdb.cte1' does not exist.", DATABASE_NAME); + cteTemplateQueries = + new String[] { + "cte2 as %s (SELECT voltage FROM cte1)", + "cte1 as %s (select deviceid, voltage from testtb where voltage > 200)" + }; + String errMsg = "550: Table 'testdb.cte1' does not exist."; + testCteFailureWithVariants(cteTemplateQueries, mainQuery, errMsg); } @Test public void testRecursive() { - String sql = - "WITH RECURSIVE t(n) AS (" + String sqlTemplate = + "WITH RECURSIVE t(n) AS %s (" + " VALUES (1)" + " UNION ALL" + " SELECT n+1 FROM t WHERE n < 100)" + " SELECT sum(n) FROM t"; - tableAssertTestFail(sql, "701: recursive cte is not supported yet", DATABASE_NAME); + for (String keyword : cteKeywords) { + tableAssertTestFail( + String.format(sqlTemplate, keyword), + "701: recursive cte is not supported yet", + DATABASE_NAME); + } } @Test @@ -295,7 +315,10 @@ public void testPrivileges() throws SQLException { .getConnection("tmpuser", "tmppw123456789", BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { statement.execute("USE testdb"); - statement.execute("with cte as (select * from testtb) select * from cte"); + for (String keyword : cteKeywords) { + statement.execute( + String.format("with cte as %s (select * from testtb) select * from cte", keyword)); + } } try (Connection connection = @@ -303,7 +326,11 @@ public void testPrivileges() throws SQLException { .getConnection("tmpuser", "tmppw123456789", BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { statement.execute("USE testdb"); - statement.execute("with cte as (select * from testtb1) select * from testtb"); + for (String keyword : cteKeywords) { + statement.execute( + String.format( + "with cte as %s (select * from testtb1) select * from testtb", keyword)); + } fail("No exception!"); } catch (Exception e) { Assert.assertTrue( @@ -329,4 +356,28 @@ private static void prepareData() { fail(e.getMessage()); } } + + private void testCteSuccessWithVariants( + String[] cteTemplateQueries, String mainQuery, String[] expectedHeader, String[] retArray) { + for (String keyword : cteKeywords) { + String cteQueries = + Arrays.stream(cteTemplateQueries) + .map(s -> String.format(s, keyword)) + .collect(Collectors.joining(", ")); + String query = String.format("with %s %s", cteQueries, mainQuery); + tableResultSetEqualTest(query, expectedHeader, retArray, DATABASE_NAME); + } + } + + private void testCteFailureWithVariants( + String[] cteTemplateQueries, String mainQuery, String expectedErrMsg) { + for (String keyword : cteKeywords) { + String cteQueries = + Arrays.stream(cteTemplateQueries) + .map(s -> String.format(s, keyword)) + .collect(Collectors.joining(", ")); + String query = String.format("with %s %s", cteQueries, mainQuery); + tableAssertTestFail(query, expectedErrMsg, DATABASE_NAME); + } + } } From 15b2343ece5008ed209db1c72cf40c2052774785 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 28 Sep 2025 16:26:00 +0800 Subject: [PATCH 23/69] fix: subquery analyze stats --- .../plan/relational/planner/ir/CteMaterializer.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java index b4a5810a33a1..755cee4c7547 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java @@ -240,10 +240,11 @@ private static void handleCteExplainAnalyzeResults( return; } + MPPQueryContext cteContext = execution.getContext(); FragmentInstanceStatisticsDrawer fragmentInstanceStatisticsDrawer = new FragmentInstanceStatisticsDrawer(); - fragmentInstanceStatisticsDrawer.renderPlanStatistics(context); - fragmentInstanceStatisticsDrawer.renderDispatchCost(context); + fragmentInstanceStatisticsDrawer.renderPlanStatistics(cteContext); + fragmentInstanceStatisticsDrawer.renderDispatchCost(cteContext); try { List lines = From 41b32cee8f12612b7926c3acc8b2ec2633329376 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 28 Sep 2025 17:35:59 +0800 Subject: [PATCH 24/69] fix: CteMaterializer single instance --- .../plan/execution/QueryExecution.java | 6 ++-- .../planner/{ir => }/CteMaterializer.java | 33 ++++++++++++------- .../planner/TableLogicalPlanner.java | 3 +- .../plan/relational/planner/PlanTester.java | 2 ++ 4 files changed, 27 insertions(+), 17 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/{ir => }/CteMaterializer.java (92%) 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 377c69ec8730..7ffc9dc1f7f1 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 @@ -47,7 +47,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeUtil; -import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.CteMaterializer; +import org.apache.iotdb.db.queryengine.plan.relational.planner.CteMaterializer; import org.apache.iotdb.db.queryengine.plan.scheduler.IScheduler; import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId; @@ -351,7 +351,7 @@ private void releaseResource() { cleanUpResultHandle(); } if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) && !context.isSubquery()) { - CteMaterializer.cleanUpCTE(context); + CteMaterializer.getInstance().cleanUpCTE(context); } } @@ -397,7 +397,7 @@ private void releaseResource(Throwable t) { cleanUpResultHandle(); } if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) && !context.isSubquery()) { - CteMaterializer.cleanUpCTE(context); + CteMaterializer.getInstance().cleanUpCTE(context); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java similarity index 92% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 755cee4c7547..7e0a35eddb56 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -19,7 +19,7 @@ * */ -package org.apache.iotdb.db.queryengine.plan.relational.planner.ir; +package org.apache.iotdb.db.queryengine.plan.relational.planner; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.IClientManager; @@ -70,13 +70,11 @@ public class CteMaterializer { private static final Logger LOGGER = LoggerFactory.getLogger(CteMaterializer.class); private static String CTE_MATERIALIZATION_FAILURE_WARNING = - "***** CTE MATERIALIZATION failed! INLINE mode is adopted in main query *****"; + "***** Failed to materialize CTE! INLINE mode is adopted in the main query *****"; private static final Coordinator coordinator = Coordinator.getInstance(); - private CteMaterializer() {} - - public static void materializeCTE(Analysis analysis, MPPQueryContext context) { + public void materializeCTE(Analysis analysis, MPPQueryContext context) { analysis .getNamedQueries() .forEach( @@ -104,7 +102,7 @@ public static void materializeCTE(Analysis analysis, MPPQueryContext context) { }); } - public static void cleanUpCTE(MPPQueryContext context) { + public void cleanUpCTE(MPPQueryContext context) { Map, CteDataStore> cteDataStores = context.getCteDataStores(); cteDataStores .values() @@ -116,8 +114,7 @@ public static void cleanUpCTE(MPPQueryContext context) { cteDataStores.clear(); } - private static CteDataStore fetchCteQueryResult( - Table table, Query query, MPPQueryContext context) { + private CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; try { @@ -184,7 +181,7 @@ private static CteDataStore fetchCteQueryResult( return null; } - private static TableSchema getTableSchema(DatasetHeader datasetHeader, String cteName) { + private TableSchema getTableSchema(DatasetHeader datasetHeader, String cteName) { final List columnNames = datasetHeader.getRespColumns(); final List columnDataTypes = datasetHeader.getRespDataTypes(); if (columnNames.size() != columnDataTypes.size()) { @@ -213,7 +210,7 @@ private static TableSchema getTableSchema(DatasetHeader datasetHeader, String ct return new TableSchema(cteName, columnSchemaList); } - private static List getCteExplainAnalyzeLines( + private List getCteExplainAnalyzeLines( FragmentInstanceStatisticsDrawer fragmentInstanceStatisticsDrawer, List instances, boolean verbose) @@ -231,7 +228,7 @@ private static List getCteExplainAnalyzeLines( return statisticLines.stream().map(StatisticLine::getValue).collect(Collectors.toList()); } - private static void handleCteExplainAnalyzeResults( + private void handleCteExplainAnalyzeResults( MPPQueryContext context, long queryId, Table table, String warnMessage) { QueryExecution execution = (QueryExecution) coordinator.getQueryExecution(queryId); DistributedQueryPlan distributedQueryPlan = execution.getDistributedPlan(); @@ -265,7 +262,7 @@ private static void handleCteExplainAnalyzeResults( } } - private static void handleCteExplainResults(MPPQueryContext context, long queryId, Table table) { + private void handleCteExplainResults(MPPQueryContext context, long queryId, Table table) { QueryExecution execution = (QueryExecution) coordinator.getQueryExecution(queryId); DistributedQueryPlan distributedQueryPlan = execution.getDistributedPlan(); if (distributedQueryPlan == null) { @@ -276,4 +273,16 @@ private static void handleCteExplainResults(MPPQueryContext context, long queryI List lines = distributedQueryPlan.getPlanText(); context.addCteExplainResult(table, new Pair<>(-1, lines)); } + + private static class CteMaterializerHolder { + private static final CteMaterializer INSTANCE = new CteMaterializer(); + + private CteMaterializerHolder() { + // Empty constructor + } + } + + public static CteMaterializer getInstance() { + return CteMaterializerHolder.INSTANCE; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index d027df7e6718..cc8111294dfc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -50,7 +50,6 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; -import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.CteMaterializer; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; @@ -374,7 +373,7 @@ private RelationPlan createRelationPlan(Analysis analysis, PipeEnriched pipeEnri private RelationPlan createRelationPlan(Analysis analysis, Query query) { // materialize cte if needed - CteMaterializer.materializeCTE(analysis, queryContext); + CteMaterializer.getInstance().materializeCTE(analysis, queryContext); return getRelationPlanner(analysis).process(query, null); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java index 850755702016..02ee999cda7a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.protocol.session.IClientSession; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; @@ -99,6 +100,7 @@ public PlanTester() { public PlanTester(Metadata metadata) { this.metadata = metadata; + IoTDBDescriptor.getInstance().getConfig().setDataNodeId(1); } public LogicalQueryPlan createPlan(String sql) { From 9f1dad1738aeaa9e24b2ee14b294b832f0b8bd05 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 29 Sep 2025 14:07:11 +0800 Subject: [PATCH 25/69] fix: Blob/String issue when folding uncorrelated scalar subquery --- ...thUncorrelatedScalarSubqueryReconstructor.java | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 2cd07ecd7b2b..03110852cdb5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; @@ -36,13 +37,16 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NotExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StringLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SubqueryExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; +import java.util.List; import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; @@ -140,7 +144,13 @@ private static Optional fetchUncorrelatedSubqueryResultForPredicate( checkArgument( tsBlock.get().getPositionCount() == 1 && !tsBlock.get().getColumn(0).isNull(0), "Scalar Subquery result should only have one row."); - switch (columns[0].getDataType()) { + + // column type + DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); + List dataTypes = datasetHeader.getRespDataTypes(); + checkArgument(dataTypes.size() == 1, "Scalar Subquery result should only have one column."); + + switch (dataTypes.get(0)) { case INT32: case DATE: return Optional.of(new LongLiteral(Long.toString(columns[0].getInt(0)))); @@ -154,9 +164,10 @@ private static Optional fetchUncorrelatedSubqueryResultForPredicate( case BOOLEAN: return Optional.of(new BooleanLiteral(Boolean.toString(columns[0].getBoolean(0)))); case BLOB: + return Optional.of(new BinaryLiteral(columns[0].getBinary(0).toString())); case TEXT: case STRING: - return Optional.of(new BinaryLiteral(columns[0].getBinary(0).toString())); + return Optional.of(new StringLiteral(columns[0].getBinary(0).toString())); default: throw new IllegalArgumentException( String.format( From 7b18aa587ed21c094a5bcc9ba14b3e132d0567e0 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 29 Sep 2025 15:51:26 +0800 Subject: [PATCH 26/69] fix: uncorrelated subquery test --- .../relational/planner/SubqueryPlanner.java | 2 +- ...correlatedScalarSubqueryReconstructor.java | 30 ++- .../planner/UncorrelatedSubqueryTest.java | 182 ++++-------------- .../assertions/DeviceTableScanMatcher.java | 11 ++ .../planner/assertions/PlanMatchPattern.java | 19 ++ .../planner/assertions/TableScanMatcher.java | 17 ++ 6 files changed, 114 insertions(+), 147 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java index f9e62930f029..1ac59f843480 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java @@ -157,7 +157,7 @@ public PlanBuilder handleSubqueries( private void tryFoldUncorrelatedScalarSubqueryInPredicate( Expression expression, MPPQueryContext context) { - PredicateWithUncorrelatedScalarSubqueryReconstructor + PredicateWithUncorrelatedScalarSubqueryReconstructor.getInstance() .reconstructPredicateWithUncorrelatedScalarSubquery(expression, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 03110852cdb5..64e037e5b91d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.ir; import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; @@ -57,11 +58,7 @@ public class PredicateWithUncorrelatedScalarSubqueryReconstructor { private static final Coordinator coordinator = Coordinator.getInstance(); - private PredicateWithUncorrelatedScalarSubqueryReconstructor() { - // utility class - } - - public static void reconstructPredicateWithUncorrelatedScalarSubquery( + public void reconstructPredicateWithUncorrelatedScalarSubquery( Expression expression, MPPQueryContext context) { if (expression instanceof LogicalExpression) { LogicalExpression logicalExpression = (LogicalExpression) expression; @@ -99,7 +96,7 @@ public static void reconstructPredicateWithUncorrelatedScalarSubquery( * Optional.empty() if the subquery cannot be executed in advance or if it does not return a * valid result. */ - private static Optional fetchUncorrelatedSubqueryResultForPredicate( + public Optional fetchUncorrelatedSubqueryResultForPredicate( SubqueryExpression subqueryExpression, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; @@ -182,4 +179,25 @@ private static Optional fetchUncorrelatedSubqueryResultForPredicate( } return Optional.empty(); } + + private static class PredicateWithUncorrelatedScalarSubqueryReconstructorHolder { + private static PredicateWithUncorrelatedScalarSubqueryReconstructor INSTANCE = + new PredicateWithUncorrelatedScalarSubqueryReconstructor(); + + private PredicateWithUncorrelatedScalarSubqueryReconstructorHolder() { + // Empty constructor + } + } + + public static PredicateWithUncorrelatedScalarSubqueryReconstructor getInstance() { + return PredicateWithUncorrelatedScalarSubqueryReconstructor + .PredicateWithUncorrelatedScalarSubqueryReconstructorHolder.INSTANCE; + } + + @TestOnly + public static void setInstance(PredicateWithUncorrelatedScalarSubqueryReconstructor instance) { + PredicateWithUncorrelatedScalarSubqueryReconstructor + .PredicateWithUncorrelatedScalarSubqueryReconstructorHolder.INSTANCE = + instance; + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java index eebd4e158439..d683359e4790 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern; +import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.PredicateWithUncorrelatedScalarSubqueryReconstructor; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; @@ -32,6 +33,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.junit.Test; +import org.mockito.Mockito; import java.util.Collections; import java.util.Optional; @@ -39,11 +41,8 @@ import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanAssert.assertPlan; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.aggregation; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.aggregationFunction; -import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.aggregationTableScan; -import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.any; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.anyTree; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.collect; -import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.enforceSingleRow; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.exchange; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.filter; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.join; @@ -55,7 +54,6 @@ import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.sort; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.tableScan; import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode.Step.FINAL; -import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode.Step.INTERMEDIATE; import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode.Step.PARTIAL; import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode.Step.SINGLE; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression.Operator.EQUAL; @@ -63,178 +61,76 @@ import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression.Operator.LESS_THAN_OR_EQUAL; public class UncorrelatedSubqueryTest { + private void mockPredicateWithUncorrelatedScalarSubquery() { + PredicateWithUncorrelatedScalarSubqueryReconstructor predicateWithUncorrelatedScalarSubquery = + Mockito.spy(new PredicateWithUncorrelatedScalarSubqueryReconstructor()); + Mockito.when( + predicateWithUncorrelatedScalarSubquery.fetchUncorrelatedSubqueryResultForPredicate( + Mockito.any(), Mockito.any())) + .thenReturn(Optional.of(new LongLiteral("1"))); + PredicateWithUncorrelatedScalarSubqueryReconstructor.setInstance( + predicateWithUncorrelatedScalarSubquery); + } @Test public void testUncorrelatedScalarSubqueryInWhereClause() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where s1 = (select max(s1) from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); - Expression filterPredicate = - new ComparisonExpression(EQUAL, new SymbolReference("s1"), new SymbolReference("max")); - PlanMatchPattern tableScan = - tableScan("testdb.table1", ImmutableList.of("s1"), ImmutableSet.of("s1")); + tableScan( + "testdb.table1", + ImmutableList.of("s1"), + ImmutableSet.of("s1"), + new ComparisonExpression(EQUAL, new SymbolReference("s1"), new LongLiteral("1"))); // Verify full LogicalPlan /* - * └──OutputNode - * └──ProjectNode - * └──FilterNode - * └──JoinNode - * |──TableScanNode - * ├──AggregationNode - * │ └──AggregationTableScanNode - - */ - assertPlan( - logicalQueryPlan, - output( - project( - filter( - filterPredicate, - join( - JoinNode.JoinType.INNER, - builder -> - builder - .left(tableScan) - .right( - aggregation( - singleGroupingSet(), - ImmutableMap.of( - Optional.of("max"), - aggregationFunction("max", ImmutableList.of("max_9"))), - Collections.emptyList(), - Optional.empty(), - FINAL, - aggregationTableScan( - singleGroupingSet(), - Collections.emptyList(), - Optional.empty(), - PARTIAL, - "testdb.table1", - ImmutableList.of("max_9"), - ImmutableSet.of("s1_6"))))))))); + * └──OutputNode + * └──DeviceTableScanNode + */ + assertPlan(logicalQueryPlan, output(tableScan)); // Verify DistributionPlan - assertPlan( - planTester.getFragmentPlan(0), - output( - project( - filter( - filterPredicate, - join( - JoinNode.JoinType.INNER, - builder -> builder.left(exchange()).right(exchange())))))); + assertPlan(planTester.getFragmentPlan(0), output(collect(exchange(), exchange(), exchange()))); - assertPlan(planTester.getFragmentPlan(1), collect(exchange(), exchange(), exchange())); + assertPlan(planTester.getFragmentPlan(1), tableScan); assertPlan(planTester.getFragmentPlan(2), tableScan); assertPlan(planTester.getFragmentPlan(3), tableScan); - assertPlan(planTester.getFragmentPlan(4), tableScan); - - assertPlan( - planTester.getFragmentPlan(5), - aggregation( - singleGroupingSet(), - ImmutableMap.of( - Optional.of("max"), aggregationFunction("max", ImmutableList.of("max_10"))), - Collections.emptyList(), - Optional.empty(), - FINAL, - collect(exchange(), exchange(), exchange()))); - - assertPlan( - planTester.getFragmentPlan(6), - aggregation( - singleGroupingSet(), - ImmutableMap.of( - Optional.of("max_10"), aggregationFunction("max", ImmutableList.of("max_9"))), - Collections.emptyList(), - Optional.empty(), - INTERMEDIATE, - aggregationTableScan( - singleGroupingSet(), - Collections.emptyList(), - Optional.empty(), - PARTIAL, - "testdb.table1", - ImmutableList.of("max_9"), - ImmutableSet.of("s1_6")))); - - assertPlan( - planTester.getFragmentPlan(7), - aggregation( - singleGroupingSet(), - ImmutableMap.of( - Optional.of("max_10"), aggregationFunction("max", ImmutableList.of("max_9"))), - Collections.emptyList(), - Optional.empty(), - INTERMEDIATE, - aggregationTableScan( - singleGroupingSet(), - Collections.emptyList(), - Optional.empty(), - PARTIAL, - "testdb.table1", - ImmutableList.of("max_9"), - ImmutableSet.of("s1_6")))); - - assertPlan( - planTester.getFragmentPlan(8), - aggregation( - singleGroupingSet(), - ImmutableMap.of( - Optional.of("max_10"), aggregationFunction("max", ImmutableList.of("max_9"))), - Collections.emptyList(), - Optional.empty(), - INTERMEDIATE, - aggregationTableScan( - singleGroupingSet(), - Collections.emptyList(), - Optional.empty(), - PARTIAL, - "testdb.table1", - ImmutableList.of("max_9"), - ImmutableSet.of("s1_6")))); } @Test public void testUncorrelatedScalarSubqueryInWhereClauseWithEnforceSingleRowNode() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where s1 = (select s2 from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); - PlanMatchPattern tableScan1 = - tableScan("testdb.table1", ImmutableList.of("s1"), ImmutableSet.of("s1")); + PlanMatchPattern tableScan = + tableScan( + "testdb.table1", + ImmutableList.of("s1"), + ImmutableSet.of("s1"), + new ComparisonExpression(EQUAL, new SymbolReference("s1"), new LongLiteral("1"))); // Verify LogicalPlan /* - * └──OutputNode - * └──ProjectNode - * └──FilterNode - * └──JoinNode - * |──TableScanNode - * ├──EnforceSingleRowNode - * │ └──TableScanNode - - */ - assertPlan( - logicalQueryPlan, - output( - project( - anyTree( - join( - JoinNode.JoinType.INNER, - builder -> builder.left(tableScan1).right(enforceSingleRow(any()))))))); + * └──OutputNode + * └──DeviceTableScanNode + */ + assertPlan(logicalQueryPlan, output(tableScan)); } @Test public void testUncorrelatedInPredicateSubquery() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where s1 in (select s1 from table1)"; @@ -288,6 +184,7 @@ public void testUncorrelatedInPredicateSubquery() { @Test public void testUncorrelatedNotInPredicateSubquery() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where s1 not in (select s1 from table1)"; @@ -324,6 +221,7 @@ public void testUncorrelatedNotInPredicateSubquery() { @Test public void testUncorrelatedAnyComparisonSubquery() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where s1 > any (select s1 from table1)"; @@ -453,6 +351,7 @@ public void testUncorrelatedAnyComparisonSubquery() { @Test public void testUncorrelatedEqualsSomeComparisonSubquery() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where s1 = some (select s1 from table1)"; @@ -489,6 +388,7 @@ public void testUncorrelatedEqualsSomeComparisonSubquery() { @Test public void testUncorrelatedAllComparisonSubquery() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where s1 != all (select s1 from table1)"; @@ -520,6 +420,7 @@ public void testUncorrelatedAllComparisonSubquery() { @Test public void testUncorrelatedExistsSubquery() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where exists(select s2 from table2)"; @@ -569,6 +470,7 @@ public void testUncorrelatedExistsSubquery() { @Test public void testUncorrelatedNotExistsSubquery() { PlanTester planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); String sql = "SELECT s1 FROM table1 where not exists(select s2 from table2)"; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/DeviceTableScanMatcher.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/DeviceTableScanMatcher.java index 061286ae4a62..9998146446be 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/DeviceTableScanMatcher.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/DeviceTableScanMatcher.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import java.util.List; import java.util.Optional; @@ -38,6 +39,15 @@ public DeviceTableScanMatcher( super(expectedTableName, hasTableLayout, outputSymbols, assignmentsKeys); } + public DeviceTableScanMatcher( + String expectedTableName, + Optional hasTableLayout, + List outputSymbols, + Set assignmentsKeys, + Expression pushDownPredicate) { + super(expectedTableName, hasTableLayout, outputSymbols, assignmentsKeys, pushDownPredicate); + } + @Override public boolean shapeMatches(PlanNode node) { return node instanceof DeviceTableScanNode; @@ -51,6 +61,7 @@ public String toString() { .add("hasTableLayout", hasTableLayout.orElse(null)) .add("outputSymbols", outputSymbols) .add("assignmentsKeys", assignmentsKeys) + .add("pushDownPredicate", pushDownPredicate) .toString(); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java index 3ec60536aec4..1a562296973e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java @@ -221,6 +221,25 @@ public static PlanMatchPattern tableScan( return pattern; } + public static PlanMatchPattern tableScan( + String expectedTableName, + List outputSymbols, + Set assignmentsKeys, + Expression pushDownPredicate) { + PlanMatchPattern pattern = + node(DeviceTableScanNode.class) + .with( + new DeviceTableScanMatcher( + expectedTableName, + Optional.empty(), + outputSymbols, + assignmentsKeys, + pushDownPredicate)); + outputSymbols.forEach( + symbol -> pattern.withAlias(symbol, new ColumnReference(expectedTableName, symbol))); + return pattern; + } + public static PlanMatchPattern tableScan( String expectedTableName, Map columnReferences) { PlanMatchPattern result = tableScan(expectedTableName); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/TableScanMatcher.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/TableScanMatcher.java index b860a18c16a3..9d1f3091b27f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/TableScanMatcher.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/TableScanMatcher.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import java.util.List; import java.util.Optional; @@ -41,6 +42,7 @@ public abstract class TableScanMatcher implements Matcher { protected final List outputSymbols; // this field empty means no need to match protected Set assignmentsKeys; + protected Expression pushDownPredicate; public TableScanMatcher( String expectedTableName, @@ -53,6 +55,16 @@ public TableScanMatcher( this.assignmentsKeys = requireNonNull(assignmentsKeys, "assignmentsKeys is null"); } + public TableScanMatcher( + String expectedTableName, + Optional hasTableLayout, + List outputSymbols, + Set assignmentsKeys, + Expression pushDownPredicate) { + this(expectedTableName, hasTableLayout, outputSymbols, assignmentsKeys); + this.pushDownPredicate = requireNonNull(pushDownPredicate, "pushDownPredicate is null"); + } + @Override public MatchResult detailMatches( PlanNode node, SessionInfo sessionInfo, Metadata metadata, SymbolAliases symbolAliases) { @@ -84,6 +96,11 @@ public MatchResult detailMatches( return NO_MATCH; } + if (pushDownPredicate != null + && !pushDownPredicate.equals(tableScanNode.getPushDownPredicate())) { + return NO_MATCH; + } + return new MatchResult(true); } } From 072d30c48e84781a16bc6828b5b6e41e53abfe6f Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 30 Sep 2025 08:36:22 +0800 Subject: [PATCH 27/69] cte explain unit test --- .../relational/planner/CteMaterializer.java | 8 +- ...correlatedScalarSubqueryReconstructor.java | 7 +- .../relational/planner/node/CteScanNode.java | 4 + .../planner/CteMaterializerTest.java | 289 ++++++++++++++++++ .../planner/assertions/AliasMatcher.java | 11 + .../planner/assertions/ColumnReference.java | 13 + .../planner/assertions/CteScanMatcher.java | 86 ++++++ .../planner/assertions/PlanMatchPattern.java | 9 + .../planner/assertions/RvalueMatcher.java | 8 + 9 files changed, 428 insertions(+), 7 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/CteScanMatcher.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 7e0a35eddb56..89d096a28008 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -114,7 +114,7 @@ public void cleanUpCTE(MPPQueryContext context) { cteDataStores.clear(); } - private CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContext context) { + public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; try { @@ -275,7 +275,7 @@ private void handleCteExplainResults(MPPQueryContext context, long queryId, Tabl } private static class CteMaterializerHolder { - private static final CteMaterializer INSTANCE = new CteMaterializer(); + private static CteMaterializer INSTANCE = new CteMaterializer(); private CteMaterializerHolder() { // Empty constructor @@ -285,4 +285,8 @@ private CteMaterializerHolder() { public static CteMaterializer getInstance() { return CteMaterializerHolder.INSTANCE; } + + public static void setInstance(CteMaterializer instance) { + CteMaterializerHolder.INSTANCE = instance; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 64e037e5b91d..7c828f0ce11c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -190,14 +190,11 @@ private PredicateWithUncorrelatedScalarSubqueryReconstructorHolder() { } public static PredicateWithUncorrelatedScalarSubqueryReconstructor getInstance() { - return PredicateWithUncorrelatedScalarSubqueryReconstructor - .PredicateWithUncorrelatedScalarSubqueryReconstructorHolder.INSTANCE; + return PredicateWithUncorrelatedScalarSubqueryReconstructorHolder.INSTANCE; } @TestOnly public static void setInstance(PredicateWithUncorrelatedScalarSubqueryReconstructor instance) { - PredicateWithUncorrelatedScalarSubqueryReconstructor - .PredicateWithUncorrelatedScalarSubqueryReconstructorHolder.INSTANCE = - instance; + PredicateWithUncorrelatedScalarSubqueryReconstructorHolder.INSTANCE = instance; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java index e92c8e56dd63..d8634fcab110 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java @@ -57,6 +57,10 @@ public CteScanNode( this.dataStore = dataStore; } + public QualifiedName getQualifiedName() { + return qualifiedName; + } + public CteDataStore getDataStore() { return dataStore; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java new file mode 100644 index 000000000000..0af63c68cfc2 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java @@ -0,0 +1,289 @@ +/* + * + * * 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; + +import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; +import org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.QuerySpecification; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SelectItem; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SingleColumn; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; +import org.apache.iotdb.db.utils.cte.CteDataStore; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.type.Type; +import org.apache.tsfile.read.common.type.TypeFactory; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanAssert.assertPlan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.aggregation; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.aggregationFunction; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.cteScan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.filter; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.limit; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.offset; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.output; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.singleGroupingSet; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.sort; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.tableScan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode.Step.SINGLE; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression.Operator.GREATER_THAN; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.NullOrdering.LAST; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.Ordering.ASCENDING; + +public class CteMaterializerTest { + private Type convertType(String columnName) { + switch (columnName) { + case "time": + return TypeFactory.getType(TSDataType.TIMESTAMP); + case "s1": + case "a1": + return TypeFactory.getType(TSDataType.INT64); + default: + } + return null; + } + + private void mockException() { + CteMaterializer cteMaterializer = Mockito.spy(new CteMaterializer()); + Mockito.when(cteMaterializer.fetchCteQueryResult(Mockito.any(), Mockito.any(), Mockito.any())) + .thenReturn(null); + CteMaterializer.setInstance(cteMaterializer); + } + + private void mockSubquery() { + CteMaterializer cteMaterializer = Mockito.spy(new CteMaterializer()); + + Mockito.when( + cteMaterializer.fetchCteQueryResult( + Mockito.any(Table.class), + Mockito.any(Query.class), + Mockito.any(MPPQueryContext.class))) + .thenAnswer( + (InvocationOnMock invocation) -> { + Table table = invocation.getArgument(0); + Query query = invocation.getArgument(1); + List selectItems = + ((QuerySpecification) query.getQueryBody()).getSelect().getSelectItems(); + List columnsSchemas = + selectItems.stream() + .map( + selectItem -> { + SingleColumn column = ((SingleColumn) selectItem); + String columnName = + column.getAlias().isPresent() + ? column.getAlias().get().toString() + : column.getExpression().toString(); + return new ColumnSchema( + columnName, + convertType(columnName), + false, + TsTableColumnCategory.FIELD); + }) + .collect(Collectors.toList()); + + TableSchema tableSchema = new TableSchema(table.getName().toString(), columnsSchemas); + return new CteDataStore(query, tableSchema, ImmutableList.of(0, 1)); + }); + CteMaterializer.setInstance(cteMaterializer); + } + + @Test + public void testSimpleCte() { + PlanTester planTester = new PlanTester(); + mockSubquery(); + + String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──CteScanNode + */ + assertPlan(logicalQueryPlan, output(cteScan)); + } + + @Test + public void testFieldFilterCte() { + PlanTester planTester = new PlanTester(); + mockSubquery(); + + String sql = + "with cte1 as materialized (SELECT time, s1 as a1 FROM table1) select * from cte1 where a1 > 10"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + Expression filterPredicate = + new ComparisonExpression(GREATER_THAN, new SymbolReference("a1"), new LongLiteral("10")); + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "a1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──FilterNode + * └──CteScanNode + */ + assertPlan(logicalQueryPlan, output(filter(filterPredicate, cteScan))); + } + + @Test + public void testTimeFilterCte() { + PlanTester planTester = new PlanTester(); + mockSubquery(); + + String sql = + "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1 where time > 1000"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + Expression filterPredicate = + new ComparisonExpression( + GREATER_THAN, new SymbolReference("time"), new LongLiteral("1000")); + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──FilterNode + * └──CteScanNode + */ + assertPlan(logicalQueryPlan, output(filter(filterPredicate, cteScan))); + } + + @Test + public void testSortCte() { + PlanTester planTester = new PlanTester(); + mockSubquery(); + + String sql = + "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1 order by s1"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + List orderBy = ImmutableList.of(sort("s1", ASCENDING, LAST)); + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──SortNode + * └──CteScanNode + */ + assertPlan(logicalQueryPlan, output(sort(orderBy, cteScan))); + } + + @Test + public void testLimitOffsetCte() { + PlanTester planTester = new PlanTester(); + mockSubquery(); + + String sql = + "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1 limit 1 offset 2"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──LimitNode + * └──OffsetNode + * └──CteScanNode + */ + assertPlan(logicalQueryPlan, output(offset(2, limit(3, cteScan)))); + } + + @Test + public void testAggCte() { + PlanTester planTester = new PlanTester(); + mockSubquery(); + + String sql = + "with cte1 as materialized (SELECT time, s1 FROM table1) select s1, max(time) from cte1 group by s1"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──AggregationNode + * └──CteScanNode + */ + assertPlan( + logicalQueryPlan, + output( + aggregation( + singleGroupingSet("s1"), + ImmutableMap.of( + Optional.of("max"), aggregationFunction("max", ImmutableList.of("time"))), + Collections.emptyList(), + Optional.empty(), + SINGLE, + cteScan))); + } + + @Test + public void testCteQueryException() { + PlanTester planTester = new PlanTester(); + mockException(); + + String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + PlanMatchPattern tableScan = + tableScan("testdb.table1", ImmutableList.of("time", "s1"), ImmutableSet.of("time", "s1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──TableScanNode + */ + assertPlan(logicalQueryPlan, output(tableScan)); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AliasMatcher.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AliasMatcher.java index 272b1a613412..2ceb0fb993a1 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AliasMatcher.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AliasMatcher.java @@ -22,7 +22,9 @@ import org.apache.iotdb.db.queryengine.common.SessionInfo; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import java.util.Optional; @@ -56,9 +58,18 @@ public boolean shapeMatches(PlanNode node) { public MatchResult detailMatches( PlanNode node, SessionInfo sessionInfo, Metadata metadata, SymbolAliases symbolAliases) { Optional symbol = matcher.getAssignedSymbol(node, sessionInfo, metadata, symbolAliases); + if (!symbol.isPresent() && node instanceof CteScanNode) { + TableSchema tableSchema = ((CteScanNode) node).getDataStore().getTableSchema(); + symbol = matcher.getCteSymbol(tableSchema); + } + if (symbol.isPresent() && alias.isPresent()) { return match(alias.get(), symbol.get().toSymbolReference()); } + if (node instanceof CteScanNode) { + System.out.println("CteScanNode"); + } + return new MatchResult(symbol.isPresent()); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/ColumnReference.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/ColumnReference.java index a93176c19a84..cff4126cde71 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/ColumnReference.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/ColumnReference.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode; @@ -90,6 +91,18 @@ private Optional getAssignedSymbol( return result; } + @Override + public Optional getCteSymbol(TableSchema tableSchema) { + Optional result = Optional.empty(); + for (ColumnSchema columnSchema : tableSchema.getColumns()) { + if (columnSchema.getName().equals(columnName)) { + result = Optional.of(Symbol.of(columnName)); + break; + } + } + return result; + } + @Override public String toString() { return format("Column %s:%s", tableName, columnName); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/CteScanMatcher.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/CteScanMatcher.java new file mode 100644 index 000000000000..8db77ad51ae8 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/CteScanMatcher.java @@ -0,0 +1,86 @@ +/* + * + * * 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.assertions; + +import org.apache.iotdb.db.queryengine.common.SessionInfo; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; + +import java.util.List; +import java.util.stream.Collectors; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkState; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.MatchResult.NO_MATCH; + +public class CteScanMatcher implements Matcher { + protected final String expectedCteName; + // this field empty means no need to match + protected final List outputSymbols; + + public CteScanMatcher(String expectedCteName, List outputSymbols) { + this.expectedCteName = expectedCteName; + this.outputSymbols = outputSymbols; + } + + @Override + public boolean shapeMatches(PlanNode node) { + return node instanceof CteScanNode; + } + + @Override + public MatchResult detailMatches( + PlanNode node, SessionInfo sessionInfo, Metadata metadata, SymbolAliases symbolAliases) { + checkState( + shapeMatches(node), + "Plan testing framework error: shapeMatches returned false in detailMatches in %s", + this.getClass().getName()); + + CteScanNode cteScanNode = (CteScanNode) node; + String actualCteName = cteScanNode.getQualifiedName().toString(); + + if (!expectedCteName.equalsIgnoreCase(actualCteName)) { + return NO_MATCH; + } + + if (!outputSymbols.isEmpty() + && !outputSymbols.equals( + cteScanNode.getOutputSymbols().stream() + .map(Symbol::getName) + .collect(Collectors.toList()))) { + return NO_MATCH; + } + + return new MatchResult(true); + } + + @Override + public String toString() { + return toStringHelper(this) + .omitNullValues() + .add("expectedCteName", expectedCteName) + .add("outputSymbols", outputSymbols) + .toString(); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java index 1a562296973e..2b2a6e557e1e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java @@ -31,6 +31,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AssignUniqueId; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CollectNode; +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.EnforceSingleRowNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExchangeNode; @@ -246,6 +247,14 @@ public static PlanMatchPattern tableScan( return result.addColumnReferences(expectedTableName, columnReferences); } + public static PlanMatchPattern cteScan(String expectedCteName, List outputSymbols) { + PlanMatchPattern pattern = + node(CteScanNode.class).with(new CteScanMatcher(expectedCteName, outputSymbols)); + outputSymbols.forEach( + symbol -> pattern.withAlias(symbol, new ColumnReference(expectedCteName, symbol))); + return pattern; + } + public static PlanMatchPattern tableFunctionProcessor( Consumer handler, PlanMatchPattern... source) { TableFunctionProcessorMatcher.Builder builder = new TableFunctionProcessorMatcher.Builder(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/RvalueMatcher.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/RvalueMatcher.java index 3af1f25cbbdb..5dc691db37f2 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/RvalueMatcher.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/RvalueMatcher.java @@ -22,6 +22,7 @@ import org.apache.iotdb.db.queryengine.common.SessionInfo; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; import java.util.Optional; @@ -39,4 +40,11 @@ public interface RvalueMatcher { */ Optional getAssignedSymbol( PlanNode node, SessionInfo sessionInfo, Metadata metadata, SymbolAliases symbolAliases); + + /** + * Get the column symbol of the CTE. It should be found in CTE's table schema instead of metadata + */ + default Optional getCteSymbol(TableSchema tableSchema) { + return Optional.empty(); + } } From fa0538f898a17cbfcc2381e999f29b4899572487 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 30 Sep 2025 10:53:18 +0800 Subject: [PATCH 28/69] adjust test cases --- .../planner/CteMaterializerTest.java | 28 +++++---------- .../planner/UncorrelatedSubqueryTest.java | 36 +++++-------------- 2 files changed, 18 insertions(+), 46 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java index 0af63c68cfc2..f6ae7e910adb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java @@ -44,6 +44,7 @@ import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.type.Type; import org.apache.tsfile.read.common.type.TypeFactory; +import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -70,6 +71,14 @@ import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.Ordering.ASCENDING; public class CteMaterializerTest { + private PlanTester planTester; + + @Before + public void setUp() { + planTester = new PlanTester(); + mockSubquery(); + } + private Type convertType(String columnName) { switch (columnName) { case "time": @@ -128,9 +137,6 @@ private void mockSubquery() { @Test public void testSimpleCte() { - PlanTester planTester = new PlanTester(); - mockSubquery(); - String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -147,9 +153,6 @@ public void testSimpleCte() { @Test public void testFieldFilterCte() { - PlanTester planTester = new PlanTester(); - mockSubquery(); - String sql = "with cte1 as materialized (SELECT time, s1 as a1 FROM table1) select * from cte1 where a1 > 10"; @@ -170,9 +173,6 @@ public void testFieldFilterCte() { @Test public void testTimeFilterCte() { - PlanTester planTester = new PlanTester(); - mockSubquery(); - String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1 where time > 1000"; @@ -194,9 +194,6 @@ public void testTimeFilterCte() { @Test public void testSortCte() { - PlanTester planTester = new PlanTester(); - mockSubquery(); - String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1 order by s1"; @@ -216,9 +213,6 @@ public void testSortCte() { @Test public void testLimitOffsetCte() { - PlanTester planTester = new PlanTester(); - mockSubquery(); - String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1 limit 1 offset 2"; @@ -238,9 +232,6 @@ public void testLimitOffsetCte() { @Test public void testAggCte() { - PlanTester planTester = new PlanTester(); - mockSubquery(); - String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select s1, max(time) from cte1 group by s1"; @@ -269,7 +260,6 @@ public void testAggCte() { @Test public void testCteQueryException() { - PlanTester planTester = new PlanTester(); mockException(); String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java index d683359e4790..ea8e098414a0 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java @@ -32,6 +32,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; @@ -61,6 +62,14 @@ import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression.Operator.LESS_THAN_OR_EQUAL; public class UncorrelatedSubqueryTest { + private PlanTester planTester; + + @Before + public void setUp() throws Exception { + planTester = new PlanTester(); + mockPredicateWithUncorrelatedScalarSubquery(); + } + private void mockPredicateWithUncorrelatedScalarSubquery() { PredicateWithUncorrelatedScalarSubqueryReconstructor predicateWithUncorrelatedScalarSubquery = Mockito.spy(new PredicateWithUncorrelatedScalarSubqueryReconstructor()); @@ -74,9 +83,6 @@ private void mockPredicateWithUncorrelatedScalarSubquery() { @Test public void testUncorrelatedScalarSubqueryInWhereClause() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where s1 = (select max(s1) from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -105,9 +111,6 @@ public void testUncorrelatedScalarSubqueryInWhereClause() { @Test public void testUncorrelatedScalarSubqueryInWhereClauseWithEnforceSingleRowNode() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where s1 = (select s2 from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -129,9 +132,6 @@ public void testUncorrelatedScalarSubqueryInWhereClauseWithEnforceSingleRowNode( @Test public void testUncorrelatedInPredicateSubquery() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where s1 in (select s1 from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -183,9 +183,6 @@ public void testUncorrelatedInPredicateSubquery() { @Test public void testUncorrelatedNotInPredicateSubquery() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where s1 not in (select s1 from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -220,9 +217,6 @@ public void testUncorrelatedNotInPredicateSubquery() { @Test public void testUncorrelatedAnyComparisonSubquery() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where s1 > any (select s1 from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -350,9 +344,6 @@ public void testUncorrelatedAnyComparisonSubquery() { @Test public void testUncorrelatedEqualsSomeComparisonSubquery() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where s1 = some (select s1 from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -387,9 +378,6 @@ public void testUncorrelatedEqualsSomeComparisonSubquery() { @Test public void testUncorrelatedAllComparisonSubquery() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where s1 != all (select s1 from table1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -419,9 +407,6 @@ public void testUncorrelatedAllComparisonSubquery() { @Test public void testUncorrelatedExistsSubquery() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where exists(select s2 from table2)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); @@ -469,9 +454,6 @@ public void testUncorrelatedExistsSubquery() { @Test public void testUncorrelatedNotExistsSubquery() { - PlanTester planTester = new PlanTester(); - mockPredicateWithUncorrelatedScalarSubquery(); - String sql = "SELECT s1 FROM table1 where not exists(select s2 from table2)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); From 75d7c242c7f7cb45f83e9fce6216e6c408049cb5 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 30 Sep 2025 11:55:01 +0800 Subject: [PATCH 29/69] add filter/sort/limit/offset/agg test cases in IT --- .../it/query/recent/IoTDBCteIT.java | 76 ++++++++++++++----- 1 file changed, 59 insertions(+), 17 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java index ec8bcebad44d..015ee3b734b1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java @@ -106,48 +106,90 @@ public void tearDown() { } @Test - public void testQuery() { - final String mainQuery = "select * from cte order by deviceid"; - + public void testFilterQuery() { // case 1 + String mainQuery = "select * from cte where time > 1000 order by deviceid"; String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; String[] retArray = new String[] { - "1970-01-01T00:00:01.000Z,d1,100.0,", "1970-01-01T00:00:02.000Z,d1,200.0,", - "1970-01-01T00:00:01.000Z,d2,300.0," }; String[] cteTemplateQueries = new String[] {"cte as %s (select * from testtb)"}; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); // case 2 - expectedHeader = new String[] {"deviceid", "voltage"}; - retArray = new String[] {"d1,100.0,", "d1,200.0,", "d2,300.0,"}; - cteTemplateQueries = new String[] {"cte as %s (select deviceid, voltage from testtb)"}; + mainQuery = "select * from cte where voltage > 200 order by deviceid"; + expectedHeader = new String[] {"time", "deviceid", "voltage"}; + retArray = new String[] {"1970-01-01T00:00:01.000Z,d2,300.0,"}; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + } - // case 3 - expectedHeader = new String[] {"deviceid", "avg_voltage"}; - retArray = new String[] {"d1,150.0,", "d2,300.0,"}; - cteTemplateQueries = + @Test + public void testSortQuery() { + final String mainQuery = "select * from cte order by deviceid, voltage desc"; + + String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:02.000Z,d1,200.0,", + "1970-01-01T00:00:01.000Z,d1,100.0,", + "1970-01-01T00:00:01.000Z,d2,300.0," + }; + String[] cteTemplateQueries = new String[] {"cte as %s (select * from testtb)"}; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + } + + @Test + public void testLimitOffsetQuery() { + final String mainQuery = "select * from cte limit 1 offset 1"; + + String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:02.000Z,d1,200.0,", + }; + String[] cteTemplateQueries = + new String[] {"cte as %s (select * from testtb where deviceid = 'd1') "}; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + } + + @Test + public void testAggQuery() { + // case 1 + String mainQuery = "select * from cte order by deviceid"; + String[] expectedHeader = new String[] {"deviceid", "avg_voltage"}; + String[] retArray = new String[] {"d1,150.0,", "d2,300.0,"}; + String[] cteTemplateQueries = new String[] { "cte as %s (select deviceid, avg(voltage) as avg_voltage from testtb group by deviceid)" }; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + + // case 2 + mainQuery = + "select deviceid, avg(voltage) as avg_voltage from cte group by deviceid order by deviceid"; + cteTemplateQueries = new String[] {"cte as %s (select deviceid, voltage from testtb)"}; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); } @Test public void testPartialColumn() { - final String mainQuery = "select * from cte order by id"; - // case 1 - String[] expectedHeader = new String[] {"id", "v"}; + String mainQuery = "select * from cte order by deviceid"; + String[] expectedHeader = new String[] {"deviceid", "voltage"}; String[] retArray = new String[] {"d1,100.0,", "d1,200.0,", "d2,300.0,"}; - String[] cteTemplateQueries = - new String[] {"cte(id, v) as %s (select deviceid, voltage from testtb)"}; + String[] cteTemplateQueries = new String[] {"cte as %s (select deviceid, voltage from testtb)"}; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + mainQuery = "select * from cte order by id"; + expectedHeader = new String[] {"id", "v"}; + retArray = new String[] {"d1,100.0,", "d1,200.0,", "d2,300.0,"}; + // case 2 + cteTemplateQueries = new String[] {"cte(id, v) as %s (select deviceid, voltage from testtb)"}; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + + // case 3 cteTemplateQueries = new String[] {"cte(v) as %s (select deviceid, voltage from testtb)"}; String errMsg = "701: Column alias list has 1 entries but relation has 2 columns"; testCteFailureWithVariants(cteTemplateQueries, mainQuery, errMsg); From f97f7cba3990fba08731861f2d4b68170afd1678 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 30 Sep 2025 13:44:19 +0800 Subject: [PATCH 30/69] add concurrent cte test case --- .../it/query/recent/IoTDBCteIT.java | 97 +++++++++++++++++++ 1 file changed, 97 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java index 015ee3b734b1..defda41a62fa 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java @@ -47,6 +47,12 @@ import java.sql.Statement; import java.util.Arrays; import java.util.Locale; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static org.apache.iotdb.db.it.utils.TestUtils.tableAssertTestFail; @@ -387,6 +393,97 @@ public void testPrivileges() throws SQLException { } } + @Test + public void testConcurrentCteQueries() throws Exception { + final int threadCount = 10; + final int queriesPerThread = 20; + final AtomicInteger successCount = new AtomicInteger(0); + final AtomicInteger failureCount = new AtomicInteger(0); + final AtomicInteger totalCount = new AtomicInteger(0); + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch finishLatch = new CountDownLatch(threadCount); + + ExecutorService executorService = Executors.newFixedThreadPool(threadCount); + + // Create CTE query tasks + Future[] futures = new Future[threadCount]; + for (int i = 0; i < threadCount; i++) { + final int threadId = i; + futures[i] = + executorService.submit( + () -> { + try { + startLatch.await(); // Wait for all threads to be ready + + try (Connection connection = + EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("USE testdb"); + + // Execute multiple CTE queries in each thread + for (int j = 0; j < queriesPerThread; j++) { + try { + // Test different types of CTE queries + String[] queries = { + String.format( + "WITH cte as %s (SELECT * FROM testtb WHERE voltage > 150) SELECT * FROM cte ORDER BY deviceid", + cteKeywords[j % cteKeywords.length]), + String.format( + "WITH cte as %s (SELECT deviceid, avg(voltage) as avg_v FROM testtb GROUP BY deviceid) SELECT * FROM cte", + cteKeywords[j % cteKeywords.length]), + String.format( + "WITH cte as %s (SELECT * FROM testtb WHERE time > 1000) SELECT count(*) as cnt FROM cte", + cteKeywords[j % cteKeywords.length]) + }; + + String query = queries[j % queries.length]; + ResultSet resultSet = statement.executeQuery(query); + + // Verify results + int rowCount = 0; + while (resultSet.next()) { + rowCount++; + } + totalCount.getAndAdd(rowCount); + + successCount.incrementAndGet(); + + } catch (SQLException e) { + failureCount.incrementAndGet(); + System.err.println( + "Thread " + threadId + " query " + j + " failed: " + e.getMessage()); + } + } + } + } catch (Exception e) { + failureCount.incrementAndGet(); + System.err.println("Thread " + threadId + " failed: " + e.getMessage()); + } finally { + finishLatch.countDown(); + } + }); + } + + // Start all threads at once + startLatch.countDown(); + + // Wait for all threads to complete + finishLatch.await(60, TimeUnit.SECONDS); + + // Shutdown executor + executorService.shutdown(); + boolean terminated = executorService.awaitTermination(10, TimeUnit.SECONDS); + if (!terminated) { + executorService.shutdownNow(); + } + + // Verify results + int totalQueries = threadCount * queriesPerThread; + assertEquals("All queries should succeed", totalQueries, successCount.get()); + assertEquals("No queries should fail", 0, failureCount.get()); + assertEquals("Total query count should match", 340, totalCount.get()); + } + private static void prepareData() { try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = connection.createStatement()) { From 6e29d673ed67c971dd37744a48fef4f201cf86fd Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 30 Sep 2025 15:21:13 +0800 Subject: [PATCH 31/69] add explain analyze test cases # Conflicts: # integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java # integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java # integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java # integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java --- .../env/cluster/config/MppCommonConfig.java | 12 + .../cluster/config/MppSharedCommonConfig.java | 14 + .../env/remote/config/RemoteCommonConfig.java | 10 + .../apache/iotdb/itbase/env/CommonConfig.java | 4 + .../it/query/recent/IoTExplainAnalyzeIT.java | 264 ++++++++++++++++++ .../queryengine/common/MPPQueryContext.java | 3 +- 6 files changed, 305 insertions(+), 2 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java index e7f4b228d4f1..746f13f04c0d 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java @@ -657,6 +657,18 @@ public CommonConfig setRestrictObjectLimit(boolean restrictObjectLimit) { return this; } + @Override + public CommonConfig setCteBufferSize(long cteBufferSize) { + setProperty("cte_buffer_size_in_bytes", String.valueOf(cteBufferSize)); + return this; + } + + @Override + public CommonConfig setMaxRowsInCteBuffer(int maxRows) { + setProperty("max_rows_in_cte_buffer", String.valueOf(maxRows)); + return this; + } + // For part of the log directory public String getClusterConfigStr() { return fromConsensusFullNameToAbbr(properties.getProperty(CONFIG_NODE_CONSENSUS_PROTOCOL_CLASS)) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java index abb0f8bf8bb0..de15176c5f1c 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java @@ -691,4 +691,18 @@ public CommonConfig setRestrictObjectLimit(boolean restrictObjectLimit) { dnConfig.setRestrictObjectLimit(restrictObjectLimit); return this; } + + @Override + public CommonConfig setCteBufferSize(long cteBufferSize) { + dnConfig.setCteBufferSize(cteBufferSize); + cnConfig.setCteBufferSize(cteBufferSize); + return this; + } + + @Override + public CommonConfig setMaxRowsInCteBuffer(int maxRows) { + dnConfig.setMaxRowsInCteBuffer(maxRows); + cnConfig.setMaxRowsInCteBuffer(maxRows); + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java index 148046423cd0..807660330a30 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java @@ -482,4 +482,14 @@ public CommonConfig setAuditableOperationResult(String auditableOperationResult) public CommonConfig setRestrictObjectLimit(boolean restrictObjectLimit) { return this; } + + @Override + public CommonConfig setCteBufferSize(long cteBufferSize) { + return this; + } + + @Override + public CommonConfig setMaxRowsInCteBuffer(int maxRows) { + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java index 531f94eec4b1..4cda485a6cbf 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java @@ -213,4 +213,8 @@ default CommonConfig setDefaultDatabaseLevel(int defaultDatabaseLevel) { CommonConfig setAuditableOperationResult(String auditableOperationResult); CommonConfig setRestrictObjectLimit(boolean restrictObjectLimit); + + CommonConfig setCteBufferSize(long cteBufferSize); + + CommonConfig setMaxRowsInCteBuffer(int maxRows); } diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java new file mode 100644 index 000000000000..eacfa6c428ee --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java @@ -0,0 +1,264 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, + * * software distributed under the License is distributed on an + * * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * * KIND, either express or implied. See the License for the + * * specific language governing permissions and limitations + * * under the License. + * + */ + +package org.apache.iotdb.relational.it.query.recent; + +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.TableClusterIT; +import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; +import org.apache.iotdb.itbase.env.BaseEnv; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +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.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.Locale; +import java.util.function.ToLongFunction; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +@RunWith(IoTDBTestRunner.class) +@Category({TableLocalStandaloneIT.class, TableClusterIT.class}) +public class IoTExplainAnalyzeIT { + private static final String DATABASE_NAME = "testdb"; + + private static final String[] creationSqls = + new String[] { + "CREATE DATABASE IF NOT EXISTS testdb", + "USE testdb", + "CREATE TABLE IF NOT EXISTS testtb(deviceid STRING TAG, voltage FLOAT FIELD)", + "INSERT INTO testtb VALUES(1000, 'd1', 100.0)", + "INSERT INTO testtb VALUES(2000, 'd1', 200.0)", + "INSERT INTO testtb VALUES(1000, 'd2', 300.0)", + }; + + private static final String dropDbSqls = "DROP DATABASE IF EXISTS testdb"; + + @BeforeClass + public static void setUpClass() { + Locale.setDefault(Locale.ENGLISH); + + EnvFactory.getEnv() + .getConfig() + .getCommonConfig() + .setPartitionInterval(1000) + .setMemtableSizeThreshold(10000) + .setMaxRowsInCteBuffer(100); + EnvFactory.getEnv().initClusterEnvironment(); + } + + @AfterClass + public static void tearDownClass() { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Before + public void setUp() throws SQLException { + prepareData(); + } + + @After + public void tearDown() { + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute(dropDbSqls); + } catch (Exception e) { + fail(e.getMessage()); + } + } + + @Test + public void testEmptyCteQuery() throws SQLException { + ResultSet resultSet = null; + String sql = + "explain analyze with cte1 as materialized (select * from testtb1) select * from testtb, cte1 where testtb.deviceid = cte1.deviceid"; + try (Connection conn = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = conn.createStatement()) { + statement.execute("Use testdb"); + statement.execute( + "CREATE TABLE IF NOT EXISTS testtb1(deviceid STRING TAG, voltage FLOAT FIELD)"); + resultSet = statement.executeQuery(sql); + StringBuilder sb = new StringBuilder(); + while (resultSet.next()) { + sb.append(resultSet.getString(1)).append(System.lineSeparator()); + } + String result = sb.toString(); + Assert.assertFalse( + "Explain Analyze should not contain ExplainAnalyze node.", + result.contains("ExplainAnalyzeNode")); + + String[] lines = result.split(System.lineSeparator()); + Assert.assertTrue(lines.length > 3); + Assert.assertEquals("CTE Query : 'cte1'", lines[0]); + Assert.assertEquals("", lines[1]); + Assert.assertEquals("Main Query", lines[2]); + statement.execute("DROP TABLE testtb1"); + } finally { + if (resultSet != null) { + resultSet.close(); + } + } + } + + @Test + public void testCteQueryExceedsThreshold() throws SQLException { + ResultSet resultSet = null; + String sql = + "explain analyze with cte1 as materialized (select * from testtb2) select * from testtb where testtb.deviceid in (select deviceid from cte1)"; + try (Connection conn = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = conn.createStatement()) { + statement.execute("Use testdb"); + statement.execute( + "CREATE TABLE IF NOT EXISTS testtb2(deviceid STRING TAG, voltage FLOAT FIELD)"); + for (int i = 0; i < 100; i++) { + statement.addBatch( + String.format("insert into testtb2(deviceid, voltage) values('d%d', %d)", i, i)); + } + statement.executeBatch(); + resultSet = statement.executeQuery(sql); + StringBuilder sb = new StringBuilder(); + while (resultSet.next()) { + sb.append(resultSet.getString(1)).append(System.lineSeparator()); + } + + String result = sb.toString(); + Assert.assertFalse( + "Main Query should not contain CteScan node when the CTE query's result set exceeds threshold.", + result.contains("CteScanNode(CteScanOperator)")); + Assert.assertTrue( + "CTE Query should contain warning message when CTE query's result set exceeds threshold.", + result.contains("Failed to materialize CTE")); + Assert.assertFalse( + "Explain Analyze should not contain ExplainAnalyze node.", + result.contains("ExplainAnalyzeNode")); + + String[] plans = result.split("Main Query"); + for (String plan : plans) { + String[] lines = plan.split(System.lineSeparator()); + long[] instanceCount = + Arrays.stream(lines) + .filter(line -> line.contains("Fragment Instances Count:")) + .mapToLong(extractNumber("Fragment Instances Count:\\s(\\d+)")) + .toArray(); + assertEquals(instanceCount.length, 1); + + long totalInstances = + Arrays.stream(lines).filter(line -> line.contains("FRAGMENT-INSTANCE")).count(); + assertEquals(totalInstances, instanceCount[0]); + } + + statement.execute("DROP TABLE testtb2"); + } finally { + if (resultSet != null) { + resultSet.close(); + } + } + } + + @Test + public void testCteQuerySuccess() throws SQLException { + ResultSet resultSet = null; + String sql = + "explain analyze with cte1 as materialized (select * from testtb3) select * from testtb where testtb.deviceid in (select deviceid from cte1)"; + try (Connection conn = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = conn.createStatement()) { + statement.execute("Use testdb"); + statement.execute( + "CREATE TABLE IF NOT EXISTS testtb3(deviceid STRING TAG, voltage FLOAT FIELD)"); + for (int i = 0; i < 50; i++) { + statement.addBatch( + String.format("insert into testtb3(deviceid, voltage) values('d%d', %d)", i, i)); + } + statement.executeBatch(); + resultSet = statement.executeQuery(sql); + StringBuilder sb = new StringBuilder(); + while (resultSet.next()) { + sb.append(resultSet.getString(1)).append(System.lineSeparator()); + } + + String result = sb.toString(); + Assert.assertTrue( + "Main Query should contain CteScan node when the CTE query's result set exceeds threshold.", + result.contains("CteScanNode(CteScanOperator)")); + Assert.assertFalse( + "Explain Analyze should not contain ExplainAnalyze node.", + result.contains("ExplainAnalyzeNode")); + + String[] plans = result.split("Main Query"); + for (String plan : plans) { + String[] lines = plan.split(System.lineSeparator()); + long[] instanceCount = + Arrays.stream(lines) + .filter(line -> line.contains("Fragment Instances Count:")) + .mapToLong(extractNumber("Fragment Instances Count:\\s(\\d+)")) + .toArray(); + assertEquals(instanceCount.length, 1); + + long totalInstances = + Arrays.stream(lines).filter(line -> line.contains("FRAGMENT-INSTANCE")).count(); + assertEquals(totalInstances, instanceCount[0]); + } + + statement.execute("DROP TABLE testtb3"); + } finally { + if (resultSet != null) { + resultSet.close(); + } + } + } + + private static ToLongFunction extractNumber(String regex) { + return line -> { + Pattern pattern = Pattern.compile(regex); + Matcher matcher = pattern.matcher(line); + if (matcher.find()) { + return Long.parseLong(matcher.group(1)); + } + return 0; + }; + } + + private static void prepareData() { + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + + for (String sql : creationSqls) { + statement.execute(sql); + } + } catch (Exception e) { + fail(e.getMessage()); + } + } +} 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 db45075909ac..048e29c0a180 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 @@ -42,12 +42,11 @@ import org.apache.tsfile.utils.Pair; import java.time.ZoneId; -import java.util.HashMap; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.List; import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; From 272c14a41755df5e9a87265a9c0e48ec7073aa7a Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 30 Sep 2025 20:13:46 +0800 Subject: [PATCH 32/69] fix: unit test cases --- .../queryengine/plan/relational/analyzer/AnalyzerTest.java | 7 +++++++ .../plan/relational/analyzer/InsertIntoQueryTest.java | 7 +++++++ .../plan/relational/analyzer/LimitOffsetPushDownTest.java | 7 +++++++ .../db/queryengine/plan/relational/analyzer/SortTest.java | 7 +++++++ .../queryengine/plan/relational/analyzer/SubQueryTest.java | 7 +++++++ 5 files changed, 35 insertions(+) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java index b50c7cbd33b9..2319ea29e09f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java @@ -29,6 +29,7 @@ import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.partition.DataPartitionQueryParam; import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.commons.schema.table.InsertNodeMeasurementInfo; import org.apache.iotdb.commons.schema.table.TsTable; import org.apache.iotdb.db.protocol.session.IClientSession; @@ -81,6 +82,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.tsfile.file.metadata.IDeviceID.Factory; import org.apache.tsfile.utils.Binary; +import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; import org.mockito.Mockito; @@ -145,6 +147,11 @@ public class AnalyzerTest { DistributedQueryPlan distributedQueryPlan; DeviceTableScanNode deviceTableScanNode; + @BeforeClass + public static void setUp() { + IoTDBDescriptor.getInstance().getConfig().setDataNodeId(1); + } + @Test public void testMockQuery() throws OperatorNotFoundException { final String sql = diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/InsertIntoQueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/InsertIntoQueryTest.java index 8c1a766f4c3f..879356acaee9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/InsertIntoQueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/InsertIntoQueryTest.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.analyzer; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.planner.plan.DistributedQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; @@ -34,6 +35,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.StreamSortNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKNode; +import org.junit.BeforeClass; import org.junit.Test; import static org.apache.iotdb.db.queryengine.plan.relational.analyzer.AnalyzerTest.analyzeSQL; @@ -60,6 +62,11 @@ public class InsertIntoQueryTest { TableDistributedPlanner distributionPlanner; DistributedQueryPlan distributedQueryPlan; + @BeforeClass + public static void setUp() { + IoTDBDescriptor.getInstance().getConfig().setDataNodeId(1); + } + @Test public void simpleInsertIntoQuery() { sql = diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/LimitOffsetPushDownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/LimitOffsetPushDownTest.java index af79ea9a6b9d..fbe814ff25bc 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/LimitOffsetPushDownTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/LimitOffsetPushDownTest.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.analyzer; import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.protocol.session.IClientSession; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.common.SessionInfo; @@ -39,6 +40,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.StreamSortNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKNode; +import org.junit.BeforeClass; import org.junit.Test; import java.time.ZoneId; @@ -79,6 +81,11 @@ public class LimitOffsetPushDownTest { DistributedQueryPlan distributedQueryPlan; DeviceTableScanNode deviceTableScanNode; + @BeforeClass + public static void setUp() { + IoTDBDescriptor.getInstance().getConfig().setDataNodeId(1); + } + // without sort operation, limit can be pushed into TableScan, pushLimitToEachDevice==false @Test public void noOrderByTest() { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SortTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SortTest.java index 047063a1b650..2f6a028ea4a4 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SortTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SortTest.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.analyzer; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.execution.warnings.WarningCollector; import org.apache.iotdb.db.queryengine.plan.planner.plan.DistributedQueryPlan; @@ -41,6 +42,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKNode; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; +import org.junit.BeforeClass; import org.junit.Test; import java.util.List; @@ -78,6 +80,11 @@ public class SortTest { DistributedQueryPlan distributedQueryPlan; DeviceTableScanNode deviceTableScanNode; + @BeforeClass + public static void setUp() { + IoTDBDescriptor.getInstance().getConfig().setDataNodeId(1); + } + // order by some_ids, time, others; has filter @Test public void someIDColumnTimeOthersSortTest() { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SubQueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SubQueryTest.java index da7fc24d684d..156123604b5d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SubQueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/SubQueryTest.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.analyzer; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.planner.plan.DistributedQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; @@ -37,6 +38,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.StreamSortNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKNode; +import org.junit.BeforeClass; import org.junit.Test; import java.util.Arrays; @@ -72,6 +74,11 @@ public class SubQueryTest { DistributedQueryPlan distributedQueryPlan; DeviceTableScanNode deviceTableScanNode; + @BeforeClass + public static void setUp() { + IoTDBDescriptor.getInstance().getConfig().setDataNodeId(1); + } + @Test public void subQueryTest1() { // outer query has limit and sort, From c184c517d3ff11f3201eae678e2a552091f57b41 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 9 Oct 2025 09:48:16 +0800 Subject: [PATCH 33/69] annotation changes --- .../org/apache/iotdb/db/conf/IoTDBDescriptor.java | 2 +- .../iotdb/db/queryengine/common/MPPQueryContext.java | 12 ++++++++---- .../fragment/FragmentInstanceExecution.java | 4 ++-- .../db/queryengine/plan/analyze/AnalyzeVisitor.java | 1 + .../plan/relational/analyzer/Analyzer.java | 3 +-- .../plan/relational/analyzer/StatementAnalyzer.java | 3 ++- .../plan/relational/planner/CteMaterializer.java | 7 ++++--- .../relational/planner/CorrelatedSubqueryTest.java | 3 +++ .../relational/planner/assertions/AliasMatcher.java | 3 --- .../resources/conf/iotdb-system.properties.template | 3 ++- 10 files changed, 24 insertions(+), 17 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index ff48b82562f2..0d489fdf8ea6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -1057,7 +1057,7 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException // The buffer for sort operator to calculate loadFixedSizeLimitForQuery(properties, "sort_buffer_size_in_bytes", conf::setSortBufferSize); - // The buffer size for cte materialization + // The buffer for cte materialization. loadFixedSizeLimitForQuery(properties, "cte_buffer_size_in_bytes", conf::setCteBufferSize); // max number of rows for cte materialization 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 048e29c0a180..4f10dc358e4d 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 @@ -88,7 +88,9 @@ public class MPPQueryContext implements IAuditEntity { private final Set acquiredLocks = new HashSet<>(); - // explainType & isVerbose are used by cte query in table model + // Previously, the boolean value 'isExplainAnalyze' was needed to determine whether query + // statistics should be recorded during the query process. Now 'explainType' is used to + // determine whether query statistics or query plan should be recorded during the query process. private ExplainType explainType = ExplainType.NONE; private boolean isVerbose = false; @@ -111,11 +113,13 @@ public class MPPQueryContext implements IAuditEntity { private boolean userQuery = false; - // table -> (maxLineLength, explain/explain analyze lines) + private Map, CteDataStore> cteDataStores = new HashMap<>(); + // table -> (maxLineLength, 'explain' or 'explain analyze' result) + // Max line length of each CTE should be remembered because we need to standardize + // the output format of main query and CTE query. private final Map, Pair>> cteExplainResults = new HashMap<>(); - private Map, CteDataStore> cteDataStores = new HashMap<>(); - // If this is a subquery, we do not release CTE query result + // Do not release CTE query result if it is a subquery. private boolean subquery = false; public MPPQueryContext(QueryId queryId) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java index 0a08144e7317..d392b2730682 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java @@ -164,8 +164,8 @@ private boolean fillFragmentInstanceStatistics( statistics.setFragmentInstanceId(context.getId().toThrift()); statistics.setQueryStatistics(context.getQueryStatistics().toThrift()); statistics.setState(getInstanceState().toString()); - - // Check if this fragment instance should be ignored for statistics + // Previously we ignore statistics when current data region is instance of + // VirtualDataRegion. Now data region of a CteScanNode is also virtual. if (shouldIgnoreForStatistics()) { // We don't need to output the region having ExplainAnalyzeOperator only. return false; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index 10a2d9383edd..66fe98f302fb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -241,6 +241,7 @@ public Analysis visitNode(StatementNode node, MPPQueryContext context) { @Override public Analysis visitExplain(ExplainStatement explainStatement, MPPQueryContext context) { Analysis analysis = visitQuery(explainStatement.getQueryStatement(), context); + context.setExplainType(ExplainType.EXPLAIN); analysis.setRealStatement(explainStatement); analysis.setFinishQueryAfterAnalyze(true); analysis.setDatabaseName(context.getDatabaseName().orElse(null)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java index d8f34b2a3ae0..783a94e848d9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java @@ -80,8 +80,7 @@ public Analysis analyze(Statement statement) { warningCollector); Analysis analysis = new Analysis(rewrittenStatement, parameterLookup); - // Here we register CTE into analysis. Later in RelationPlanner, we create CteScanNode or - // DeviceTableDeviceScanNode based on analysis.getNamedQuery result. + // Register CTE passed by parent query. context .getCteDataStores() .forEach( 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 f8285ecf2f52..3689b66c2e55 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 @@ -3097,7 +3097,8 @@ protected Scope visitTable(Table table, Optional scope) { table, QualifiedName.of(name.getDatabaseName(), name.getObjectName())); Optional tableSchema = metadata.getTableSchema(sessionContext, name); - // if table schema is not found in metada, we check if it's a CTE defined in the parent query + // if table schema is not found in metadata, we check if it's a CTE defined in the parent + // query if (!tableSchema.isPresent()) { CteDataStore dataStore = queryContext.getCteDataStore(table); if (dataStore != null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 89d096a28008..558a37822f03 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -69,8 +69,6 @@ public class CteMaterializer { private static final Logger LOGGER = LoggerFactory.getLogger(CteMaterializer.class); - private static String CTE_MATERIALIZATION_FAILURE_WARNING = - "***** Failed to materialize CTE! INLINE mode is adopted in the main query *****"; private static final Coordinator coordinator = Coordinator.getInstance(); @@ -160,7 +158,10 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex "Fail to materialize CTE because the data size exceeded memory or the row count threshold"); if (context.isExplainAnalyze()) { handleCteExplainAnalyzeResults( - context, queryId, table, CTE_MATERIALIZATION_FAILURE_WARNING); + context, + queryId, + table, + "!!! Failed to materialize CTE. The main query falls back to INLINE mode !!!"); } return null; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CorrelatedSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CorrelatedSubqueryTest.java index ccc6db6620bf..e7baf8078f11 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CorrelatedSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CorrelatedSubqueryTest.java @@ -63,6 +63,9 @@ public void testCorrelatedExistsSubquery() { PlanMatchPattern tableScan2 = tableScan("testdb.table2", ImmutableMap.of("s2_7", "s2")); + Expression filterPredicate = + new CoalesceExpression(new BooleanLiteral("true"), new BooleanLiteral("false")); + // Verify full LogicalPlan /* * └──OutputNode diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AliasMatcher.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AliasMatcher.java index 2ceb0fb993a1..178a9e1b3e66 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AliasMatcher.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AliasMatcher.java @@ -66,9 +66,6 @@ public MatchResult detailMatches( if (symbol.isPresent() && alias.isPresent()) { return match(alias.get(), symbol.get().toSymbolReference()); } - if (node instanceof CteScanNode) { - System.out.println("CteScanNode"); - } return new MatchResult(symbol.isPresent()); } diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 3a64f80f1e8f..6a236e2f278b 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -1123,7 +1123,8 @@ batch_size=100000 # Datatype: long sort_buffer_size_in_bytes=0 -# The buffer size for CTE materialization +# The buffer size for CTE materialization. If cte_buffer_size_in_bytes <= 0, default value will be used, default value = +# min(32MB, memory for query operators / query_thread_count / 2). if cte_buffer_size_in_bytes > 0, the specified value will be used. # effectiveMode: hot_reload # Datatype: long cte_buffer_size_in_bytes=0 From 9eab079d12be22cf600a7a2193ee3220e0bf2a11 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 9 Oct 2025 10:11:30 +0800 Subject: [PATCH 34/69] fix: handle MemoryNotEnoughException --- .../plan/relational/planner/CteMaterializer.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 558a37822f03..a2e0043808a8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -93,7 +93,6 @@ public void materializeCTE(Analysis analysis, MPPQueryContext context) { return; } - context.reserveMemoryForFrontEnd(dataStore.getCachedBytes()); context.addCteDataStore(table, dataStore); query.setCteDataStore(dataStore); } @@ -115,6 +114,7 @@ public void cleanUpCTE(MPPQueryContext context) { public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContext context) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; + CteDataStore cteDataStore = null; try { final ExecutionResult executionResult = coordinator.executeForTableModel( @@ -139,7 +139,7 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); - CteDataStore cteDataStore = + cteDataStore = new CteDataStore( query, tableSchema, datasetHeader.getColumnIndex2TsBlockColumnIndexList()); while (execution.hasNextResult()) { @@ -153,6 +153,7 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex if (!tsBlock.isPresent() || tsBlock.get().isEmpty()) { continue; } + context.reserveMemoryForFrontEnd(tsBlock.get().getRetainedSizeInBytes()); if (!cteDataStore.addTsBlock(tsBlock.get())) { LOGGER.warn( "Fail to materialize CTE because the data size exceeded memory or the row count threshold"); @@ -163,6 +164,7 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex table, "!!! Failed to materialize CTE. The main query falls back to INLINE mode !!!"); } + context.releaseMemoryReservedForFrontEnd(cteDataStore.getCachedBytes()); return null; } } @@ -175,6 +177,9 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex return cteDataStore; } catch (final Throwable throwable) { + if (cteDataStore != null) { + context.releaseMemoryReservedForFrontEnd(cteDataStore.getCachedBytes()); + } t = throwable; } finally { coordinator.cleanupQueryExecution(queryId, null, t); From 2f1e7a5d65bb0b61901f874a829b12d290f87e13 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 9 Oct 2025 14:58:52 +0800 Subject: [PATCH 35/69] fix: data partition issue in cluster env --- .../relational/planner/distribute/AddExchangeNodes.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/AddExchangeNodes.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/AddExchangeNodes.java index 9cef48070285..a1c26e1b7017 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/AddExchangeNodes.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/AddExchangeNodes.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.TableDeviceSourceNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CollectNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; 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.TableFunctionProcessorNode; @@ -100,6 +101,14 @@ public PlanNode visitTableScan( return node; } + @Override + public PlanNode visitCteScan( + CteScanNode node, TableDistributedPlanGenerator.PlanContext context) { + context.nodeDistributionMap.put( + node.getPlanNodeId(), new NodeDistribution(NO_CHILD, DataPartition.NOT_ASSIGNED)); + return node; + } + @Override public PlanNode visitTableDeviceFetch( final TableDeviceFetchNode node, final TableDistributedPlanGenerator.PlanContext context) { From 8dc5789f0ed15cf0eb8dea81bbc71f8fc40cebfb Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 9 Oct 2025 19:45:53 +0800 Subject: [PATCH 36/69] fix: explain analyze IT test should not run in TableCluster mode --- .../it/query/recent/IoTExplainAnalyzeIT.java | 54 +++++++++---------- 1 file changed, 25 insertions(+), 29 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java index eacfa6c428ee..6108704a6e6e 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java @@ -23,7 +23,6 @@ import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.framework.IoTDBTestRunner; -import org.apache.iotdb.itbase.category.TableClusterIT; import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; import org.apache.iotdb.itbase.env.BaseEnv; @@ -50,21 +49,21 @@ import static org.junit.Assert.fail; @RunWith(IoTDBTestRunner.class) -@Category({TableLocalStandaloneIT.class, TableClusterIT.class}) +@Category({TableLocalStandaloneIT.class}) public class IoTExplainAnalyzeIT { private static final String DATABASE_NAME = "testdb"; private static final String[] creationSqls = new String[] { - "CREATE DATABASE IF NOT EXISTS testdb", - "USE testdb", + "CREATE DATABASE IF NOT EXISTS " + DATABASE_NAME, + "USE " + DATABASE_NAME, "CREATE TABLE IF NOT EXISTS testtb(deviceid STRING TAG, voltage FLOAT FIELD)", "INSERT INTO testtb VALUES(1000, 'd1', 100.0)", "INSERT INTO testtb VALUES(2000, 'd1', 200.0)", "INSERT INTO testtb VALUES(1000, 'd2', 300.0)", }; - private static final String dropDbSqls = "DROP DATABASE IF EXISTS testdb"; + private static final String dropDbSqls = "DROP DATABASE IF EXISTS " + DATABASE_NAME; @BeforeClass public static void setUpClass() { @@ -100,20 +99,22 @@ public void tearDown() { } @Test - public void testEmptyCteQuery() throws SQLException { - ResultSet resultSet = null; + public void testEmptyCteQuery() { String sql = "explain analyze with cte1 as materialized (select * from testtb1) select * from testtb, cte1 where testtb.deviceid = cte1.deviceid"; try (Connection conn = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = conn.createStatement()) { - statement.execute("Use testdb"); + statement.execute("Use " + DATABASE_NAME); statement.execute( "CREATE TABLE IF NOT EXISTS testtb1(deviceid STRING TAG, voltage FLOAT FIELD)"); - resultSet = statement.executeQuery(sql); + ResultSet resultSet = statement.executeQuery(sql); StringBuilder sb = new StringBuilder(); while (resultSet.next()) { + System.out.println(resultSet.getString(1)); sb.append(resultSet.getString(1)).append(System.lineSeparator()); } + resultSet.close(); + String result = sb.toString(); Assert.assertFalse( "Explain Analyze should not contain ExplainAnalyze node.", @@ -125,21 +126,19 @@ public void testEmptyCteQuery() throws SQLException { Assert.assertEquals("", lines[1]); Assert.assertEquals("Main Query", lines[2]); statement.execute("DROP TABLE testtb1"); - } finally { - if (resultSet != null) { - resultSet.close(); - } + + } catch (SQLException e) { + fail(e.getMessage()); } } @Test - public void testCteQueryExceedsThreshold() throws SQLException { - ResultSet resultSet = null; + public void testCteQueryExceedsThreshold() { String sql = "explain analyze with cte1 as materialized (select * from testtb2) select * from testtb where testtb.deviceid in (select deviceid from cte1)"; try (Connection conn = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = conn.createStatement()) { - statement.execute("Use testdb"); + statement.execute("Use " + DATABASE_NAME); statement.execute( "CREATE TABLE IF NOT EXISTS testtb2(deviceid STRING TAG, voltage FLOAT FIELD)"); for (int i = 0; i < 100; i++) { @@ -147,11 +146,12 @@ public void testCteQueryExceedsThreshold() throws SQLException { String.format("insert into testtb2(deviceid, voltage) values('d%d', %d)", i, i)); } statement.executeBatch(); - resultSet = statement.executeQuery(sql); + ResultSet resultSet = statement.executeQuery(sql); StringBuilder sb = new StringBuilder(); while (resultSet.next()) { sb.append(resultSet.getString(1)).append(System.lineSeparator()); } + resultSet.close(); String result = sb.toString(); Assert.assertFalse( @@ -180,21 +180,18 @@ public void testCteQueryExceedsThreshold() throws SQLException { } statement.execute("DROP TABLE testtb2"); - } finally { - if (resultSet != null) { - resultSet.close(); - } + } catch (SQLException e) { + fail(e.getMessage()); } } @Test - public void testCteQuerySuccess() throws SQLException { - ResultSet resultSet = null; + public void testCteQuerySuccess() { String sql = "explain analyze with cte1 as materialized (select * from testtb3) select * from testtb where testtb.deviceid in (select deviceid from cte1)"; try (Connection conn = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = conn.createStatement()) { - statement.execute("Use testdb"); + statement.execute("Use " + DATABASE_NAME); statement.execute( "CREATE TABLE IF NOT EXISTS testtb3(deviceid STRING TAG, voltage FLOAT FIELD)"); for (int i = 0; i < 50; i++) { @@ -202,11 +199,12 @@ public void testCteQuerySuccess() throws SQLException { String.format("insert into testtb3(deviceid, voltage) values('d%d', %d)", i, i)); } statement.executeBatch(); - resultSet = statement.executeQuery(sql); + ResultSet resultSet = statement.executeQuery(sql); StringBuilder sb = new StringBuilder(); while (resultSet.next()) { sb.append(resultSet.getString(1)).append(System.lineSeparator()); } + resultSet.close(); String result = sb.toString(); Assert.assertTrue( @@ -232,10 +230,8 @@ public void testCteQuerySuccess() throws SQLException { } statement.execute("DROP TABLE testtb3"); - } finally { - if (resultSet != null) { - resultSet.close(); - } + } catch (SQLException e) { + fail(e.getMessage()); } } From 294b51115017035c02e89fff5966ac0a925e2930 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 21 Oct 2025 14:19:49 +0800 Subject: [PATCH 37/69] fix: release memory occupied by CteDataStore --- .../queryengine/plan/relational/planner/CteMaterializer.java | 2 ++ .../main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java | 4 +--- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index a2e0043808a8..9959933d0d37 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -165,6 +165,7 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex "!!! Failed to materialize CTE. The main query falls back to INLINE mode !!!"); } context.releaseMemoryReservedForFrontEnd(cteDataStore.getCachedBytes()); + cteDataStore.clear(); return null; } } @@ -179,6 +180,7 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex } catch (final Throwable throwable) { if (cteDataStore != null) { context.releaseMemoryReservedForFrontEnd(cteDataStore.getCachedBytes()); + cteDataStore.clear(); } t = throwable; } finally { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index bdedc496b58f..8b77f3dafb73 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -56,9 +56,6 @@ public boolean addTsBlock(TsBlock tsBlock) { int rows = tsBlock.getPositionCount(); if (bytesSize + cachedBytes >= iotConfig.getCteBufferSize() || rows + cachedRows >= iotConfig.getMaxRowsInCteBuffer()) { - cachedData.clear(); - cachedBytes = 0; - cachedRows = 0; return false; } cachedData.add(tsBlock); @@ -70,6 +67,7 @@ public boolean addTsBlock(TsBlock tsBlock) { public void clear() { cachedData.clear(); cachedBytes = 0L; + cachedRows = 0; } public List getCachedData() { From 995aedf4d3e0ae194e8974523d4ea67e70eaebfe Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 22 Oct 2025 09:41:48 +0800 Subject: [PATCH 38/69] code optimization --- .../it/query/recent/IoTExplainAnalyzeIT.java | 4 ++-- .../plan/relational/planner/CteMaterializer.java | 12 +++--------- .../relational/planner/TableLogicalPlanner.java | 4 +++- .../plan/relational/sql/ast/Query.java | 16 +++++++--------- 4 files changed, 15 insertions(+), 21 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java index 6108704a6e6e..8ee9a819239c 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java @@ -188,7 +188,7 @@ public void testCteQueryExceedsThreshold() { @Test public void testCteQuerySuccess() { String sql = - "explain analyze with cte1 as materialized (select * from testtb3) select * from testtb where testtb.deviceid in (select deviceid from cte1)"; + "explain analyze with cte1 as materialized (select voltage, deviceid from testtb3) select * from testtb where testtb.deviceid in (select deviceid from cte1)"; try (Connection conn = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); Statement statement = conn.createStatement()) { statement.execute("Use " + DATABASE_NAME); @@ -208,7 +208,7 @@ public void testCteQuerySuccess() { String result = sb.toString(); Assert.assertTrue( - "Main Query should contain CteScan node when the CTE query's result set exceeds threshold.", + "Main Query should contain CteScan node when the CTE query's result set does not exceeds threshold.", result.contains("CteScanNode(CteScanOperator)")); Assert.assertFalse( "Explain Analyze should not contain ExplainAnalyze node.", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 9959933d0d37..0fd75dc6db5d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -78,14 +78,8 @@ public void materializeCTE(Analysis analysis, MPPQueryContext context) { .forEach( (tableRef, query) -> { Table table = tableRef.getNode(); - if (query.isMaterialized()) { - CteDataStore dataStore = query.getCteDataStore(); - if (dataStore != null) { - context.addCteDataStore(table, dataStore); - return; - } - - dataStore = fetchCteQueryResult(table, query, context); + if (query.isMaterialized() && !query.isDone()) { + CteDataStore dataStore = fetchCteQueryResult(table, query, context); if (dataStore == null) { // CTE query execution failed. Use inline instead of materialization // in the outer query @@ -94,7 +88,7 @@ public void materializeCTE(Analysis analysis, MPPQueryContext context) { } context.addCteDataStore(table, dataStore); - query.setCteDataStore(dataStore); + query.setDone(true); } }); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index cc8111294dfc..8210d8443d68 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -373,7 +373,9 @@ private RelationPlan createRelationPlan(Analysis analysis, PipeEnriched pipeEnri private RelationPlan createRelationPlan(Analysis analysis, Query query) { // materialize cte if needed - CteMaterializer.getInstance().materializeCTE(analysis, queryContext); + if (!queryContext.isSubquery()) { + CteMaterializer.getInstance().materializeCTE(analysis, queryContext); + } return getRelationPlanner(analysis).process(query, null); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java index ed11f9f5865e..22d3994262b2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java @@ -19,8 +19,6 @@ package org.apache.iotdb.db.queryengine.plan.relational.sql.ast; -import org.apache.iotdb.db.utils.cte.CteDataStore; - import com.google.common.collect.ImmutableList; import org.apache.tsfile.utils.RamUsageEstimator; @@ -42,10 +40,10 @@ public class Query extends Statement { private final Optional orderBy; private final Optional offset; private final Optional limit; - // whether this query needs serialization + // whether this query needs materialization private boolean materialized = false; - // query result for common table expression - private CteDataStore cteDataStore = null; + // whether materialization is done + private boolean done = false; public Query( Optional with, @@ -116,12 +114,12 @@ public void setMaterialized(boolean materialized) { this.materialized = materialized; } - public CteDataStore getCteDataStore() { - return cteDataStore; + public boolean isDone() { + return done; } - public void setCteDataStore(CteDataStore cteDataStore) { - this.cteDataStore = cteDataStore; + public void setDone(boolean done) { + this.done = done; } @Override From 600d4b7a865c26b26d74a21358bdc679de3c4afc Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 22 Oct 2025 15:45:45 +0800 Subject: [PATCH 39/69] list materialization total cost in logical plan cost --- .../queryengine/common/MPPQueryContext.java | 9 +++++++++ .../operator/ExplainAnalyzeOperator.java | 19 +++++++++++++++++++ .../relational/planner/CteMaterializer.java | 3 +++ 3 files changed, 31 insertions(+) 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 4f10dc358e4d..255f6daac672 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 @@ -113,6 +113,7 @@ public class MPPQueryContext implements IAuditEntity { private boolean userQuery = false; + private final Map, Long> cteMaterializationCosts = new HashMap<>(); private Map, CteDataStore> cteDataStores = new HashMap<>(); // table -> (maxLineLength, 'explain' or 'explain analyze' result) // Max line length of each CTE should be remembered because we need to standardize @@ -478,6 +479,14 @@ public void setSubquery(boolean subquery) { this.subquery = subquery; } + public void addCteMaterializationCost(Table table, long cost) { + cteMaterializationCosts.put(NodeRef.of(table), cost); + } + + public Map, Long> getCteMaterializationCosts() { + return cteMaterializationCosts; + } + public void addCteDataStore(Table table, CteDataStore dataStore) { cteDataStores.put(NodeRef.of(table), dataStore); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java index 378e4668071f..b4f1a5c7261b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java @@ -71,6 +71,8 @@ public class ExplainAnalyzeOperator implements ProcessOperator { RamUsageEstimator.shallowSizeOfInstance(ExplainAnalyzeOperator.class); private static final String LOG_TITLE = "---------------------Intermediate Results of EXPLAIN ANALYZE---------------------:"; + private static final double NS_TO_MS_FACTOR = 1.0 / 1000000; + private final OperatorContext operatorContext; private final Operator child; private final boolean verbose; @@ -233,6 +235,23 @@ private List mergeAnalyzeResults( sb.append(SPACE); } analyzeResult.add(sb.toString()); + if (line.contains("Logical Plan Cost:")) { + mppQueryContext + .getCteMaterializationCosts() + .forEach( + (tableRef, cost) -> { + sb.setLength(0); + sb.append( + String.format( + " %s Materialization Total Cost: %.3f ms", + tableRef.getNode().getName().toString(), cost * NS_TO_MS_FACTOR)); + int currentLength = sb.length(); + for (int i = 0; i < maxLineLength - currentLength; i++) { + sb.append(SPACE); + } + analyzeResult.add(sb.toString()); + }); + } }); return analyzeResult; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 0fd75dc6db5d..e6b587ad06be 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -109,6 +109,7 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; CteDataStore cteDataStore = null; + long startTime = System.nanoTime(); try { final ExecutionResult executionResult = coordinator.executeForTableModel( @@ -178,6 +179,8 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex } t = throwable; } finally { + long cost = System.nanoTime() - startTime; + context.addCteMaterializationCost(table, cost); coordinator.cleanupQueryExecution(queryId, null, t); } return null; From 6f83810f2dd9d1965ba62742bcba31d7a6fa5d70 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 22 Oct 2025 15:57:46 +0800 Subject: [PATCH 40/69] add multiple with it test cases --- .../relational/it/query/recent/IoTDBCteIT.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java index defda41a62fa..1c22c8c3f108 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java @@ -111,6 +111,20 @@ public void tearDown() { } } + @Test + public void testMultipleWith() { + String mainQuery = + "select * from cte1 where voltage > " + + "(with cte2 as materialized (select avg(voltage) as avg_voltage from testtb) select avg_voltage from cte2)"; + String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:01.000Z,d2,300.0,", + }; + String[] cteTemplateQueries = new String[] {"cte1 as %s (select * from testtb)"}; + testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); + } + @Test public void testFilterQuery() { // case 1 From dff5ad8ce42a295b0efdfe98c3dbb39ef5bd6571 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 3 Nov 2025 11:34:31 +0800 Subject: [PATCH 41/69] change default value for cte parameters --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 4 ++-- .../org/apache/iotdb/db/conf/IoTDBDescriptor.java | 15 ++++++++++++--- .../conf/iotdb-system.properties.template | 6 +++--- 3 files changed, 17 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 4fd31e817806..35d56ecc3c22 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -422,10 +422,10 @@ public class IoTDBConfig { private long sortBufferSize = 32 * 1024 * 1024L; /** The buffer for cte scan operation */ - private long cteBufferSize = 32 * 1024 * 1024L; + private long cteBufferSize = 128 * 1024L; /** Max number of rows for cte materialization */ - private int maxRowsInCteBuffer = 10000; + private int maxRowsInCteBuffer = 1000; /** Mods cache size limit per fi */ private long modsCacheSizeLimitPerFI = 32 * 1024 * 1024; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 0d489fdf8ea6..f0fea44b65f7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -1058,13 +1058,22 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException loadFixedSizeLimitForQuery(properties, "sort_buffer_size_in_bytes", conf::setSortBufferSize); // The buffer for cte materialization. - loadFixedSizeLimitForQuery(properties, "cte_buffer_size_in_bytes", conf::setCteBufferSize); + long cteBufferSizeInBytes = + Long.parseLong( + properties.getProperty( + "cte_buffer_size_in_bytes", Long.toString(conf.getCteBufferSize()))); + if (cteBufferSizeInBytes > 0) { + conf.setCteBufferSize(cteBufferSizeInBytes); + } // max number of rows for cte materialization - conf.setMaxRowsInCteBuffer( + int maxRowsInCteBuffer = Integer.parseInt( properties.getProperty( - "max_rows_in_cte_buffer", Integer.toString(conf.getMaxRowsInCteBuffer())))); + "max_rows_in_cte_buffer", Integer.toString(conf.getMaxRowsInCteBuffer()))); + if (maxRowsInCteBuffer > 0) { + conf.setMaxRowsInCteBuffer(maxRowsInCteBuffer); + } loadFixedSizeLimitForQuery( properties, "mods_cache_size_limit_per_fi_in_bytes", conf::setModsCacheSizeLimitPerFI); diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 6a236e2f278b..7f78cf22b2d2 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -1123,8 +1123,8 @@ batch_size=100000 # Datatype: long sort_buffer_size_in_bytes=0 -# The buffer size for CTE materialization. If cte_buffer_size_in_bytes <= 0, default value will be used, default value = -# min(32MB, memory for query operators / query_thread_count / 2). if cte_buffer_size_in_bytes > 0, the specified value will be used. +# The buffer size for CTE materialization. If cte_buffer_size_in_bytes <= 0, a default value of 128 KB is used; otherwise the specified value +# will be used. # effectiveMode: hot_reload # Datatype: long cte_buffer_size_in_bytes=0 @@ -1133,7 +1133,7 @@ cte_buffer_size_in_bytes=0 # Max rows for CTE materialization # effectiveMode: hot_reload # Datatype: int -max_rows_in_cte_buffer=10000 +max_rows_in_cte_buffer=1000 # The maximum mod entries size that each FragmentInstance can cache. # if mods_cache_size_limit_per_fi_in_bytes <= 0, default value will be used, default value = min(32MB, memory for query operators / query_thread_count / 2) From 1c8ba766dd0d3541e173425a4fc5e0b96d7d947f Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 25 Nov 2025 16:02:48 +0800 Subject: [PATCH 42/69] fix unit test --- .../PushAggregationThroughUnionTest.java | 6 ++++++ .../relational/planner/CteMaterializerTest.java | 17 ++++++++--------- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/PushAggregationThroughUnionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/PushAggregationThroughUnionTest.java index e1702a6f2fe7..722c8ce992d4 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/PushAggregationThroughUnionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/PushAggregationThroughUnionTest.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.analyzer; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.planner.plan.DistributedQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; @@ -31,6 +32,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.junit.Before; import org.junit.Test; import java.util.Optional; @@ -53,6 +55,10 @@ import static org.junit.Assert.assertEquals; public class PushAggregationThroughUnionTest { + @Before + public void setUp() { + IoTDBDescriptor.getInstance().getConfig().setDataNodeId(1); + } @Test public void UnionAggregationTest() { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java index f6ae7e910adb..71755b098767 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java @@ -93,20 +93,16 @@ private Type convertType(String columnName) { private void mockException() { CteMaterializer cteMaterializer = Mockito.spy(new CteMaterializer()); - Mockito.when(cteMaterializer.fetchCteQueryResult(Mockito.any(), Mockito.any(), Mockito.any())) - .thenReturn(null); + Mockito.doReturn(null) + .when(cteMaterializer) + .fetchCteQueryResult(Mockito.any(), Mockito.any(), Mockito.any()); CteMaterializer.setInstance(cteMaterializer); } private void mockSubquery() { CteMaterializer cteMaterializer = Mockito.spy(new CteMaterializer()); - Mockito.when( - cteMaterializer.fetchCteQueryResult( - Mockito.any(Table.class), - Mockito.any(Query.class), - Mockito.any(MPPQueryContext.class))) - .thenAnswer( + Mockito.doAnswer( (InvocationOnMock invocation) -> { Table table = invocation.getArgument(0); Query query = invocation.getArgument(1); @@ -131,7 +127,10 @@ private void mockSubquery() { TableSchema tableSchema = new TableSchema(table.getName().toString(), columnsSchemas); return new CteDataStore(query, tableSchema, ImmutableList.of(0, 1)); - }); + }) + .when(cteMaterializer) + .fetchCteQueryResult( + Mockito.any(Table.class), Mockito.any(Query.class), Mockito.any(MPPQueryContext.class)); CteMaterializer.setInstance(cteMaterializer); } From 98b3942b75620d49ea668138ed7e99808e9e8314 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 26 Nov 2025 15:04:04 +0800 Subject: [PATCH 43/69] fix: CteScanNode replica set & query done --- .../queryengine/plan/relational/planner/CteMaterializer.java | 2 +- .../queryengine/plan/relational/planner/RelationPlanner.java | 2 +- .../queryengine/plan/relational/planner/node/CteScanNode.java | 4 +++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index e6b587ad06be..096e11db3d91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -83,7 +83,7 @@ public void materializeCTE(Analysis analysis, MPPQueryContext context) { if (dataStore == null) { // CTE query execution failed. Use inline instead of materialization // in the outer query - query.setMaterialized(false); + query.setDone(false); return; } 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 7a24917f5add..4b7032e8b059 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 @@ -249,7 +249,7 @@ protected RelationPlan visitTable(final Table table, final Void context) { throw new SemanticException("unexpected recursive cte"); } - if (namedQuery.isMaterialized()) { + if (namedQuery.isMaterialized() && namedQuery.isDone()) { CteDataStore dataStore = queryContext.getCteDataStore(table); if (dataStore != null) { List cteSymbols = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java index d8634fcab110..48eb58c980d0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/CteScanNode.java @@ -22,6 +22,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.node; import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.db.queryengine.common.DataNodeEndPoints; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; @@ -81,7 +82,8 @@ public void close() throws Exception {} @Override public TRegionReplicaSet getRegionReplicaSet() { - return null; + return new TRegionReplicaSet( + null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())); } @Override From d83f4745ff2f0d9673ea4155a335a7810ea44c6e Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 27 Nov 2025 09:23:18 +0800 Subject: [PATCH 44/69] fix: memory control & fetchCteQueryResult --- .../source/relational/CteScanOperator.java | 27 ++++++++++++------- .../relational/planner/CteMaterializer.java | 11 ++------ .../iotdb/db/utils/cte/CteDataStore.java | 13 +++++++++ 3 files changed, 33 insertions(+), 18 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 5d516e0d5b38..6bb7c21acaac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -29,6 +29,7 @@ import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.iotdb.db.utils.cte.MemoryReader; +import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.utils.RamUsageEstimator; import org.slf4j.Logger; @@ -42,10 +43,14 @@ public class CteScanOperator extends AbstractSourceOperator { private final CteDataStore dataStore; private CteDataReader dataReader; + private final long maxReturnSize = + TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + public CteScanOperator( OperatorContext operatorContext, PlanNodeId sourceId, CteDataStore dataStore) { this.operatorContext = operatorContext; this.sourceId = sourceId; + dataStore.increaseRefCount(); this.dataStore = dataStore; prepareReader(); } @@ -84,27 +89,31 @@ public boolean isFinished() throws Exception { @Override public long calculateMaxPeekMemory() { - if (dataReader == null) { - return 0; - } - return dataReader.bytesUsed(); + return calculateRetainedSizeAfterCallingNext() + calculateMaxReturnSize(); } @Override public long calculateMaxReturnSize() { - // The returned object is a reference to TsBlock in CteDataReader - return RamUsageEstimator.NUM_BYTES_OBJECT_REF; + return maxReturnSize; } @Override public long calculateRetainedSizeAfterCallingNext() { - return calculateMaxPeekMemoryWithCounter(); + return 0L; } @Override public long ramBytesUsed() { - return INSTANCE_SIZE - + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext); + long bytes = + INSTANCE_SIZE + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext); + if (dataReader != null) { + bytes += dataReader.bytesUsed(); + } + if (dataStore.getRefCount() == 1) { + bytes += dataStore.getCachedBytes(); + } + + return bytes; } private void prepareReader() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 096e11db3d91..b529e3091df1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -53,7 +53,6 @@ import org.apache.iotdb.rpc.TSStatusCode; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.type.TypeFactory; @@ -95,13 +94,7 @@ public void materializeCTE(Analysis analysis, MPPQueryContext context) { public void cleanUpCTE(MPPQueryContext context) { Map, CteDataStore> cteDataStores = context.getCteDataStores(); - cteDataStores - .values() - .forEach( - dataStore -> { - context.releaseMemoryReservedForFrontEnd(dataStore.getCachedBytes()); - dataStore.clear(); - }); + cteDataStores.values().forEach(CteDataStore::clear); cteDataStores.clear(); } @@ -121,7 +114,7 @@ public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContex .getSessionInfoOfTableModel(SessionManager.getInstance().getCurrSession()), String.format("Materialize query for CTE '%s'", table.getName()), LocalExecutionPlanner.getInstance().metadata, - ImmutableMap.of(), + context.getCteDataStores(), context.getExplainType(), context.getTimeOut(), false); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index 8b77f3dafb73..03313b312aa8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; public class CteDataStore { private final Query query; @@ -40,6 +41,9 @@ public class CteDataStore { private long cachedBytes; private int cachedRows; + // reference count by CteScanOperator + private final AtomicInteger count; + public CteDataStore( Query query, TableSchema tableSchema, List columnIndex2TsBlockColumnIndexList) { this.query = query; @@ -48,6 +52,7 @@ public CteDataStore( this.cachedData = new ArrayList<>(); this.cachedBytes = 0L; this.cachedRows = 0; + this.count = new AtomicInteger(0); } public boolean addTsBlock(TsBlock tsBlock) { @@ -89,4 +94,12 @@ public Query getQuery() { public List getColumnIndex2TsBlockColumnIndexList() { return columnIndex2TsBlockColumnIndexList; } + + public int getRefCount() { + return count.get(); + } + + public void increaseRefCount() { + count.getAndIncrement(); + } } From e1caa8923048d6f9986a1ec50af0c0ca99169879 Mon Sep 17 00:00:00 2001 From: shizy Date: Sat, 29 Nov 2025 17:57:40 +0800 Subject: [PATCH 45/69] fix: with in cte materialization subquery --- .../queryengine/common/MPPQueryContext.java | 9 +++++ .../plan/relational/analyzer/Scope.java | 31 +++++++++++++-- .../analyzer/StatementAnalyzer.java | 35 +++++++++++------ .../relational/planner/CteMaterializer.java | 38 +++++++++++++++++-- .../planner/TableLogicalPlanner.java | 2 +- .../planner/CteMaterializerTest.java | 8 +++- 6 files changed, 102 insertions(+), 21 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 255f6daac672..d81da26dd2ca 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 @@ -34,6 +34,8 @@ import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.planner.memory.NotThreadSafeMemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.statistics.QueryPlanStatistics; import org.apache.iotdb.db.utils.cte.CteDataStore; @@ -123,6 +125,9 @@ public class MPPQueryContext implements IAuditEntity { // Do not release CTE query result if it is a subquery. private boolean subquery = false; + // Tables in the subquery + private final Map> subQueryTables = new HashMap<>(); + public MPPQueryContext(QueryId queryId) { this.queryId = queryId; this.endPointBlackList = ConcurrentHashMap.newKeySet(); @@ -503,6 +508,10 @@ public void setCteDataStores(Map, CteDataStore> cteDataStores) { this.cteDataStores = cteDataStores; } + public Map> getSubQueryTables() { + return subQueryTables; + } + public void addCteExplainResult(Table table, Pair> cteExplainResult) { cteExplainResults.put(NodeRef.of(table), cteExplainResult); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java index 0e7588a59f49..a979b1f6c3ef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java @@ -23,12 +23,15 @@ import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AllColumns; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.QualifiedName; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WithQuery; import org.apache.iotdb.rpc.TSStatusCode; import com.google.common.collect.ImmutableMap; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -50,6 +53,7 @@ public class Scope { private final RelationId relationId; private final RelationType relation; private final Map namedQueries; + private final List tables; public static Scope create() { return builder().build(); @@ -64,16 +68,30 @@ private Scope( boolean queryBoundary, RelationId relationId, RelationType relation, - Map namedQueries) { + Map namedQueries, + List tables) { this.parent = requireNonNull(parent, "parent is null"); this.relationId = requireNonNull(relationId, "relationId is null"); this.queryBoundary = queryBoundary; this.relation = requireNonNull(relation, "relation is null"); this.namedQueries = ImmutableMap.copyOf(requireNonNull(namedQueries, "namedQueries is null")); + this.tables = new ArrayList<>(requireNonNull(tables, "tables is null")); + } + + public void addTable(Table table) { + tables.add(new Identifier(table.getName().getSuffix())); + } + + public Scope clone() { + return builder().like(this).build(); + } + + public List getTables() { + return tables; } public Scope withRelationType(RelationType relationType) { - return new Scope(parent, queryBoundary, relationId, relationType, namedQueries); + return new Scope(parent, queryBoundary, relationId, relationType, namedQueries, tables); } public Scope getQueryBoundaryScope() { @@ -327,6 +345,7 @@ public static final class Builder { private RelationId relationId = RelationId.anonymous(); private RelationType relationType = new RelationType(); private final Map namedQueries = new HashMap<>(); + private final List tables = new ArrayList<>(); private Optional parent = Optional.empty(); private boolean queryBoundary; @@ -334,6 +353,7 @@ public Builder like(Scope other) { relationId = other.relationId; relationType = other.relation; namedQueries.putAll(other.namedQueries); + tables.addAll(other.tables); parent = other.parent; queryBoundary = other.queryBoundary; return this; @@ -364,12 +384,17 @@ public Builder withNamedQuery(String name, WithQuery withQuery) { return this; } + public Builder withTables(List tables) { + this.tables.addAll(tables); + return this; + } + public boolean containsNamedQuery(String name) { return namedQueries.containsKey(name); } public Scope build() { - return new Scope(parent, queryBoundary, relationId, relationType, namedQueries); + return new Scope(parent, queryBoundary, relationId, relationType, namedQueries, tables); } } 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 3689b66c2e55..d9d0f33309a0 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 @@ -895,6 +895,7 @@ protected Scope visitQuery(Query node, Optional context) { Scope.builder() .withParent(withScope) .withRelationType(RelationId.of(node), queryBodyScope.getRelationType()) + .withTables(queryBodyScope.getTables()) .build(); analysis.setScope(node, queryScope); @@ -936,7 +937,7 @@ private Scope analyzeWith(Query node, Optional scope) { if (!isRecursive) { Query query = withQuery.getQuery(); - analyze(query, withScopeBuilder.build()); + Scope queryScope = analyze(query, withScopeBuilder.build()); // check if all or none of the columns are explicitly alias if (withQuery.getColumnNames().isPresent()) { @@ -946,6 +947,7 @@ private Scope analyzeWith(Query node, Optional scope) { } withScopeBuilder.withNamedQuery(name, withQuery); + queryContext.getSubQueryTables().put(withQuery.getQuery(), queryScope.getTables()); } } Scope withScope = withScopeBuilder.build(); @@ -3065,6 +3067,7 @@ protected Scope visitSetOperation(SetOperation node, Optional scope) { @Override protected Scope visitTable(Table table, Optional scope) { if (!table.getName().getPrefix().isPresent()) { + scope.ifPresent(s -> s.addTable(table)); // is this a reference to a WITH query? Optional withQuery = createScope(scope).getNamedQuery(table.getName().getSuffix()); @@ -3096,15 +3099,15 @@ protected Scope visitTable(Table table, Optional scope) { analysis.setRelationName( table, QualifiedName.of(name.getDatabaseName(), name.getObjectName())); - Optional tableSchema = metadata.getTableSchema(sessionContext, name); - // if table schema is not found in metadata, we check if it's a CTE defined in the parent - // query + // check if table schema is found in CTE data stores + CteDataStore dataStore = queryContext.getCteDataStore(table); + Optional tableSchema = + dataStore != null ? Optional.of(dataStore.getTableSchema()) : Optional.empty(); + // If table schema is not found, check if it is in metadata if (!tableSchema.isPresent()) { - CteDataStore dataStore = queryContext.getCteDataStore(table); - if (dataStore != null) { - tableSchema = Optional.of(dataStore.getTableSchema()); - } + tableSchema = metadata.getTableSchema(sessionContext, name); } + // This can only be a table if (!tableSchema.isPresent()) { TableMetadataImpl.throwTableNotExistsException( @@ -3594,8 +3597,15 @@ protected Scope visitJoin(Join node, Optional scope) { joinConditionCheck(criteria); - Scope left = process(node.getLeft(), scope); - Scope right = process(node.getRight(), scope); + Optional leftScope = scope.map(Scope::clone); + Scope left = process(node.getLeft(), leftScope); + Optional rightScope = scope.map(Scope::clone); + Scope right = process(node.getRight(), rightScope); + + if (scope.isPresent()) { + leftScope.ifPresent(l -> scope.get().getTables().addAll(l.getTables())); + rightScope.ifPresent(l -> scope.get().getTables().addAll(l.getTables())); + } if (criteria instanceof JoinUsing) { return analyzeJoinUsing(node, ((JoinUsing) criteria).getColumns(), scope, left, right); @@ -4447,7 +4457,10 @@ private Scope createAndAssignScope(Node node, Optional parentScope, List< private Scope createAndAssignScope( Node node, Optional parentScope, RelationType relationType) { Scope scope = - scopeBuilder(parentScope).withRelationType(RelationId.of(node), relationType).build(); + scopeBuilder(parentScope) + .withRelationType(RelationId.of(node), relationType) + .withTables(parentScope.get().getTables()) + .build(); analysis.setScope(node, scope); return scope; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index b529e3091df1..28744d5f97e9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -42,8 +42,11 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.With; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WithQuery; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; import org.apache.iotdb.db.queryengine.statistics.FragmentInstanceStatisticsDrawer; import org.apache.iotdb.db.queryengine.statistics.QueryStatisticsFetcher; @@ -71,14 +74,15 @@ public class CteMaterializer { private static final Coordinator coordinator = Coordinator.getInstance(); - public void materializeCTE(Analysis analysis, MPPQueryContext context) { + public void materializeCTE(MPPQueryContext context, Analysis analysis, Query mainQuery) { analysis .getNamedQueries() .forEach( (tableRef, query) -> { Table table = tableRef.getNode(); if (query.isMaterialized() && !query.isDone()) { - CteDataStore dataStore = fetchCteQueryResult(table, query, context); + CteDataStore dataStore = + fetchCteQueryResult(context, table, query, mainQuery.getWith().orElse(null)); if (dataStore == null) { // CTE query execution failed. Use inline instead of materialization // in the outer query @@ -98,15 +102,41 @@ public void cleanUpCTE(MPPQueryContext context) { cteDataStores.clear(); } - public CteDataStore fetchCteQueryResult(Table table, Query query, MPPQueryContext context) { + public CteDataStore fetchCteQueryResult( + MPPQueryContext context, Table table, Query query, With with) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; CteDataStore cteDataStore = null; long startTime = System.nanoTime(); try { + Query q = query; + if (with != null) { + List tables = + context.getSubQueryTables().getOrDefault(query, ImmutableList.of()); + List withQueries = + with.getQueries().stream() + .filter( + x -> + tables.contains(x.getName()) + && !x.getQuery().isMaterialized() + && !x.getQuery().isDone()) + .collect(Collectors.toList()); + + if (!withQueries.isEmpty()) { + With w = new With(with.getLocation().orElse(null), with.isRecursive(), withQueries); + q = + new Query( + Optional.of(w), + query.getQueryBody(), + query.getFill(), + query.getOrderBy(), + query.getOffset(), + query.getLimit()); + } + } final ExecutionResult executionResult = coordinator.executeForTableModel( - query, + q, new SqlParser(), SessionManager.getInstance().getCurrSession(), queryId, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index 8210d8443d68..5c24010f6356 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -374,7 +374,7 @@ private RelationPlan createRelationPlan(Analysis analysis, PipeEnriched pipeEnri private RelationPlan createRelationPlan(Analysis analysis, Query query) { // materialize cte if needed if (!queryContext.isSubquery()) { - CteMaterializer.getInstance().materializeCTE(analysis, queryContext); + CteMaterializer.getInstance().materializeCTE(queryContext, analysis, query); } return getRelationPlanner(analysis).process(query, null); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java index 71755b098767..12cd93dfd9ac 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java @@ -36,6 +36,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SingleColumn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.With; import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.collect.ImmutableList; @@ -95,7 +96,7 @@ private void mockException() { CteMaterializer cteMaterializer = Mockito.spy(new CteMaterializer()); Mockito.doReturn(null) .when(cteMaterializer) - .fetchCteQueryResult(Mockito.any(), Mockito.any(), Mockito.any()); + .fetchCteQueryResult(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any()); CteMaterializer.setInstance(cteMaterializer); } @@ -130,7 +131,10 @@ private void mockSubquery() { }) .when(cteMaterializer) .fetchCteQueryResult( - Mockito.any(Table.class), Mockito.any(Query.class), Mockito.any(MPPQueryContext.class)); + Mockito.any(MPPQueryContext.class), + Mockito.any(Table.class), + Mockito.any(Query.class), + Mockito.any(With.class)); CteMaterializer.setInstance(cteMaterializer); } From 91629b79f23d778774978d83f3696451ec5a39d4 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 1 Dec 2025 08:32:54 +0800 Subject: [PATCH 46/69] fix: with in scalar subquery --- .../plan/relational/analyzer/Analysis.java | 11 +++++ .../analyzer/ExpressionAnalyzer.java | 8 +-- .../analyzer/StatementAnalyzer.java | 10 ++-- .../relational/planner/CteMaterializer.java | 4 +- .../relational/planner/SubqueryPlanner.java | 2 +- .../planner/TableLogicalPlanner.java | 2 +- ...correlatedScalarSubqueryReconstructor.java | 49 ++++++++++++++++--- .../planner/CteMaterializerTest.java | 4 +- .../planner/UncorrelatedSubqueryTest.java | 2 +- 9 files changed, 69 insertions(+), 23 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java index 5d373dd4a06b..306c9b51c28a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java @@ -69,6 +69,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.TableFunctionInvocation; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WindowFrame; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.With; import org.apache.iotdb.db.queryengine.plan.statement.component.FillPolicy; import com.google.common.collect.ArrayListMultimap; @@ -123,6 +124,8 @@ public class Analysis implements IAnalysis { private final Map, Query> namedQueries = new LinkedHashMap<>(); + private With with; + // map expandable query to the node being the inner recursive reference private final Map, Node> expandableNamedQueries = new LinkedHashMap<>(); @@ -282,6 +285,14 @@ public Map, Query> getNamedQueries() { return namedQueries; } + public With getWith() { + return with; + } + + public void setWith(With with) { + this.with = with; + } + public boolean isAnalyzed(Expression expression) { return expression instanceof DataType || types.containsKey(NodeRef.of(expression)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java index 6157612b4238..8ea35c2b94e8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java @@ -1717,12 +1717,12 @@ private Type analyzePredicateWithSubquery( return subqueryType; } - private Type analyzeSubquery( - SubqueryExpression node, StackableAstVisitorContext context) { + private Type analyzeSubquery(SubqueryExpression node, StackableAstVisitorContext ctx) { StatementAnalyzer analyzer = - statementAnalyzerFactory.apply(node, context.getContext().getCorrelationSupport()); - Scope subqueryScope = Scope.builder().withParent(context.getContext().getScope()).build(); + statementAnalyzerFactory.apply(node, ctx.getContext().getCorrelationSupport()); + Scope subqueryScope = Scope.builder().withParent(ctx.getContext().getScope()).build(); Scope queryScope = analyzer.analyze(node.getQuery(), subqueryScope); + context.getSubQueryTables().put(node.getQuery(), queryScope.getTables()); ImmutableList.Builder fields = ImmutableList.builder(); for (int i = 0; i < queryScope.getRelationType().getAllFieldCount(); i++) { 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 d9d0f33309a0..949a839eb345 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 @@ -921,6 +921,7 @@ private Scope analyzeWith(Query node, Optional scope) { // analyze WITH clause With with = node.getWith().get(); + analysis.setWith(with); Scope.Builder withScopeBuilder = scopeBuilder(scope); for (WithQuery withQuery : with.getQueries()) { @@ -4456,11 +4457,10 @@ private Scope createAndAssignScope(Node node, Optional parentScope, List< private Scope createAndAssignScope( Node node, Optional parentScope, RelationType relationType) { - Scope scope = - scopeBuilder(parentScope) - .withRelationType(RelationId.of(node), relationType) - .withTables(parentScope.get().getTables()) - .build(); + Scope.Builder scopeBuilder = + scopeBuilder(parentScope).withRelationType(RelationId.of(node), relationType); + parentScope.ifPresent(scope -> scopeBuilder.withTables(scope.getTables())); + Scope scope = scopeBuilder.build(); analysis.setScope(node, scope); return scope; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 28744d5f97e9..28d998dad7e6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -74,7 +74,7 @@ public class CteMaterializer { private static final Coordinator coordinator = Coordinator.getInstance(); - public void materializeCTE(MPPQueryContext context, Analysis analysis, Query mainQuery) { + public void materializeCTE(Analysis analysis, MPPQueryContext context) { analysis .getNamedQueries() .forEach( @@ -82,7 +82,7 @@ public void materializeCTE(MPPQueryContext context, Analysis analysis, Query mai Table table = tableRef.getNode(); if (query.isMaterialized() && !query.isDone()) { CteDataStore dataStore = - fetchCteQueryResult(context, table, query, mainQuery.getWith().orElse(null)); + fetchCteQueryResult(context, table, query, analysis.getWith()); if (dataStore == null) { // CTE query execution failed. Use inline instead of materialization // in the outer query diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java index 1ac59f843480..64f6f4e85a99 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java @@ -158,7 +158,7 @@ public PlanBuilder handleSubqueries( private void tryFoldUncorrelatedScalarSubqueryInPredicate( Expression expression, MPPQueryContext context) { PredicateWithUncorrelatedScalarSubqueryReconstructor.getInstance() - .reconstructPredicateWithUncorrelatedScalarSubquery(expression, context); + .reconstructPredicateWithUncorrelatedScalarSubquery(context, analysis, expression); } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index 5c24010f6356..8210d8443d68 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -374,7 +374,7 @@ private RelationPlan createRelationPlan(Analysis analysis, PipeEnriched pipeEnri private RelationPlan createRelationPlan(Analysis analysis, Query query) { // materialize cte if needed if (!queryContext.isSubquery()) { - CteMaterializer.getInstance().materializeCTE(queryContext, analysis, query); + CteMaterializer.getInstance().materializeCTE(analysis, queryContext); } return getRelationPlanner(analysis).process(query, null); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 7c828f0ce11c..1e6d3a276f38 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BinaryLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; @@ -38,17 +39,22 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NotExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StringLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SubqueryExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.With; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WithQuery; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; import org.apache.iotdb.rpc.TSStatusCode; +import com.google.common.collect.ImmutableList; import org.apache.tsfile.block.column.Column; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; import static com.google.common.base.Preconditions.checkArgument; @@ -59,29 +65,32 @@ public class PredicateWithUncorrelatedScalarSubqueryReconstructor { private static final Coordinator coordinator = Coordinator.getInstance(); public void reconstructPredicateWithUncorrelatedScalarSubquery( - Expression expression, MPPQueryContext context) { + MPPQueryContext context, Analysis analysis, Expression expression) { if (expression instanceof LogicalExpression) { LogicalExpression logicalExpression = (LogicalExpression) expression; for (Expression term : logicalExpression.getTerms()) { - reconstructPredicateWithUncorrelatedScalarSubquery(term, context); + reconstructPredicateWithUncorrelatedScalarSubquery(context, analysis, term); } } else if (expression instanceof NotExpression) { NotExpression notExpression = (NotExpression) expression; - reconstructPredicateWithUncorrelatedScalarSubquery(notExpression.getValue(), context); + reconstructPredicateWithUncorrelatedScalarSubquery( + context, analysis, notExpression.getValue()); } else if (expression instanceof ComparisonExpression) { ComparisonExpression comparisonExpression = (ComparisonExpression) expression; Expression left = comparisonExpression.getLeft(); Expression right = comparisonExpression.getRight(); if (left instanceof Identifier && right instanceof SubqueryExpression) { Optional result = - fetchUncorrelatedSubqueryResultForPredicate((SubqueryExpression) right, context); + fetchUncorrelatedSubqueryResultForPredicate( + context, (SubqueryExpression) right, analysis.getWith()); // If the subquery result is not present, we cannot reconstruct the predicate. if (result.isPresent()) { right = result.get(); } } else if (right instanceof Identifier && left instanceof SubqueryExpression) { Optional result = - fetchUncorrelatedSubqueryResultForPredicate((SubqueryExpression) left, context); + fetchUncorrelatedSubqueryResultForPredicate( + context, (SubqueryExpression) left, analysis.getWith()); if (result.isPresent()) { left = result.get(); } @@ -97,14 +106,40 @@ public void reconstructPredicateWithUncorrelatedScalarSubquery( * valid result. */ public Optional fetchUncorrelatedSubqueryResultForPredicate( - SubqueryExpression subqueryExpression, MPPQueryContext context) { + MPPQueryContext context, SubqueryExpression subqueryExpression, With with) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; try { + Query query = subqueryExpression.getQuery(); + Query q = query; + if (with != null) { + List tables = + context.getSubQueryTables().getOrDefault(query, ImmutableList.of()); + List withQueries = + with.getQueries().stream() + .filter( + x -> + tables.contains(x.getName()) + && !x.getQuery().isMaterialized() + && !x.getQuery().isDone()) + .collect(Collectors.toList()); + + if (!withQueries.isEmpty()) { + With w = new With(with.getLocation().orElse(null), with.isRecursive(), withQueries); + q = + new Query( + Optional.of(w), + query.getQueryBody(), + query.getFill(), + query.getOrderBy(), + query.getOffset(), + query.getLimit()); + } + } final ExecutionResult executionResult = coordinator.executeForTableModel( - subqueryExpression.getQuery(), + q, relationSqlParser, SessionManager.getInstance().getCurrSession(), queryId, diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java index 12cd93dfd9ac..c5a5fca28908 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java @@ -105,8 +105,8 @@ private void mockSubquery() { Mockito.doAnswer( (InvocationOnMock invocation) -> { - Table table = invocation.getArgument(0); - Query query = invocation.getArgument(1); + Table table = invocation.getArgument(1); + Query query = invocation.getArgument(2); List selectItems = ((QuerySpecification) query.getQueryBody()).getSelect().getSelectItems(); List columnsSchemas = diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java index ea8e098414a0..8cddd6fe3158 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java @@ -75,7 +75,7 @@ private void mockPredicateWithUncorrelatedScalarSubquery() { Mockito.spy(new PredicateWithUncorrelatedScalarSubqueryReconstructor()); Mockito.when( predicateWithUncorrelatedScalarSubquery.fetchUncorrelatedSubqueryResultForPredicate( - Mockito.any(), Mockito.any())) + Mockito.any(), Mockito.any(), Mockito.any())) .thenReturn(Optional.of(new LongLiteral("1"))); PredicateWithUncorrelatedScalarSubqueryReconstructor.setInstance( predicateWithUncorrelatedScalarSubquery); From 83978972720e1fffa9e61a60d5c4e10c5a336840 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 2 Dec 2025 09:09:43 +0800 Subject: [PATCH 47/69] fix: data store ref count --- .../operator/source/relational/CteScanOperator.java | 5 +++-- .../java/org/apache/iotdb/db/utils/cte/CteDataStore.java | 8 ++------ 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 6bb7c21acaac..44e22e300dbb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -41,6 +41,7 @@ public class CteScanOperator extends AbstractSourceOperator { RamUsageEstimator.shallowSizeOfInstance(CteScanOperator.class); private final CteDataStore dataStore; + private final int dataStoreRefCount; private CteDataReader dataReader; private final long maxReturnSize = @@ -50,8 +51,8 @@ public CteScanOperator( OperatorContext operatorContext, PlanNodeId sourceId, CteDataStore dataStore) { this.operatorContext = operatorContext; this.sourceId = sourceId; - dataStore.increaseRefCount(); this.dataStore = dataStore; + this.dataStoreRefCount = dataStore.increaseRefCount(); prepareReader(); } @@ -109,7 +110,7 @@ public long ramBytesUsed() { if (dataReader != null) { bytes += dataReader.bytesUsed(); } - if (dataStore.getRefCount() == 1) { + if (dataStoreRefCount == 1) { bytes += dataStore.getCachedBytes(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index 03313b312aa8..8ff79a3ca3dc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -95,11 +95,7 @@ public List getColumnIndex2TsBlockColumnIndexList() { return columnIndex2TsBlockColumnIndexList; } - public int getRefCount() { - return count.get(); - } - - public void increaseRefCount() { - count.getAndIncrement(); + public int increaseRefCount() { + return count.incrementAndGet(); } } From 1503f79266b177eccb8ce5a551481dd1352bc573 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 2 Dec 2025 09:18:04 +0800 Subject: [PATCH 48/69] fix rebase --- .../execution/fragment/FragmentInstanceExecution.java | 2 +- .../db/queryengine/plan/relational/analyzer/AnalyzerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java index d392b2730682..8d4cdcd88860 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java @@ -171,7 +171,7 @@ private boolean fillFragmentInstanceStatistics( return false; } - statistics.setDataRegion(context.getDataRegion().getDataRegionId()); + statistics.setDataRegion(context.getDataRegion().getDataRegionIdString()); statistics.setIp(CONFIG.getInternalAddress() + ":" + CONFIG.getInternalPort()); statistics.setStartTimeInMS(context.getStartTime()); statistics.setEndTimeInMS( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java index 2319ea29e09f..6666c180c5e0 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AnalyzerTest.java @@ -29,9 +29,9 @@ import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.partition.DataPartitionQueryParam; import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.commons.schema.table.InsertNodeMeasurementInfo; import org.apache.iotdb.commons.schema.table.TsTable; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.protocol.session.IClientSession; import org.apache.iotdb.db.protocol.session.InternalClientSession; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; From 11f0169322e05155a0a8ee468a8fe0641e9a092d Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 3 Dec 2025 16:37:23 +0800 Subject: [PATCH 49/69] mock executeForTableModel instead of fetchCteQueryResult --- .../relational/planner/CteMaterializer.java | 9 +- .../planner/CteMaterializerTest.java | 197 +++++++++++------- .../plan/relational/planner/PlanTester.java | 15 +- 3 files changed, 144 insertions(+), 77 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 28d998dad7e6..4807c7526b54 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -73,6 +73,7 @@ public class CteMaterializer { private static final Logger LOGGER = LoggerFactory.getLogger(CteMaterializer.class); private static final Coordinator coordinator = Coordinator.getInstance(); + private static final SessionManager sessionManager = SessionManager.getInstance(); public void materializeCTE(Analysis analysis, MPPQueryContext context) { analysis @@ -104,7 +105,7 @@ public void cleanUpCTE(MPPQueryContext context) { public CteDataStore fetchCteQueryResult( MPPQueryContext context, Table table, Query query, With with) { - final long queryId = SessionManager.getInstance().requestQueryId(); + final long queryId = sessionManager.requestQueryId(); Throwable t = null; CteDataStore cteDataStore = null; long startTime = System.nanoTime(); @@ -138,10 +139,9 @@ public CteDataStore fetchCteQueryResult( coordinator.executeForTableModel( q, new SqlParser(), - SessionManager.getInstance().getCurrSession(), + sessionManager.getCurrSession(), queryId, - SessionManager.getInstance() - .getSessionInfoOfTableModel(SessionManager.getInstance().getCurrSession()), + sessionManager.getSessionInfoOfTableModel(sessionManager.getCurrSession()), String.format("Materialize query for CTE '%s'", table.getName()), LocalExecutionPlanner.getInstance().metadata, context.getCteDataStores(), @@ -153,6 +153,7 @@ public CteDataStore fetchCteQueryResult( } // query execution QueryExecution execution = (QueryExecution) coordinator.getQueryExecution(queryId); + // get table schema DatasetHeader datasetHeader = coordinator.getQueryExecution(queryId).getDatasetHeader(); TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java index c5a5fca28908..44f66f31476c 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java @@ -21,44 +21,47 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner; -import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; -import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.commons.schema.column.ColumnHeader; +import org.apache.iotdb.db.protocol.session.SessionManager; +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; +import org.apache.iotdb.db.queryengine.plan.Coordinator; +import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; +import org.apache.iotdb.db.queryengine.plan.execution.QueryExecution; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; -import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; -import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.QuerySpecification; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SelectItem; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SingleColumn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.With; -import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.read.common.type.Type; -import org.apache.tsfile.read.common.type.TypeFactory; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.column.LongColumn; +import org.apache.tsfile.read.common.block.column.TimeColumn; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.stream.Collectors; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanAssert.assertPlan; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.aggregation; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.aggregationFunction; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.cteScan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.explainAnalyze; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.filter; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.limit; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.offset; @@ -70,26 +73,79 @@ import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression.Operator.GREATER_THAN; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.NullOrdering.LAST; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.Ordering.ASCENDING; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; +@PowerMockIgnore({"com.sun.org.apache.xerces.*", "javax.xml.*", "org.xml.*", "javax.management.*"}) +@RunWith(PowerMockRunner.class) +@PrepareForTest({Coordinator.class, SessionManager.class}) public class CteMaterializerTest { - private PlanTester planTester; + private static PlanTester planTester; + private static Coordinator mockCoordinator; - @Before - public void setUp() { + @BeforeClass + public static void prepareEnv() { planTester = new PlanTester(); - mockSubquery(); + mockStatic(Coordinator.class); + mockStatic(SessionManager.class); + + // Create a mock Coordinator instance + mockCoordinator = Mockito.mock(Coordinator.class); + when(Coordinator.getInstance()).thenReturn(mockCoordinator); + + // Create mock SessionManager + SessionManager mockSessionManager = Mockito.mock(SessionManager.class); + when(SessionManager.getInstance()).thenReturn(mockSessionManager); + + // Mock TSStatus with success status + TSStatus mockStatus = Mockito.mock(TSStatus.class); + when(mockStatus.getCode()).thenReturn(200); // Success status code + + // Create a real ExecutionResult instance + ExecutionResult mockResult = new ExecutionResult(new QueryId("1"), mockStatus); + + // Mock the executeForTableModel method + when(mockCoordinator.executeForTableModel( + Mockito.any(), // Statement + Mockito.any(), // SqlParser + Mockito.any(), // IClientSession + Mockito.anyLong(), // queryId + Mockito.any(), // SessionInfo + Mockito.anyString(), // String + Mockito.any(), // Metadata + Mockito.anyMap(), // Map, CteDataStore> + Mockito.any(), // ExplainType + Mockito.anyLong(), // timeOut + Mockito.anyBoolean())) // userQuery + .thenReturn(mockResult); } - private Type convertType(String columnName) { - switch (columnName) { - case "time": - return TypeFactory.getType(TSDataType.TIMESTAMP); - case "s1": - case "a1": - return TypeFactory.getType(TSDataType.INT64); - default: - } - return null; + @Before + public void setUp() throws IoTDBException { + // Create QueryExecution mock + QueryExecution mockQueryExecution = Mockito.mock(QueryExecution.class); + when(mockQueryExecution.hasNextResult()) + .thenReturn(true) // First call returns true + .thenReturn(false); // Subsequent calls return false + + // Create a real DatasetHeader with time and s1 columns + List columnHeaders = + ImmutableList.of( + new ColumnHeader("time", TSDataType.TIMESTAMP), + new ColumnHeader("s1", TSDataType.INT64)); + DatasetHeader mockDatasetHeader = new DatasetHeader(columnHeaders, false); + when(mockQueryExecution.getDatasetHeader()).thenReturn(mockDatasetHeader); + + // Create a TSBlock with sample data for getBatchResult + long[] timestamps = {1000L, 2000L, 3000L}; + long[] values = {10L, 20L, 30L}; + TimeColumn timeColumn = new TimeColumn(3, timestamps); + LongColumn valueColumn = new LongColumn(3, Optional.empty(), values); + TsBlock sampleTsBlock = new TsBlock(timeColumn, valueColumn); + when(mockQueryExecution.getBatchResult()).thenReturn(Optional.of(sampleTsBlock)); + + // Mock coordinator methods + when(mockCoordinator.getQueryExecution(Mockito.anyLong())).thenReturn(mockQueryExecution); } private void mockException() { @@ -100,44 +156,6 @@ private void mockException() { CteMaterializer.setInstance(cteMaterializer); } - private void mockSubquery() { - CteMaterializer cteMaterializer = Mockito.spy(new CteMaterializer()); - - Mockito.doAnswer( - (InvocationOnMock invocation) -> { - Table table = invocation.getArgument(1); - Query query = invocation.getArgument(2); - List selectItems = - ((QuerySpecification) query.getQueryBody()).getSelect().getSelectItems(); - List columnsSchemas = - selectItems.stream() - .map( - selectItem -> { - SingleColumn column = ((SingleColumn) selectItem); - String columnName = - column.getAlias().isPresent() - ? column.getAlias().get().toString() - : column.getExpression().toString(); - return new ColumnSchema( - columnName, - convertType(columnName), - false, - TsTableColumnCategory.FIELD); - }) - .collect(Collectors.toList()); - - TableSchema tableSchema = new TableSchema(table.getName().toString(), columnsSchemas); - return new CteDataStore(query, tableSchema, ImmutableList.of(0, 1)); - }) - .when(cteMaterializer) - .fetchCteQueryResult( - Mockito.any(MPPQueryContext.class), - Mockito.any(Table.class), - Mockito.any(Query.class), - Mockito.any(With.class)); - CteMaterializer.setInstance(cteMaterializer); - } - @Test public void testSimpleCte() { String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; @@ -157,13 +175,13 @@ public void testSimpleCte() { @Test public void testFieldFilterCte() { String sql = - "with cte1 as materialized (SELECT time, s1 as a1 FROM table1) select * from cte1 where a1 > 10"; + "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1 where s1 > 10"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); Expression filterPredicate = - new ComparisonExpression(GREATER_THAN, new SymbolReference("a1"), new LongLiteral("10")); - PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "a1")); + new ComparisonExpression(GREATER_THAN, new SymbolReference("s1"), new LongLiteral("10")); + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); // Verify full LogicalPlan /* @@ -263,6 +281,7 @@ public void testAggCte() { @Test public void testCteQueryException() { + CteMaterializer originalCteMaterializer = CteMaterializer.getInstance(); mockException(); String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; @@ -278,5 +297,43 @@ public void testCteQueryException() { * └──TableScanNode */ assertPlan(logicalQueryPlan, output(tableScan)); + + // reset original CteMaterializer + CteMaterializer.setInstance(originalCteMaterializer); + } + + @Test + public void testExplainAnalyze() { + String sql = + "explain analyze with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──ExplainAnalyzeNode + * └──CteScanNode + */ + assertPlan(logicalQueryPlan, output(explainAnalyze(cteScan))); + } + + /** This test primarily ensures code coverage for handleCteExplainResults method. */ + @Test + public void testExplain() { + String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql, true); + + PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──CteScanNode + */ + assertPlan(logicalQueryPlan, output(cteScan)); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java index 02ee999cda7a..6cf914ee3b60 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.protocol.session.IClientSession; +import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.common.SessionInfo; @@ -104,20 +105,28 @@ public PlanTester(Metadata metadata) { } public LogicalQueryPlan createPlan(String sql) { - return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector()); + return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), false); + } + + public LogicalQueryPlan createPlan(String sql, boolean explain) { + return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), explain); } public LogicalQueryPlan createPlan(SessionInfo sessionInfo, String sql) { - return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector()); + return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), false); } public LogicalQueryPlan createPlan( SessionInfo sessionInfo, String sql, WarningCollector warningCollector, - PlanOptimizersStatsCollector planOptimizersStatsCollector) { + PlanOptimizersStatsCollector planOptimizersStatsCollector, + boolean explain) { distributedQueryPlan = null; MPPQueryContext context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); + if (explain) { + context.setExplainType(ExplainType.EXPLAIN); + } Analysis analysis = analyze(sql, metadata, context); this.analysis = analysis; From 970aa489ff95d677af39ce9140c011c162184952 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 4 Dec 2025 09:04:00 +0800 Subject: [PATCH 50/69] add IT test for nest CTE --- .../it/query/recent/IoTDBCteIT.java | 58 +++++- .../planner/CteMaterializerTest.java | 11 +- .../relational/planner/CteSubqueryTest.java | 168 ++++++++++++++++++ 3 files changed, 233 insertions(+), 4 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java index 1c22c8c3f108..d9443bf66840 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java @@ -58,6 +58,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.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @RunWith(IoTDBTestRunner.class) @@ -344,6 +346,60 @@ public void testNest() { testCteFailureWithVariants(cteTemplateQueries, mainQuery, errMsg); } + @Test + public void testNestExplain1() throws SQLException { + String sql = + "explain with cte1 as (select * from testtb), " + + "cte2 as materialized (select time, voltage from cte1) " + + "select * from cte2"; + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("USE testdb"); + + // explain + ResultSet resultSet = statement.executeQuery(sql); + ResultSetMetaData metaData = resultSet.getMetaData(); + assertEquals(metaData.getColumnCount(), 1); + assertEquals(metaData.getColumnName(1), "distribution plan"); + + StringBuilder sb = new StringBuilder(); + while (resultSet.next()) { + sb.append(resultSet.getString(1)).append(System.lineSeparator()); + } + String result = sb.toString(); + assertFalse(result.contains("CTE Query : 'cte1'")); + assertTrue(result.contains("CTE Query : 'cte2'")); + assertTrue(result.contains("Main Query")); + } + } + + @Test + public void testNestExplain2() throws SQLException { + String sql = + "explain with cte1 as materialized (select * from testtb), " + + "cte2 as materialized (select time, voltage from cte1) " + + "select * from cte2"; + try (Connection connection = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("USE testdb"); + + // explain + ResultSet resultSet = statement.executeQuery(sql); + ResultSetMetaData metaData = resultSet.getMetaData(); + assertEquals(metaData.getColumnCount(), 1); + assertEquals(metaData.getColumnName(1), "distribution plan"); + + StringBuilder sb = new StringBuilder(); + while (resultSet.next()) { + sb.append(resultSet.getString(1)).append(System.lineSeparator()); + } + String result = sb.toString(); + assertTrue(result.contains("CTE Query : 'cte1'")); + assertTrue(result.contains("CTE Query : 'cte2'")); + assertTrue(result.contains("Main Query")); + } + } + @Test public void testRecursive() { String sqlTemplate = @@ -395,7 +451,7 @@ public void testPrivileges() throws SQLException { } fail("No exception!"); } catch (Exception e) { - Assert.assertTrue( + assertTrue( e.getMessage(), e.getMessage() .contains( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java index 44f66f31476c..3394b1fbb58c 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java @@ -320,14 +320,19 @@ public void testExplainAnalyze() { assertPlan(logicalQueryPlan, output(explainAnalyze(cteScan))); } - /** This test primarily ensures code coverage for handleCteExplainResults method. */ + /** + * This test primarily ensures code coverage: materializeCTE.handleCteExplainResults & + * materializeCTE.fetchCteQueryResult + */ @Test public void testExplain() { - String sql = "with cte1 as materialized (SELECT time, s1 FROM table1) select * from cte1"; + String sql = + "with cte1 as (select time, s1 from table1), " + + "cte2 as materialized (select * from cte1) select * from cte2"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql, true); - PlanMatchPattern cteScan = cteScan("cte1", ImmutableList.of("time", "s1")); + PlanMatchPattern cteScan = cteScan("cte2", ImmutableList.of("time", "s1")); // Verify full LogicalPlan /* diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java new file mode 100644 index 000000000000..8b35a2df6def --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java @@ -0,0 +1,168 @@ +/* + * + * * 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; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.commons.schema.column.ColumnHeader; +import org.apache.iotdb.db.protocol.session.SessionManager; +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; +import org.apache.iotdb.db.queryengine.plan.Coordinator; +import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; +import org.apache.iotdb.db.queryengine.plan.execution.QueryExecution; +import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; +import org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.column.LongColumn; +import org.apache.tsfile.read.common.block.column.TimeColumn; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.List; +import java.util.Optional; + +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanAssert.assertPlan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.collect; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.exchange; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.output; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.tableScan; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression.Operator.EQUAL; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; + +@PowerMockIgnore({"com.sun.org.apache.xerces.*", "javax.xml.*", "org.xml.*", "javax.management.*"}) +@RunWith(PowerMockRunner.class) +@PrepareForTest({Coordinator.class, SessionManager.class}) +public class CteSubqueryTest { + private PlanTester planTester; + + @Before + public void setUp() throws Exception { + planTester = new PlanTester(); + mockExecuteForTableModel(); + } + + /** + * This test primarily ensures code coverage: + * PredicateWithUncorrelatedScalarSubqueryReconstructor.fetchUncorrelatedSubqueryResultForPredicate + */ + @Test + public void testCteSubquery() throws IoTDBException { + mockExecuteForTableModel(); + + String sql = + "with cte1 as (select time, s1 from table1) select s1 from table1 " + + "where s1 = (select s2 from cte1)"; + + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + PlanMatchPattern tableScan = + tableScan( + "testdb.table1", + ImmutableList.of("s1"), + ImmutableSet.of("s1"), + new ComparisonExpression(EQUAL, new SymbolReference("s1"), new LongLiteral("1"))); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──DeviceTableScanNode + */ + assertPlan(logicalQueryPlan, output(tableScan)); + + // Verify DistributionPlan + assertPlan(planTester.getFragmentPlan(0), output(collect(exchange(), exchange(), exchange()))); + + assertPlan(planTester.getFragmentPlan(1), tableScan); + assertPlan(planTester.getFragmentPlan(2), tableScan); + assertPlan(planTester.getFragmentPlan(3), tableScan); + } + + private void mockExecuteForTableModel() throws IoTDBException { + mockStatic(Coordinator.class); + mockStatic(SessionManager.class); + + // Create a mock Coordinator instance + Coordinator mockCoordinator = Mockito.mock(Coordinator.class); + when(Coordinator.getInstance()).thenReturn(mockCoordinator); + + // Create mock SessionManager + SessionManager mockSessionManager = Mockito.mock(SessionManager.class); + when(SessionManager.getInstance()).thenReturn(mockSessionManager); + + // Mock TSStatus with success status + TSStatus mockStatus = Mockito.mock(TSStatus.class); + when(mockStatus.getCode()).thenReturn(200); // Success status code + + // Create a real ExecutionResult instance + ExecutionResult mockResult = new ExecutionResult(new QueryId("1"), mockStatus); + + // Mock the executeForTableModel method + when(mockCoordinator.executeForTableModel( + Mockito.any(), // Statement + Mockito.any(), // SqlParser + Mockito.any(), // IClientSession + Mockito.anyLong(), // queryId + Mockito.any(), // SessionInfo + Mockito.anyString(), // String + Mockito.any(), // Metadata + Mockito.anyMap(), // Map, CteDataStore> + Mockito.any(), // ExplainType + Mockito.anyLong(), // timeOut + Mockito.anyBoolean())) // userQuery + .thenReturn(mockResult); + + // Create QueryExecution mock + QueryExecution mockQueryExecution = Mockito.mock(QueryExecution.class); + when(mockQueryExecution.hasNextResult()) + .thenReturn(true) // First call returns true + .thenReturn(false); // Subsequent calls return false + + // Create a real DatasetHeader with time and s1 columns + List columnHeaders = ImmutableList.of(new ColumnHeader("s2", TSDataType.INT64)); + DatasetHeader mockDatasetHeader = new DatasetHeader(columnHeaders, false); + when(mockQueryExecution.getDatasetHeader()).thenReturn(mockDatasetHeader); + + // Create a TSBlock with sample data for getBatchResult + + TimeColumn timeColumn = new TimeColumn(1, new long[] {1000L}); + LongColumn valueColumn = new LongColumn(1, Optional.empty(), new long[] {1L}); + TsBlock sampleTsBlock = new TsBlock(timeColumn, valueColumn); + when(mockQueryExecution.getBatchResult()).thenReturn(Optional.of(sampleTsBlock)); + + // Mock coordinator methods + when(mockCoordinator.getQueryExecution(Mockito.anyLong())).thenReturn(mockQueryExecution); + } +} From f66fc8859cff82cbecd48718d0dec8fbd68ff6b8 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 4 Dec 2025 14:35:07 +0800 Subject: [PATCH 51/69] add CteScanOperator test --- .../source/relational/CteScanOperator.java | 6 + .../operator/CteScanOperatorTest.java | 253 ++++++++++++++++++ 2 files changed, 259 insertions(+) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 44e22e300dbb..31059b217949 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.AbstractSourceOperator; @@ -122,4 +123,9 @@ private void prepareReader() { dataReader = new MemoryReader(dataStore.getCachedData()); } } + + @TestOnly + public int getDataStoreRefCount() { + return dataStoreRefCount; + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java new file mode 100644 index 000000000000..ed43c6aa119f --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java @@ -0,0 +1,253 @@ +/* + * + * * 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; + +import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.CteScanOperator; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; +import org.apache.iotdb.db.utils.cte.CteDataStore; + +import com.google.common.collect.ImmutableList; +import org.apache.tsfile.common.conf.TSFileDescriptor; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.BinaryColumnBuilder; +import org.apache.tsfile.read.common.block.column.DoubleColumnBuilder; +import org.apache.tsfile.read.common.block.column.TimeColumnBuilder; +import org.apache.tsfile.read.common.type.DoubleType; +import org.apache.tsfile.read.common.type.StringType; +import org.apache.tsfile.read.common.type.TimestampType; +import org.apache.tsfile.utils.Binary; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; + +public class CteScanOperatorTest { + private OperatorContext operatorContext; + private PlanNodeId planNodeId; + private CteDataStore cteDataStore; + private CteScanOperator cteScanOperator; + + @Before + public void setUp() { + // Set up mock objects + operatorContext = mock(OperatorContext.class); + planNodeId = new PlanNodeId("test-plan-node"); + + // Create a simple table schema for testing + TableSchema tableSchema = createTestTableSchema(); + + // Create mock query + Query mockQuery = mock(Query.class); + + // Create column index mapping + List columnIndex2TsBlockColumnIndexList = Arrays.asList(0, 1, 2); + + // Initialize CteDataStore + cteDataStore = new CteDataStore(mockQuery, tableSchema, columnIndex2TsBlockColumnIndexList); + + // Add test data to the data store + List testData = createTestTsBlocks(); + for (TsBlock tsBlock : testData) { + cteDataStore.addTsBlock(tsBlock); + } + } + + @After + public void tearDown() throws Exception { + if (cteScanOperator != null) { + cteScanOperator.close(); + } + } + + @Test + public void testConstructor() throws Exception { + cteScanOperator = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + assertEquals(1, cteScanOperator.getDataStoreRefCount()); + cteScanOperator.close(); + } + + @Test + public void testEmptyDataStore() throws Exception { + // Create empty data store + Query mockQuery = mock(Query.class); + TableSchema tableSchema = createTestTableSchema(); + CteDataStore emptyDataStore = new CteDataStore(mockQuery, tableSchema, Arrays.asList(0, 1, 2)); + + cteScanOperator = new CteScanOperator(operatorContext, planNodeId, emptyDataStore); + // Should not have data + assertFalse(cteScanOperator.hasNext()); + + cteScanOperator.close(); + } + + @Test + public void testNextWithData() throws Exception { + cteScanOperator = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + // Should have data + assertTrue(cteScanOperator.hasNext()); + TsBlock firstBlock = cteScanOperator.next(); + assertNotNull(firstBlock); + assertEquals(2, firstBlock.getValueColumnCount()); + assertEquals(3, firstBlock.getPositionCount()); + + // Should have data + assertTrue(cteScanOperator.hasNext()); + TsBlock secondBlock = cteScanOperator.next(); + assertNotNull(secondBlock); + assertEquals(2, secondBlock.getValueColumnCount()); + assertEquals(2, secondBlock.getPositionCount()); + + // should return null + TsBlock thirdBlock = cteScanOperator.next(); + assertNull(thirdBlock); + + cteScanOperator.close(); + } + + @Test + public void testIsFinished() throws Exception { + cteScanOperator = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + + // Initially not finished + assertFalse(cteScanOperator.isFinished()); + // Consume all data + while (cteScanOperator.hasNext()) { + cteScanOperator.next(); + } + // Now should be finished + assertTrue(cteScanOperator.isFinished()); + + cteScanOperator.close(); + } + + @Test + public void testMemory() throws Exception { + cteScanOperator = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + + // maxPeekMemory + maxReturnSize + retainedSize + long maxPeekMemory = cteScanOperator.calculateMaxPeekMemory(); + assertEquals( + TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(), maxPeekMemory); + long maxReturnSize = cteScanOperator.calculateMaxReturnSize(); + assertEquals( + TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(), maxReturnSize); + long retainedSize = cteScanOperator.calculateRetainedSizeAfterCallingNext(); + assertEquals(0L, retainedSize); + + cteScanOperator.close(); + } + + @Test + public void testMultipleCteScanOperators() throws Exception { + // Test reference counting with multiple operators + CteScanOperator operator1 = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + CteScanOperator operator2 = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + + // CteDataStore Reference count + assertEquals(1, operator1.getDataStoreRefCount()); + assertEquals(2, operator2.getDataStoreRefCount()); + + // Operator Memory + assertTrue(operator1.ramBytesUsed() > operator2.ramBytesUsed()); + + // Both operators should be able to read data + assertTrue(operator1.hasNext()); + assertTrue(operator2.hasNext()); + + // Clean up + operator1.close(); + operator2.close(); + } + + private TableSchema createTestTableSchema() { + List columnSchemas = new ArrayList<>(); + columnSchemas.add( + new ColumnSchema("time", TimestampType.TIMESTAMP, false, TsTableColumnCategory.TIME)); + columnSchemas.add( + new ColumnSchema("name", StringType.STRING, false, TsTableColumnCategory.FIELD)); + columnSchemas.add( + new ColumnSchema("value", DoubleType.DOUBLE, false, TsTableColumnCategory.FIELD)); + + return new TableSchema("test_table", columnSchemas); + } + + private List createTestTsBlocks() { + List blocks = new ArrayList<>(); + + // Create first TsBlock + blocks.add( + createTsBlock( + new long[] {1000L, 2000L, 3000L}, + new String[] {"Alice", "Bob", "Charlie"}, + new double[] {10.5, 20.3, 30.7})); + + // Create second TsBlock + blocks.add( + createTsBlock( + new long[] {4000L, 5000L}, new String[] {"David", "Eve"}, new double[] {40.2, 50.8})); + + return blocks; + } + + private TsBlock createTsBlock(long[] times, String[] names, double[] values) { + TsBlockBuilder builder = + new TsBlockBuilder(ImmutableList.of(TSDataType.STRING, TSDataType.DOUBLE)); + + // Time column + TimeColumnBuilder timeColumn = builder.getTimeColumnBuilder(); + for (long time : times) { + timeColumn.writeLong(time); + } + + // Name column + BinaryColumnBuilder nameColumn = (BinaryColumnBuilder) builder.getColumnBuilder(0); + for (String name : names) { + nameColumn.writeBinary(new Binary(name, StandardCharsets.UTF_8)); + } + + // Value column + DoubleColumnBuilder valueColumn = (DoubleColumnBuilder) builder.getColumnBuilder(1); + for (double value : values) { + valueColumn.writeDouble(value); + } + + builder.declarePositions(times.length); + return builder.build(); + } +} From ae9c504243eecf9385bc1ce17a376fd3ac8d4af9 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 4 Dec 2025 16:57:58 +0800 Subject: [PATCH 52/69] fix sonar issues --- .../plan/relational/analyzer/Scope.java | 2 +- .../analyzer/StatementAnalyzer.java | 4 +- .../relational/planner/RelationPlanner.java | 115 ++++++++++-------- 3 files changed, 67 insertions(+), 54 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java index a979b1f6c3ef..f780e29a79ed 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java @@ -82,7 +82,7 @@ public void addTable(Table table) { tables.add(new Identifier(table.getName().getSuffix())); } - public Scope clone() { + public Scope copy() { return builder().like(this).build(); } 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 949a839eb345..da3804d858b7 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 @@ -3598,9 +3598,9 @@ protected Scope visitJoin(Join node, Optional scope) { joinConditionCheck(criteria); - Optional leftScope = scope.map(Scope::clone); + Optional leftScope = scope.map(Scope::copy); Scope left = process(node.getLeft(), leftScope); - Optional rightScope = scope.map(Scope::clone); + Optional rightScope = scope.map(Scope::copy); Scope right = process(node.getRight(), rightScope); if (scope.isPresent()) { 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 4b7032e8b059..295239f09536 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 @@ -240,66 +240,86 @@ protected RelationPlan visitTable(final Table table, final Void context) { } final Scope scope = analysis.getScope(table); + final Query namedQuery = analysis.getNamedQuery(table); // Common Table Expression - final Query namedQuery = analysis.getNamedQuery(table); if (namedQuery != null) { - if (analysis.isExpandableQuery(namedQuery)) { - // recursive cte - throw new SemanticException("unexpected recursive cte"); + return processNamedQuery(table, namedQuery, scope); + } + + return processPhysicalTable(table, scope); + } + + private RelationPlan processNamedQuery(Table table, Query namedQuery, Scope scope) { + if (analysis.isExpandableQuery(namedQuery)) { + throw new SemanticException("unexpected recursive cte"); + } + + if (namedQuery.isMaterialized() && namedQuery.isDone()) { + RelationPlan materializedCtePlan = processMaterializedCte(table, scope); + if (materializedCtePlan != null) { + return materializedCtePlan; } + } - if (namedQuery.isMaterialized() && namedQuery.isDone()) { - CteDataStore dataStore = queryContext.getCteDataStore(table); - if (dataStore != null) { - List cteSymbols = - dataStore.getTableSchema().getColumns().stream() - .map(column -> symbolAllocator.newSymbol(column.getName(), column.getType())) - .collect(Collectors.toList()); - - // CTE Scan Node - CteScanNode cteScanNode = - new CteScanNode(idAllocator.genPlanNodeId(), table.getName(), cteSymbols, dataStore); - - List columnIndex2TsBlockColumnIndexList = - dataStore.getColumnIndex2TsBlockColumnIndexList(); - if (columnIndex2TsBlockColumnIndexList == null) { - return new RelationPlan(cteScanNode, scope, cteSymbols, outerContext); - } + return processRegularCte(table, namedQuery, scope); + } - List outputSymbols = new ArrayList<>(); - Assignments.Builder assignments = Assignments.builder(); - for (int index : columnIndex2TsBlockColumnIndexList) { - Symbol columnSymbol = cteSymbols.get(index); - outputSymbols.add(columnSymbol); - assignments.put(columnSymbol, columnSymbol.toSymbolReference()); - } + private RelationPlan processMaterializedCte(Table table, Scope scope) { + CteDataStore dataStore = queryContext.getCteDataStore(table); + if (dataStore == null) { + return null; + } - // Project Node - ProjectNode projectNode = - new ProjectNode( - queryContext.getQueryId().genPlanNodeId(), cteScanNode, assignments.build()); + List cteSymbols = + dataStore.getTableSchema().getColumns().stream() + .map(column -> symbolAllocator.newSymbol(column.getName(), column.getType())) + .collect(Collectors.toList()); - return new RelationPlan(projectNode, scope, outputSymbols, outerContext); - } - } + CteScanNode cteScanNode = + new CteScanNode(idAllocator.genPlanNodeId(), table.getName(), cteSymbols, dataStore); - RelationPlan subPlan = process(namedQuery, null); - // Add implicit coercions if view query produces types that don't match the declared output - // types of the view (e.g., if the underlying tables referenced by the view changed) - List types = - analysis.getOutputDescriptor(table).getAllFields().stream() - .map(Field::getType) - .collect(toImmutableList()); + List columnIndex2TsBlockColumnIndexList = + dataStore.getColumnIndex2TsBlockColumnIndexList(); + if (columnIndex2TsBlockColumnIndexList == null) { + return new RelationPlan(cteScanNode, scope, cteSymbols, outerContext); + } - NodeAndMappings coerced = coerce(subPlan, types, symbolAllocator, idAllocator); - return new RelationPlan(coerced.getNode(), scope, coerced.getFields(), outerContext); + List outputSymbols = new ArrayList<>(); + Assignments.Builder assignments = Assignments.builder(); + for (int index : columnIndex2TsBlockColumnIndexList) { + Symbol columnSymbol = cteSymbols.get(index); + outputSymbols.add(columnSymbol); + assignments.put(columnSymbol, columnSymbol.toSymbolReference()); } + // Project Node + ProjectNode projectNode = + new ProjectNode( + queryContext.getQueryId().genPlanNodeId(), cteScanNode, assignments.build()); + + return new RelationPlan(projectNode, scope, outputSymbols, outerContext); + } + + private RelationPlan processRegularCte(Table table, Query namedQuery, Scope scope) { + RelationPlan subPlan = process(namedQuery, null); + // Add implicit coercions if view query produces types that don't match the declared output + // types of the view (e.g., if the underlying tables referenced by the view changed) + List types = + analysis.getOutputDescriptor(table).getAllFields().stream() + .map(Field::getType) + .collect(toImmutableList()); + + NodeAndMappings coerced = coerce(subPlan, types, symbolAllocator, idAllocator); + return new RelationPlan(coerced.getNode(), scope, coerced.getFields(), outerContext); + } + + private RelationPlan processPhysicalTable(Table table, Scope scope) { final ImmutableList.Builder outputSymbolsBuilder = ImmutableList.builder(); final ImmutableMap.Builder symbolToColumnSchema = ImmutableMap.builder(); final Collection fields = scope.getRelationType().getAllFields(); final QualifiedName qualifiedName = analysis.getRelationName(table); + if (!qualifiedName.getPrefix().isPresent()) { throw new IllegalStateException("Table " + table.getName() + " has no prefix!"); } @@ -327,7 +347,6 @@ protected RelationPlan visitTable(final Table table, final Void context) { } final List outputSymbols = outputSymbolsBuilder.build(); - final Map tableColumnSchema = symbolToColumnSchema.build(); analysis.addTableSchema(qualifiedObjectName, tableColumnSchema); @@ -359,12 +378,6 @@ protected RelationPlan visitTable(final Table table, final Void context) { tableColumnSchema, tagAndAttributeIndexMap); return new RelationPlan(tableScanNode, scope, outputSymbols, outerContext); - - // Collection fields = analysis.getMaterializedViewStorageTableFields(node); - // Query namedQuery = analysis.getNamedQuery(node); - // Collection fields = analysis.getMaterializedViewStorageTableFields(node); - // plan = addRowFilters(node, plan); - // plan = addColumnMasks(node, plan); } @Override From 2201ebe4b970868a2c748956e606fcc22790dda9 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 5 Dec 2025 19:00:32 +0800 Subject: [PATCH 53/69] fix explain & explain analyze output --- .../apache/iotdb/db/queryengine/common/MPPQueryContext.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 d81da26dd2ca..748806c54830 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 @@ -47,6 +47,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -121,7 +122,7 @@ public class MPPQueryContext implements IAuditEntity { // Max line length of each CTE should be remembered because we need to standardize // the output format of main query and CTE query. private final Map, Pair>> cteExplainResults = - new HashMap<>(); + new LinkedHashMap<>(); // Do not release CTE query result if it is a subquery. private boolean subquery = false; From 3e2b7bd83ccb1e21072467a5717ec2e35834f57e Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 8 Dec 2025 17:50:45 +0800 Subject: [PATCH 54/69] add shadow expression for ComparisonExpression --- .../plan/relational/planner/QueryPlanner.java | 11 +++++-- ...correlatedScalarSubqueryReconstructor.java | 32 +++++++++++++----- .../sql/ast/ComparisonExpression.java | 22 +++++++++++-- .../planner/UncorrelatedSubqueryTest.java | 33 +++++++++++++++++++ 4 files changed, 85 insertions(+), 13 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/QueryPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/QueryPlanner.java index 106b406257f6..ad22504b826c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/QueryPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/QueryPlanner.java @@ -30,6 +30,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.RelationType; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.GapFillStartAndEndTimeExtractVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.PredicateWithUncorrelatedScalarSubqueryReconstructor; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode.Aggregation; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; @@ -763,9 +764,13 @@ private PlanBuilder filter(PlanBuilder subPlan, Expression predicate, Node node) } subPlan = subqueryPlanner.handleSubqueries(subPlan, predicate, analysis.getSubqueries(node)); - return subPlan.withNewRoot( - new FilterNode( - queryIdAllocator.genPlanNodeId(), subPlan.getRoot(), subPlan.rewrite(predicate))); + PlanBuilder planBuilder = + subPlan.withNewRoot( + new FilterNode( + queryIdAllocator.genPlanNodeId(), subPlan.getRoot(), subPlan.rewrite(predicate))); + PredicateWithUncorrelatedScalarSubqueryReconstructor.getInstance() + .clearShadowExpression(predicate); + return planBuilder; } private PlanBuilder aggregate(PlanBuilder subPlan, QuerySpecification node) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 1e6d3a276f38..e57db82de3fd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -34,6 +34,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DoubleLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression; @@ -79,21 +80,19 @@ public void reconstructPredicateWithUncorrelatedScalarSubquery( ComparisonExpression comparisonExpression = (ComparisonExpression) expression; Expression left = comparisonExpression.getLeft(); Expression right = comparisonExpression.getRight(); - if (left instanceof Identifier && right instanceof SubqueryExpression) { + if ((left instanceof Identifier || left instanceof FunctionCall) + && right instanceof SubqueryExpression) { Optional result = fetchUncorrelatedSubqueryResultForPredicate( context, (SubqueryExpression) right, analysis.getWith()); // If the subquery result is not present, we cannot reconstruct the predicate. - if (result.isPresent()) { - right = result.get(); - } - } else if (right instanceof Identifier && left instanceof SubqueryExpression) { + result.ifPresent(comparisonExpression::setShadowRight); + } else if ((right instanceof Identifier || right instanceof FunctionCall) + && left instanceof SubqueryExpression) { Optional result = fetchUncorrelatedSubqueryResultForPredicate( context, (SubqueryExpression) left, analysis.getWith()); - if (result.isPresent()) { - left = result.get(); - } + result.ifPresent(comparisonExpression::setShadowLeft); } comparisonExpression.setLeft(left); comparisonExpression.setRight(right); @@ -215,6 +214,23 @@ public Optional fetchUncorrelatedSubqueryResultForPredicate( return Optional.empty(); } + public void clearShadowExpression(Expression expression) { + if (expression instanceof LogicalExpression) { + LogicalExpression logicalExpression = (LogicalExpression) expression; + for (Expression term : logicalExpression.getTerms()) { + clearShadowExpression(term); + } + } else if (expression instanceof NotExpression) { + NotExpression notExpression = (NotExpression) expression; + clearShadowExpression(notExpression.getValue()); + } else if (expression instanceof ComparisonExpression) { + ComparisonExpression comparisonExpression = (ComparisonExpression) expression; + comparisonExpression.clearShadow(); + clearShadowExpression(comparisonExpression.getLeft()); + clearShadowExpression(comparisonExpression.getRight()); + } + } + private static class PredicateWithUncorrelatedScalarSubqueryReconstructorHolder { private static PredicateWithUncorrelatedScalarSubqueryReconstructor INSTANCE = new PredicateWithUncorrelatedScalarSubqueryReconstructor(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java index 65168dc7e915..891a455cd514 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java @@ -101,7 +101,9 @@ public Operator negate() { private final Operator operator; private Expression left; + private Expression shadowLeft; private Expression right; + private Expression shadowRight; public ComparisonExpression(Operator operator, Expression left, Expression right) { super(null); @@ -131,11 +133,11 @@ public Operator getOperator() { } public Expression getLeft() { - return left; + return shadowLeft != null ? shadowLeft : left; } public Expression getRight() { - return right; + return shadowRight != null ? shadowRight : right; } public void setLeft(Expression left) { @@ -156,6 +158,22 @@ public List getChildren() { return ImmutableList.of(left, right); } + // set by unfold of subquery + public void setShadowLeft(Expression shadowLeft) { + this.shadowLeft = shadowLeft; + } + + // set by unfold of subquery + public void setShadowRight(Expression shadowRight) { + this.shadowRight = shadowRight; + } + + // called after the stage is finished + public void clearShadow() { + this.shadowLeft = null; + this.shadowRight = null; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java index 8cddd6fe3158..65231a9f5e21 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java @@ -500,4 +500,37 @@ public void testUncorrelatedNotExistsSubquery() { SINGLE, tableScan2))))))); } + + @Test + public void testUncorrelatedHavingSubquery() { + String sql = + "SELECT min(time) as min FROM table1 group by s1 having min(time) > (select max(time) from table2)"; + LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + + PlanMatchPattern tableScan = + tableScan("testdb.table1", ImmutableList.of("time", "s1"), ImmutableSet.of("time", "s1")); + PlanMatchPattern agg = + aggregation( + singleGroupingSet("s1"), + ImmutableMap.of( + Optional.of("min"), aggregationFunction("min", ImmutableList.of("time"))), + ImmutableList.of(), + Optional.empty(), + SINGLE, + tableScan); + + Expression filterPredicate = + new ComparisonExpression(GREATER_THAN, new SymbolReference("min"), new LongLiteral("1")); + + // Verify full LogicalPlan + /* + * └──OutputNode + * └──FilterNode + * ├──ProjectNode + * └──Aggregation + * └──TableScanNode + */ + + assertPlan(logicalQueryPlan, output(filter(filterPredicate, project(agg)))); + } } From 936fcce5578eedc673dfbc11a7aa39e8bd6aaeec Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 9 Dec 2025 22:45:01 +0800 Subject: [PATCH 55/69] fix: sorted original indices --- .../it/query/recent/IoTDBCteIT.java | 68 ++++++++++--------- .../analyzer/StatementAnalyzer.java | 36 +++++++++- .../relational/planner/CteMaterializer.java | 34 +++++++++- 3 files changed, 101 insertions(+), 37 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java index d9443bf66840..13a2b0dd28f2 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBCteIT.java @@ -71,10 +71,10 @@ public class IoTDBCteIT { new String[] { "CREATE DATABASE IF NOT EXISTS testdb", "USE testdb", - "CREATE TABLE IF NOT EXISTS testtb(deviceid STRING TAG, voltage FLOAT FIELD)", - "INSERT INTO testtb VALUES(1000, 'd1', 100.0)", - "INSERT INTO testtb VALUES(2000, 'd1', 200.0)", - "INSERT INTO testtb VALUES(1000, 'd2', 300.0)", + "CREATE TABLE IF NOT EXISTS testtb(voltage FLOAT FIELD, manufacturer STRING FIELD, deviceid STRING TAG)", + "INSERT INTO testtb VALUES(1000, 100.0, 'a', 'd1')", + "INSERT INTO testtb VALUES(2000, 200.0, 'b', 'd1')", + "INSERT INTO testtb VALUES(1000, 300.0, 'c', 'd2')", }; private static final String dropDbSqls = "DROP DATABASE IF EXISTS testdb"; @@ -118,10 +118,10 @@ public void testMultipleWith() { String mainQuery = "select * from cte1 where voltage > " + "(with cte2 as materialized (select avg(voltage) as avg_voltage from testtb) select avg_voltage from cte2)"; - String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; + String[] expectedHeader = new String[] {"time", "voltage", "manufacturer", "deviceid"}; String[] retArray = new String[] { - "1970-01-01T00:00:01.000Z,d2,300.0,", + "1970-01-01T00:00:01.000Z,300.0,c,d2,", }; String[] cteTemplateQueries = new String[] {"cte1 as %s (select * from testtb)"}; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); @@ -131,18 +131,18 @@ public void testMultipleWith() { public void testFilterQuery() { // case 1 String mainQuery = "select * from cte where time > 1000 order by deviceid"; - String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; + String[] expectedHeader = new String[] {"time", "voltage", "manufacturer", "deviceid"}; String[] retArray = new String[] { - "1970-01-01T00:00:02.000Z,d1,200.0,", + "1970-01-01T00:00:02.000Z,200.0,b,d1,", }; String[] cteTemplateQueries = new String[] {"cte as %s (select * from testtb)"}; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); // case 2 mainQuery = "select * from cte where voltage > 200 order by deviceid"; - expectedHeader = new String[] {"time", "deviceid", "voltage"}; - retArray = new String[] {"1970-01-01T00:00:01.000Z,d2,300.0,"}; + expectedHeader = new String[] {"time", "voltage", "manufacturer", "deviceid"}; + retArray = new String[] {"1970-01-01T00:00:01.000Z,300.0,c,d2,"}; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); } @@ -150,12 +150,12 @@ public void testFilterQuery() { public void testSortQuery() { final String mainQuery = "select * from cte order by deviceid, voltage desc"; - String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; + String[] expectedHeader = new String[] {"time", "voltage", "manufacturer", "deviceid"}; String[] retArray = new String[] { - "1970-01-01T00:00:02.000Z,d1,200.0,", - "1970-01-01T00:00:01.000Z,d1,100.0,", - "1970-01-01T00:00:01.000Z,d2,300.0," + "1970-01-01T00:00:02.000Z,200.0,b,d1,", + "1970-01-01T00:00:01.000Z,100.0,a,d1,", + "1970-01-01T00:00:01.000Z,300.0,c,d2," }; String[] cteTemplateQueries = new String[] {"cte as %s (select * from testtb)"}; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); @@ -165,10 +165,10 @@ public void testSortQuery() { public void testLimitOffsetQuery() { final String mainQuery = "select * from cte limit 1 offset 1"; - String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; + String[] expectedHeader = new String[] {"time", "voltage", "manufacturer", "deviceid"}; String[] retArray = new String[] { - "1970-01-01T00:00:02.000Z,d1,200.0,", + "1970-01-01T00:00:02.000Z,200.0,b,d1,", }; String[] cteTemplateQueries = new String[] {"cte as %s (select * from testtb where deviceid = 'd1') "}; @@ -248,8 +248,8 @@ public void testExplain() throws SQLException { @Test public void testMultiReference() { - String[] expectedHeader = new String[] {"time", "deviceid", "voltage"}; - String[] retArray = new String[] {"1970-01-01T00:00:01.000Z,d2,300.0,"}; + String[] expectedHeader = new String[] {"time", "voltage", "manufacturer", "deviceid"}; + String[] retArray = new String[] {"1970-01-01T00:00:01.000Z,300.0,c,d2,"}; String[] cteTemplateQueries = new String[] {"cte as %s (select * from testtb)"}; String mainQuery = "select * from cte where voltage > (select avg(voltage) from cte)"; testCteSuccessWithVariants(cteTemplateQueries, mainQuery, expectedHeader, retArray); @@ -280,10 +280,11 @@ public void testSession() throws IoTDBConnectionException, StatementExecutionExc session.executeQueryStatement( String.format("with cte as %s (select * from testtb) select * from cte", keyword)); - assertEquals(dataSet.getColumnNames().size(), 3); + assertEquals(dataSet.getColumnNames().size(), 4); assertEquals(dataSet.getColumnNames().get(0), "time"); - assertEquals(dataSet.getColumnNames().get(1), "deviceid"); - assertEquals(dataSet.getColumnNames().get(2), "voltage"); + assertEquals(dataSet.getColumnNames().get(1), "voltage"); + assertEquals(dataSet.getColumnNames().get(2), "manufacturer"); + assertEquals(dataSet.getColumnNames().get(3), "deviceid"); int cnt = 0; while (dataSet.hasNext()) { dataSet.next(); @@ -310,10 +311,11 @@ public void testJdbc() throws ClassNotFoundException, SQLException { String.format("with cte as %s (select * from testtb) select * from cte", keyword)); final ResultSetMetaData metaData = resultSet.getMetaData(); - assertEquals(metaData.getColumnCount(), 3); + assertEquals(metaData.getColumnCount(), 4); assertEquals(metaData.getColumnLabel(1), "time"); - assertEquals(metaData.getColumnLabel(2), "deviceid"); - assertEquals(metaData.getColumnLabel(3), "voltage"); + assertEquals(metaData.getColumnLabel(2), "voltage"); + assertEquals(metaData.getColumnLabel(3), "manufacturer"); + assertEquals(metaData.getColumnLabel(4), "deviceid"); int cnt = 0; while (resultSet.next()) { @@ -326,12 +328,12 @@ public void testJdbc() throws ClassNotFoundException, SQLException { @Test public void testNest() { - final String mainQuery = "SELECT * FROM cte2"; + final String mainQuery = "select * from cte2"; String[] cteTemplateQueries = new String[] { "cte1 as %s (select deviceid, voltage from testtb where voltage > 200)", - "cte2 as %s (SELECT voltage FROM cte1)" + "cte2 as %s (select voltage from cte1)" }; String[] expectedHeader = new String[] {"voltage"}; String[] retArray = new String[] {"300.0,"}; @@ -339,7 +341,7 @@ public void testNest() { cteTemplateQueries = new String[] { - "cte2 as %s (SELECT voltage FROM cte1)", + "cte2 as %s (select voltage from cte1)", "cte1 as %s (select deviceid, voltage from testtb where voltage > 200)" }; String errMsg = "550: Table 'testdb.cte1' does not exist."; @@ -406,8 +408,8 @@ public void testRecursive() { "WITH RECURSIVE t(n) AS %s (" + " VALUES (1)" + " UNION ALL" - + " SELECT n+1 FROM t WHERE n < 100)" - + " SELECT sum(n) FROM t"; + + " select n+1 from t WHERE n < 100)" + + " select sum(n) from t"; for (String keyword : cteKeywords) { tableAssertTestFail( @@ -426,7 +428,7 @@ public void testPrivileges() throws SQLException { adminStmt.execute("USE testdb"); adminStmt.execute( "CREATE TABLE IF NOT EXISTS testtb1(deviceid STRING TAG, voltage FLOAT FIELD)"); - adminStmt.execute("GRANT SELECT ON testdb.testtb TO USER tmpuser"); + adminStmt.execute("GRANT select ON testdb.testtb TO USER tmpuser"); try (Connection connection = EnvFactory.getEnv() @@ -496,13 +498,13 @@ public void testConcurrentCteQueries() throws Exception { // Test different types of CTE queries String[] queries = { String.format( - "WITH cte as %s (SELECT * FROM testtb WHERE voltage > 150) SELECT * FROM cte ORDER BY deviceid", + "WITH cte as %s (select * from testtb WHERE voltage > 150) select * from cte ORDER BY deviceid", cteKeywords[j % cteKeywords.length]), String.format( - "WITH cte as %s (SELECT deviceid, avg(voltage) as avg_v FROM testtb GROUP BY deviceid) SELECT * FROM cte", + "WITH cte as %s (select deviceid, avg(voltage) as avg_v from testtb GROUP BY deviceid) select * from cte", cteKeywords[j % cteKeywords.length]), String.format( - "WITH cte as %s (SELECT * FROM testtb WHERE time > 1000) SELECT count(*) as cnt FROM cte", + "WITH cte as %s (select * from testtb WHERE time > 1000) select count(*) as cnt from cte", cteKeywords[j % cteKeywords.length]) }; 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 da3804d858b7..8be2a727a23f 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 @@ -3102,8 +3102,29 @@ protected Scope visitTable(Table table, Optional scope) { // check if table schema is found in CTE data stores CteDataStore dataStore = queryContext.getCteDataStore(table); - Optional tableSchema = - dataStore != null ? Optional.of(dataStore.getTableSchema()) : Optional.empty(); + Optional tableSchema = Optional.empty(); + if (dataStore != null) { + tableSchema = Optional.of(dataStore.getTableSchema()); + List columnIndex2TsBlockColumnIndexList = + dataStore.getColumnIndex2TsBlockColumnIndexList(); + if (columnIndex2TsBlockColumnIndexList != null + && !columnIndex2TsBlockColumnIndexList.isEmpty()) { + // Check if the list is completely sequential (0, 1, 2, ...) + boolean isSequential = true; + for (int i = 0; i < columnIndex2TsBlockColumnIndexList.size(); i++) { + if (columnIndex2TsBlockColumnIndexList.get(i) != i) { + isSequential = false; + break; + } + } + + // Generate new TableSchema with reordered columns only if not sequential + if (!isSequential) { + tableSchema = + reorderTableSchemaColumns(tableSchema.get(), columnIndex2TsBlockColumnIndexList); + } + } + } // If table schema is not found, check if it is in metadata if (!tableSchema.isPresent()) { tableSchema = metadata.getTableSchema(sessionContext, name); @@ -3127,6 +3148,17 @@ protected Scope visitTable(Table table, Optional scope) { return createAndAssignScope(table, scope, relationType); } + private Optional reorderTableSchemaColumns( + TableSchema tableSchema, List columnIndex2TsBlockColumnIndexList) { + List columnSchemas = tableSchema.getColumns(); + final List columnSchemaList = + columnIndex2TsBlockColumnIndexList.stream() + .map(columnSchemas::get) + .collect(Collectors.toList()); + + return Optional.of(new TableSchema(tableSchema.getTableName(), columnSchemaList)); + } + private Scope createScopeForCommonTableExpression( Table table, Optional scope, WithQuery withQuery) { Query query = withQuery.getQuery(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 4807c7526b54..e6ee347f691f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -63,6 +63,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -225,9 +226,14 @@ private TableSchema getTableSchema(DatasetHeader datasetHeader, String cteName) columnIndex2TsBlockColumnIndexList = IntStream.range(0, columnNames.size()).boxed().collect(Collectors.toList()); } - // build column schema list of cte table based on column2BlockColumnIndex + + // Get original column indices in the TsBlock + List tsBlockColumnIndices = + adjustColumnIndexMapping(columnIndex2TsBlockColumnIndexList); + + // build column schema list of cte table based on sorted original indices final List columnSchemaList = - columnIndex2TsBlockColumnIndexList.stream() + tsBlockColumnIndices.stream() .map( index -> new ColumnSchema( @@ -239,6 +245,30 @@ private TableSchema getTableSchema(DatasetHeader datasetHeader, String cteName) return new TableSchema(cteName, columnSchemaList); } + /** + * Adjust column index mapping by sorting and preserving original indices. For example, if input + * is {0, 3, 1, 2}, the output will be {0, 2, 3, 1}. This method doesn't modify the original list. + * + * @param originalIndexList original column index list + * @return adjusted column index list with sorted values preserving original positions + */ + private List adjustColumnIndexMapping(List originalIndexList) { + if (originalIndexList == null || originalIndexList.isEmpty()) { + return originalIndexList; + } + + // Create LinkedHashMap to maintain value-position mapping + Map valueToPositionMap = new LinkedHashMap<>(); + IntStream.range(0, originalIndexList.size()) + .forEach(i -> valueToPositionMap.put(originalIndexList.get(i), i)); + + // Sort by key (value) and collect positions in sorted order + return valueToPositionMap.entrySet().stream() + .sorted(Map.Entry.comparingByKey()) + .map(Map.Entry::getValue) + .collect(Collectors.toList()); + } + private List getCteExplainAnalyzeLines( FragmentInstanceStatisticsDrawer fragmentInstanceStatisticsDrawer, List instances, From 0f328127d7480142885aaabad99dc7223464068f Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 10 Dec 2025 14:43:26 +0800 Subject: [PATCH 56/69] cteDataStores -> cteQueries --- .../queryengine/common/MPPQueryContext.java | 20 ++++++---- .../db/queryengine/plan/Coordinator.java | 6 +-- .../plan/relational/analyzer/Analyzer.java | 6 +-- .../relational/planner/CteMaterializer.java | 38 ++++++++++--------- ...correlatedScalarSubqueryReconstructor.java | 2 +- .../plan/relational/sql/ast/Query.java | 15 +++++--- .../iotdb/db/utils/cte/CteDataStore.java | 10 +---- .../operator/CteScanOperatorTest.java | 9 +---- 8 files changed, 52 insertions(+), 54 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 748806c54830..53e6b3a334d6 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 @@ -117,7 +117,7 @@ public class MPPQueryContext implements IAuditEntity { private boolean userQuery = false; private final Map, Long> cteMaterializationCosts = new HashMap<>(); - private Map, CteDataStore> cteDataStores = new HashMap<>(); + private Map, Query> cteQueries = new HashMap<>(); // table -> (maxLineLength, 'explain' or 'explain analyze' result) // Max line length of each CTE should be remembered because we need to standardize // the output format of main query and CTE query. @@ -493,20 +493,24 @@ public Map, Long> getCteMaterializationCosts() { return cteMaterializationCosts; } - public void addCteDataStore(Table table, CteDataStore dataStore) { - cteDataStores.put(NodeRef.of(table), dataStore); + public void addCteQuery(Table table, Query query) { + cteQueries.put(NodeRef.of(table), query); } - public Map, CteDataStore> getCteDataStores() { - return cteDataStores; + public Map, Query> getCteQueries() { + return cteQueries; } public CteDataStore getCteDataStore(Table table) { - return cteDataStores.get(NodeRef.of(table)); + Query query = cteQueries.get(NodeRef.of(table)); + if (query == null) { + return null; + } + return query.getCteDataStore(); } - public void setCteDataStores(Map, CteDataStore> cteDataStores) { - this.cteDataStores = cteDataStores; + public void setCteQueries(Map, Query> cteQueries) { + this.cteQueries = cteQueries; } public Map> getSubQueryTables() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index afd719739fd9..f7906cf19b35 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -93,6 +93,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.MigrateRegion; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Parameter; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.PipeStatement; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Prepare; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ReconstructRegion; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.RelationalAuthorStatement; @@ -143,7 +144,6 @@ import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement; import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.utils.SetThreadName; -import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.thrift.TBase; import org.apache.tsfile.utils.Accountable; @@ -418,7 +418,7 @@ public ExecutionResult executeForTableModel( SessionInfo session, String sql, Metadata metadata, - Map, CteDataStore> cteDataStoreMap, + Map, Query> cteQueries, ExplainType explainType, long timeOut, boolean userQuery) { @@ -429,7 +429,7 @@ public ExecutionResult executeForTableModel( userQuery, ((queryContext, startTime) -> { queryContext.setSubquery(true); - queryContext.setCteDataStores(cteDataStoreMap); + queryContext.setCteQueries(cteQueries); queryContext.setExplainType(explainType); return createQueryExecutionForTableModel( statement, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java index 783a94e848d9..43c993fdeccc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java @@ -82,10 +82,8 @@ public Analysis analyze(Statement statement) { Analysis analysis = new Analysis(rewrittenStatement, parameterLookup); // Register CTE passed by parent query. context - .getCteDataStores() - .forEach( - (tableRef, dataStore) -> - analysis.registerNamedQuery(tableRef.getNode(), dataStore.getQuery())); + .getCteQueries() + .forEach((tableRef, query) -> analysis.registerNamedQuery(tableRef.getNode(), query)); Statement innerStatement = rewrittenStatement instanceof PipeEnriched diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index e6ee347f691f..4857983857b3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -82,26 +82,31 @@ public void materializeCTE(Analysis analysis, MPPQueryContext context) { .forEach( (tableRef, query) -> { Table table = tableRef.getNode(); - if (query.isMaterialized() && !query.isDone()) { - CteDataStore dataStore = - fetchCteQueryResult(context, table, query, analysis.getWith()); - if (dataStore == null) { - // CTE query execution failed. Use inline instead of materialization - // in the outer query - query.setDone(false); - return; + if (query.isMaterialized()) { + if (!query.isDone()) { + CteDataStore dataStore = + fetchCteQueryResult(context, table, query, analysis.getWith()); + if (dataStore == null) { + // CTE query execution failed. Use inline instead of materialization + // in the outer query + query.setCteDataStore(null); + return; + } + query.setCteDataStore(dataStore); } - - context.addCteDataStore(table, dataStore); - query.setDone(true); + context.addCteQuery(table, query); } }); } public void cleanUpCTE(MPPQueryContext context) { - Map, CteDataStore> cteDataStores = context.getCteDataStores(); - cteDataStores.values().forEach(CteDataStore::clear); - cteDataStores.clear(); + Map, Query> cteQueries = context.getCteQueries(); + cteQueries.values().stream() + .map(Query::getCteDataStore) + .filter(java.util.Objects::nonNull) + .distinct() + .forEach(CteDataStore::clear); + cteQueries.clear(); } public CteDataStore fetchCteQueryResult( @@ -145,7 +150,7 @@ public CteDataStore fetchCteQueryResult( sessionManager.getSessionInfoOfTableModel(sessionManager.getCurrSession()), String.format("Materialize query for CTE '%s'", table.getName()), LocalExecutionPlanner.getInstance().metadata, - context.getCteDataStores(), + context.getCteQueries(), context.getExplainType(), context.getTimeOut(), false); @@ -160,8 +165,7 @@ public CteDataStore fetchCteQueryResult( TableSchema tableSchema = getTableSchema(datasetHeader, table.getName().toString()); cteDataStore = - new CteDataStore( - query, tableSchema, datasetHeader.getColumnIndex2TsBlockColumnIndexList()); + new CteDataStore(tableSchema, datasetHeader.getColumnIndex2TsBlockColumnIndexList()); while (execution.hasNextResult()) { final Optional tsBlock; try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index e57db82de3fd..b8ffbe6b3872 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -146,7 +146,7 @@ public Optional fetchUncorrelatedSubqueryResultForPredicate( .getSessionInfoOfTableModel(SessionManager.getInstance().getCurrSession()), "Try to Fetch Uncorrelated Scalar Subquery Result for Predicate", LocalExecutionPlanner.getInstance().metadata, - context.getCteDataStores(), + context.getCteQueries(), ExplainType.NONE, context.getTimeOut(), false); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java index 22d3994262b2..7ef8e56ceb2f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.queryengine.plan.relational.sql.ast; +import org.apache.iotdb.db.utils.cte.CteDataStore; + import com.google.common.collect.ImmutableList; import org.apache.tsfile.utils.RamUsageEstimator; @@ -42,8 +44,7 @@ public class Query extends Statement { private final Optional limit; // whether this query needs materialization private boolean materialized = false; - // whether materialization is done - private boolean done = false; + private CteDataStore cteDataStore = null; public Query( Optional with, @@ -115,11 +116,15 @@ public void setMaterialized(boolean materialized) { } public boolean isDone() { - return done; + return cteDataStore != null; + } + + public void setCteDataStore(CteDataStore cteDataStore) { + this.cteDataStore = cteDataStore; } - public void setDone(boolean done) { - this.done = done; + public CteDataStore getCteDataStore() { + return this.cteDataStore; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index 8ff79a3ca3dc..613f6f87fa55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -24,7 +24,6 @@ import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.tsfile.read.common.block.TsBlock; @@ -33,7 +32,6 @@ import java.util.concurrent.atomic.AtomicInteger; public class CteDataStore { - private final Query query; private final TableSchema tableSchema; private final List columnIndex2TsBlockColumnIndexList; @@ -44,9 +42,7 @@ public class CteDataStore { // reference count by CteScanOperator private final AtomicInteger count; - public CteDataStore( - Query query, TableSchema tableSchema, List columnIndex2TsBlockColumnIndexList) { - this.query = query; + public CteDataStore(TableSchema tableSchema, List columnIndex2TsBlockColumnIndexList) { this.tableSchema = tableSchema; this.columnIndex2TsBlockColumnIndexList = columnIndex2TsBlockColumnIndexList; this.cachedData = new ArrayList<>(); @@ -87,10 +83,6 @@ public TableSchema getTableSchema() { return tableSchema; } - public Query getQuery() { - return query; - } - public List getColumnIndex2TsBlockColumnIndexList() { return columnIndex2TsBlockColumnIndexList; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java index ed43c6aa119f..0f44f68b8cbd 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java @@ -26,7 +26,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.collect.ImmutableList; @@ -72,14 +71,11 @@ public void setUp() { // Create a simple table schema for testing TableSchema tableSchema = createTestTableSchema(); - // Create mock query - Query mockQuery = mock(Query.class); - // Create column index mapping List columnIndex2TsBlockColumnIndexList = Arrays.asList(0, 1, 2); // Initialize CteDataStore - cteDataStore = new CteDataStore(mockQuery, tableSchema, columnIndex2TsBlockColumnIndexList); + cteDataStore = new CteDataStore(tableSchema, columnIndex2TsBlockColumnIndexList); // Add test data to the data store List testData = createTestTsBlocks(); @@ -105,9 +101,8 @@ public void testConstructor() throws Exception { @Test public void testEmptyDataStore() throws Exception { // Create empty data store - Query mockQuery = mock(Query.class); TableSchema tableSchema = createTestTableSchema(); - CteDataStore emptyDataStore = new CteDataStore(mockQuery, tableSchema, Arrays.asList(0, 1, 2)); + CteDataStore emptyDataStore = new CteDataStore(tableSchema, Arrays.asList(0, 1, 2)); cteScanOperator = new CteScanOperator(operatorContext, planNodeId, emptyDataStore); // Should not have data From d390f525e94286727c90579800ee29643376d2d0 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 10 Dec 2025 16:20:49 +0800 Subject: [PATCH 57/69] remove cleanUpCTE --- .../db/queryengine/plan/execution/QueryExecution.java | 7 ------- .../plan/relational/planner/CteMaterializer.java | 11 ----------- .../plan/relational/planner/CteSubqueryTest.java | 2 +- 3 files changed, 1 insertion(+), 19 deletions(-) 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 7ffc9dc1f7f1..4734db5850a4 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 @@ -47,7 +47,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeUtil; -import org.apache.iotdb.db.queryengine.plan.relational.planner.CteMaterializer; import org.apache.iotdb.db.queryengine.plan.scheduler.IScheduler; import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId; @@ -350,9 +349,6 @@ private void releaseResource() { resultHandle.close(); cleanUpResultHandle(); } - if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) && !context.isSubquery()) { - CteMaterializer.getInstance().cleanUpCTE(context); - } } private void cleanUpResultHandle() { @@ -396,9 +392,6 @@ private void releaseResource(Throwable t) { } cleanUpResultHandle(); } - if (getSQLDialect().equals(IClientSession.SqlDialect.TABLE) && !context.isSubquery()) { - CteMaterializer.getInstance().cleanUpCTE(context); - } } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 4857983857b3..ffc551d591c5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -39,7 +39,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.DistributedQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; -import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; @@ -99,16 +98,6 @@ public void materializeCTE(Analysis analysis, MPPQueryContext context) { }); } - public void cleanUpCTE(MPPQueryContext context) { - Map, Query> cteQueries = context.getCteQueries(); - cteQueries.values().stream() - .map(Query::getCteDataStore) - .filter(java.util.Objects::nonNull) - .distinct() - .forEach(CteDataStore::clear); - cteQueries.clear(); - } - public CteDataStore fetchCteQueryResult( MPPQueryContext context, Table table, Query query, With with) { final long queryId = sessionManager.requestQueryId(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java index 8b35a2df6def..59fbc1a1f8d9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java @@ -83,7 +83,7 @@ public void testCteSubquery() throws IoTDBException { mockExecuteForTableModel(); String sql = - "with cte1 as (select time, s1 from table1) select s1 from table1 " + "with cte1 as (select time, s2 from table1) select s1 from table1 " + "where s1 = (select s2 from cte1)"; LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); From fcff59578082bd5e8dc70571f2864f138a098e7e Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 11 Dec 2025 11:21:31 +0800 Subject: [PATCH 58/69] simplify process --- .../operator/source/relational/CteScanOperator.java | 7 ++----- .../apache/iotdb/db/queryengine/plan/Coordinator.java | 3 +-- .../plan/relational/planner/CteMaterializer.java | 11 +++++++++++ .../plan/relational/planner/RelationPlanner.java | 6 +++--- .../org/apache/iotdb/db/utils/cte/CteDataStore.java | 2 +- .../execution/operator/CteScanOperatorTest.java | 8 +++----- 6 files changed, 21 insertions(+), 16 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 31059b217949..1d5a4ba95ca7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -30,7 +30,6 @@ import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.iotdb.db.utils.cte.MemoryReader; -import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.utils.RamUsageEstimator; import org.slf4j.Logger; @@ -45,9 +44,6 @@ public class CteScanOperator extends AbstractSourceOperator { private final int dataStoreRefCount; private CteDataReader dataReader; - private final long maxReturnSize = - TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); - public CteScanOperator( OperatorContext operatorContext, PlanNodeId sourceId, CteDataStore dataStore) { this.operatorContext = operatorContext; @@ -96,7 +92,8 @@ public long calculateMaxPeekMemory() { @Override public long calculateMaxReturnSize() { - return maxReturnSize; + // The returned object is a reference to TsBlock in CteDataReader + return RamUsageEstimator.NUM_BYTES_OBJECT_REF; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index f7906cf19b35..5a35195be013 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -93,8 +93,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.MigrateRegion; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Parameter; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.PipeStatement; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Prepare; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ReconstructRegion; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.RelationalAuthorStatement; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.RemoveAINode; @@ -160,7 +160,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.BlockingDeque; -import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index ffc551d591c5..99a06cff2eeb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -250,6 +250,17 @@ private List adjustColumnIndexMapping(List originalIndexList) return originalIndexList; } + boolean isSequential = true; + for (int i = 0; i < originalIndexList.size(); i++) { + if (originalIndexList.get(i) != i) { + isSequential = false; + break; + } + } + if (isSequential) { + return originalIndexList; + } + // Create LinkedHashMap to maintain value-position mapping Map valueToPositionMap = new LinkedHashMap<>(); IntStream.range(0, originalIndexList.size()) 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 295239f09536..b73e815962b0 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 @@ -256,7 +256,7 @@ private RelationPlan processNamedQuery(Table table, Query namedQuery, Scope scop } if (namedQuery.isMaterialized() && namedQuery.isDone()) { - RelationPlan materializedCtePlan = processMaterializedCte(table, scope); + RelationPlan materializedCtePlan = processMaterializedCte(table, namedQuery, scope); if (materializedCtePlan != null) { return materializedCtePlan; } @@ -265,8 +265,8 @@ private RelationPlan processNamedQuery(Table table, Query namedQuery, Scope scop return processRegularCte(table, namedQuery, scope); } - private RelationPlan processMaterializedCte(Table table, Scope scope) { - CteDataStore dataStore = queryContext.getCteDataStore(table); + private RelationPlan processMaterializedCte(Table table, Query query, Scope scope) { + CteDataStore dataStore = query.getCteDataStore(); if (dataStore == null) { return null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index 613f6f87fa55..b5e8fb897227 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -53,7 +53,7 @@ public CteDataStore(TableSchema tableSchema, List columnIndex2TsBlockCo public boolean addTsBlock(TsBlock tsBlock) { IoTDBConfig iotConfig = IoTDBDescriptor.getInstance().getConfig(); - long bytesSize = tsBlock.getRetainedSizeInBytes(); + long bytesSize = tsBlock.getSizeInBytes(); int rows = tsBlock.getPositionCount(); if (bytesSize + cachedBytes >= iotConfig.getCteBufferSize() || rows + cachedRows >= iotConfig.getMaxRowsInCteBuffer()) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java index 0f44f68b8cbd..482b85d6dc7f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java @@ -29,7 +29,6 @@ import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.collect.ImmutableList; -import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.block.TsBlockBuilder; @@ -40,6 +39,7 @@ import org.apache.tsfile.read.common.type.StringType; import org.apache.tsfile.read.common.type.TimestampType; import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.RamUsageEstimator; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -157,11 +157,9 @@ public void testMemory() throws Exception { // maxPeekMemory + maxReturnSize + retainedSize long maxPeekMemory = cteScanOperator.calculateMaxPeekMemory(); - assertEquals( - TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(), maxPeekMemory); + assertEquals(RamUsageEstimator.NUM_BYTES_OBJECT_REF, maxPeekMemory); long maxReturnSize = cteScanOperator.calculateMaxReturnSize(); - assertEquals( - TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(), maxReturnSize); + assertEquals(RamUsageEstimator.NUM_BYTES_OBJECT_REF, maxReturnSize); long retainedSize = cteScanOperator.calculateRetainedSizeAfterCallingNext(); assertEquals(0L, retainedSize); From 57c2d65562a256b2ff83dd03ada7a34ecaa345ed Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 12 Dec 2025 10:35:01 +0800 Subject: [PATCH 59/69] copilot comments --- .../it/query/recent/IoTExplainAnalyzeIT.java | 6 +++++- .../iotdb/db/queryengine/common/MPPQueryContext.java | 11 ++++++++--- .../plan/relational/analyzer/ExpressionAnalyzer.java | 2 +- .../queryengine/plan/relational/analyzer/Scope.java | 7 ++++++- .../plan/relational/analyzer/StatementAnalyzer.java | 6 +++--- .../plan/relational/planner/CteMaterializer.java | 3 +-- ...teWithUncorrelatedScalarSubqueryReconstructor.java | 4 +--- 7 files changed, 25 insertions(+), 14 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java index 8ee9a819239c..4c252dbf3ca1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTExplainAnalyzeIT.java @@ -240,7 +240,11 @@ private static ToLongFunction extractNumber(String regex) { Pattern pattern = Pattern.compile(regex); Matcher matcher = pattern.matcher(line); if (matcher.find()) { - return Long.parseLong(matcher.group(1)); + try { + return Long.parseLong(matcher.group(1)); + } catch (NumberFormatException e) { + return 0; + } } return 0; }; 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 53e6b3a334d6..ce646332451f 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 @@ -40,6 +40,7 @@ import org.apache.iotdb.db.queryengine.statistics.QueryPlanStatistics; import org.apache.iotdb.db.utils.cte.CteDataStore; +import com.google.common.collect.ImmutableList; import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.utils.Pair; @@ -127,7 +128,7 @@ public class MPPQueryContext implements IAuditEntity { private boolean subquery = false; // Tables in the subquery - private final Map> subQueryTables = new HashMap<>(); + private final Map, List> subQueryTables = new HashMap<>(); public MPPQueryContext(QueryId queryId) { this.queryId = queryId; @@ -513,8 +514,12 @@ public void setCteQueries(Map, Query> cteQueries) { this.cteQueries = cteQueries; } - public Map> getSubQueryTables() { - return subQueryTables; + public void addSubQueryTables(Query query, List tables) { + subQueryTables.put(NodeRef.of(query), tables); + } + + public List getTables(Query query) { + return subQueryTables.getOrDefault(NodeRef.of(query), ImmutableList.of()); } public void addCteExplainResult(Table table, Pair> cteExplainResult) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java index 8ea35c2b94e8..663c92ad83e9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java @@ -1722,7 +1722,7 @@ private Type analyzeSubquery(SubqueryExpression node, StackableAstVisitorContext statementAnalyzerFactory.apply(node, ctx.getContext().getCorrelationSupport()); Scope subqueryScope = Scope.builder().withParent(ctx.getContext().getScope()).build(); Scope queryScope = analyzer.analyze(node.getQuery(), subqueryScope); - context.getSubQueryTables().put(node.getQuery(), queryScope.getTables()); + context.addSubQueryTables(node.getQuery(), queryScope.getTables()); ImmutableList.Builder fields = ImmutableList.builder(); for (int i = 0; i < queryScope.getRelationType().getAllFieldCount(); i++) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java index f780e29a79ed..c309458f7d6d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java @@ -32,6 +32,7 @@ import com.google.common.collect.ImmutableMap; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -82,12 +83,16 @@ public void addTable(Table table) { tables.add(new Identifier(table.getName().getSuffix())); } + public void addTables(List tables) { + this.tables.addAll(tables); + } + public Scope copy() { return builder().like(this).build(); } public List getTables() { - return tables; + return Collections.unmodifiableList(tables); } public Scope withRelationType(RelationType relationType) { 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 8be2a727a23f..ece1a5def551 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 @@ -948,7 +948,7 @@ private Scope analyzeWith(Query node, Optional scope) { } withScopeBuilder.withNamedQuery(name, withQuery); - queryContext.getSubQueryTables().put(withQuery.getQuery(), queryScope.getTables()); + queryContext.addSubQueryTables(withQuery.getQuery(), queryScope.getTables()); } } Scope withScope = withScopeBuilder.build(); @@ -3636,8 +3636,8 @@ protected Scope visitJoin(Join node, Optional scope) { Scope right = process(node.getRight(), rightScope); if (scope.isPresent()) { - leftScope.ifPresent(l -> scope.get().getTables().addAll(l.getTables())); - rightScope.ifPresent(l -> scope.get().getTables().addAll(l.getTables())); + leftScope.ifPresent(l -> scope.get().addTables(l.getTables())); + rightScope.ifPresent(l -> scope.get().addTables(l.getTables())); } if (criteria instanceof JoinUsing) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 99a06cff2eeb..2a706444aafe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -107,8 +107,7 @@ public CteDataStore fetchCteQueryResult( try { Query q = query; if (with != null) { - List tables = - context.getSubQueryTables().getOrDefault(query, ImmutableList.of()); + List tables = context.getTables(query); List withQueries = with.getQueries().stream() .filter( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index b8ffbe6b3872..9cbef8cf1149 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -48,7 +48,6 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; import org.apache.iotdb.rpc.TSStatusCode; -import com.google.common.collect.ImmutableList; import org.apache.tsfile.block.column.Column; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; @@ -113,8 +112,7 @@ public Optional fetchUncorrelatedSubqueryResultForPredicate( Query query = subqueryExpression.getQuery(); Query q = query; if (with != null) { - List tables = - context.getSubQueryTables().getOrDefault(query, ImmutableList.of()); + List tables = context.getTables(query); List withQueries = with.getQueries().stream() .filter( From bfc9174da43a64ad5750b8b09e71a0d30594e821 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 12 Dec 2025 14:48:51 +0800 Subject: [PATCH 60/69] revert getRetainedSizeInBytes --- .../main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index b5e8fb897227..613f6f87fa55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -53,7 +53,7 @@ public CteDataStore(TableSchema tableSchema, List columnIndex2TsBlockCo public boolean addTsBlock(TsBlock tsBlock) { IoTDBConfig iotConfig = IoTDBDescriptor.getInstance().getConfig(); - long bytesSize = tsBlock.getSizeInBytes(); + long bytesSize = tsBlock.getRetainedSizeInBytes(); int rows = tsBlock.getPositionCount(); if (bytesSize + cachedBytes >= iotConfig.getCteBufferSize() || rows + cachedRows >= iotConfig.getMaxRowsInCteBuffer()) { From 5016903e9b100b66a5f6046b2807a5d5363b0762 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 12 Dec 2025 14:49:50 +0800 Subject: [PATCH 61/69] simplify CteScanOperator --- .../source/relational/CteScanOperator.java | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 1d5a4ba95ca7..79da8d6fff6a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -24,7 +24,7 @@ import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; -import org.apache.iotdb.db.queryengine.execution.operator.source.AbstractSourceOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.SourceOperator; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.utils.cte.CteDataReader; import org.apache.iotdb.db.utils.cte.CteDataStore; @@ -35,22 +35,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class CteScanOperator extends AbstractSourceOperator { +public class CteScanOperator implements SourceOperator { private static final Logger LOGGER = LoggerFactory.getLogger(CteScanOperator.class); private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(CteScanOperator.class); + private final OperatorContext operatorContext; + private final PlanNodeId sourceId; + private final CteDataStore dataStore; + private final CteDataReader dataReader; private final int dataStoreRefCount; - private CteDataReader dataReader; public CteScanOperator( OperatorContext operatorContext, PlanNodeId sourceId, CteDataStore dataStore) { this.operatorContext = operatorContext; this.sourceId = sourceId; this.dataStore = dataStore; + this.dataReader = new MemoryReader(dataStore.getCachedData()); this.dataStoreRefCount = dataStore.increaseRefCount(); - prepareReader(); } @Override @@ -104,10 +107,9 @@ public long calculateRetainedSizeAfterCallingNext() { @Override public long ramBytesUsed() { long bytes = - INSTANCE_SIZE + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext); - if (dataReader != null) { - bytes += dataReader.bytesUsed(); - } + INSTANCE_SIZE + + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext) + + dataReader.bytesUsed(); if (dataStoreRefCount == 1) { bytes += dataStore.getCachedBytes(); } @@ -115,10 +117,14 @@ public long ramBytesUsed() { return bytes; } - private void prepareReader() { - if (dataStore.getCachedBytes() != 0) { - dataReader = new MemoryReader(dataStore.getCachedData()); - } + @Override + public OperatorContext getOperatorContext() { + return operatorContext; + } + + @Override + public PlanNodeId getSourceId() { + return sourceId; } @TestOnly From e80713a806055267a97b8fb0cb0e0bcf0b672ad9 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 16 Dec 2025 17:04:39 +0800 Subject: [PATCH 62/69] claude ai: move ExplainType inside MPPQueryContext --- .../db/queryengine/common/ExplainType.java | 28 ------------------- .../queryengine/common/MPPQueryContext.java | 7 +++++ .../db/queryengine/plan/Coordinator.java | 2 +- .../plan/analyze/AnalyzeVisitor.java | 2 +- .../analyzer/StatementAnalyzer.java | 2 +- ...correlatedScalarSubqueryReconstructor.java | 2 +- .../plan/relational/planner/PlanTester.java | 2 +- 7 files changed, 12 insertions(+), 33 deletions(-) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/ExplainType.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/ExplainType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/ExplainType.java deleted file mode 100644 index 65d04feb8b43..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/ExplainType.java +++ /dev/null @@ -1,28 +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.common; - -public enum ExplainType { - NONE, - EXPLAIN, - EXPLAIN_ANALYZE, -} 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 ce646332451f..24bdf5997782 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 @@ -64,6 +64,13 @@ public class MPPQueryContext implements IAuditEntity { private String sql; private final QueryId queryId; + /** The type of explanation for a query. */ + public enum ExplainType { + NONE, + EXPLAIN, + EXPLAIN_ANALYZE, + } + // LocalQueryId is kept to adapt to the old client, it's unique in current datanode. // Now it's only be used by EXPLAIN ANALYZE to get queryExecution. private long localQueryId; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index 5a35195be013..585cd6a82564 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -39,8 +39,8 @@ import org.apache.iotdb.db.protocol.session.IClientSession; import org.apache.iotdb.db.protocol.session.PreparedStatementInfo; import org.apache.iotdb.db.queryengine.common.DataNodeEndPoints; -import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext.ExplainType; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.common.SessionInfo; import org.apache.iotdb.db.queryengine.execution.QueryIdGenerator; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index 66fe98f302fb..4cae2c0f2884 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -47,8 +47,8 @@ import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.exception.sql.StatementAnalyzeException; import org.apache.iotdb.db.queryengine.common.DeviceContext; -import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext.ExplainType; import org.apache.iotdb.db.queryengine.common.TimeseriesContext; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.common.header.DatasetHeaderFactory; 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 ece1a5def551..7ba8f311d604 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 @@ -25,8 +25,8 @@ import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema; import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; import org.apache.iotdb.db.exception.sql.SemanticException; -import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext.ExplainType; import org.apache.iotdb.db.queryengine.common.SessionInfo; import org.apache.iotdb.db.queryengine.execution.warnings.IoTDBWarning; import org.apache.iotdb.db.queryengine.execution.warnings.WarningCollector; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 9cbef8cf1149..c5e24262899c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -22,8 +22,8 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.protocol.session.SessionManager; -import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext.ExplainType; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java index 6cf914ee3b60..4d73cc9d003b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java @@ -23,8 +23,8 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.protocol.session.IClientSession; -import org.apache.iotdb.db.queryengine.common.ExplainType; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext.ExplainType; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.common.SessionInfo; import org.apache.iotdb.db.queryengine.execution.warnings.WarningCollector; From c8c559d255b054c6740b243003bc69d2834b8820 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 16 Dec 2025 17:26:23 +0800 Subject: [PATCH 63/69] claude ai: MppQueryContext notation --- .../queryengine/common/MPPQueryContext.java | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 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 24bdf5997782..603e5a8276a5 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 @@ -99,9 +99,10 @@ public enum ExplainType { private final Set acquiredLocks = new HashSet<>(); - // Previously, the boolean value 'isExplainAnalyze' was needed to determine whether query - // statistics should be recorded during the query process. Now 'explainType' is used to - // determine whether query statistics or query plan should be recorded during the query process. + // Determines the explanation mode for the query: + // - NONE: Normal query execution without explanation + // - EXPLAIN: Show the logical and physical query plan without execution + // - EXPLAIN_ANALYZE: Execute the query and collect detailed execution statistics private ExplainType explainType = ExplainType.NONE; private boolean isVerbose = false; @@ -124,14 +125,19 @@ public enum ExplainType { private boolean userQuery = false; - private final Map, Long> cteMaterializationCosts = new HashMap<>(); private Map, Query> cteQueries = new HashMap<>(); - // table -> (maxLineLength, 'explain' or 'explain analyze' result) - // Max line length of each CTE should be remembered because we need to standardize - // the output format of main query and CTE query. + + // Stores the EXPLAIN/EXPLAIN ANALYZE results for Common Table Expressions (CTEs) + // Key: CTE table reference + // Value: Pair containing (max line length of the explain output, list of formatted explain lines) + // This ensures consistent formatting between the main query and its CTE sub-queries private final Map, Pair>> cteExplainResults = new LinkedHashMap<>(); - // Do not release CTE query result if it is a subquery. + // Tracks the materialization time cost (in nanoseconds) for each CTE to help optimize query + // planning + private final Map, Long> cteMaterializationCosts = new HashMap<>(); + + // Never materialize CTE in a subquery. private boolean subquery = false; // Tables in the subquery From f102cbbb4159434748cd76e7485c9a98f5d3a309 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 23 Dec 2025 16:51:51 +0800 Subject: [PATCH 64/69] fix: left & right in ComparisonExpression should be final --- ...eWithUncorrelatedScalarSubqueryReconstructor.java | 2 -- .../relational/sql/ast/ComparisonExpression.java | 12 ++---------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index c5e24262899c..95f2226802cf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -93,8 +93,6 @@ public void reconstructPredicateWithUncorrelatedScalarSubquery( context, (SubqueryExpression) left, analysis.getWith()); result.ifPresent(comparisonExpression::setShadowLeft); } - comparisonExpression.setLeft(left); - comparisonExpression.setRight(right); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java index 891a455cd514..793543540784 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/ComparisonExpression.java @@ -100,9 +100,9 @@ public Operator negate() { } private final Operator operator; - private Expression left; + private final Expression left; private Expression shadowLeft; - private Expression right; + private final Expression right; private Expression shadowRight; public ComparisonExpression(Operator operator, Expression left, Expression right) { @@ -140,14 +140,6 @@ public Expression getRight() { return shadowRight != null ? shadowRight : right; } - public void setLeft(Expression left) { - this.left = left; - } - - public void setRight(Expression right) { - this.right = right; - } - @Override public R accept(AstVisitor visitor, C context) { return visitor.visitComparisonExpression(this, context); From 9f133c22200958ae1a477b8b010357116af664e8 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 24 Dec 2025 14:35:19 +0800 Subject: [PATCH 65/69] cleanup cte in main query --- .../iotdb/db/queryengine/common/MPPQueryContext.java | 9 +++++++++ 1 file changed, 9 insertions(+) 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 603e5a8276a5..de200aae751d 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 @@ -210,10 +210,19 @@ public void releaseMemoryForSchemaTree() { } public void prepareForRetry() { + if (!isSubquery()) { + cleanUpCte(); + } this.initResultNodeContext(); this.releaseAllMemoryReservedForFrontEnd(); } + private void cleanUpCte() { + cteQueries.clear(); + cteExplainResults.clear(); + cteMaterializationCosts.clear(); + subQueryTables.clear(); + } private void initResultNodeContext() { this.resultNodeContext = new ResultNodeContext(queryId); } From e0f6bf03d03a9d7e82506eb1b561ff18da0be61e Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 25 Dec 2025 08:30:53 +0800 Subject: [PATCH 66/69] code review --- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 18 ++++++++++++++++++ .../db/queryengine/common/MPPQueryContext.java | 17 ++++++++++------- .../fragment/FragmentInstanceExecution.java | 10 ++++++++-- .../source/relational/CteScanOperator.java | 9 ++++++--- .../iotdb/db/queryengine/plan/Coordinator.java | 2 +- .../plan/relational/analyzer/Analysis.java | 16 ++++++++++++++++ .../plan/relational/analyzer/Analyzer.java | 1 + .../plan/relational/analyzer/Scope.java | 14 +++++++------- .../relational/analyzer/StatementAnalyzer.java | 14 ++++++-------- .../analyzer/StatementAnalyzerFactory.java | 4 ++++ .../ConvertPredicateToTimeFilterVisitor.java | 10 +--------- .../planner/TableLogicalPlanner.java | 4 ++-- .../distribute/TableDistributedPlanner.java | 2 +- ...ncorrelatedScalarSubqueryReconstructor.java | 11 ++++++----- .../planner/UncorrelatedSubqueryTest.java | 2 +- .../conf/iotdb-system.properties.template | 5 ++++- 16 files changed, 92 insertions(+), 47 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index f0fea44b65f7..73b8764994c7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -2199,6 +2199,24 @@ public synchronized void loadHotModifiedProps(TrimProperties properties) Long.parseLong( properties.getProperty( "max_object_file_size_in_byte", String.valueOf(conf.getMaxObjectSizeInByte())))); + + // The buffer for cte materialization. + long cteBufferSizeInBytes = + Long.parseLong( + properties.getProperty( + "cte_buffer_size_in_bytes", Long.toString(conf.getCteBufferSize()))); + if (cteBufferSizeInBytes > 0) { + conf.setCteBufferSize(cteBufferSizeInBytes); + } + // max number of rows for cte materialization + int maxRowsInCteBuffer = + Integer.parseInt( + properties.getProperty( + "max_rows_in_cte_buffer", Integer.toString(conf.getMaxRowsInCteBuffer()))); + if (maxRowsInCteBuffer > 0) { + conf.setMaxRowsInCteBuffer(maxRowsInCteBuffer); + } + } catch (Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); 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 de200aae751d..fac3afff8b0b 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 @@ -137,8 +137,10 @@ public enum ExplainType { // planning private final Map, Long> cteMaterializationCosts = new HashMap<>(); - // Never materialize CTE in a subquery. - private boolean subquery = false; + // Indicates whether this query context is for a sub-query triggered by the main query. + // Sub-queries are independent queries spawned from the main query (e.g., CTE sub-queries). + // When true, CTE materialization is skipped as it's handled by the main query context. + private boolean innerTriggeredQuery = false; // Tables in the subquery private final Map, List> subQueryTables = new HashMap<>(); @@ -210,7 +212,7 @@ public void releaseMemoryForSchemaTree() { } public void prepareForRetry() { - if (!isSubquery()) { + if (!isInnerTriggeredQuery()) { cleanUpCte(); } this.initResultNodeContext(); @@ -223,6 +225,7 @@ private void cleanUpCte() { cteMaterializationCosts.clear(); subQueryTables.clear(); } + private void initResultNodeContext() { this.resultNodeContext = new ResultNodeContext(queryId); } @@ -500,12 +503,12 @@ public void setUserQuery(boolean userQuery) { this.userQuery = userQuery; } - public boolean isSubquery() { - return subquery; + public boolean isInnerTriggeredQuery() { + return innerTriggeredQuery; } - public void setSubquery(boolean subquery) { - this.subquery = subquery; + public void setInnerTriggeredQuery(boolean innerTriggeredQuery) { + this.innerTriggeredQuery = innerTriggeredQuery; } public void addCteMaterializationCost(Table table, long cost) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java index 8d4cdcd88860..9d099859e37d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java @@ -59,6 +59,11 @@ public class FragmentInstanceExecution { // It will be set to null while this FI is FINISHED private List drivers; + // Indicates whether this fragment instance should be ignored for statistics collection. + // This is true when the fragment instance contains ExplainAnalyzeOperator, which is + // a virtual fragment used for EXPLAIN ANALYZE and should not be included in query statistics. + boolean shouldIgnoreForStatistics; + // It will be set to null while this FI is FINISHED private ISink sink; @@ -110,6 +115,7 @@ private FragmentInstanceExecution( this.stateMachine = stateMachine; this.timeoutInMs = timeoutInMs; this.exchangeManager = exchangeManager; + this.shouldIgnoreForStatistics = shouldIgnoreForStatistics(); } public FragmentInstanceState getInstanceState() { @@ -141,7 +147,7 @@ public FragmentInstanceStateMachine getStateMachine() { } // Check if this fragment instance should be ignored for statistics - // (i.e., it contains ExplainAnalyzeOperator only) + // (i.e., it contains ExplainAnalyzeOperator) private boolean shouldIgnoreForStatistics() { if (drivers == null || drivers.isEmpty()) { return false; @@ -166,7 +172,7 @@ private boolean fillFragmentInstanceStatistics( statistics.setState(getInstanceState().toString()); // Previously we ignore statistics when current data region is instance of // VirtualDataRegion. Now data region of a CteScanNode is also virtual. - if (shouldIgnoreForStatistics()) { + if (shouldIgnoreForStatistics) { // We don't need to output the region having ExplainAnalyzeOperator only. return false; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 79da8d6fff6a..5e60b5cef105 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -30,6 +30,7 @@ import org.apache.iotdb.db.utils.cte.CteDataStore; import org.apache.iotdb.db.utils.cte.MemoryReader; +import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.utils.RamUsageEstimator; import org.slf4j.Logger; @@ -40,6 +41,9 @@ public class CteScanOperator implements SourceOperator { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(CteScanOperator.class); + private final long maxReturnSize = + TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + private final OperatorContext operatorContext; private final PlanNodeId sourceId; @@ -79,7 +83,7 @@ public void close() throws Exception { dataReader.close(); } } catch (Exception e) { - LOGGER.error("Fail to close fileChannel", e); + LOGGER.error("Fail to close CteDataReader", e); } } @@ -95,8 +99,7 @@ public long calculateMaxPeekMemory() { @Override public long calculateMaxReturnSize() { - // The returned object is a reference to TsBlock in CteDataReader - return RamUsageEstimator.NUM_BYTES_OBJECT_REF; + return maxReturnSize; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index 585cd6a82564..5d8d0e03a4a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -427,7 +427,7 @@ public ExecutionResult executeForTableModel( sql, userQuery, ((queryContext, startTime) -> { - queryContext.setSubquery(true); + queryContext.setInnerTriggeredQuery(true); queryContext.setCteQueries(cteQueries); queryContext.setExplainType(explainType); return createQueryExecutionForTableModel( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java index 306c9b51c28a..4a0fe9daa570 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analysis.java @@ -70,6 +70,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.TableFunctionInvocation; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WindowFrame; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.With; +import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; import org.apache.iotdb.db.queryengine.plan.statement.component.FillPolicy; import com.google.common.collect.ArrayListMultimap; @@ -124,6 +125,8 @@ public class Analysis implements IAnalysis { private final Map, Query> namedQueries = new LinkedHashMap<>(); + // WITH clause stored during analyze phase. Required for constant folding and CTE materialization + // subqueries, which cannot directly access the WITH clause private With with; // map expandable query to the node being the inner recursive reference @@ -255,6 +258,11 @@ public class Analysis implements IAnalysis { private boolean isQuery = false; + // SqlParser is needed during query planning phase for executing uncorrelated scalar subqueries + // in advance (predicate folding). The planner needs to parse and execute these subqueries + // independently to utilize predicate pushdown optimization. + private SqlParser sqlParser; + public Analysis(@Nullable Statement root, Map, Expression> parameters) { this.root = root; this.parameters = ImmutableMap.copyOf(requireNonNull(parameters, "parameters is null")); @@ -277,6 +285,14 @@ public void setUpdateType(String updateType) { this.updateType = updateType; } + public SqlParser getSqlParser() { + return sqlParser; + } + + public void setSqlParser(SqlParser sqlParser) { + this.sqlParser = sqlParser; + } + public Query getNamedQuery(Table table) { return namedQueries.get(NodeRef.of(table)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java index 43c993fdeccc..cab532fd2d38 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Analyzer.java @@ -80,6 +80,7 @@ public Analysis analyze(Statement statement) { warningCollector); Analysis analysis = new Analysis(rewrittenStatement, parameterLookup); + analysis.setSqlParser(statementAnalyzerFactory.getSqlParser()); // Register CTE passed by parent query. context .getCteQueries() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java index c309458f7d6d..e2fd0cc72767 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/Scope.java @@ -54,7 +54,11 @@ public class Scope { private final RelationId relationId; private final RelationType relation; private final Map namedQueries; - private final List tables; + + // Tables to access for the current relation. For CTE materialization and constant folding + // subqueries, non-materialized CTEs in tables must be identified, and their definitions + // attached to the subquery context. + private List tables; public static Scope create() { return builder().build(); @@ -83,12 +87,8 @@ public void addTable(Table table) { tables.add(new Identifier(table.getName().getSuffix())); } - public void addTables(List tables) { - this.tables.addAll(tables); - } - - public Scope copy() { - return builder().like(this).build(); + public void setTables(List tables) { + this.tables = tables; } public List getTables() { 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 7ba8f311d604..9220ad55afa5 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 @@ -3630,15 +3630,13 @@ protected Scope visitJoin(Join node, Optional scope) { joinConditionCheck(criteria); - Optional leftScope = scope.map(Scope::copy); - Scope left = process(node.getLeft(), leftScope); - Optional rightScope = scope.map(Scope::copy); - Scope right = process(node.getRight(), rightScope); + // remember current tables in the scope + List tables = new ArrayList<>(); + scope.ifPresent(s -> tables.addAll(s.getTables())); - if (scope.isPresent()) { - leftScope.ifPresent(l -> scope.get().addTables(l.getTables())); - rightScope.ifPresent(l -> scope.get().addTables(l.getTables())); - } + Scope left = process(node.getLeft(), scope); + scope.ifPresent(s -> s.setTables(tables)); + Scope right = process(node.getRight(), scope); if (criteria instanceof JoinUsing) { return analyzeJoinUsing(node, ((JoinUsing) criteria).getColumns(), scope, left, right); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzerFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzerFactory.java index 53c300f3ce35..c532731850c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzerFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzerFactory.java @@ -77,4 +77,8 @@ public static StatementAnalyzerFactory createTestingStatementAnalyzerFactory( public AccessControl getAccessControl() { return accessControl; } + + public SqlParser getSqlParser() { + return sqlParser; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/ConvertPredicateToTimeFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/ConvertPredicateToTimeFilterVisitor.java index b608a4b674aa..7368c5d28c2c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/ConvertPredicateToTimeFilterVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/ConvertPredicateToTimeFilterVisitor.java @@ -21,7 +21,6 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BetweenPredicate; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; -import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DoubleLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Extract; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.IfExpression; @@ -300,13 +299,6 @@ protected Filter visitBetweenPredicate(BetweenPredicate node, Void context) { } public static long getLongValue(Expression expression) { - if (expression instanceof LongLiteral) { - return ((LongLiteral) expression).getParsedValue(); - } else if (expression instanceof DoubleLiteral) { - return (long) ((DoubleLiteral) expression).getValue(); - } else { - throw new IllegalArgumentException( - "Expression should be LongLiteral or DoubleLiteral, but got: " + expression.getClass()); - } + return ((LongLiteral) expression).getParsedValue(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index 8210d8443d68..5b3ceac38337 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -316,7 +316,7 @@ private PlanNode createOutputPlan(RelationPlan plan, Analysis analysis) { int columnNumber = 0; // TODO perfect the logic of outputDescriptor - if (queryContext.isExplainAnalyze() && !queryContext.isSubquery()) { + if (queryContext.isExplainAnalyze() && !queryContext.isInnerTriggeredQuery()) { outputs.add(new Symbol(ColumnHeaderConstant.EXPLAIN_ANALYZE)); names.add(ColumnHeaderConstant.EXPLAIN_ANALYZE); columnHeaders.add(new ColumnHeader(ColumnHeaderConstant.EXPLAIN_ANALYZE, TSDataType.TEXT)); @@ -373,7 +373,7 @@ private RelationPlan createRelationPlan(Analysis analysis, PipeEnriched pipeEnri private RelationPlan createRelationPlan(Analysis analysis, Query query) { // materialize cte if needed - if (!queryContext.isSubquery()) { + if (!queryContext.isInnerTriggeredQuery()) { CteMaterializer.getInstance().materializeCTE(analysis, queryContext); } return getRelationPlanner(analysis).process(query, null); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java index e8ffd74a1568..ff7686fe8689 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanner.java @@ -103,7 +103,7 @@ public DistributedQueryPlan plan() { new TableDistributedPlanGenerator.PlanContext(); PlanNode outputNodeWithExchange = generateDistributedPlanWithOptimize(planContext); List planText = null; - if (mppQueryContext.isExplain() && mppQueryContext.isSubquery()) { + if (mppQueryContext.isExplain() && mppQueryContext.isInnerTriggeredQuery()) { planText = outputNodeWithExchange.accept( new PlanGraphPrinter(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 95f2226802cf..5165a2ad097d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -60,8 +60,6 @@ public class PredicateWithUncorrelatedScalarSubqueryReconstructor { - private static final SqlParser relationSqlParser = new SqlParser(); - private static final Coordinator coordinator = Coordinator.getInstance(); public void reconstructPredicateWithUncorrelatedScalarSubquery( @@ -83,14 +81,14 @@ public void reconstructPredicateWithUncorrelatedScalarSubquery( && right instanceof SubqueryExpression) { Optional result = fetchUncorrelatedSubqueryResultForPredicate( - context, (SubqueryExpression) right, analysis.getWith()); + context, analysis.getSqlParser(), (SubqueryExpression) right, analysis.getWith()); // If the subquery result is not present, we cannot reconstruct the predicate. result.ifPresent(comparisonExpression::setShadowRight); } else if ((right instanceof Identifier || right instanceof FunctionCall) && left instanceof SubqueryExpression) { Optional result = fetchUncorrelatedSubqueryResultForPredicate( - context, (SubqueryExpression) left, analysis.getWith()); + context, analysis.getSqlParser(), (SubqueryExpression) left, analysis.getWith()); result.ifPresent(comparisonExpression::setShadowLeft); } } @@ -102,7 +100,10 @@ public void reconstructPredicateWithUncorrelatedScalarSubquery( * valid result. */ public Optional fetchUncorrelatedSubqueryResultForPredicate( - MPPQueryContext context, SubqueryExpression subqueryExpression, With with) { + MPPQueryContext context, + SqlParser relationSqlParser, + SubqueryExpression subqueryExpression, + With with) { final long queryId = SessionManager.getInstance().requestQueryId(); Throwable t = null; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java index 65231a9f5e21..2f02b3f758eb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java @@ -75,7 +75,7 @@ private void mockPredicateWithUncorrelatedScalarSubquery() { Mockito.spy(new PredicateWithUncorrelatedScalarSubqueryReconstructor()); Mockito.when( predicateWithUncorrelatedScalarSubquery.fetchUncorrelatedSubqueryResultForPredicate( - Mockito.any(), Mockito.any(), Mockito.any())) + Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any())) .thenReturn(Optional.of(new LongLiteral("1"))); PredicateWithUncorrelatedScalarSubqueryReconstructor.setInstance( predicateWithUncorrelatedScalarSubquery); diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 7f78cf22b2d2..9d63529355a5 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -1127,12 +1127,15 @@ sort_buffer_size_in_bytes=0 # will be used. # effectiveMode: hot_reload # Datatype: long -cte_buffer_size_in_bytes=0 +# Privilege: SYSTEM +# Unit: bytes +cte_buffer_size_in_bytes=131072 # Max rows for CTE materialization # effectiveMode: hot_reload # Datatype: int +# Privilege: SYSTEM max_rows_in_cte_buffer=1000 # The maximum mod entries size that each FragmentInstance can cache. From 63d49b40e4b436640f1738799ba8be99b7662623 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 26 Dec 2025 11:17:11 +0800 Subject: [PATCH 67/69] PredicateWithUncorrelatedScalarSubqueryReconstructor part --- .../plan/relational/planner/QueryPlanner.java | 43 ++++++++++++++++--- .../relational/planner/RelationPlanner.java | 43 +++++++++++++++---- .../relational/planner/SubqueryPlanner.java | 19 ++++++-- .../planner/TableLogicalPlanner.java | 22 +++++++++- ...correlatedScalarSubqueryReconstructor.java | 30 ++++--------- .../plan/relational/planner/PlanTester.java | 28 ++++++++++-- .../planner/UncorrelatedSubqueryTest.java | 36 ++++++++++------ 7 files changed, 165 insertions(+), 56 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/QueryPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/QueryPlanner.java index ad22504b826c..071a6a05da67 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/QueryPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/QueryPlanner.java @@ -125,6 +125,9 @@ public class QueryPlanner { private final Optional outerContext; private final Map, RelationPlan> recursiveSubqueries; + private final PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor; + // private final Map, Symbol> lambdaDeclarationToSymbolMap; // private final SubqueryPlanner subqueryPlanner; @@ -134,13 +137,18 @@ public QueryPlanner( MPPQueryContext queryContext, Optional outerContext, SessionInfo session, - Map, RelationPlan> recursiveSubqueries) { + Map, RelationPlan> recursiveSubqueries, + PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor) { requireNonNull(analysis, "analysis is null"); requireNonNull(symbolAllocator, "symbolAllocator is null"); requireNonNull(queryContext, "queryContext is null"); requireNonNull(outerContext, "outerContext is null"); requireNonNull(session, "session is null"); requireNonNull(recursiveSubqueries, "recursiveSubqueries is null"); + requireNonNull( + predicateWithUncorrelatedScalarSubqueryReconstructor, + "predicateWithUncorrelatedScalarSubqueryReconstructor is null"); this.analysis = analysis; this.symbolAllocator = symbolAllocator; @@ -150,8 +158,16 @@ public QueryPlanner( this.outerContext = outerContext; this.subqueryPlanner = new SubqueryPlanner( - analysis, symbolAllocator, queryContext, outerContext, session, recursiveSubqueries); + analysis, + symbolAllocator, + queryContext, + outerContext, + session, + recursiveSubqueries, + predicateWithUncorrelatedScalarSubqueryReconstructor); this.recursiveSubqueries = recursiveSubqueries; + this.predicateWithUncorrelatedScalarSubqueryReconstructor = + predicateWithUncorrelatedScalarSubqueryReconstructor; } public RelationPlan plan(Query query) { @@ -297,6 +313,9 @@ public RelationPlan plan(QuerySpecification node) { builder = limit(builder, node.getLimit(), orderingScheme); builder = builder.appendProjections(outputs, symbolAllocator, queryContext); + for (Expression expr : expressions) { + predicateWithUncorrelatedScalarSubqueryReconstructor.clearShadowExpression(expr); + } return new RelationPlan( builder.getRoot(), analysis.getScope(node), computeOutputs(builder, outputs), outerContext); @@ -351,6 +370,9 @@ private PlanBuilder planWindowFunctions( subPlan = subqueryPlanner.handleSubqueries(subPlan, inputs, analysis.getSubqueries(node)); subPlan = subPlan.appendProjections(inputs, symbolAllocator, queryContext); + for (Expression input : inputs) { + predicateWithUncorrelatedScalarSubqueryReconstructor.clearShadowExpression(input); + } // Add projection to coerce inputs to their site-specific types. // This is important because the same lexical expression may need to be coerced @@ -735,7 +757,13 @@ private static List computeOutputs( private PlanBuilder planQueryBody(QueryBody queryBody) { RelationPlan relationPlan = new RelationPlanner( - analysis, symbolAllocator, queryContext, outerContext, session, recursiveSubqueries) + analysis, + symbolAllocator, + queryContext, + outerContext, + session, + recursiveSubqueries, + predicateWithUncorrelatedScalarSubqueryReconstructor) .process(queryBody, null); return newPlanBuilder(relationPlan, analysis); @@ -750,7 +778,8 @@ private PlanBuilder planFrom(QuerySpecification node) { queryContext, outerContext, session, - recursiveSubqueries) + recursiveSubqueries, + predicateWithUncorrelatedScalarSubqueryReconstructor) .process(node.getFrom().orElse(null), null); return newPlanBuilder(relationPlan, analysis); } else { @@ -768,8 +797,7 @@ private PlanBuilder filter(PlanBuilder subPlan, Expression predicate, Node node) subPlan.withNewRoot( new FilterNode( queryIdAllocator.genPlanNodeId(), subPlan.getRoot(), subPlan.rewrite(predicate))); - PredicateWithUncorrelatedScalarSubqueryReconstructor.getInstance() - .clearShadowExpression(predicate); + predicateWithUncorrelatedScalarSubqueryReconstructor.clearShadowExpression(predicate); return planBuilder; } @@ -806,6 +834,9 @@ private PlanBuilder aggregate(PlanBuilder subPlan, QuerySpecification node) { List inputs = inputBuilder.build(); subPlan = subqueryPlanner.handleSubqueries(subPlan, inputs, analysis.getSubqueries(node)); subPlan = subPlan.appendProjections(inputs, symbolAllocator, queryContext); + for (Expression input : inputs) { + predicateWithUncorrelatedScalarSubqueryReconstructor.clearShadowExpression(input); + } Function rewrite = subPlan.getTranslations()::rewrite; 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 b73e815962b0..d442d51a2141 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 @@ -54,6 +54,7 @@ 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; +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.ExceptNode; @@ -189,19 +190,27 @@ public class RelationPlanner extends AstVisitor { private final SubqueryPlanner subqueryPlanner; private final Map, RelationPlan> recursiveSubqueries; + private final PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor; + public RelationPlanner( final Analysis analysis, final SymbolAllocator symbolAllocator, final MPPQueryContext queryContext, final Optional outerContext, final SessionInfo sessionInfo, - final Map, RelationPlan> recursiveSubqueries) { + final Map, RelationPlan> recursiveSubqueries, + PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor) { requireNonNull(analysis, "analysis is null"); requireNonNull(symbolAllocator, "symbolAllocator is null"); requireNonNull(queryContext, "queryContext is null"); requireNonNull(outerContext, "outerContext is null"); requireNonNull(sessionInfo, "session is null"); requireNonNull(recursiveSubqueries, "recursiveSubqueries is null"); + requireNonNull( + predicateWithUncorrelatedScalarSubqueryReconstructor, + "predicateWithUncorrelatedScalarSubqueryReconstructor is null"); this.analysis = analysis; this.symbolAllocator = symbolAllocator; @@ -209,6 +218,8 @@ public RelationPlanner( this.idAllocator = queryContext.getQueryId(); this.outerContext = outerContext; this.sessionInfo = sessionInfo; + this.predicateWithUncorrelatedScalarSubqueryReconstructor = + predicateWithUncorrelatedScalarSubqueryReconstructor; this.subqueryPlanner = new SubqueryPlanner( analysis, @@ -216,14 +227,21 @@ public RelationPlanner( queryContext, outerContext, sessionInfo, - recursiveSubqueries); + recursiveSubqueries, + predicateWithUncorrelatedScalarSubqueryReconstructor); this.recursiveSubqueries = recursiveSubqueries; } @Override protected RelationPlan visitQuery(final Query node, final Void context) { return new QueryPlanner( - analysis, symbolAllocator, queryContext, outerContext, sessionInfo, recursiveSubqueries) + analysis, + symbolAllocator, + queryContext, + outerContext, + sessionInfo, + recursiveSubqueries, + predicateWithUncorrelatedScalarSubqueryReconstructor) .plan(node); } @@ -384,7 +402,13 @@ private RelationPlan processPhysicalTable(Table table, Scope scope) { protected RelationPlan visitQuerySpecification( final QuerySpecification node, final Void context) { return new QueryPlanner( - analysis, symbolAllocator, queryContext, outerContext, sessionInfo, recursiveSubqueries) + analysis, + symbolAllocator, + queryContext, + outerContext, + sessionInfo, + recursiveSubqueries, + predicateWithUncorrelatedScalarSubqueryReconstructor) .plan(node); } @@ -911,11 +935,10 @@ protected RelationPlan visitPatternRecognitionRelation( .forEach(outputLayout::add); } + List expressions = + extractPatternRecognitionExpressions(node.getVariableDefinitions(), node.getMeasures()); planBuilder = - subqueryPlanner.handleSubqueries( - planBuilder, - extractPatternRecognitionExpressions(node.getVariableDefinitions(), node.getMeasures()), - analysis.getSubqueries(node)); + subqueryPlanner.handleSubqueries(planBuilder, expressions, analysis.getSubqueries(node)); PatternRecognitionComponents components = planPatternRecognitionComponents( @@ -928,6 +951,10 @@ protected RelationPlan visitPatternRecognitionRelation( outputLayout.addAll(components.getMeasureOutputs()); + for (Expression expr : expressions) { + predicateWithUncorrelatedScalarSubqueryReconstructor.clearShadowExpression(expr); + } + if (!oneRowOutput) { Set inputSymbolsOnOutput = ImmutableSet.copyOf(outputLayout.build()); subPlan.getFieldMappings().stream() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java index 64f6f4e85a99..1484a47d5afe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SubqueryPlanner.java @@ -81,6 +81,8 @@ class SubqueryPlanner { private final MPPQueryContext plannerContext; private final SessionInfo session; private final Map, RelationPlan> recursiveSubqueries; + private final PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor; SubqueryPlanner( Analysis analysis, @@ -88,13 +90,18 @@ class SubqueryPlanner { MPPQueryContext plannerContext, Optional outerContext, SessionInfo session, - Map, RelationPlan> recursiveSubqueries) { + Map, RelationPlan> recursiveSubqueries, + PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor) { requireNonNull(analysis, "analysis is null"); requireNonNull(symbolAllocator, "symbolAllocator is null"); requireNonNull(plannerContext, "plannerContext is null"); requireNonNull(outerContext, "outerContext is null"); requireNonNull(session, "session is null"); requireNonNull(recursiveSubqueries, "recursiveSubqueries is null"); + requireNonNull( + predicateWithUncorrelatedScalarSubqueryReconstructor, + "predicateWithUncorrelatedScalarSubqueryReconstructor is null"); this.analysis = analysis; this.symbolAllocator = symbolAllocator; @@ -102,6 +109,8 @@ class SubqueryPlanner { this.plannerContext = plannerContext; this.session = session; this.recursiveSubqueries = recursiveSubqueries; + this.predicateWithUncorrelatedScalarSubqueryReconstructor = + predicateWithUncorrelatedScalarSubqueryReconstructor; } public PlanBuilder handleSubqueries( @@ -157,7 +166,7 @@ public PlanBuilder handleSubqueries( private void tryFoldUncorrelatedScalarSubqueryInPredicate( Expression expression, MPPQueryContext context) { - PredicateWithUncorrelatedScalarSubqueryReconstructor.getInstance() + predicateWithUncorrelatedScalarSubqueryReconstructor .reconstructPredicateWithUncorrelatedScalarSubquery(context, analysis, expression); } @@ -219,6 +228,7 @@ private PlanBuilder planInPredicate( subPlan = planInPredicate( subPlan, value, subquery, output, predicate, analysis.getPredicateCoercions(predicate)); + predicateWithUncorrelatedScalarSubqueryReconstructor.clearShadowExpression(value); return new PlanBuilder( subPlan @@ -355,7 +365,8 @@ private RelationPlan planSubquery(Expression subquery, TranslationMap outerConte plannerContext, Optional.of(outerContext), session, - recursiveSubqueries) + recursiveSubqueries, + predicateWithUncorrelatedScalarSubqueryReconstructor) .process(subquery, null); } @@ -385,6 +396,7 @@ private PlanBuilder planQuantifiedComparison( subPlan = planQuantifiedComparison( subPlan, operator, quantifier, value, subquery, output, predicateCoercions); + predicateWithUncorrelatedScalarSubqueryReconstructor.clearShadowExpression(value); return new PlanBuilder( subPlan .getTranslations() @@ -399,6 +411,7 @@ private PlanBuilder planQuantifiedComparison( subPlan = planInPredicate( subPlan, value, subquery, output, quantifiedComparison, predicateCoercions); + predicateWithUncorrelatedScalarSubqueryReconstructor.clearShadowExpression(value); return new PlanBuilder( subPlan .getTranslations() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index 5b3ceac38337..2d7ed174d2ad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -50,6 +50,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; +import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.PredicateWithUncorrelatedScalarSubqueryReconstructor; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; @@ -115,6 +116,9 @@ public class TableLogicalPlanner { private final Metadata metadata; private final WarningCollector warningCollector; + private PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor; + @TestOnly public TableLogicalPlanner( MPPQueryContext queryContext, @@ -145,6 +149,16 @@ public TableLogicalPlanner( this.symbolAllocator = requireNonNull(symbolAllocator, "symbolAllocator is null"); this.warningCollector = requireNonNull(warningCollector, "warningCollector is null"); this.planOptimizers = planOptimizers; + this.predicateWithUncorrelatedScalarSubqueryReconstructor = + new PredicateWithUncorrelatedScalarSubqueryReconstructor(); + } + + @TestOnly + public void setPredicateWithUncorrelatedScalarSubqueryReconstructor( + PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor) { + this.predicateWithUncorrelatedScalarSubqueryReconstructor = + predicateWithUncorrelatedScalarSubqueryReconstructor; } public LogicalQueryPlan plan(final Analysis analysis) { @@ -389,7 +403,13 @@ private RelationPlan createRelationPlan(Analysis analysis, Delete statement) { private RelationPlanner getRelationPlanner(Analysis analysis) { return new RelationPlanner( - analysis, symbolAllocator, queryContext, Optional.empty(), sessionInfo, ImmutableMap.of()); + analysis, + symbolAllocator, + queryContext, + Optional.empty(), + sessionInfo, + ImmutableMap.of(), + predicateWithUncorrelatedScalarSubqueryReconstructor); } private PlanNode planCreateOrUpdateDevice( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 5165a2ad097d..bce197c2681a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.ir; import org.apache.iotdb.commons.exception.IoTDBException; -import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.MPPQueryContext.ExplainType; @@ -32,6 +31,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BinaryLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DereferenceExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DoubleLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall; @@ -62,6 +62,8 @@ public class PredicateWithUncorrelatedScalarSubqueryReconstructor { private static final Coordinator coordinator = Coordinator.getInstance(); + public PredicateWithUncorrelatedScalarSubqueryReconstructor() {} + public void reconstructPredicateWithUncorrelatedScalarSubquery( MPPQueryContext context, Analysis analysis, Expression expression) { if (expression instanceof LogicalExpression) { @@ -77,14 +79,18 @@ public void reconstructPredicateWithUncorrelatedScalarSubquery( ComparisonExpression comparisonExpression = (ComparisonExpression) expression; Expression left = comparisonExpression.getLeft(); Expression right = comparisonExpression.getRight(); - if ((left instanceof Identifier || left instanceof FunctionCall) + if ((left instanceof Identifier + || left instanceof FunctionCall + || left instanceof DereferenceExpression) && right instanceof SubqueryExpression) { Optional result = fetchUncorrelatedSubqueryResultForPredicate( context, analysis.getSqlParser(), (SubqueryExpression) right, analysis.getWith()); // If the subquery result is not present, we cannot reconstruct the predicate. result.ifPresent(comparisonExpression::setShadowRight); - } else if ((right instanceof Identifier || right instanceof FunctionCall) + } else if ((right instanceof Identifier + || right instanceof FunctionCall + || right instanceof DereferenceExpression) && left instanceof SubqueryExpression) { Optional result = fetchUncorrelatedSubqueryResultForPredicate( @@ -227,22 +233,4 @@ public void clearShadowExpression(Expression expression) { clearShadowExpression(comparisonExpression.getRight()); } } - - private static class PredicateWithUncorrelatedScalarSubqueryReconstructorHolder { - private static PredicateWithUncorrelatedScalarSubqueryReconstructor INSTANCE = - new PredicateWithUncorrelatedScalarSubqueryReconstructor(); - - private PredicateWithUncorrelatedScalarSubqueryReconstructorHolder() { - // Empty constructor - } - } - - public static PredicateWithUncorrelatedScalarSubqueryReconstructor getInstance() { - return PredicateWithUncorrelatedScalarSubqueryReconstructorHolder.INSTANCE; - } - - @TestOnly - public static void setInstance(PredicateWithUncorrelatedScalarSubqueryReconstructor instance) { - PredicateWithUncorrelatedScalarSubqueryReconstructorHolder.INSTANCE = instance; - } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java index 4d73cc9d003b..d7d4169a3603 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanTester.java @@ -39,6 +39,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.execution.querystats.PlanOptimizersStatsCollector; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.planner.distribute.TableDistributedPlanner; +import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.PredicateWithUncorrelatedScalarSubqueryReconstructor; import org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.DataNodeLocationSupplierFactory; import org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.PlanOptimizer; import org.apache.iotdb.db.queryengine.plan.relational.security.AllowAllAccessControl; @@ -105,15 +106,28 @@ public PlanTester(Metadata metadata) { } public LogicalQueryPlan createPlan(String sql) { - return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), false); + return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), false, null); + } + + public LogicalQueryPlan createPlan( + String sql, + PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor) { + return createPlan( + sessionInfo, + sql, + NOOP, + createPlanOptimizersStatsCollector(), + false, + predicateWithUncorrelatedScalarSubqueryReconstructor); } public LogicalQueryPlan createPlan(String sql, boolean explain) { - return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), explain); + return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), explain, null); } public LogicalQueryPlan createPlan(SessionInfo sessionInfo, String sql) { - return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), false); + return createPlan(sessionInfo, sql, NOOP, createPlanOptimizersStatsCollector(), false, null); } public LogicalQueryPlan createPlan( @@ -121,7 +135,9 @@ public LogicalQueryPlan createPlan( String sql, WarningCollector warningCollector, PlanOptimizersStatsCollector planOptimizersStatsCollector, - boolean explain) { + boolean explain, + PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubqueryReconstructor) { distributedQueryPlan = null; MPPQueryContext context = new MPPQueryContext(sql, queryId, sessionInfo, null, null); if (explain) { @@ -135,6 +151,10 @@ public LogicalQueryPlan createPlan( TableLogicalPlanner logicalPlanner = new TableLogicalPlanner( context, metadata, sessionInfo, symbolAllocator, WarningCollector.NOOP); + if (predicateWithUncorrelatedScalarSubqueryReconstructor != null) { + logicalPlanner.setPredicateWithUncorrelatedScalarSubqueryReconstructor( + predicateWithUncorrelatedScalarSubqueryReconstructor); + } plan = logicalPlanner.plan(analysis); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java index 2f02b3f758eb..3a0a0acd2036 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/UncorrelatedSubqueryTest.java @@ -63,6 +63,8 @@ public class UncorrelatedSubqueryTest { private PlanTester planTester; + private PredicateWithUncorrelatedScalarSubqueryReconstructor + predicateWithUncorrelatedScalarSubquery; @Before public void setUp() throws Exception { @@ -71,21 +73,20 @@ public void setUp() throws Exception { } private void mockPredicateWithUncorrelatedScalarSubquery() { - PredicateWithUncorrelatedScalarSubqueryReconstructor predicateWithUncorrelatedScalarSubquery = + predicateWithUncorrelatedScalarSubquery = Mockito.spy(new PredicateWithUncorrelatedScalarSubqueryReconstructor()); Mockito.when( predicateWithUncorrelatedScalarSubquery.fetchUncorrelatedSubqueryResultForPredicate( Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any())) .thenReturn(Optional.of(new LongLiteral("1"))); - PredicateWithUncorrelatedScalarSubqueryReconstructor.setInstance( - predicateWithUncorrelatedScalarSubquery); } @Test public void testUncorrelatedScalarSubqueryInWhereClause() { String sql = "SELECT s1 FROM table1 where s1 = (select max(s1) from table1)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); PlanMatchPattern tableScan = tableScan( @@ -113,7 +114,8 @@ public void testUncorrelatedScalarSubqueryInWhereClause() { public void testUncorrelatedScalarSubqueryInWhereClauseWithEnforceSingleRowNode() { String sql = "SELECT s1 FROM table1 where s1 = (select s2 from table1)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); PlanMatchPattern tableScan = tableScan( @@ -134,7 +136,8 @@ public void testUncorrelatedScalarSubqueryInWhereClauseWithEnforceSingleRowNode( public void testUncorrelatedInPredicateSubquery() { String sql = "SELECT s1 FROM table1 where s1 in (select s1 from table1)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); Expression filterPredicate = new SymbolReference("expr"); @@ -185,7 +188,8 @@ public void testUncorrelatedInPredicateSubquery() { public void testUncorrelatedNotInPredicateSubquery() { String sql = "SELECT s1 FROM table1 where s1 not in (select s1 from table1)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); Expression filterPredicate = new NotExpression(new SymbolReference("expr")); @@ -219,7 +223,8 @@ public void testUncorrelatedNotInPredicateSubquery() { public void testUncorrelatedAnyComparisonSubquery() { String sql = "SELECT s1 FROM table1 where s1 > any (select s1 from table1)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); PlanMatchPattern tableScan1 = tableScan("testdb.table1", ImmutableList.of("s1"), ImmutableSet.of("s1")); @@ -346,7 +351,8 @@ public void testUncorrelatedAnyComparisonSubquery() { public void testUncorrelatedEqualsSomeComparisonSubquery() { String sql = "SELECT s1 FROM table1 where s1 = some (select s1 from table1)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); Expression filterPredicate = new SymbolReference("expr"); @@ -380,7 +386,8 @@ public void testUncorrelatedEqualsSomeComparisonSubquery() { public void testUncorrelatedAllComparisonSubquery() { String sql = "SELECT s1 FROM table1 where s1 != all (select s1 from table1)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); PlanMatchPattern tableScan1 = tableScan("testdb.table1", ImmutableList.of("s1"), ImmutableSet.of("s1")); @@ -409,7 +416,8 @@ public void testUncorrelatedAllComparisonSubquery() { public void testUncorrelatedExistsSubquery() { String sql = "SELECT s1 FROM table1 where exists(select s2 from table2)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); PlanMatchPattern tableScan1 = tableScan("testdb.table1", ImmutableList.of("s1"), ImmutableSet.of("s1")); @@ -456,7 +464,8 @@ public void testUncorrelatedExistsSubquery() { public void testUncorrelatedNotExistsSubquery() { String sql = "SELECT s1 FROM table1 where not exists(select s2 from table2)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); PlanMatchPattern tableScan1 = tableScan("testdb.table1", ImmutableList.of("s1"), ImmutableSet.of("s1")); @@ -505,7 +514,8 @@ public void testUncorrelatedNotExistsSubquery() { public void testUncorrelatedHavingSubquery() { String sql = "SELECT min(time) as min FROM table1 group by s1 having min(time) > (select max(time) from table2)"; - LogicalQueryPlan logicalQueryPlan = planTester.createPlan(sql); + LogicalQueryPlan logicalQueryPlan = + planTester.createPlan(sql, predicateWithUncorrelatedScalarSubquery); PlanMatchPattern tableScan = tableScan("testdb.table1", ImmutableList.of("time", "s1"), ImmutableSet.of("time", "s1")); From 22cfc091fbc14f58528f673c27b04c90f5661d0d Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 26 Dec 2025 11:30:06 +0800 Subject: [PATCH 68/69] materialize the query only once --- .../plan/relational/planner/CteMaterializer.java | 3 ++- .../db/queryengine/plan/relational/sql/ast/Query.java | 11 +++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 2a706444aafe..be37ec172265 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -82,9 +82,10 @@ public void materializeCTE(Analysis analysis, MPPQueryContext context) { (tableRef, query) -> { Table table = tableRef.getNode(); if (query.isMaterialized()) { - if (!query.isDone()) { + if (!query.isExecuted()) { CteDataStore dataStore = fetchCteQueryResult(context, table, query, analysis.getWith()); + query.setExecuted(true); if (dataStore == null) { // CTE query execution failed. Use inline instead of materialization // in the outer query diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java index 7ef8e56ceb2f..5bed50b17d89 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Query.java @@ -44,6 +44,9 @@ public class Query extends Statement { private final Optional limit; // whether this query needs materialization private boolean materialized = false; + // whether this query has ever been executed + private boolean isExecuted = false; + // materialization has been executed successfully if cteDataStore is not null private CteDataStore cteDataStore = null; public Query( @@ -115,6 +118,14 @@ public void setMaterialized(boolean materialized) { this.materialized = materialized; } + public boolean isExecuted() { + return isExecuted; + } + + public void setExecuted(boolean executed) { + isExecuted = executed; + } + public boolean isDone() { return cteDataStore != null; } From fe47b8f7e51a40dac9b8a470816f443b6374fd35 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 26 Dec 2025 14:28:05 +0800 Subject: [PATCH 69/69] memory for CteScanOperator / CteScanReader --- .../source/relational/CteScanOperator.java | 44 +++++----------- .../plan/planner/TableOperatorGenerator.java | 6 ++- .../relational/planner/CteMaterializer.java | 4 +- .../iotdb/db/utils/cte/CteDataReader.java | 10 +--- .../iotdb/db/utils/cte/CteDataStore.java | 30 ++++++++--- .../iotdb/db/utils/cte/MemoryReader.java | 31 +++++++---- .../operator/CteScanOperatorTest.java | 52 +++++++++++-------- 7 files changed, 95 insertions(+), 82 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java index 5e60b5cef105..dc2cafab1791 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/CteScanOperator.java @@ -21,10 +21,10 @@ package org.apache.iotdb.db.queryengine.execution.operator.source.relational; -import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.execution.operator.source.SourceOperator; +import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.utils.cte.CteDataReader; import org.apache.iotdb.db.utils.cte.CteDataStore; @@ -36,6 +36,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static java.util.Objects.requireNonNull; + public class CteScanOperator implements SourceOperator { private static final Logger LOGGER = LoggerFactory.getLogger(CteScanOperator.class); private static final long INSTANCE_SIZE = @@ -46,42 +48,33 @@ public class CteScanOperator implements SourceOperator { private final OperatorContext operatorContext; private final PlanNodeId sourceId; - - private final CteDataStore dataStore; private final CteDataReader dataReader; - private final int dataStoreRefCount; public CteScanOperator( - OperatorContext operatorContext, PlanNodeId sourceId, CteDataStore dataStore) { + OperatorContext operatorContext, + PlanNodeId sourceId, + CteDataStore dataStore, + MemoryReservationManager memoryReservationManager) { + requireNonNull(dataStore, "dataStore is null"); this.operatorContext = operatorContext; this.sourceId = sourceId; - this.dataStore = dataStore; - this.dataReader = new MemoryReader(dataStore.getCachedData()); - this.dataStoreRefCount = dataStore.increaseRefCount(); + this.dataReader = new MemoryReader(dataStore, memoryReservationManager); } @Override public TsBlock next() throws Exception { - if (dataReader == null) { - return null; - } return dataReader.next(); } @Override public boolean hasNext() throws Exception { - if (dataReader == null) { - return false; - } return dataReader.hasNext(); } @Override public void close() throws Exception { try { - if (dataReader != null) { - dataReader.close(); - } + dataReader.close(); } catch (Exception e) { LOGGER.error("Fail to close CteDataReader", e); } @@ -109,15 +102,9 @@ public long calculateRetainedSizeAfterCallingNext() { @Override public long ramBytesUsed() { - long bytes = - INSTANCE_SIZE - + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext) - + dataReader.bytesUsed(); - if (dataStoreRefCount == 1) { - bytes += dataStore.getCachedBytes(); - } - - return bytes; + return INSTANCE_SIZE + + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext) + + dataReader.ramBytesUsed(); } @Override @@ -129,9 +116,4 @@ public OperatorContext getOperatorContext() { public PlanNodeId getSourceId() { return sourceId; } - - @TestOnly - public int getDataStoreRefCount() { - return dataStoreRefCount; - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index d2da556ee832..cd88853793b9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -1202,7 +1202,11 @@ public Operator visitCteScan(CteScanNode node, LocalExecutionPlanContext context context.getNextOperatorId(), node.getPlanNodeId(), CteScanOperator.class.getSimpleName()); - return new CteScanOperator(operatorContext, node.getPlanNodeId(), node.getDataStore()); + return new CteScanOperator( + operatorContext, + node.getPlanNodeId(), + node.getDataStore(), + context.getInstanceContext().getMemoryReservationContext()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index be37ec172265..a6ac7a422464 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -177,7 +177,7 @@ public CteDataStore fetchCteQueryResult( table, "!!! Failed to materialize CTE. The main query falls back to INLINE mode !!!"); } - context.releaseMemoryReservedForFrontEnd(cteDataStore.getCachedBytes()); + context.releaseMemoryReservedForFrontEnd(cteDataStore.ramBytesUsed()); cteDataStore.clear(); return null; } @@ -192,7 +192,7 @@ public CteDataStore fetchCteQueryResult( return cteDataStore; } catch (final Throwable throwable) { if (cteDataStore != null) { - context.releaseMemoryReservedForFrontEnd(cteDataStore.getCachedBytes()); + context.releaseMemoryReservedForFrontEnd(cteDataStore.ramBytesUsed()); cteDataStore.clear(); } t = throwable; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java index fabb13439aa5..10a9cc93d659 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataReader.java @@ -24,8 +24,9 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.utils.Accountable; -public interface CteDataReader { +public interface CteDataReader extends Accountable { /** * Check if there is more data in CteDataReader. DiskSpillerReader may run out of current TsBlocks * , then it needs to read from file and cache more data. This method should be called before @@ -48,11 +49,4 @@ public interface CteDataReader { * @throws IoTDBException the error occurs when closing fileChannel */ void close() throws IoTDBException; - - /** - * Get the bytes used by this CteDataReader. - * - * @return the bytes used by this CteDataReader - */ - long bytesUsed(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java index 613f6f87fa55..cb37759e2009 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/CteDataStore.java @@ -21,17 +21,23 @@ package org.apache.iotdb.db.utils.cte; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.utils.Accountable; +import org.apache.tsfile.utils.RamUsageEstimator; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -public class CteDataStore { +public class CteDataStore implements Accountable { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(CteDataStore.class); + private final TableSchema tableSchema; private final List columnIndex2TsBlockColumnIndexList; @@ -39,7 +45,7 @@ public class CteDataStore { private long cachedBytes; private int cachedRows; - // reference count by CteScanOperator + // reference count by CteScanReader private final AtomicInteger count; public CteDataStore(TableSchema tableSchema, List columnIndex2TsBlockColumnIndexList) { @@ -75,10 +81,6 @@ public List getCachedData() { return cachedData; } - public long getCachedBytes() { - return cachedBytes; - } - public TableSchema getTableSchema() { return tableSchema; } @@ -87,7 +89,21 @@ public List getColumnIndex2TsBlockColumnIndexList() { return columnIndex2TsBlockColumnIndexList; } - public int increaseRefCount() { + public int incrementAndGetCount() { return count.incrementAndGet(); } + + public int decrementAndGetCount() { + return count.decrementAndGet(); + } + + @Override + public long ramBytesUsed() { + return INSTANCE_SIZE + cachedBytes; + } + + @TestOnly + public int getCount() { + return count.get(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java index 1e0ebd6c2891..cd5103f396ee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/cte/MemoryReader.java @@ -22,43 +22,54 @@ package org.apache.iotdb.db.utils.cte; import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.utils.RamUsageEstimator; -import java.util.List; - public class MemoryReader implements CteDataReader { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(MemoryReader.class); + // thread-safe memory manager + private final MemoryReservationManager memoryReservationManager; // all the data in MemoryReader lies in memory - private final List cachedData; + private final CteDataStore dataStore; private int tsBlockIndex; - public MemoryReader(List cachedTsBlock) { - this.cachedData = cachedTsBlock; + public MemoryReader(CteDataStore dataStore, MemoryReservationManager memoryReservationManager) { + this.dataStore = dataStore; this.tsBlockIndex = 0; + this.memoryReservationManager = memoryReservationManager; + if (dataStore.incrementAndGetCount() == 1) { + memoryReservationManager.reserveMemoryCumulatively(dataStore.ramBytesUsed()); + } } @Override public boolean hasNext() throws IoTDBException { - return cachedData != null && tsBlockIndex < cachedData.size(); + return dataStore.getCachedData() != null && tsBlockIndex < dataStore.getCachedData().size(); } @Override public TsBlock next() throws IoTDBException { - if (cachedData == null || tsBlockIndex >= cachedData.size()) { + if (dataStore.getCachedData() == null || tsBlockIndex >= dataStore.getCachedData().size()) { return null; } - return cachedData.get(tsBlockIndex++); + return dataStore.getCachedData().get(tsBlockIndex++); } @Override - public void close() throws IoTDBException {} + public void close() throws IoTDBException { + if (dataStore.decrementAndGetCount() == 0) { + memoryReservationManager.releaseMemoryCumulatively(dataStore.ramBytesUsed()); + } + } @Override - public long bytesUsed() { + public long ramBytesUsed() { + // The calculation excludes the memory occupied by the CteDataStore. + // memory allocate/release for CteDataStore is handled during constructor and close return INSTANCE_SIZE; } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java index 482b85d6dc7f..9a36c47f8950 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/CteScanOperatorTest.java @@ -22,13 +22,16 @@ package org.apache.iotdb.db.queryengine.execution.operator; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.CteScanOperator; +import org.apache.iotdb.db.queryengine.plan.planner.memory.ThreadSafeMemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.utils.cte.CteDataStore; import com.google.common.collect.ImmutableList; +import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.block.TsBlockBuilder; @@ -39,7 +42,6 @@ import org.apache.tsfile.read.common.type.StringType; import org.apache.tsfile.read.common.type.TimestampType; import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.utils.RamUsageEstimator; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -60,6 +62,7 @@ public class CteScanOperatorTest { private OperatorContext operatorContext; private PlanNodeId planNodeId; private CteDataStore cteDataStore; + private ThreadSafeMemoryReservationManager memoryReservationManager; private CteScanOperator cteScanOperator; @Before @@ -82,6 +85,8 @@ public void setUp() { for (TsBlock tsBlock : testData) { cteDataStore.addTsBlock(tsBlock); } + + memoryReservationManager = new ThreadSafeMemoryReservationManager(new QueryId("1"), "test"); } @After @@ -93,8 +98,9 @@ public void tearDown() throws Exception { @Test public void testConstructor() throws Exception { - cteScanOperator = new CteScanOperator(operatorContext, planNodeId, cteDataStore); - assertEquals(1, cteScanOperator.getDataStoreRefCount()); + cteScanOperator = + new CteScanOperator(operatorContext, planNodeId, cteDataStore, memoryReservationManager); + assertEquals(1, cteDataStore.getCount()); cteScanOperator.close(); } @@ -104,7 +110,8 @@ public void testEmptyDataStore() throws Exception { TableSchema tableSchema = createTestTableSchema(); CteDataStore emptyDataStore = new CteDataStore(tableSchema, Arrays.asList(0, 1, 2)); - cteScanOperator = new CteScanOperator(operatorContext, planNodeId, emptyDataStore); + cteScanOperator = + new CteScanOperator(operatorContext, planNodeId, emptyDataStore, memoryReservationManager); // Should not have data assertFalse(cteScanOperator.hasNext()); @@ -113,7 +120,8 @@ public void testEmptyDataStore() throws Exception { @Test public void testNextWithData() throws Exception { - cteScanOperator = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + cteScanOperator = + new CteScanOperator(operatorContext, planNodeId, cteDataStore, memoryReservationManager); // Should have data assertTrue(cteScanOperator.hasNext()); TsBlock firstBlock = cteScanOperator.next(); @@ -137,7 +145,8 @@ public void testNextWithData() throws Exception { @Test public void testIsFinished() throws Exception { - cteScanOperator = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + cteScanOperator = + new CteScanOperator(operatorContext, planNodeId, cteDataStore, memoryReservationManager); // Initially not finished assertFalse(cteScanOperator.isFinished()); @@ -153,15 +162,13 @@ public void testIsFinished() throws Exception { @Test public void testMemory() throws Exception { - cteScanOperator = new CteScanOperator(operatorContext, planNodeId, cteDataStore); + cteScanOperator = + new CteScanOperator(operatorContext, planNodeId, cteDataStore, memoryReservationManager); - // maxPeekMemory + maxReturnSize + retainedSize - long maxPeekMemory = cteScanOperator.calculateMaxPeekMemory(); - assertEquals(RamUsageEstimator.NUM_BYTES_OBJECT_REF, maxPeekMemory); - long maxReturnSize = cteScanOperator.calculateMaxReturnSize(); - assertEquals(RamUsageEstimator.NUM_BYTES_OBJECT_REF, maxReturnSize); - long retainedSize = cteScanOperator.calculateRetainedSizeAfterCallingNext(); - assertEquals(0L, retainedSize); + long maxReturnSize = TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + assertEquals(maxReturnSize, cteScanOperator.calculateMaxPeekMemory()); + assertEquals(maxReturnSize, cteScanOperator.calculateMaxPeekMemory()); + assertEquals(0L, cteScanOperator.calculateRetainedSizeAfterCallingNext()); cteScanOperator.close(); } @@ -169,15 +176,14 @@ public void testMemory() throws Exception { @Test public void testMultipleCteScanOperators() throws Exception { // Test reference counting with multiple operators - CteScanOperator operator1 = new CteScanOperator(operatorContext, planNodeId, cteDataStore); - CteScanOperator operator2 = new CteScanOperator(operatorContext, planNodeId, cteDataStore); - - // CteDataStore Reference count - assertEquals(1, operator1.getDataStoreRefCount()); - assertEquals(2, operator2.getDataStoreRefCount()); - - // Operator Memory - assertTrue(operator1.ramBytesUsed() > operator2.ramBytesUsed()); + CteScanOperator operator1 = + new CteScanOperator(operatorContext, planNodeId, cteDataStore, memoryReservationManager); + assertEquals(1, cteDataStore.getCount()); + CteScanOperator operator2 = + new CteScanOperator(operatorContext, planNodeId, cteDataStore, memoryReservationManager); + assertEquals(2, cteDataStore.getCount()); + + assertEquals(896, cteDataStore.ramBytesUsed()); // Both operators should be able to read data assertTrue(operator1.hasNext());