From 97adac19f8c02600e1306a39e61b8237bf5f856b Mon Sep 17 00:00:00 2001 From: daidai Date: Sun, 1 Dec 2024 20:17:20 +0800 Subject: [PATCH 01/31] [fix](maxcompute)add mc catalog read partition table partition prune (#44508) ### What problem does this PR solve? Based on the `requiredPartitions` API interface of the maxcompute API, we implemented partition pruning when reading partitioned tables to avoid queries that scan the entire table. --- .../doris/datasource/ExternalTable.java | 9 +- .../datasource/hive/HMSExternalTable.java | 4 +- .../datasource/hive/source/HiveScanNode.java | 2 +- .../maxcompute/MaxComputeExternalTable.java | 33 +- .../maxcompute/source/MaxComputeScanNode.java | 50 +++- .../translator/PhysicalPlanTranslator.java | 3 +- .../expression/rules/PartitionPruner.java | 2 +- .../rules/rewrite/PruneFileScanPartition.java | 4 +- .../doris/planner/SingleNodePlanner.java | 5 +- .../test_max_compute_partition_prune.out | 125 ++++++++ .../test_max_compute_partition_prune.groovy | 282 ++++++++++++++++++ 11 files changed, 497 insertions(+), 22 deletions(-) create mode 100644 regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out create mode 100644 regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index d82959954f26072..6c72d0a67d4732c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -377,7 +377,7 @@ public TableIndexes getTableIndexes() { * @return */ public SelectedPartitions initSelectedPartitions(Optional snapshot) { - if (!supportPartitionPruned()) { + if (!supportInternalPartitionPruned()) { return SelectedPartitions.NOT_PRUNED; } if (CollectionUtils.isEmpty(this.getPartitionColumns(snapshot))) { @@ -394,7 +394,7 @@ public SelectedPartitions initSelectedPartitions(Optional snapshot * @param snapshot if not support mvcc, ignore this * @return partitionName ==> PartitionItem */ - public Map getNameToPartitionItems(Optional snapshot) { + protected Map getNameToPartitionItems(Optional snapshot) { return Collections.emptyMap(); } @@ -410,11 +410,12 @@ public List getPartitionColumns(Optional snapshot) { } /** - * Does it support partition cpruned, If so, this method needs to be overridden in subclasses + * Does it support Internal partition pruned, If so, this method needs to be overridden in subclasses + * Internal partition pruned : Implement partition pruning logic without relying on external APIs. * * @return */ - public boolean supportPartitionPruned() { + public boolean supportInternalPartitionPruned() { return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index 6d65f8bcdbccb7e..134ad362fa1eedb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -301,12 +301,12 @@ public List getPartitionColumns(Optional snapshot) { } @Override - public boolean supportPartitionPruned() { + public boolean supportInternalPartitionPruned() { return getDlaType() == DLAType.HIVE; } @Override - public Map getNameToPartitionItems(Optional snapshot) { + protected Map getNameToPartitionItems(Optional snapshot) { return getNameToPartitionItems(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index 3a2a4d3eb5c6ae0..99d3cd1cd216226 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -82,7 +82,7 @@ public class HiveScanNode extends FileQueryScanNode { // will only be set in Nereids, for lagency planner, it should be null @Setter - private SelectedPartitions selectedPartitions = null; + protected SelectedPartitions selectedPartitions = null; private boolean partitionInit = false; private final AtomicReference batchException = new AtomicReference<>(null); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java index dc3232f79f5f716..0f748f59e927bc5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MapType; +import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; @@ -28,6 +29,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.thrift.TMCTable; import org.apache.doris.thrift.TTableDescriptor; import org.apache.doris.thrift.TTableType; @@ -50,6 +52,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; import java.util.stream.Collectors; @@ -71,6 +74,15 @@ protected synchronized void makeSureInitialized() { } } + @Override + public boolean supportInternalPartitionPruned() { + return true; + } + + @Override + public List getPartitionColumns(Optional snapshot) { + return getPartitionColumns(); + } public List getPartitionColumns() { makeSureInitialized(); @@ -79,7 +91,24 @@ public List getPartitionColumns() { .orElse(Collections.emptyList()); } - public TablePartitionValues getPartitionValues() { + @Override + protected Map getNameToPartitionItems(Optional snapshot) { + if (getPartitionColumns().isEmpty()) { + return Collections.emptyMap(); + } + + TablePartitionValues tablePartitionValues = getPartitionValues(); + Map idToPartitionItem = tablePartitionValues.getIdToPartitionItem(); + Map idToNameMap = tablePartitionValues.getPartitionIdToNameMap(); + + Map nameToPartitionItem = Maps.newHashMapWithExpectedSize(idToPartitionItem.size()); + for (Entry entry : idToPartitionItem.entrySet()) { + nameToPartitionItem.put(idToNameMap.get(entry.getKey()), entry.getValue()); + } + return nameToPartitionItem; + } + + private TablePartitionValues getPartitionValues() { makeSureInitialized(); Optional schemaCacheValue = getSchemaCacheValue(); if (!schemaCacheValue.isPresent()) { @@ -110,6 +139,8 @@ private TablePartitionValues loadPartitionValues(MaxComputeSchemaCacheValue sche /** * parse all values from partitionPath to a single list. + * In MaxCompute : Support special characters : _$#.!@ + * Ref : MaxCompute Error Code: ODPS-0130071 Invalid partition value. * * @param partitionColumns partitionColumns can contain the part1,part2,part3... * @param partitionPath partitionPath format is like the 'part1=123/part2=abc/part3=1bc' diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java index e0b84b0860e5514..e177e9d8b7c88c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java @@ -40,6 +40,7 @@ import org.apache.doris.datasource.maxcompute.MaxComputeExternalTable; import org.apache.doris.datasource.maxcompute.source.MaxComputeSplit.SplitType; import org.apache.doris.datasource.property.constants.MCProperties; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.nereids.util.DateUtils; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.spi.Split; @@ -50,6 +51,7 @@ import org.apache.doris.thrift.TTableFormatFileDesc; import com.aliyun.odps.OdpsType; +import com.aliyun.odps.PartitionSpec; import com.aliyun.odps.table.TableIdentifier; import com.aliyun.odps.table.configuration.ArrowOptions; import com.aliyun.odps.table.configuration.ArrowOptions.TimestampUnit; @@ -60,6 +62,7 @@ import com.aliyun.odps.table.read.split.impl.IndexedInputSplit; import com.google.common.collect.Maps; import jline.internal.Log; +import lombok.Setter; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -86,14 +89,28 @@ public class MaxComputeScanNode extends FileQueryScanNode { private static final LocationPath ROW_OFFSET_PATH = new LocationPath("/row_offset", Maps.newHashMap()); private static final LocationPath BYTE_SIZE_PATH = new LocationPath("/byte_size", Maps.newHashMap()); + @Setter + private SelectedPartitions selectedPartitions = null; + + // For new planner + public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, + SelectedPartitions selectedPartitions, boolean needCheckColumnPriv) { + this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, + selectedPartitions, needCheckColumnPriv); + } + + // For old planner public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv) { - this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, needCheckColumnPriv); + this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, + SelectedPartitions.NOT_PRUNED, needCheckColumnPriv); } - public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, - StatisticalType statisticalType, boolean needCheckColumnPriv) { + private MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, + StatisticalType statisticalType, SelectedPartitions selectedPartitions, + boolean needCheckColumnPriv) { super(id, desc, planNodeName, statisticalType, needCheckColumnPriv); table = (MaxComputeExternalTable) desc.getTable(); + this.selectedPartitions = selectedPartitions; } @Override @@ -117,10 +134,27 @@ private void setScanParams(TFileRangeDesc rangeDesc, MaxComputeSplit maxComputeS rangeDesc.setSize(maxComputeSplit.getLength()); } - void createTableBatchReadSession() throws UserException { + // Return false if no need to read any partition data. + // Return true if need to read partition data. + boolean createTableBatchReadSession() throws UserException { List requiredPartitionColumns = new ArrayList<>(); List orderedRequiredDataColumns = new ArrayList<>(); + List requiredPartitionSpecs = new ArrayList<>(); + //if requiredPartitionSpecs is empty, get all partition data. + if (!table.getPartitionColumns().isEmpty() && selectedPartitions != SelectedPartitions.NOT_PRUNED) { + this.totalPartitionNum = selectedPartitions.totalPartitionNum; + this.selectedPartitionNum = selectedPartitions.selectedPartitions.size(); + + if (selectedPartitions.selectedPartitions.isEmpty()) { + //no need read any partition data. + return false; + } + selectedPartitions.selectedPartitions.forEach( + (key, value) -> requiredPartitionSpecs.add(new PartitionSpec(key)) + ); + } + Set requiredSlots = desc.getSlots().stream().map(e -> e.getColumn().getName()).collect(Collectors.toSet()); @@ -150,6 +184,7 @@ void createTableBatchReadSession() throws UserException { .withSettings(mcCatalog.getSettings()) .withSplitOptions(mcCatalog.getSplitOption()) .requiredPartitionColumns(requiredPartitionColumns) + .requiredPartitions(requiredPartitionSpecs) .requiredDataColumns(orderedRequiredDataColumns) .withArrowOptions( ArrowOptions.newBuilder() @@ -162,7 +197,7 @@ void createTableBatchReadSession() throws UserException { } catch (java.io.IOException e) { throw new RuntimeException(e); } - + return true; } @Override @@ -430,7 +465,10 @@ public List getSplits() throws UserException { if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { return result; } - createTableBatchReadSession(); + + if (!createTableBatchReadSession()) { + return result; + } try { String scanSessionSerialize = serializeSession(tableBatchReadSession); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 56ae65ec7229413..c474e6bd56e1d7e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -579,7 +579,8 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla } else if (table instanceof TrinoConnectorExternalTable) { scanNode = new TrinoConnectorScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else if (table instanceof MaxComputeExternalTable) { - scanNode = new MaxComputeScanNode(context.nextPlanNodeId(), tupleDescriptor, false); + scanNode = new MaxComputeScanNode(context.nextPlanNodeId(), tupleDescriptor, + fileScan.getSelectedPartitions(), false); } else if (table instanceof LakeSoulExternalTable) { scanNode = new LakeSoulScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java index fac1a7f82d2cfb8..ed783aa3d5a9b69 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java @@ -55,7 +55,7 @@ public class PartitionPruner extends DefaultExpressionRewriter { /** Different type of table may have different partition prune behavior. */ public enum PartitionTableType { OLAP, - HIVE + EXTERNAL } private PartitionPruner(List partitions, Expression partitionPredicate) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java index 4bbb0a8aa762708..ba8b270d1f397d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java @@ -59,7 +59,7 @@ public Rule build() { ExternalTable tbl = scan.getTable(); SelectedPartitions selectedPartitions; - if (tbl.supportPartitionPruned()) { + if (tbl.supportInternalPartitionPruned()) { selectedPartitions = pruneExternalPartitions(tbl, filter, scan, ctx.cascadesContext); } else { // set isPruned so that it won't go pass the partition prune again @@ -91,7 +91,7 @@ private SelectedPartitions pruneExternalPartitions(ExternalTable externalTable, Map nameToPartitionItem = scan.getSelectedPartitions().selectedPartitions; List prunedPartitions = new ArrayList<>(PartitionPruner.prune( - partitionSlots, filter.getPredicate(), nameToPartitionItem, ctx, PartitionTableType.HIVE)); + partitionSlots, filter.getPredicate(), nameToPartitionItem, ctx, PartitionTableType.EXTERNAL)); for (String name : prunedPartitions) { selectedPartitionItems.put(name, nameToPartitionItem.get(name)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java index d94ad0a2552240f..4091640066c1d7a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java @@ -76,7 +76,6 @@ import org.apache.doris.datasource.trinoconnector.source.TrinoConnectorScanNode; import org.apache.doris.qe.ConnectContext; import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException; -import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TPushAggOp; import com.google.common.base.Preconditions; @@ -1993,9 +1992,7 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef, SelectStmt s scanNode = new TrinoConnectorScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); break; case MAX_COMPUTE_EXTERNAL_TABLE: - // TODO: support max compute scan node - scanNode = new MaxComputeScanNode(ctx.getNextNodeId(), tblRef.getDesc(), "MCScanNode", - StatisticalType.MAX_COMPUTE_SCAN_NODE, true); + scanNode = new MaxComputeScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); break; case ES_EXTERNAL_TABLE: scanNode = new EsScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); diff --git a/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out b/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out new file mode 100644 index 000000000000000..8f443829c4c3943 --- /dev/null +++ b/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out @@ -0,0 +1,125 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + diff --git a/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy new file mode 100644 index 000000000000000..e34569117a167fe --- /dev/null +++ b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy @@ -0,0 +1,282 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + + +/* +CREATE TABLE one_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 INT); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (1, 'Alice'); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (2, 'Bob'); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (3, 'Charlie'); +INSERT INTO one_partition_tb PARTITION (part1=2025) VALUES (4, 'David'); +INSERT INTO one_partition_tb PARTITION (part1=2025) VALUES (5, 'Eva'); +CREATE TABLE two_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 STRING, part2 int); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (1, 'Alice'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (2, 'Bob'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (3, 'Charlie'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=2) VALUES (4, 'David'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=2) VALUES (5, 'Eva'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=1) VALUES (6, 'Frank'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=1) VALUES (7, 'Grace'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (8, 'Hannah'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (9, 'Ivy'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (10, 'Jack'); +CREATE TABLE three_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 STRING, part2 INT, part3 STRING); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (1, 'Alice'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (2, 'Bob'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (3, 'Charlie'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q2') VALUES (4, 'David'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q2') VALUES (5, 'Eva'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2025, part3='Q1') VALUES (6, 'Frank'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2025, part3='Q2') VALUES (7, 'Grace'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2024, part3='Q1') VALUES (8, 'Hannah'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2024, part3='Q1') VALUES (9, 'Ivy'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q2') VALUES (10, 'Jack'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q2') VALUES (11, 'Leo'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q3') VALUES (12, 'Mia'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q1') VALUES (13, 'Nina'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q2') VALUES (14, 'Oscar'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q3') VALUES (15, 'Paul'); +select * from one_partition_tb; +select * from two_partition_tb; +select * from three_partition_tb; +show partitions one_partition_tb; +show partitions two_partition_tb; +show partitions three_partition_tb; +*/ + +suite("test_max_compute_partition_prune", "p2,external,maxcompute,external_remote,external_remote_maxcompute") { + + + def one_partition_1_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2024 ORDER BY id;""" + def one_partition_2_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2025 ORDER BY id;""" + def one_partition_3_all = """SELECT * FROM one_partition_tb ORDER BY id;""" + def one_partition_4_all = """SELECT * FROM one_partition_tb WHERE id = 5 ORDER BY id;""" + def one_partition_5_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2024 AND id >= 3 ORDER BY id;""" + + def two_partition_1_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 1 ORDER BY id;""" + def two_partition_2_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'EU' AND part2 = 2 ORDER BY id;""" + def two_partition_3_2 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' ORDER BY id;""" + def two_partition_4_all = """SELECT * FROM two_partition_tb ORDER BY id;""" + def two_partition_5_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 2 AND id > 5 ORDER BY id;""" + def two_partition_6_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'EU' AND part2 = 2 ORDER BY id;""" + + def three_partition_1_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2024 AND part3 = 'Q1' ORDER BY id;""" + def three_partition_2_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2025 AND part3 = 'Q2' ORDER BY id;""" + def three_partition_3_3 = """SELECT * FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 ORDER BY id;""" + def three_partition_4_2 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part3 = 'Q1' ORDER BY id;""" + def three_partition_5_all = """SELECT * FROM three_partition_tb ORDER BY id;""" + def three_partition_6_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q1' ORDER BY id;""" + def three_partition_7_7 = """SELECT * FROM three_partition_tb WHERE part2 = 2025 ORDER BY id;""" + def three_partition_8_2 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part3 = 'Q2' AND id BETWEEN 6 AND 10 ORDER BY id;""" + + + String enabled = context.config.otherConfigs.get("enableMaxComputeTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String ak = context.config.otherConfigs.get("ak") + String sk = context.config.otherConfigs.get("sk"); + String mc_db = "mc_datalake" + String mc_catalog_name = "test_max_compute_partition_prune" + + sql """drop catalog if exists ${mc_catalog_name};""" + sql """ + create catalog if not exists ${mc_catalog_name} properties ( + "type" = "max_compute", + "mc.default.project" = "${mc_db}", + "mc.access_key" = "${ak}", + "mc.secret_key" = "${sk}", + "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api" + ); + """ + sql """ switch ${mc_catalog_name} """ + sql """ use ${mc_db}""" + + qt_one_partition_1_1 one_partition_1_1 + explain { + sql("${one_partition_1_1}") + contains "partition=1/2" + } + + qt_one_partition_2_1 one_partition_2_1 + explain { + sql("${one_partition_2_1}") + contains "partition=1/2" + } + + qt_one_partition_3_all one_partition_3_all + explain { + sql("${one_partition_3_all}") + contains "partition=2/2" + } + + qt_one_partition_4_all one_partition_4_all + explain { + sql("${one_partition_4_all}") + contains "partition=2/2" + } + + qt_one_partition_5_1 one_partition_5_1 + explain { + sql("${one_partition_5_1}") + contains "partition=1/2" + } + + + qt_two_partition_1_1 two_partition_1_1 + explain { + sql("${two_partition_1_1}") + contains "partition=1/4" + } + + qt_two_partition_2_1 two_partition_2_1 + explain { + sql("${two_partition_2_1}") + contains "partition=1/4" + } + + qt_two_partition_3_2 two_partition_3_2 + explain { + sql("${two_partition_3_2}") + contains "partition=2/4" + } + + qt_two_partition_4_all two_partition_4_all + explain { + sql("${two_partition_4_all}") + contains "partition=4/4" + } + + qt_two_partition_5_1 two_partition_5_1 + explain { + sql("${two_partition_5_1}") + contains "partition=1/4" + } + + qt_two_partition_6_1 two_partition_6_1 + explain { + sql("${two_partition_6_1}") + contains "partition=1/4" + } + + + + qt_three_partition_1_1 three_partition_1_1 + explain { + sql("${three_partition_1_1}") + contains "partition=1/10" + } + + qt_three_partition_2_1 three_partition_2_1 + explain { + sql("${three_partition_2_1}") + contains "partition=1/10" + } + + qt_three_partition_3_3 three_partition_3_3 + explain { + sql("${three_partition_3_3}") + contains "partition=3/10" + } + + qt_three_partition_4_2 three_partition_4_2 + explain { + sql("${three_partition_4_2}") + contains "partition=2/10" + } + + qt_three_partition_5_all three_partition_5_all + explain { + sql("${three_partition_5_all}") + contains "partition=10/10" + } + + qt_three_partition_6_1 three_partition_6_1 + explain { + sql("${three_partition_6_1}") + contains "partition=1/10" + } + + qt_three_partition_7_7 three_partition_7_7 + explain { + sql("${three_partition_7_7}") + contains "partition=7/10" + } + + qt_three_partition_8_2 three_partition_8_2 + explain { + sql("${three_partition_8_2}") + contains "partition=2/10" + } + + + // 0 partitions + def one_partition_6_0 = """SELECT * FROM one_partition_tb WHERE part1 = 2023 ORDER BY id;""" + qt_one_partition_6_0 one_partition_6_0 + explain { + sql("${one_partition_6_0}") + contains "partition=0/2" + } + + def two_partition_7_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'CN' AND part2 = 1 ORDER BY id;""" + qt_two_partition_7_0 two_partition_7_0 + explain { + sql("${two_partition_7_0}") + contains "partition=0/4" + } + + def two_partition_8_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 3 ORDER BY id;""" + qt_two_partition_8_0 two_partition_8_0 + explain { + sql("${two_partition_8_0}") + contains "partition=0/4" + } + + def three_partition_9_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2023 AND part3 = 'Q1' ORDER BY id;""" + qt_three_partition_9_0 three_partition_9_0 + explain { + sql("${three_partition_9_0}") + contains "partition=0/10" + } + + def three_partition_10_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_10_0 three_partition_10_0 + explain { + sql("${three_partition_10_0}") + contains "partition=0/10" + } + + def three_partition_11_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_11_0 three_partition_11_0 + explain { + sql("${three_partition_11_0}") + contains "partition=0/10" + } + + } +} \ No newline at end of file From 9ca8048d4d3e60f431b8c46212bdc2d29d844f0b Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Sun, 1 Dec 2024 22:18:42 +0800 Subject: [PATCH 02/31] [fix](iceberg)Bring field_id with parquet files And fix map type's key optional (#44470) ### What problem does this PR solve? 1. Column IDs are required to be stored as [field IDs](http://github.com/apache/parquet-format/blob/40699d05bd24181de6b1457babbee2c16dce3803/src/main/thrift/parquet.thrift#L459) on the parquet schema. ref: https://iceberg.apache.org/spec/?h=field+id#parquet So, we should add field ids. 2. For `MapType`, its key is always required. --- .../table/iceberg/arrow_schema_util.cpp | 134 ++++++++ .../format/table/iceberg/arrow_schema_util.h | 45 +++ .../vec/exec/format/table/iceberg/types.cpp | 3 +- be/src/vec/exec/format/table/iceberg/types.h | 4 + be/src/vec/runtime/vparquet_transformer.cpp | 51 +-- be/src/vec/runtime/vparquet_transformer.h | 5 +- .../iceberg/viceberg_partition_writer.cpp | 2 +- .../table/iceberg/arrow_schema_util_test.cpp | 304 ++++++++++++++++++ .../table/iceberg/schema_parser_test.cpp | 24 ++ 9 files changed, 547 insertions(+), 25 deletions(-) create mode 100644 be/src/vec/exec/format/table/iceberg/arrow_schema_util.cpp create mode 100644 be/src/vec/exec/format/table/iceberg/arrow_schema_util.h create mode 100644 be/test/vec/exec/format/table/iceberg/arrow_schema_util_test.cpp diff --git a/be/src/vec/exec/format/table/iceberg/arrow_schema_util.cpp b/be/src/vec/exec/format/table/iceberg/arrow_schema_util.cpp new file mode 100644 index 000000000000000..35a4d51b7f19596 --- /dev/null +++ b/be/src/vec/exec/format/table/iceberg/arrow_schema_util.cpp @@ -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. + +#include "vec/exec/format/table/iceberg/arrow_schema_util.h" + +#include +#include + +namespace doris { +namespace iceberg { + +const char* ArrowSchemaUtil::PARQUET_FIELD_ID = "PARQUET:field_id"; +const char* ArrowSchemaUtil::ORIGINAL_TYPE = "originalType"; +const char* ArrowSchemaUtil::MAP_TYPE_VALUE = "mapType"; + +Status ArrowSchemaUtil::convert(const Schema* schema, const std::string& timezone, + std::vector>& fields) { + for (const auto& column : schema->columns()) { + std::shared_ptr arrow_field; + RETURN_IF_ERROR(convert_to(column, &arrow_field, timezone)); + fields.push_back(arrow_field); + } + return Status::OK(); +} + +Status ArrowSchemaUtil::convert_to(const iceberg::NestedField& field, + std::shared_ptr* arrow_field, + const std::string& timezone) { + std::shared_ptr arrow_type; + std::unordered_map metadata; + metadata[PARQUET_FIELD_ID] = std::to_string(field.field_id()); + + switch (field.field_type()->type_id()) { + case iceberg::TypeID::BOOLEAN: + arrow_type = arrow::boolean(); + break; + + case iceberg::TypeID::INTEGER: + arrow_type = arrow::int32(); + break; + + case iceberg::TypeID::LONG: + arrow_type = arrow::int64(); + break; + + case iceberg::TypeID::FLOAT: + arrow_type = arrow::float32(); + break; + + case iceberg::TypeID::DOUBLE: + arrow_type = arrow::float64(); + break; + + case iceberg::TypeID::DATE: + arrow_type = arrow::date32(); + break; + + case iceberg::TypeID::TIMESTAMP: { + arrow_type = std::make_shared(arrow::TimeUnit::MICRO, timezone); + break; + } + + case iceberg::TypeID::BINARY: + case iceberg::TypeID::STRING: + case iceberg::TypeID::UUID: + case iceberg::TypeID::FIXED: + arrow_type = arrow::utf8(); + break; + + case iceberg::TypeID::DECIMAL: { + auto dt = dynamic_cast(field.field_type()); + arrow_type = arrow::decimal(dt->get_precision(), dt->get_scale()); + break; + } + + case iceberg::TypeID::STRUCT: { + std::vector> element_fields; + StructType* st = field.field_type()->as_struct_type(); + for (const auto& column : st->fields()) { + std::shared_ptr element_field; + RETURN_IF_ERROR(convert_to(column, &element_field, timezone)); + element_fields.push_back(element_field); + } + arrow_type = arrow::struct_(element_fields); + break; + } + + case iceberg::TypeID::LIST: { + std::shared_ptr item_field; + ListType* list_type = field.field_type()->as_list_type(); + RETURN_IF_ERROR(convert_to(list_type->element_field(), &item_field, timezone)); + arrow_type = arrow::list(item_field); + break; + } + + case iceberg::TypeID::MAP: { + std::shared_ptr key_field; + std::shared_ptr value_field; + MapType* map_type = field.field_type()->as_map_type(); + RETURN_IF_ERROR(convert_to(map_type->key_field(), &key_field, timezone)); + RETURN_IF_ERROR(convert_to(map_type->value_field(), &value_field, timezone)); + metadata[ORIGINAL_TYPE] = MAP_TYPE_VALUE; + arrow_type = std::make_shared(key_field, value_field); + break; + } + + case iceberg::TypeID::TIME: + default: + return Status::InternalError("Unsupported field type:" + field.field_type()->to_string()); + } + + std::shared_ptr schema_metadata = + std::make_shared(metadata); + *arrow_field = + arrow::field(field.field_name(), arrow_type, field.is_optional(), schema_metadata); + return Status::OK(); +} + +} // namespace iceberg +} // namespace doris \ No newline at end of file diff --git a/be/src/vec/exec/format/table/iceberg/arrow_schema_util.h b/be/src/vec/exec/format/table/iceberg/arrow_schema_util.h new file mode 100644 index 000000000000000..20b7dbc627cc44a --- /dev/null +++ b/be/src/vec/exec/format/table/iceberg/arrow_schema_util.h @@ -0,0 +1,45 @@ +// 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. + +#pragma once + +#include + +#include + +#include "vec/exec/format/table/iceberg/schema.h" + +namespace doris { +namespace iceberg { + +class ArrowSchemaUtil { +public: + static Status convert(const Schema* schema, const std::string& timezone, + std::vector>& fields); + +private: + static const char* PARQUET_FIELD_ID; + static const char* ORIGINAL_TYPE; + static const char* MAP_TYPE_VALUE; + + static Status convert_to(const iceberg::NestedField& field, + std::shared_ptr* arrow_field, + const std::string& timezone); +}; + +} // namespace iceberg +} // namespace doris diff --git a/be/src/vec/exec/format/table/iceberg/types.cpp b/be/src/vec/exec/format/table/iceberg/types.cpp index b56a231979ace10..bf643655ab8810f 100644 --- a/be/src/vec/exec/format/table/iceberg/types.cpp +++ b/be/src/vec/exec/format/table/iceberg/types.cpp @@ -25,8 +25,9 @@ namespace iceberg { std::unique_ptr MapType::of_optional(int key_id, int value_id, std::unique_ptr key_type, std::unique_ptr value_type) { + // key is always required auto key_field = - std::make_unique(true, key_id, "key", std::move(key_type), std::nullopt); + std::make_unique(false, key_id, "key", std::move(key_type), std::nullopt); auto value_field = std::make_unique(true, value_id, "value", std::move(value_type), std::nullopt); return std::unique_ptr(new MapType(std::move(key_field), std::move(value_field))); diff --git a/be/src/vec/exec/format/table/iceberg/types.h b/be/src/vec/exec/format/table/iceberg/types.h index f5262b36f55cd33..91a2f705df0d0ba 100644 --- a/be/src/vec/exec/format/table/iceberg/types.h +++ b/be/src/vec/exec/format/table/iceberg/types.h @@ -265,6 +265,10 @@ class DecimalType : public PrimitiveType { ss << "decimal(" << precision << ", " << scale << ")"; return ss.str(); } + + int get_precision() const { return precision; } + + int get_scale() const { return scale; } }; class BinaryType : public PrimitiveType { diff --git a/be/src/vec/runtime/vparquet_transformer.cpp b/be/src/vec/runtime/vparquet_transformer.cpp index ab288537313973f..5409dc0abf3466c 100644 --- a/be/src/vec/runtime/vparquet_transformer.cpp +++ b/be/src/vec/runtime/vparquet_transformer.cpp @@ -65,6 +65,7 @@ #include "vec/core/types.h" #include "vec/data_types/data_type_decimal.h" #include "vec/data_types/data_type_nullable.h" +#include "vec/exec/format/table/iceberg/arrow_schema_util.h" #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" #include "vec/functions/function_helpers.h" @@ -202,21 +203,20 @@ void ParquetBuildHelper::build_version(parquet::WriterProperties::Builder& build } } -VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer, - const VExprContextSPtrs& output_vexpr_ctxs, - std::vector column_names, - TParquetCompressionType::type compression_type, - bool parquet_disable_dictionary, - TParquetVersion::type parquet_version, - bool output_object_data, - const std::string* iceberg_schema_json) +VParquetTransformer::VParquetTransformer( + RuntimeState* state, doris::io::FileWriter* file_writer, + const VExprContextSPtrs& output_vexpr_ctxs, std::vector column_names, + TParquetCompressionType::type compression_type, bool parquet_disable_dictionary, + TParquetVersion::type parquet_version, bool output_object_data, + const std::string* iceberg_schema_json, const iceberg::Schema* iceberg_schema) : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data), _column_names(std::move(column_names)), _parquet_schemas(nullptr), _compression_type(compression_type), _parquet_disable_dictionary(parquet_disable_dictionary), _parquet_version(parquet_version), - _iceberg_schema_json(iceberg_schema_json) { + _iceberg_schema_json(iceberg_schema_json), + _iceberg_schema(iceberg_schema) { _outstream = std::shared_ptr(new ParquetOutputStream(file_writer)); } @@ -234,6 +234,7 @@ VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWri _parquet_disable_dictionary(parquet_disable_dictionary), _parquet_version(parquet_version), _iceberg_schema_json(iceberg_schema_json) { + _iceberg_schema = nullptr; _outstream = std::shared_ptr(new ParquetOutputStream(file_writer)); } @@ -265,21 +266,27 @@ Status VParquetTransformer::_parse_properties() { Status VParquetTransformer::_parse_schema() { std::vector> fields; - for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) { - std::shared_ptr type; - RETURN_IF_ERROR(convert_to_arrow_type(_output_vexpr_ctxs[i]->root()->type(), &type, - _state->timezone())); - if (_parquet_schemas != nullptr) { - std::shared_ptr field = - arrow::field(_parquet_schemas->operator[](i).schema_column_name, type, - _output_vexpr_ctxs[i]->root()->is_nullable()); - fields.emplace_back(field); - } else { - std::shared_ptr field = arrow::field( - _column_names[i], type, _output_vexpr_ctxs[i]->root()->is_nullable()); - fields.emplace_back(field); + if (_iceberg_schema != nullptr) { + RETURN_IF_ERROR( + iceberg::ArrowSchemaUtil::convert(_iceberg_schema, _state->timezone(), fields)); + } else { + for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) { + std::shared_ptr type; + RETURN_IF_ERROR(convert_to_arrow_type(_output_vexpr_ctxs[i]->root()->type(), &type, + _state->timezone())); + if (_parquet_schemas != nullptr) { + std::shared_ptr field = + arrow::field(_parquet_schemas->operator[](i).schema_column_name, type, + _output_vexpr_ctxs[i]->root()->is_nullable()); + fields.emplace_back(field); + } else { + std::shared_ptr field = arrow::field( + _column_names[i], type, _output_vexpr_ctxs[i]->root()->is_nullable()); + fields.emplace_back(field); + } } } + if (_iceberg_schema_json != nullptr) { std::shared_ptr schema_metadata = arrow::KeyValueMetadata::Make({"iceberg.schema"}, {*_iceberg_schema_json}); diff --git a/be/src/vec/runtime/vparquet_transformer.h b/be/src/vec/runtime/vparquet_transformer.h index 9fdbb271373212b..ecc4a8ddeac4bc1 100644 --- a/be/src/vec/runtime/vparquet_transformer.h +++ b/be/src/vec/runtime/vparquet_transformer.h @@ -27,6 +27,7 @@ #include #include +#include "vec/exec/format/table/iceberg/schema.h" #include "vfile_format_transformer.h" namespace doris { @@ -95,7 +96,8 @@ class VParquetTransformer final : public VFileFormatTransformer { std::vector column_names, TParquetCompressionType::type compression_type, bool parquet_disable_dictionary, TParquetVersion::type parquet_version, - bool output_object_data, const std::string* iceberg_schema_json = nullptr); + bool output_object_data, const std::string* iceberg_schema_json = nullptr, + const iceberg::Schema* iceberg_schema = nullptr); VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer, const VExprContextSPtrs& output_vexpr_ctxs, @@ -132,6 +134,7 @@ class VParquetTransformer final : public VFileFormatTransformer { const TParquetVersion::type _parquet_version; const std::string* _iceberg_schema_json; uint64_t _write_size = 0; + const iceberg::Schema* _iceberg_schema; }; } // namespace doris::vectorized diff --git a/be/src/vec/sink/writer/iceberg/viceberg_partition_writer.cpp b/be/src/vec/sink/writer/iceberg/viceberg_partition_writer.cpp index 924adf68145a7a8..23ee389a8b7d104 100644 --- a/be/src/vec/sink/writer/iceberg/viceberg_partition_writer.cpp +++ b/be/src/vec/sink/writer/iceberg/viceberg_partition_writer.cpp @@ -84,7 +84,7 @@ Status VIcebergPartitionWriter::open(RuntimeState* state, RuntimeProfile* profil _file_format_transformer.reset(new VParquetTransformer( state, _file_writer.get(), _write_output_expr_ctxs, _write_column_names, parquet_compression_type, parquet_disable_dictionary, TParquetVersion::PARQUET_1_0, - false, _iceberg_schema_json)); + false, _iceberg_schema_json, &_schema)); return _file_format_transformer->open(); } case TFileFormatType::FORMAT_ORC: { diff --git a/be/test/vec/exec/format/table/iceberg/arrow_schema_util_test.cpp b/be/test/vec/exec/format/table/iceberg/arrow_schema_util_test.cpp new file mode 100644 index 000000000000000..b5f61c9d2e3cdf3 --- /dev/null +++ b/be/test/vec/exec/format/table/iceberg/arrow_schema_util_test.cpp @@ -0,0 +1,304 @@ +// 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. + +#include "vec/exec/format/table/iceberg/arrow_schema_util.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "io/fs/local_file_system.h" +#include "vec/exec/format/table/iceberg/schema.h" +#include "vec/exec/format/table/iceberg/schema_parser.h" + +namespace doris { +namespace iceberg { + +class ArrowSchemaUtilTest : public testing::Test { +public: + ArrowSchemaUtilTest() = default; + virtual ~ArrowSchemaUtilTest() = default; +}; + +const std::string_view pfid = "PARQUET:field_id"; + +TEST(ArrowSchemaUtilTest, test_simple_field) { + std::vector nested_fields; + nested_fields.reserve(2); + NestedField field1(false, 1, "field1", std::make_unique(), std::nullopt); + NestedField field2(false, 2, "field2", std::make_unique(), std::nullopt); + nested_fields.emplace_back(std::move(field1)); + nested_fields.emplace_back(std::move(field2)); + + Schema schema(1, std::move(nested_fields)); + + std::vector> fields; + Status st; + st = ArrowSchemaUtil::convert(&schema, "utc", fields); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(2, fields.size()); + EXPECT_EQ("field1", fields[0]->name()); + EXPECT_EQ("field2", fields[1]->name()); + EXPECT_TRUE(fields[0]->HasMetadata()); + EXPECT_TRUE(fields[1]->HasMetadata()); + EXPECT_EQ("1", fields[0]->metadata()->Get(pfid).ValueUnsafe()); + EXPECT_EQ("2", fields[1]->metadata()->Get(pfid).ValueUnsafe()); +} + +TEST(ArrowSchemaUtilTest, test_stuct_field) { + // struct_json comes from : + // Schema schema = new Schema( + // Types.NestedField.optional( + // 21, "st_col", Types.StructType.of( + // Types.NestedField.optional(32, "st_col_c1", Types.IntegerType.get()), + // Types.NestedField.optional(43, "st_col_c2", Types.StringType.get()) + // ) + // ) + // ); + // StringWriter writer = new StringWriter(); + // JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + // SchemaParser.toJson(schema.asStruct(), generator); + // generator.flush(); + // System.out.println(writer.toString()); + + const std::string struct_json = R"({ + "type": "struct", + "fields": [ + { + "id": 21, + "name": "st_col", + "required": false, + "type": { + "type": "struct", + "fields": [ + { + "id": 32, + "name": "st_col_c1", + "required": false, + "type": "int" + }, + { + "id": 43, + "name": "st_col_c2", + "required": false, + "type": "string" + } + ] + } + } + ] + })"; + std::unique_ptr schema = SchemaParser::from_json(struct_json); + + std::vector> fields; + Status st; + st = ArrowSchemaUtil::convert(schema.get(), "utc", fields); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(1, fields.size()); + EXPECT_EQ("st_col", fields[0]->name()); + EXPECT_EQ("21", fields[0]->metadata()->Get(pfid).ValueUnsafe()); + + arrow::StructType* arrow_struct = dynamic_cast(fields[0]->type().get()); + auto map_fields = arrow_struct->fields(); + EXPECT_EQ(2, arrow_struct->fields().size()); + EXPECT_EQ("st_col_c1", map_fields.at(0).get()->name()); + EXPECT_EQ("st_col_c2", map_fields.at(1).get()->name()); + EXPECT_EQ("32", map_fields.at(0).get()->metadata()->Get(pfid).ValueUnsafe()); + EXPECT_EQ("43", map_fields.at(1).get()->metadata()->Get(pfid).ValueUnsafe()); +} + +TEST(ArrowSchemaUtilTest, test_map_field) { + // map_json comes from : + // Schema schema = new Schema( + // Types.NestedField.optional( + // 21, "map_col", Types.MapType.ofOptional( + // 32, 43, Types.IntegerType.get(), Types.StringType.get() + // ) + // ) + // ); + // StringWriter writer = new StringWriter(); + // JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + // SchemaParser.toJson(schema.asStruct(), generator); + // generator.flush(); + // System.out.println(writer.toString()); + + const std::string map_json = R"({ + "type": "struct", + "fields": [ + { + "id": 21, + "name": "map_col", + "required": false, + "type": { + "type": "map", + "key-id": 32, + "key": "int", + "value-id": 43, + "value": "string", + "value-required": false + } + } + ] + })"; + std::unique_ptr schema = SchemaParser::from_json(map_json); + + std::vector> fields; + Status st; + st = ArrowSchemaUtil::convert(schema.get(), "utc", fields); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(1, fields.size()); + EXPECT_EQ("map_col", fields[0]->name()); + EXPECT_EQ("21", fields[0]->metadata()->Get(pfid).ValueUnsafe()); + + arrow::MapType* arrow_map = dynamic_cast(fields[0]->type().get()); + auto map_fields = arrow_map->fields(); + EXPECT_EQ(1, arrow_map->fields().size()); + EXPECT_EQ("key", arrow_map->key_field()->name()); + EXPECT_EQ("value", arrow_map->item_field()->name()); + EXPECT_EQ("32", arrow_map->key_field()->metadata()->Get(pfid).ValueUnsafe()); + EXPECT_EQ("43", arrow_map->item_field()->metadata()->Get(pfid).ValueUnsafe()); +} + +TEST(ArrowSchemaUtilTest, test_list_field) { + // list_json comes from : + // Schema schema = new Schema( + // Types.NestedField.optional( + // 21, "list_col", Types.ListType.ofOptional( + // 32, Types.IntegerType.get()))); + // StringWriter writer = new StringWriter(); + // JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + // SchemaParser.toJson(schema.asStruct(), generator); + // generator.flush(); + // System.out.println(writer.toString()); + + const std::string list_json = R"({ + "type": "struct", + "fields": [ + { + "id": 21, + "name": "list_col", + "required": false, + "type": { + "type": "list", + "element-id": 32, + "element": "int", + "element-required": false + } + } + ] + })"; + std::unique_ptr schema = SchemaParser::from_json(list_json); + + std::vector> fields; + Status st; + st = ArrowSchemaUtil::convert(schema.get(), "utc", fields); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(1, fields.size()); + EXPECT_EQ("list_col", fields[0]->name()); + EXPECT_EQ("21", fields[0]->metadata()->Get(pfid).ValueUnsafe()); + + arrow::ListType* arrow_list = dynamic_cast(fields[0]->type().get()); + auto map_fields = arrow_list->fields(); + EXPECT_EQ(1, arrow_list->fields().size()); + EXPECT_EQ("element", arrow_list->value_field()->name()); + EXPECT_EQ("32", arrow_list->value_field()->metadata()->Get(pfid).ValueUnsafe()); +} + +TEST(ArrowSchemaUtilTest, test_parquet_filed_id) { + std::string test_dir = "ut_dir/test_parquet_filed_id"; + Status st; + st = io::global_local_filesystem()->delete_directory(test_dir); + ASSERT_TRUE(st.ok()) << st; + st = io::global_local_filesystem()->create_directory(test_dir); + ASSERT_TRUE(st.ok()) << st; + + std::shared_ptr id_array; + std::shared_ptr name_array; + + arrow::Int32Builder id_builder; + ASSERT_TRUE(id_builder.Append(1).ok()); + ASSERT_TRUE(id_builder.Append(2).ok()); + ASSERT_TRUE(id_builder.Append(3).ok()); + auto&& result_id = id_builder.Finish(); + ASSERT_TRUE(result_id.ok()); + id_array = std::move(result_id).ValueUnsafe(); + + arrow::StringBuilder name_builder; + ASSERT_TRUE(name_builder.Append("Alice").ok()); + ASSERT_TRUE(name_builder.Append("Bob").ok()); + ASSERT_TRUE(name_builder.Append("Charlie").ok()); + auto&& result_name = name_builder.Finish(); + ASSERT_TRUE(result_name.ok()); + name_array = std::move(result_name).ValueUnsafe(); + + // 定义表的 Schema + std::vector nested_fields; + nested_fields.reserve(2); + NestedField field1(false, 17, "field_1", std::make_unique(), std::nullopt); + NestedField field2(false, 36, "field_2", std::make_unique(), std::nullopt); + nested_fields.emplace_back(std::move(field1)); + nested_fields.emplace_back(std::move(field2)); + + Schema schema(1, std::move(nested_fields)); + + std::vector> fields; + st = ArrowSchemaUtil::convert(&schema, "utc", fields); + auto arrow_schema = arrow::schema(fields); + + // create arrow table + auto table = arrow::Table::Make(arrow_schema, {id_array, name_array}); + + std::string file_path = test_dir + "/f1.parquet"; + std::shared_ptr outfile; + auto&& result_file = arrow::io::FileOutputStream::Open(file_path); + ASSERT_TRUE(result_file.ok()); + outfile = std::move(result_file).ValueUnsafe(); + + // arrow table to parquet file + PARQUET_THROW_NOT_OK( + parquet::arrow::WriteTable(*table, arrow::default_memory_pool(), outfile, 1024)); + + // open parquet with parquet's API + std::unique_ptr parquet_reader = + parquet::ParquetFileReader::OpenFile(file_path, false); + + // get MessageType + std::shared_ptr file_metadata = parquet_reader->metadata(); + auto schema_descriptor = file_metadata->schema(); + const parquet::schema::Node& root = *schema_descriptor->group_node(); + const auto& group_node = static_cast(root); + + EXPECT_EQ(2, group_node.field_count()); + auto filed1 = group_node.field(0); + auto filed2 = group_node.field(1); + EXPECT_EQ("field_1", filed1->name()); + EXPECT_EQ(17, filed1->field_id()); + EXPECT_EQ("field_2", filed2->name()); + EXPECT_EQ(36, filed2->field_id()); + + st = io::global_local_filesystem()->delete_directory(test_dir); + EXPECT_TRUE(st.ok()) << st; +} + +} // namespace iceberg +} // namespace doris diff --git a/be/test/vec/exec/format/table/iceberg/schema_parser_test.cpp b/be/test/vec/exec/format/table/iceberg/schema_parser_test.cpp index 4c3f58cdd10491a..f464525a7f99bcf 100644 --- a/be/test/vec/exec/format/table/iceberg/schema_parser_test.cpp +++ b/be/test/vec/exec/format/table/iceberg/schema_parser_test.cpp @@ -78,6 +78,15 @@ const std::string valid_map_json = R"({ "value-required": true })"; +const std::string valid_map_json2 = R"({ + "type": "map", + "key-id": 4, + "key": "string", + "value-id": 5, + "value": "int", + "value-required": false +})"; + const std::string nested_list_json = R"({ "type": "list", "element-id": 6, @@ -209,6 +218,21 @@ TEST(SchemaParserTest, parse_valid_map) { SchemaParser::_type_from_json(rapidjson::Document().Parse(valid_map_json.c_str())); ASSERT_NE(type, nullptr); EXPECT_EQ(type->to_string(), "map"); + EXPECT_TRUE(type->is_map_type()); + MapType* mt = type->as_map_type(); + EXPECT_TRUE(mt->field(4)->is_required()); + EXPECT_TRUE(mt->field(5)->is_required()); +} + +TEST(SchemaParserTest, parse_valid_map2) { + std::unique_ptr type = + SchemaParser::_type_from_json(rapidjson::Document().Parse(valid_map_json2.c_str())); + ASSERT_NE(type, nullptr); + EXPECT_EQ(type->to_string(), "map"); + EXPECT_TRUE(type->is_map_type()); + MapType* mt = type->as_map_type(); + EXPECT_TRUE(mt->field(4)->is_required()); + EXPECT_TRUE(mt->field(5)->is_optional()); } TEST(SchemaParserTest, parse_nested_list) { From 456a08ba746900ad1167a7c5397775195ade8195 Mon Sep 17 00:00:00 2001 From: starocean999 Date: Mon, 2 Dec 2024 09:41:10 +0800 Subject: [PATCH 03/31] [refactor](nereids)add NotNullableAggregateFunction as base class for all non-nullable agg functions (#44671) AlwaysNotNullable agg functions must return a non-null value if it's a scalar agg. So a new base class NotNullableAggregateFunction is introduced, all sub classes must implement resultForEmptyInput method to get the non-null value correctly --- .../rules/analysis/SubqueryToApply.java | 8 ++-- .../functions/AlwaysNotNullable.java | 9 ---- .../expressions/functions/agg/ArrayAgg.java | 5 +- .../expressions/functions/agg/BitmapAgg.java | 5 +- .../functions/agg/BitmapIntersect.java | 5 +- .../functions/agg/BitmapUnion.java | 5 +- .../functions/agg/BitmapUnionCount.java | 5 +- .../functions/agg/BitmapUnionInt.java | 5 +- .../functions/agg/CollectList.java | 5 +- .../expressions/functions/agg/CollectSet.java | 5 +- .../expressions/functions/agg/Count.java | 5 +- .../functions/agg/CountByEnum.java | 3 +- .../functions/agg/GroupArrayIntersect.java | 5 +- .../expressions/functions/agg/Histogram.java | 5 +- .../expressions/functions/agg/HllUnion.java | 5 +- .../functions/agg/HllUnionAgg.java | 5 +- .../functions/agg/IntersectCount.java | 5 +- .../functions/agg/LinearHistogram.java | 9 +++- .../expressions/functions/agg/MapAgg.java | 5 +- .../functions/agg/MultiDistinctCount.java | 5 +- .../functions/agg/MultiDistinctSum0.java | 5 +- .../trees/expressions/functions/agg/Ndv.java | 5 +- .../agg/NotNullableAggregateFunction.java | 48 +++++++++++++++++++ .../agg/OrthogonalBitmapIntersect.java | 11 +++-- .../agg/OrthogonalBitmapIntersectCount.java | 13 +++-- .../agg/OrthogonalBitmapUnionCount.java | 11 +++-- .../functions/agg/PercentileArray.java | 5 +- .../functions/agg/QuantileUnion.java | 5 +- .../functions/agg/SequenceCount.java | 5 +- .../trees/expressions/functions/agg/Sum0.java | 5 +- 30 files changed, 129 insertions(+), 93 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java index 14700b030d68b74..4bb55eab9d43787 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java @@ -42,9 +42,9 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.SubqueryExpr; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.NotNullableAggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.scalar.AssertTrue; import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; @@ -424,9 +424,9 @@ private Pair> addApply(SubqueryExpr subquery, Map replaceMap = new HashMap<>(); NamedExpression agg = ((ScalarSubquery) subquery).getTopLevelScalarAggFunction().get(); if (agg instanceof Alias) { - if (((Alias) agg).child() instanceof AlwaysNotNullable) { - AlwaysNotNullable notNullableAggFunc = - (AlwaysNotNullable) ((Alias) agg).child(); + if (((Alias) agg).child() instanceof NotNullableAggregateFunction) { + NotNullableAggregateFunction notNullableAggFunc = + (NotNullableAggregateFunction) ((Alias) agg).child(); if (subquery.getQueryPlan() instanceof LogicalProject) { LogicalProject logicalProject = (LogicalProject) subquery.getQueryPlan(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java index 6b12f9cd6429926..8fda4d4b020f2b7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java @@ -17,9 +17,6 @@ package org.apache.doris.nereids.trees.expressions.functions; -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Expression; - /** * nullable is always false. * @@ -30,10 +27,4 @@ public interface AlwaysNotNullable extends ComputeNullable { default boolean nullable() { return false; } - - // return value of this function if the input data is empty. - // for example, count(*) of empty table is 0; - default Expression resultForEmptyInput() { - throw new AnalysisException("should implement resultForEmptyInput() for " + this.getClass()); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java index bc91207e31f3a62..6218569261a7757 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -37,8 +36,8 @@ /** * AggregateFunction 'array_agg'. */ -public class ArrayAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class ArrayAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)) ); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java index eaf766b908d7af5..a0eaf88efaf13b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -37,8 +36,8 @@ /** * AggregateFunction 'bitmap_agg'. */ -public class BitmapAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class BitmapAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BigIntType.INSTANCE), FunctionSignature.ret(BitmapType.INSTANCE).args(IntegerType.INSTANCE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java index 1b7d2d3c3cddec6..80a9887d325634a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -35,8 +34,8 @@ /** * AggregateFunction 'bitmap_intersect'. This class is generated by GenerateFunction. */ -public class BitmapIntersect extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction { +public class BitmapIntersect extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java index cd0756a1c9361fc..fce2f4fa70c483e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; @@ -36,8 +35,8 @@ /** * AggregateFunction 'bitmap_union'. This class is generated by GenerateFunction. */ -public class BitmapUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction, RollUpTrait { +public class BitmapUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java index 593c814f22d1597..d1e2d6010a27a74 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'bitmap_union_count'. This class is generated by GenerateFunction. */ -public class BitmapUnionCount extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction, RollUpTrait { +public class BitmapUnionCount extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java index 2efe1631176c86d..af569982c4e62e8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -41,8 +40,8 @@ /** * AggregateFunction 'bitmap_union_int'. This class is generated by GenerateFunction. */ -public class BitmapUnionInt extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class BitmapUnionInt extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(SmallIntType.INSTANCE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java index d6cca2d0b90b6d1..ed0dd2406211ce3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -38,8 +37,8 @@ /** * AggregateFunction 'collect_list'. This class is generated by GenerateFunction. */ -public class CollectList extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class CollectList extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java index d9e7e7227c6f869..2aba485fc7daaa6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -38,8 +37,8 @@ /** * AggregateFunction 'collect_set'. This class is generated by GenerateFunction. */ -public class CollectSet extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class CollectSet extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java index 10874d47ee3df95..e86e90974da1bd5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; @@ -37,8 +36,8 @@ import java.util.List; /** count agg function. */ -public class Count extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, SupportWindowAnalytic, RollUpTrait { +public class Count extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, SupportWindowAnalytic, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( // count(*) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java index 2a4ee7be3f49c23..f956854d6871bd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -31,7 +30,7 @@ import java.util.List; /** count_by_enum agg function. */ -public class CountByEnum extends AggregateFunction implements ExplicitlyCastableSignature, AlwaysNotNullable { +public class CountByEnum extends NotNullableAggregateFunction implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java index 0720d6838bb1b63..1cc0db1bd5ed5a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'group_array_intersect'. */ -public class GroupArrayIntersect extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class GroupArrayIntersect extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.retArgType(0) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java index 6b0a2759823f3a9..827c57facd7498e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.SearchSignature; import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'histogram'. This class is generated by GenerateFunction. */ -public class Histogram extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable { +public class Histogram extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java index b81fad270b0cda8..8f6224f66b0043f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllEmpty; @@ -36,8 +35,8 @@ /** * AggregateFunction 'hll_union'. This class is generated by GenerateFunction. */ -public class HllUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, HllFunction, RollUpTrait { +public class HllUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, HllFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(HllType.INSTANCE).args(HllType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java index b14b61b5be07e99..b2c15b402aa2c4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'hll_union_agg'. This class is generated by GenerateFunction. */ -public class HllUnionAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, HllFunction, RollUpTrait { +public class HllUnionAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, HllFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(HllType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java index c013b2e8b4c31d3..3eed9b9a6e4ddb5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.BitmapIntersectFunction; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'intersect_count'. This class is generated by GenerateFunction. */ -public class IntersectCount extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, BitmapIntersectFunction { +public class IntersectCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, BitmapIntersectFunction { public static final List SIGNATURES = DataType.trivialTypes().stream() .map(type -> FunctionSignature.ret(BigIntType.INSTANCE).varArgs(BitmapType.INSTANCE, type, type)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/LinearHistogram.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/LinearHistogram.java index ac172fc7881c7c7..ca8cf3a2f71560c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/LinearHistogram.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/LinearHistogram.java @@ -20,9 +20,9 @@ import org.apache.doris.catalog.FunctionSet; import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.SearchSignature; +import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DoubleType; import org.apache.doris.nereids.types.VarcharType; @@ -36,7 +36,7 @@ /** * AggregateFunction 'linear_histogram'. */ -public class LinearHistogram extends AggregateFunction implements ExplicitlyCastableSignature, AlwaysNotNullable { +public class LinearHistogram extends NotNullableAggregateFunction implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT) @@ -86,4 +86,9 @@ public R accept(ExpressionVisitor visitor, C context) { public List getSignatures() { return SIGNATURES; } + + @Override + public Expression resultForEmptyInput() { + return new VarcharLiteral("{\"num_buckets\":0,\"buckets\":[]}"); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java index 744d4a23a667e5b..798b3677ff127d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.MapLiteral; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'map_agg'. This class is generated by GenerateFunction. */ -public class MapAgg extends AggregateFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class MapAgg extends NotNullableAggregateFunction + implements BinaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(MapType.of(new FollowToAnyDataType(0), new FollowToAnyDataType(1))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java index 68d31e3e7bd19d3..81768ef48dd960d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -36,8 +35,8 @@ import java.util.List; /** MultiDistinctCount */ -public class MultiDistinctCount extends AggregateFunction - implements AlwaysNotNullable, ExplicitlyCastableSignature, MultiDistinction { +public class MultiDistinctCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, MultiDistinction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).varArgs(AnyDataType.INSTANCE_WITHOUT_INDEX) ); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java index 2b0eda06b4264a9..abd5292e31a2339 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ComputePrecisionForSum; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -39,8 +38,8 @@ import java.util.List; /** MultiDistinctSum0 */ -public class MultiDistinctSum0 extends AggregateFunction implements UnaryExpression, - ExplicitlyCastableSignature, ComputePrecisionForSum, MultiDistinction, AlwaysNotNullable { +public class MultiDistinctSum0 extends NotNullableAggregateFunction implements UnaryExpression, + ExplicitlyCastableSignature, ComputePrecisionForSum, MultiDistinction { private final boolean mustUseMultiDistinctAgg; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java index ea90bc587911be8..c2a04b4fd9931be 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java @@ -21,7 +21,6 @@ import org.apache.doris.catalog.Type; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -38,8 +37,8 @@ /** * AggregateFunction 'ndv'. This class is generated by GenserateFunction. */ -public class Ndv extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, RollUpTrait { +public class Ndv extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(AnyDataType.INSTANCE_WITHOUT_INDEX) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java new file mode 100644 index 000000000000000..f38decf439c2c6c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java @@ -0,0 +1,48 @@ +// 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.doris.nereids.trees.expressions.functions.agg; + +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; + +import java.util.List; + +/** + * base class for AlwaysNotNullable aggregate function + */ +public abstract class NotNullableAggregateFunction extends AggregateFunction implements AlwaysNotNullable { + protected NotNullableAggregateFunction(String name, Expression ...expressions) { + super(name, false, expressions); + } + + protected NotNullableAggregateFunction(String name, List expressions) { + super(name, false, expressions); + } + + protected NotNullableAggregateFunction(String name, boolean distinct, Expression ...expressions) { + super(name, distinct, expressions); + } + + protected NotNullableAggregateFunction(String name, boolean distinct, List expressions) { + super(name, distinct, expressions); + } + + // return value of this function if the input data is empty. + // for example, count(*) of empty table is 0; + public abstract Expression resultForEmptyInput(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java index 956f585f0a1e4ce..7837ea1bec7569d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.BitmapIntersectFunction; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BitmapType; import org.apache.doris.nereids.util.ExpressionUtils; @@ -33,8 +33,8 @@ /** * AggregateFunction 'orthogonal_bitmap_intersect'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapIntersect extends AggregateFunction - implements AlwaysNotNullable, OrthogonalBitmapFunction, BitmapIntersectFunction { +public class OrthogonalBitmapIntersect extends NotNullableAggregateFunction + implements OrthogonalBitmapFunction, BitmapIntersectFunction { static final List FUNCTION_SIGNATURES = SUPPORTED_TYPES.stream() .map(type -> FunctionSignature.ret(BitmapType.INSTANCE).varArgs(BitmapType.INSTANCE, type, type)) @@ -75,4 +75,9 @@ public OrthogonalBitmapIntersect withDistinctAndChildren(boolean distinct, List< public R accept(ExpressionVisitor visitor, C context) { return visitor.visitOrthogonalBitmapIntersect(this, context); } + + @Override + public Expression resultForEmptyInput() { + return new BitmapEmpty(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java index 82bfa911c26fafb..fd282341249e0a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java @@ -18,8 +18,8 @@ package org.apache.doris.nereids.trees.expressions.functions.agg; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.BitmapIntersectFunction; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.util.ExpressionUtils; @@ -30,8 +30,8 @@ /** * AggregateFunction 'orthogonal_bitmap_intersect_count'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapIntersectCount extends AggregateFunction - implements AlwaysNotNullable, OrthogonalBitmapFunction, BitmapIntersectFunction { +public class OrthogonalBitmapIntersectCount extends NotNullableAggregateFunction + implements OrthogonalBitmapFunction, BitmapIntersectFunction { /** * constructor with 3 or more arguments. @@ -44,7 +44,7 @@ public OrthogonalBitmapIntersectCount(Expression arg0, Expression arg1, Expressi * constructor with 3 or more arguments. */ public OrthogonalBitmapIntersectCount(boolean distinct, Expression arg0, Expression arg1, - Expression arg2, Expression... varArgs) { + Expression arg2, Expression... varArgs) { super("orthogonal_bitmap_intersect_count", distinct, ExpressionUtils.mergeArguments(arg0, arg1, arg2, varArgs)); } @@ -63,4 +63,9 @@ public OrthogonalBitmapIntersectCount withDistinctAndChildren(boolean distinct, public R accept(ExpressionVisitor visitor, C context) { return visitor.visitOrthogonalBitmapIntersectCount(this, context); } + + @Override + public Expression resultForEmptyInput() { + return new BigIntLiteral(0); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java index c08bb08aeecf648..081e5d7f6c4250c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; @@ -34,8 +34,8 @@ /** * AggregateFunction 'orthogonal_bitmap_union_count'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapUnionCount extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class OrthogonalBitmapUnionCount extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(BitmapType.INSTANCE) @@ -73,4 +73,9 @@ public R accept(ExpressionVisitor visitor, C context) { public List getSignatures() { return SIGNATURES; } + + @Override + public Expression resultForEmptyInput() { + return new BigIntLiteral(0); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java index bf189d67e89b0b8..1abbe4d54505314 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; @@ -42,8 +41,8 @@ /** * AggregateFunction 'percentile_array'. This class is generated by GenerateFunction. */ -public class PercentileArray extends AggregateFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class PercentileArray extends NotNullableAggregateFunction + implements BinaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(DoubleType.INSTANCE)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java index 3d0729775a5cecf..3d107ec282b606c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'quantile_union'. This class is generated by GenerateFunction. */ -public class QuantileUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class QuantileUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(QuantileStateType.INSTANCE).args(QuantileStateType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java index 7af112ef8e800fe..5ac114f00e83add 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -39,8 +38,8 @@ /** * AggregateFunction 'sequence_count'. This class is generated by GenerateFunction. */ -public class SequenceCount extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, SequenceFunction { +public class SequenceCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, SequenceFunction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java index fd052a69c0e90ee..5a1f0f9fb93d34d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ComputePrecisionForSum; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; @@ -53,8 +52,8 @@ * AggregateFunction 'sum0'. sum0 returns the sum of the values which go into it like sum. * It differs in that when no non null values are applied zero is returned instead of null. */ -public class Sum0 extends AggregateFunction - implements UnaryExpression, AlwaysNotNullable, ExplicitlyCastableSignature, ComputePrecisionForSum, +public class Sum0 extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum, SupportWindowAnalytic, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( From c0b8478f0a659c0e93dae9b9a86e529085944d99 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Mon, 2 Dec 2024 09:59:52 +0800 Subject: [PATCH 04/31] [Fix](http)Enhanced Security Checks for Audit Log File Names (#44612) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Purpose: To improve the security of audit log files, a new method checkAuditLogFileName has been added to validate the file name and path to ensure they meet security requirements. This method is designed to prevent invalid file names and path traversal attacks, ensuring that only files within the designated directory can be accessed.↳ ### Changes: #### File Name Validation: A regular expression check has been added to validate the file name: ^[a-zA-Z0-9._-]+$, restricting the file name to letters, numbers, dots, underscores, and hyphens. If the file name contains invalid characters (e.g., spaces, path traversal characters), a SecurityException is thrown with the message “Invalid file name.” Path Validation: The file name is resolved into a normalized path, and it is checked to ensure that it is within the allowed directory. The path is constructed using Paths.get(Config.audit_log_dir).resolve(logFile).normalize(). If the path does not start with the specified audit log directory (Config.audit_log_dir), indicating an attempt to access outside the permitted directory (e.g., a path traversal attack), a SecurityException is thrown with the message “Invalid file path: Access outside of permitted directory.” --- .../doris/httpv2/rest/GetLogFileAction.java | 38 +++++++++++- .../doris/httpv2/GetLogFileActionTest.java | 60 +++++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java index 475ee5ace1ea0d8..87c4c4cfa90a68b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java @@ -32,6 +32,8 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Map; import java.util.Set; import javax.servlet.http.HttpServletRequest; @@ -51,6 +53,23 @@ */ @RestController public class GetLogFileAction extends RestBaseController { + /** + * This method fetches internal logs via HTTP, which is no longer recommended and will + * be deprecated in future versions. + *

+ * Using HTTP to fetch logs introduces serious security and performance issues: + * - **Security Risks**: Log content may expose sensitive information, allowing attackers to exploit the exposed + * HTTP endpoints. + * - **Performance Problems**: Frequent HTTP requests can cause significant system load, affecting the + * responsiveness and stability of the application. + *

+ * It is strongly advised not to use this approach for accessing logs. Any new requirements should be + * handled using more secure, reliable, and efficient methods such as log aggregation tools (e.g., ELK, Splunk) + * or dedicated internal APIs. + *

+ * **Note**: No new HTTP endpoints or types for log access will be accepted. + * Any further attempts to extend this HTTP-based log retrieval method will not be supported. + */ private final Set logFileTypes = Sets.newHashSet("fe.audit.log"); @RequestMapping(path = "/api/get_log_file", method = {RequestMethod.GET, RequestMethod.HEAD}) @@ -79,7 +98,13 @@ public Object execute(HttpServletRequest request, HttpServletResponse response) String fileInfos = getFileInfos(logType); response.setHeader("file_infos", fileInfos); return ResponseEntityBuilder.ok(); - } else if (method.equals(RequestMethod.GET.name())) { + } + if (method.equals(RequestMethod.GET.name())) { + try { + checkAuditLogFileName(logFile); + } catch (SecurityException e) { + return ResponseEntityBuilder.internalError(e.getMessage()); + } File log = getLogFile(logType, logFile); if (!log.exists() || !log.isFile()) { return ResponseEntityBuilder.okWithCommonError("Log file not exist: " + log.getName()); @@ -97,6 +122,17 @@ public Object execute(HttpServletRequest request, HttpServletResponse response) return ResponseEntityBuilder.ok(); } + private void checkAuditLogFileName(String logFile) { + if (!logFile.matches("^[a-zA-Z0-9._-]+$")) { + throw new SecurityException("Invalid file name"); + } + Path normalizedPath = Paths.get(Config.audit_log_dir).resolve(logFile).normalize(); + // check path is valid or not + if (!normalizedPath.startsWith(Config.audit_log_dir)) { + throw new SecurityException("Invalid file path: Access outside of permitted directory"); + } + } + private String getFileInfos(String logType) { Map fileInfos = Maps.newTreeMap(); if (logType.equals("fe.audit.log")) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java b/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java new file mode 100644 index 000000000000000..8d4cac9b6ad9f49 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java @@ -0,0 +1,60 @@ +// 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.doris.httpv2; + +import org.apache.doris.common.Config; +import org.apache.doris.httpv2.rest.GetLogFileAction; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +public class GetLogFileActionTest { + + @TempDir + public File tempDir; + + @BeforeAll + public static void before() { + File tempDir = new File("test/audit.log"); + tempDir.mkdir(); + Config.audit_log_dir = tempDir.getAbsolutePath(); + } + + @Test + public void testCheckAuditLogFileName() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + //private method checkAuditLogFileName + GetLogFileAction action = new GetLogFileAction(); + Method method = GetLogFileAction.class.getDeclaredMethod("checkAuditLogFileName", String.class); + method.setAccessible(true); + method.invoke(action, "audit.log"); + method.invoke(action, "fe.audit.log.20241104-1"); + Assertions.assertThrows(InvocationTargetException.class, () -> method.invoke(action, "../etc/passwd")); + Assertions.assertThrows(InvocationTargetException.class, () -> method.invoke(action, + "fe.audit.log.20241104-1/../../etc/passwd")); + Assertions.assertThrows(InvocationTargetException.class, + () -> method.invoke(action, "fe.audit.log.20241104-1; rm -rf /")); + + + } +} From 2d9d3f79df01c475a1a0bbc1b68d2084323f04f5 Mon Sep 17 00:00:00 2001 From: Pxl Date: Mon, 2 Dec 2024 10:26:56 +0800 Subject: [PATCH 05/31] [Bug](runtime-filter) fix publish not inited rf when broadcast join meet wake up by downsteam (#44823) ### What problem does this PR solve? fix publish not inited rf when broadcast join meet wake up by downsteam related with #44408 #41751 --- be/src/exprs/runtime_filter.cpp | 2 +- be/src/pipeline/exec/hashjoin_build_sink.cpp | 38 +++++++++++--------- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index e26452c9ef69e52..c983af0fb3ea716 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1542,7 +1542,7 @@ void IRuntimeFilter::update_runtime_filter_type_to_profile(uint64_t local_merge_ std::string IRuntimeFilter::debug_string() const { return fmt::format( - "RuntimeFilter: (id = {}, type = {}, is_broadcast: {}, ignored: {}" + "RuntimeFilter: (id = {}, type = {}, is_broadcast: {}, ignored: {}, " "build_bf_cardinality: {}, dependency: {}, synced_size: {}, has_local_target: {}, " "has_remote_target: {},error_msg: [{}]", _filter_id, to_string(_runtime_filter_type), _is_broadcast_join, diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 0a71b86bed0e869..cec0c77da8a61d9 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -139,28 +139,32 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu return Base::close(state, exec_status); } - if (_should_build_hash_table) { - if (state->get_task()->wake_up_by_downstream()) { + if (state->get_task()->wake_up_by_downstream()) { + if (_should_build_hash_table) { + // partitial ignore rf to make global rf work RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); } else { - if (p._shared_hashtable_controller && - !p._shared_hash_table_context->complete_build_stage) { - return Status::InternalError("close before sink meet eos"); - } - auto* block = _shared_state->build_block.get(); - uint64_t hash_table_size = block ? block->rows() : 0; - { - SCOPED_TIMER(_runtime_filter_init_timer); - RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); - RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); - } - if (hash_table_size > 1) { - SCOPED_TIMER(_runtime_filter_compute_timer); - _runtime_filter_slots->insert(block); - } + // do not publish filter coz local rf not inited and useless + return Base::close(state, exec_status); + } + } else if (_should_build_hash_table) { + if (p._shared_hashtable_controller && !p._shared_hash_table_context->complete_build_stage) { + return Status::InternalError("close before sink meet eos"); + } + auto* block = _shared_state->build_block.get(); + uint64_t hash_table_size = block ? block->rows() : 0; + { + SCOPED_TIMER(_runtime_filter_init_timer); + RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); + RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); + } + if (hash_table_size > 1) { + SCOPED_TIMER(_runtime_filter_compute_timer); + _runtime_filter_slots->insert(block); } } + SCOPED_TIMER(_publish_runtime_filter_timer); RETURN_IF_ERROR_OR_CATCH_EXCEPTION( _runtime_filter_slots->publish(state, !_should_build_hash_table)); From 3ce581ec0cdad4f2e953b5fd554917e6c3761d7c Mon Sep 17 00:00:00 2001 From: Gabriel Date: Mon, 2 Dec 2024 10:33:31 +0800 Subject: [PATCH 06/31] [minor](scan) Delete unused function in ScanNode (#44733) --- .../src/main/java/org/apache/doris/planner/ScanNode.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java index b4033a0535ef3d0..2b10520e67d3173 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java @@ -727,15 +727,6 @@ public static TScanRangeLocations createSingleScanRangeLocations(FederationBacke return scanRangeLocation; } - public boolean ignoreStorageDataDistribution(ConnectContext context, int numBackends) { - return context != null - && context.getSessionVariable().isIgnoreStorageDataDistribution() - && !fragment.hasNullAwareLeftAntiJoin() - && getScanRangeNum() - < ConnectContext.get().getSessionVariable().getParallelExecInstanceNum() - * (numScanBackends() > 0 ? numScanBackends() : numBackends); - } - public int numScanBackends() { return scanBackendIds.size(); } From 59ae8003866a752044b8e450760f2b3077d08e58 Mon Sep 17 00:00:00 2001 From: walter Date: Mon, 2 Dec 2024 10:43:18 +0800 Subject: [PATCH 07/31] [chore](binlog) Save table type in CreateTableRecord (#44735) --- .../main/java/org/apache/doris/binlog/CreateTableRecord.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/CreateTableRecord.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/CreateTableRecord.java index 89d11fc014e979b..602afe88081d5b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/CreateTableRecord.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/CreateTableRecord.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.persist.CreateTableInfo; import org.apache.doris.persist.gson.GsonUtils; @@ -43,6 +44,8 @@ public class CreateTableRecord { private long tableId; @SerializedName(value = "tableName") private String tableName; + @SerializedName(value = "tableType") + protected TableType type; @SerializedName(value = "sql") private String sql; @@ -56,6 +59,8 @@ public CreateTableRecord(long commitSeq, CreateTableInfo info) { String dbName = info.getDbName(); this.dbName = dbName; + this.type = table.getType(); + Database db = Env.getCurrentInternalCatalog().getDbNullable(dbName); if (db == null) { LOG.warn("db not found. dbId: {}", dbId); From 4bd1cc1a01b82ea7e74d99b94c5369ae77d08336 Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Mon, 2 Dec 2024 08:26:11 +0530 Subject: [PATCH 08/31] [Enhancement] (nereids)implement showCreateDatabaseCommand in nereids (#43034) Issue Number: close #42739 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 20 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/ShowCreateDatabaseCommand.java | 122 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../show/test_show_create_db_nereids.out | 7 + .../show/test_show_create_db_nereids.groovy | 42 ++++++ 7 files changed, 198 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java create mode 100644 regression-test/data/nereids_p0/show/test_show_create_db_nereids.out create mode 100644 regression-test/suites/nereids_p0/show/test_show_create_db_nereids.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 1b1e218a9581462..f78a1c4b53179a2 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -214,6 +214,7 @@ supportedDropStatement supportedShowStatement : SHOW (GLOBAL | SESSION | LOCAL)? VARIABLES wildWhere? #showVariables | SHOW AUTHORS #showAuthors + | SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier #showCreateDatabase | SHOW BROKER #showBroker | SHOW DYNAMIC PARTITION TABLES ((FROM | IN) database=multipartIdentifier)? #showDynamicPartition | SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEvents @@ -295,7 +296,6 @@ unsupportedShowStatement | SHOW FULL? PROCESSLIST #showProcessList | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView - | SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier #showCreateDatabase | SHOW CREATE (GLOBAL | SESSION | LOCAL)? FUNCTION functionIdentifier LEFT_PAREN functionArguments? RIGHT_PAREN ((FROM | IN) database=multipartIdentifier)? #showCreateFunction diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index e5f4322225de530..d52d2696a2951c9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -20,6 +20,7 @@ import org.apache.doris.analysis.ArithmeticExpr.Operator; import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.ColumnNullableType; +import org.apache.doris.analysis.DbName; import org.apache.doris.analysis.EncryptKeyName; import org.apache.doris.analysis.PassVar; import org.apache.doris.analysis.SetType; @@ -226,6 +227,7 @@ import org.apache.doris.nereids.DorisParser.ShowConfigContext; import org.apache.doris.nereids.DorisParser.ShowConstraintContext; import org.apache.doris.nereids.DorisParser.ShowCreateCatalogContext; +import org.apache.doris.nereids.DorisParser.ShowCreateDatabaseContext; import org.apache.doris.nereids.DorisParser.ShowCreateMTMVContext; import org.apache.doris.nereids.DorisParser.ShowCreateMaterializedViewContext; import org.apache.doris.nereids.DorisParser.ShowCreateProcedureContext; @@ -512,6 +514,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCreateDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; @@ -4617,6 +4620,23 @@ public LogicalPlan visitShowFrontends(ShowFrontendsContext ctx) { return new ShowFrontendsCommand(detail); } + @Override + public LogicalPlan visitShowCreateDatabase(ShowCreateDatabaseContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.name); + String databaseName = ""; + String catalogName = ""; + if (nameParts.size() == 2) { + // The identifier is in the form "internalcatalog.databasename" + catalogName = nameParts.get(0); + databaseName = nameParts.get(1); + } else if (nameParts.size() == 1) { + // The identifier is in the form "databasename" + databaseName = nameParts.get(0); + } + + return new ShowCreateDatabaseCommand(new DbName(catalogName, databaseName)); + } + @Override public LogicalPlan visitCleanAllProfile(CleanAllProfileContext ctx) { return new CleanAllProfileCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8f474ea395f8890..8625d103f3b4536 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -198,6 +198,7 @@ public enum PlanType { SHOW_BROKER_COMMAND, SHOW_CONFIG_COMMAND, SHOW_CREATE_CATALOG_COMMAND, + SHOW_CREATE_DATABASE_COMMAND, SHOW_CREATE_MATERIALIZED_VIEW_COMMAND, SHOW_CREATE_TABLE_COMMAND, SHOW_CREATE_VIEW_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java new file mode 100644 index 000000000000000..29186e9e5cddeeb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.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.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.DbName; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.util.PrintableMap; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.hive.HMSExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalDatabase; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Objects; + +/** + * Represents the command for SHOW CREATE DATABASE. + */ +public class ShowCreateDatabaseCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Database", ScalarType.createVarchar(20))) + .addColumn(new Column("Create Database", ScalarType.createVarchar(30))) + .build(); + + private final String databaseName; + private final String catalogName; + + public ShowCreateDatabaseCommand(DbName dbName) { + super(PlanType.SHOW_CREATE_DATABASE_COMMAND); + this.databaseName = Objects.requireNonNull(dbName.getDb(), "Database name cannot be null"); + this.catalogName = dbName.getCtl(); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + String ctlgName = catalogName; + if (Strings.isNullOrEmpty(catalogName)) { + ctlgName = Env.getCurrentEnv().getCurrentCatalog().getName(); + } + if (Strings.isNullOrEmpty(databaseName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_DB_NAME, databaseName); + } + + if (!Env.getCurrentEnv().getAccessManager().checkDbPriv(ConnectContext.get(), ctlgName, databaseName, + PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, + PrivPredicate.SHOW.getPrivs().toString(), databaseName); + } + + List> rows = Lists.newArrayList(); + + StringBuilder sb = new StringBuilder(); + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalogOrAnalysisException(ctlgName); + if (catalog instanceof HMSExternalCatalog) { + String simpleDBName = ClusterNamespace.getNameFromFullName(databaseName); + org.apache.hadoop.hive.metastore.api.Database db = ((HMSExternalCatalog) catalog).getClient() + .getDatabase(simpleDBName); + sb.append("CREATE DATABASE `").append(simpleDBName).append("`") + .append(" LOCATION '") + .append(db.getLocationUri()) + .append("'"); + } else if (catalog instanceof IcebergExternalCatalog) { + IcebergExternalDatabase db = (IcebergExternalDatabase) catalog.getDbOrAnalysisException(databaseName); + sb.append("CREATE DATABASE `").append(databaseName).append("`") + .append(" LOCATION '") + .append(db.getLocation()) + .append("'"); + } else { + DatabaseIf db = catalog.getDbOrAnalysisException(databaseName); + sb.append("CREATE DATABASE `").append(ClusterNamespace.getNameFromFullName(databaseName)).append("`"); + if (db.getDbProperties().getProperties().size() > 0) { + sb.append("\nPROPERTIES (\n"); + sb.append(new PrintableMap<>(db.getDbProperties().getProperties(), "=", true, true, false)); + sb.append("\n)"); + } + } + + rows.add(Lists.newArrayList(ClusterNamespace.getNameFromFullName(databaseName), sb.toString())); + return new ShowResultSet(this.getMetaData(), rows); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowCreateDatabaseCommand(this, context); + } + + public ShowResultSetMetaData getMetaData() { + return META_DATA; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index c00c0ef90ca0186..49485cda51d0c26 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -75,6 +75,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCreateDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; @@ -398,6 +399,10 @@ default R visitShowCreateMaterializedViewCommand(ShowCreateMaterializedViewComma return visitCommand(showCreateMtlzViewCommand, context); } + default R visitShowCreateDatabaseCommand(ShowCreateDatabaseCommand showCreateDatabaseCommand, C context) { + return visitCommand(showCreateDatabaseCommand, context); + } + default R visitShowCreateViewCommand(ShowCreateViewCommand showCreateViewCommand, C context) { return visitCommand(showCreateViewCommand, context); } diff --git a/regression-test/data/nereids_p0/show/test_show_create_db_nereids.out b/regression-test/data/nereids_p0/show/test_show_create_db_nereids.out new file mode 100644 index 000000000000000..df88b46f8fbe77d --- /dev/null +++ b/regression-test/data/nereids_p0/show/test_show_create_db_nereids.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cmd -- +db_test_show_create CREATE DATABASE `db_test_show_create` + +-- !cmd -- +db_test_show_create CREATE DATABASE `db_test_show_create`\nPROPERTIES (\n"property_key" = "property_value"\n) + diff --git a/regression-test/suites/nereids_p0/show/test_show_create_db_nereids.groovy b/regression-test/suites/nereids_p0/show/test_show_create_db_nereids.groovy new file mode 100644 index 000000000000000..bfc84c9a60f72b8 --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_create_db_nereids.groovy @@ -0,0 +1,42 @@ +// 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. + +suite("test_show_create_db_nereids", "query,create_database") { + String dbName = "db_test_show_create"; + + try { + // Create a new database to test the SHOW CREATE DATABASE command + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + + // Run the SHOW CREATE DATABASE command and validate the output using checkNereidsExecute and qt_cmd + checkNereidsExecute("""SHOW CREATE DATABASE ${dbName}""") + qt_cmd("""SHOW CREATE DATABASE ${dbName}""") + + // Drop the database and verify that the command runs successfully + sql "DROP DATABASE IF EXISTS ${dbName}" + + // Re-create the database with additional properties + sql "CREATE DATABASE IF NOT EXISTS ${dbName} PROPERTIES ('property_key'='property_value')" + + // Verify the SHOW CREATE DATABASE command captures the properties using checkNereidsExecute and qt_cmd + checkNereidsExecute("""SHOW CREATE DATABASE ${dbName}""") + qt_cmd("""SHOW CREATE DATABASE ${dbName}""") + } finally { + // Clean up by dropping the database if it still exists + try_sql("DROP DATABASE IF EXISTS ${dbName}") + } +} From 70415afa864af41d5028605b897ae2ff67a62c66 Mon Sep 17 00:00:00 2001 From: echo-dundun <50286010+echo-hhj@users.noreply.github.com> Date: Mon, 2 Dec 2024 10:57:27 +0800 Subject: [PATCH 09/31] [Feat] (Nereids)support showTrash Command (#44684) Issue Number: close #42763 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 13 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowTrashCommand.java | 109 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../ddl/show_trash/test_nereids_trash.groovy | 22 ++++ 6 files changed, 151 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java create mode 100644 regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index f78a1c4b53179a2..e5c9a997073df84 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -246,6 +246,7 @@ supportedShowStatement | SHOW TABLET DIAGNOSIS tabletId=INTEGER_VALUE #showDiagnoseTablet | SHOW FRONTENDS name=identifier? #showFrontends | SHOW TABLE tableId=INTEGER_VALUE #showTableId + | SHOW TRASH (ON backend=STRING_LITERAL)? #showTrash | SHOW WHITELIST #showWhitelist | SHOW TABLETS BELONG tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletsBelong @@ -332,7 +333,6 @@ unsupportedShowStatement | SHOW BRIEF? RESTORE ((FROM | IN) database=multipartIdentifier)? wildWhere? #showRestore | SHOW RESOURCES wildWhere? sortClause? limitClause? #showResources | SHOW WORKLOAD GROUPS wildWhere? #showWorkloadGroups - | SHOW TRASH (ON backend=STRING_LITERAL)? #showTrash | SHOW SNAPSHOT ON repo=identifier wildWhere? #showSnapshot | SHOW FULL? BUILTIN? FUNCTIONS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showFunctions diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index d52d2696a2951c9..5ab217a1d5eb2cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -255,6 +255,7 @@ import org.apache.doris.nereids.DorisParser.ShowStorageEnginesContext; import org.apache.doris.nereids.DorisParser.ShowTableIdContext; import org.apache.doris.nereids.DorisParser.ShowTabletsBelongContext; +import org.apache.doris.nereids.DorisParser.ShowTrashContext; import org.apache.doris.nereids.DorisParser.ShowTriggersContext; import org.apache.doris.nereids.DorisParser.ShowVariablesContext; import org.apache.doris.nereids.DorisParser.ShowViewContext; @@ -541,6 +542,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; @@ -4462,6 +4464,17 @@ public LogicalPlan visitShowTriggers(ShowTriggersContext ctx) { return new ShowTriggersCommand(); } + @Override + public LogicalPlan visitShowTrash(ShowTrashContext ctx) { + if (ctx.ON() != null) { + String backend = stripQuotes(ctx.STRING_LITERAL().getText()); + new ShowTrashCommand(backend); + } else { + return new ShowTrashCommand(); + } + return new ShowTrashCommand(); + } + @Override public LogicalPlan visitShowRepositories(ShowRepositoriesContext ctx) { return new ShowRepositoriesCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8625d103f3b4536..683af9fe2c47006 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -220,6 +220,7 @@ public enum PlanType { SHOW_SMALL_FILES_COMMAND, SHOW_STORAGE_ENGINES_COMMAND, SHOW_TABLE_ID_COMMAND, + SHOW_TRASH_COMMAND, SHOW_TRIGGERS_COMMAND, SHOW_VARIABLES_COMMAND, SHOW_AUTHORS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java new file mode 100644 index 000000000000000..bcf30490cc991a6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java @@ -0,0 +1,109 @@ +// 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.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.proc.TrashProcDir; +import org.apache.doris.common.util.NetUtils; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.system.Backend; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * show trash command + */ +public class ShowTrashCommand extends ShowCommand { + private List backends = Lists.newArrayList(); + private String backendQuery; + + public ShowTrashCommand() { + super(PlanType.SHOW_TRASH_COMMAND); + } + + public ShowTrashCommand(String backendQuery) { + super(PlanType.SHOW_TRASH_COMMAND); + this.backendQuery = backendQuery; + } + + public List getBackends() { + return backends; + } + + public String getBackend() { + return backendQuery; + } + + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TrashProcDir.TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + private ShowResultSet handleShowTrash(String backendQuery) throws Exception { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN) + && !Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), + PrivPredicate.OPERATOR)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN/OPERATOR"); + } + ImmutableMap backendsInfo = Env.getCurrentSystemInfo().getAllBackendsByAllCluster(); + if (backendQuery == null || backendQuery.isEmpty()) { + for (Backend backend : backendsInfo.values()) { + this.backends.add(backend); + } + } else { + for (Backend backend : backendsInfo.values()) { + String backendStr = NetUtils.getHostPortInAccessibleFormat(backend.getHost(), + backend.getHeartbeatPort()); + if (backendQuery.equals(backendStr)) { + this.backends.add(backend); + break; + } + } + } + List> infos = Lists.newArrayList(); + TrashProcDir.getTrashInfo(backends, infos); + return new ShowResultSet(getMetaData(), infos); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowTrashCommand(this, context); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + return handleShowTrash(backendQuery); + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 49485cda51d0c26..98050c598e1c55f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -102,6 +102,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; @@ -370,6 +371,10 @@ default R visitShowPluginsCommand(ShowPluginsCommand showPluginsCommand, C conte return visitCommand(showPluginsCommand, context); } + default R visitShowTrashCommand(ShowTrashCommand showTrashCommand, C context) { + return visitCommand(showTrashCommand, context); + } + default R visitShowTriggersCommand(ShowTriggersCommand showTriggersCommand, C context) { return visitCommand(showTriggersCommand, context); } diff --git a/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy b/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy new file mode 100644 index 000000000000000..5d795e6fd119a15 --- /dev/null +++ b/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("show_trash_nereids") { + // can not use qt command since the output change based on cluster and backend ip + checkNereidsExecute("""show trash;""") + checkNereidsExecute("""show trash on "127.0.0.1:9050";""") +} From b8555c4dfa9fbcf122c3df077d919991388dcb3b Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Mon, 2 Dec 2024 08:27:58 +0530 Subject: [PATCH 10/31] [Enhancement] (nereids)implement showCollationCommand in nereids (#43157) Issue Number: close #42748 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 15 ++++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowCollationCommand.java | 89 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../nereids_p0/show/test_show_collation.out | 5 ++ .../show/test_show_collation.groovy | 29 ++++++ 7 files changed, 145 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCollationCommand.java create mode 100644 regression-test/data/nereids_p0/show/test_show_collation.out create mode 100644 regression-test/suites/nereids_p0/show/test_show_collation.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index e5c9a997073df84..8edcbb40c3c13b2 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -236,6 +236,7 @@ supportedShowStatement | SHOW FILE ((FROM | IN) database=multipartIdentifier)? #showSmallFiles | SHOW STORAGE? ENGINES #showStorageEngines | SHOW CREATE CATALOG name=identifier #showCreateCatalog + | SHOW COLLATION wildWhere? #showCollation | SHOW SQL_BLOCK_RULE (FOR ruleName=identifier)? #showSqlBlockRule | SHOW CREATE VIEW name=multipartIdentifier #showCreateView | SHOW CREATE MATERIALIZED VIEW mvName=identifier @@ -307,7 +308,6 @@ unsupportedShowStatement | SHOW CATALOG name=identifier #showCatalog | SHOW FULL? (COLUMNS | FIELDS) (FROM | IN) tableName=multipartIdentifier ((FROM | IN) database=multipartIdentifier)? wildWhere? #showColumns - | SHOW COLLATION wildWhere? #showCollation | SHOW ((CHAR SET) | CHARSET) wildWhere? #showCharset | SHOW COUNT LEFT_PAREN ASTERISK RIGHT_PAREN (WARNINGS | ERRORS) #showWaringErrorCount | SHOW (WARNINGS | ERRORS) limitClause? #showWaringErrors diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 5ab217a1d5eb2cc..60eda00ce896207 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -224,6 +224,7 @@ import org.apache.doris.nereids.DorisParser.ShowAuthorsContext; import org.apache.doris.nereids.DorisParser.ShowBackendsContext; import org.apache.doris.nereids.DorisParser.ShowBrokerContext; +import org.apache.doris.nereids.DorisParser.ShowCollationContext; import org.apache.doris.nereids.DorisParser.ShowConfigContext; import org.apache.doris.nereids.DorisParser.ShowConstraintContext; import org.apache.doris.nereids.DorisParser.ShowCreateCatalogContext; @@ -512,6 +513,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowAuthorsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBackendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBrokerCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCollationCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; @@ -4782,4 +4784,17 @@ public LogicalPlan visitShowTabletsBelong(ShowTabletsBelongContext ctx) { }); return new ShowTabletsBelongCommand(tabletIdLists); } + + @Override + public LogicalPlan visitShowCollation(ShowCollationContext ctx) { + String wild = null; + if (ctx.wildWhere() != null) { + if (ctx.wildWhere().LIKE() != null) { + wild = stripQuotes(ctx.wildWhere().STRING_LITERAL().getText()); + } else if (ctx.wildWhere().WHERE() != null) { + wild = ctx.wildWhere().expression().getText(); + } + } + return new ShowCollationCommand(wild); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 683af9fe2c47006..d07c90b7ccd5e72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -196,6 +196,7 @@ public enum PlanType { SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, SHOW_BROKER_COMMAND, + SHOW_COLLATION_COMMAND, SHOW_CONFIG_COMMAND, SHOW_CREATE_CATALOG_COMMAND, SHOW_CREATE_DATABASE_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCollationCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCollationCommand.java new file mode 100644 index 000000000000000..945db68bf5901ad --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCollationCommand.java @@ -0,0 +1,89 @@ +// 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.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * Represents the command for SHOW COLLATION + */ +public class ShowCollationCommand extends ShowCommand { + private static final ShowResultSetMetaData COLLATION_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Collation", ScalarType.createVarchar(20))) + .addColumn(new Column("Charset", ScalarType.createVarchar(20))) + .addColumn(new Column("Id", ScalarType.createVarchar(10))) + .addColumn(new Column("Default", ScalarType.createVarchar(10))) + .addColumn(new Column("Compiled", ScalarType.createVarchar(10))) + .addColumn(new Column("Sortlen", ScalarType.createVarchar(10))) + .build(); + + private final String wild; + + public ShowCollationCommand(String wild) { + super(PlanType.SHOW_COLLATION_COMMAND); + this.wild = wild; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + List> rows = Lists.newArrayList(); + List utf8mb40900Bin = Lists.newArrayList(); + // | utf8mb4_0900_bin | utf8mb4 | 309 | Yes | Yes | 1 | + utf8mb40900Bin.add(ctx.getSessionVariable().getCollationConnection()); + utf8mb40900Bin.add(ctx.getSessionVariable().getCharsetServer()); + utf8mb40900Bin.add("309"); + utf8mb40900Bin.add("Yes"); + utf8mb40900Bin.add("Yes"); + utf8mb40900Bin.add("1"); + rows.add(utf8mb40900Bin); + // ATTN: we must have this collation for compatible with some bi tools + List utf8mb3GeneralCi = Lists.newArrayList(); + // | utf8mb3_general_ci | utf8mb3 | 33 | Yes | Yes | 1 | + utf8mb3GeneralCi.add("utf8mb3_general_ci"); + utf8mb3GeneralCi.add("utf8mb3"); + utf8mb3GeneralCi.add("33"); + utf8mb3GeneralCi.add("Yes"); + utf8mb3GeneralCi.add("Yes"); + utf8mb3GeneralCi.add("1"); + rows.add(utf8mb3GeneralCi); + // Set the result set and send it using the executor + return new ShowResultSet(COLLATION_META_DATA, rows); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowCollationCommand(this, context); + } + + @Override + public String toString() { + return "SHOW COLLATION" + (wild != null ? " LIKE '" + wild + "'" : ""); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 98050c598e1c55f..f3bf05b2377d1ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -72,6 +72,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowAuthorsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBackendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBrokerCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCollationCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; @@ -521,6 +522,10 @@ default R visitShowTabletsBelongCommand(ShowTabletsBelongCommand showTabletBelon return visitCommand(showTabletBelongCommand, context); } + default R visitShowCollationCommand(ShowCollationCommand showCollationCommand, C context) { + return visitCommand(showCollationCommand, context); + } + default R visitCreateRoutineLoadCommand(CreateRoutineLoadCommand createRoutineLoadCommand, C context) { return visitCommand(createRoutineLoadCommand, context); } diff --git a/regression-test/data/nereids_p0/show/test_show_collation.out b/regression-test/data/nereids_p0/show/test_show_collation.out new file mode 100644 index 000000000000000..60bac6866a761ef --- /dev/null +++ b/regression-test/data/nereids_p0/show/test_show_collation.out @@ -0,0 +1,5 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cmd -- +utf8mb4_0900_bin utf8mb4 309 Yes Yes 1 +utf8mb3_general_ci utf8mb3 33 Yes Yes 1 + diff --git a/regression-test/suites/nereids_p0/show/test_show_collation.groovy b/regression-test/suites/nereids_p0/show/test_show_collation.groovy new file mode 100644 index 000000000000000..00837d72161a97a --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_collation.groovy @@ -0,0 +1,29 @@ +// 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. + +suite("test_show_collation", "query,collation") { + try { + // Execute the SHOW COLLATION command and verify the output + checkNereidsExecute("SHOW COLLATION") + qt_cmd("SHOW COLLATION") + } catch (Exception e) { + // Log any exceptions that occur during testing + log.error("Failed to execute SHOW COLLATION command", e) + throw e + } +} + From b49374d30f3b515a5e6f86837c7caec7633ebfc5 Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Mon, 2 Dec 2024 08:28:19 +0530 Subject: [PATCH 11/31] [Enhancement] (nereids)implement showProcessListCommand in nereids (#43158) Issue Number: close #42729 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 7 + .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/ShowProcessListCommand.java | 131 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../test_show_process_list_command.groovy | 30 ++++ 6 files changed, 175 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowProcessListCommand.java create mode 100644 regression-test/suites/nereids_p0/show/test_show_process_list_command.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 8edcbb40c3c13b2..d30ef144e52cd04 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -229,6 +229,7 @@ supportedShowStatement | SHOW PLUGINS #showPlugins | SHOW REPOSITORIES #showRepositories | SHOW BRIEF? CREATE TABLE name=multipartIdentifier #showCreateTable + | SHOW FULL? PROCESSLIST #showProcessList | SHOW ROLES #showRoles | SHOW PARTITION partitionId=INTEGER_VALUE #showPartitionId | SHOW PRIVILEGES #showPrivileges @@ -295,7 +296,6 @@ unsupportedShowStatement | SHOW TABLE STATUS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTableStatus | SHOW FULL? TABLES ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTables | SHOW FULL? VIEWS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showViews - | SHOW FULL? PROCESSLIST #showProcessList | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView | SHOW CREATE (GLOBAL | SESSION | LOCAL)? FUNCTION functionIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 60eda00ce896207..1422d807b71bd9f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -248,6 +248,7 @@ import org.apache.doris.nereids.DorisParser.ShowPrivilegesContext; import org.apache.doris.nereids.DorisParser.ShowProcContext; import org.apache.doris.nereids.DorisParser.ShowProcedureStatusContext; +import org.apache.doris.nereids.DorisParser.ShowProcessListContext; import org.apache.doris.nereids.DorisParser.ShowReplicaDistributionContext; import org.apache.doris.nereids.DorisParser.ShowRepositoriesContext; import org.apache.doris.nereids.DorisParser.ShowRolesContext; @@ -536,6 +537,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowProcessListCommand; import org.apache.doris.nereids.trees.plans.commands.ShowReplicaDistributionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; @@ -4756,6 +4758,11 @@ public LogicalPlan visitShowTableId(ShowTableIdContext ctx) { return new ShowTableIdCommand(tableId); } + @Override + public LogicalPlan visitShowProcessList(ShowProcessListContext ctx) { + return new ShowProcessListCommand(ctx.FULL() != null); + } + @Override public LogicalPlan visitSync(SyncContext ctx) { return new SyncCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index d07c90b7ccd5e72..177e6b4fcff1570 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -212,6 +212,7 @@ public enum PlanType { SHOW_LAST_INSERT_COMMAND, SHOW_LOAD_PROFILE_COMMAND, SHOW_PARTITIONID_COMMAND, + SHOW_PROCESSLIST_COMMAND, SHOW_PROC_COMMAND, SHOW_PLUGINS_COMMAND, SHOW_PRIVILEGES_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowProcessListCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowProcessListCommand.java new file mode 100644 index 000000000000000..fe04d61eab689e1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowProcessListCommand.java @@ -0,0 +1,131 @@ +// 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.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.ClientPool; +import org.apache.doris.common.Pair; +import org.apache.doris.common.proc.FrontendsProcNode; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.thrift.FrontendService; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TShowProcessListRequest; +import org.apache.doris.thrift.TShowProcessListResult; + +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +/** + * Represents the command for SHOW PROCESSLIST + */ +public class ShowProcessListCommand extends ShowCommand { + private static final Logger LOG = LogManager.getLogger(ShowProcessListCommand.class); + private static final ShowResultSetMetaData PROCESSLIST_META_DATA = ShowResultSetMetaData.builder() + .addColumn(new Column("CurrentConnected", ScalarType.createVarchar(16))) + .addColumn(new Column("Id", ScalarType.createType(PrimitiveType.BIGINT))) + .addColumn(new Column("User", ScalarType.createVarchar(16))) + .addColumn(new Column("Host", ScalarType.createVarchar(16))) + .addColumn(new Column("LoginTime", ScalarType.createVarchar(16))) + .addColumn(new Column("Catalog", ScalarType.createVarchar(16))) + .addColumn(new Column("Db", ScalarType.createVarchar(16))) + .addColumn(new Column("Command", ScalarType.createVarchar(16))) + .addColumn(new Column("Time", ScalarType.createType(PrimitiveType.INT))) + .addColumn(new Column("State", ScalarType.createVarchar(64))) + .addColumn(new Column("QueryId", ScalarType.createVarchar(64))) + .addColumn(new Column("Info", ScalarType.STRING)) + .addColumn(new Column("FE", ScalarType.createVarchar(16))) + .addColumn(new Column("CloudCluster", ScalarType.createVarchar(16))).build(); + + private final boolean isFull; + + public ShowProcessListCommand(boolean isFull) { + super(PlanType.SHOW_PROCESSLIST_COMMAND); + this.isFull = isFull; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + boolean isShowFullSql = isFull; + boolean isShowAllFe = ConnectContext.get().getSessionVariable().getShowAllFeConnection(); + + List> rowSet = Lists.newArrayList(); + List threadInfos = ctx.getConnectScheduler() + .listConnection(ctx.getQualifiedUser(), isShowFullSql); + long nowMs = System.currentTimeMillis(); + for (ConnectContext.ThreadInfo info : threadInfos) { + rowSet.add(info.toRow(ctx.getConnectionId(), nowMs)); + } + + if (isShowAllFe) { + try { + TShowProcessListRequest request = new TShowProcessListRequest(); + request.setShowFullSql(isShowFullSql); + request.setCurrentUserIdent(ConnectContext.get().getCurrentUserIdentity().toThrift()); + List> frontends = FrontendsProcNode.getFrontendWithRpcPort(Env.getCurrentEnv(), + false); + FrontendService.Client client = null; + for (Pair fe : frontends) { + TNetworkAddress thriftAddress = new TNetworkAddress(fe.key(), fe.value()); + try { + client = ClientPool.frontendPool.borrowObject(thriftAddress, 3000); + } catch (Exception e) { + LOG.warn("Failed to get frontend {} client. exception: {}", fe.key(), e); + continue; + } + + boolean isReturnToPool = false; + try { + TShowProcessListResult result = client.showProcessList(request); + if (result.process_list != null && result.process_list.size() > 0) { + rowSet.addAll(result.process_list); + } + isReturnToPool = true; + } catch (Exception e) { + LOG.warn("Failed to request processlist to fe: {} . exception: {}", fe.key(), e); + } finally { + if (isReturnToPool) { + ClientPool.frontendPool.returnObject(thriftAddress, client); + } else { + ClientPool.frontendPool.invalidateObject(thriftAddress, client); + } + } + } + } catch (Throwable t) { + LOG.warn(" fetch process list from other fe failed, ", t); + } + } + + return new ShowResultSet(PROCESSLIST_META_DATA, rowSet); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowProcessListCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index f3bf05b2377d1ff..de5228f498178b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -95,6 +95,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowProcessListCommand; import org.apache.doris.nereids.trees.plans.commands.ShowReplicaDistributionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; @@ -529,4 +530,8 @@ default R visitShowCollationCommand(ShowCollationCommand showCollationCommand, C default R visitCreateRoutineLoadCommand(CreateRoutineLoadCommand createRoutineLoadCommand, C context) { return visitCommand(createRoutineLoadCommand, context); } + + default R visitShowProcessListCommand(ShowProcessListCommand showProcessListCommand, C context) { + return visitCommand(showProcessListCommand, context); + } } diff --git a/regression-test/suites/nereids_p0/show/test_show_process_list_command.groovy b/regression-test/suites/nereids_p0/show/test_show_process_list_command.groovy new file mode 100644 index 000000000000000..b67ab1148c0eacc --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_process_list_command.groovy @@ -0,0 +1,30 @@ +// 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. + +suite("test_show_process_list_command", "query,process_list") { + try { + // Execute the SHOW PROCESSLIST command and verify the output + checkNereidsExecute("SHOW PROCESSLIST") + + // Execute the SHOW FULL PROCESSLIST command and verify the output + checkNereidsExecute("SHOW FULL PROCESSLIST") + } catch (Exception e) { + // Log any exceptions that occur during testing + log.error("Failed to execute SHOW PROCESSLIST command", e) + throw e + } +} From 3e41c91246f0ed615c546e6d7f6dfc8b3e698a87 Mon Sep 17 00:00:00 2001 From: Petrichor <31833513+vinlee19@users.noreply.github.com> Date: Mon, 2 Dec 2024 10:58:37 +0800 Subject: [PATCH 12/31] [feat](Nereids) support refresh database command (#44298) Issue Number: close #https://github.com/apache/doris/issues/42579 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 22 ++- .../doris/nereids/trees/plans/PlanType.java | 1 + .../refresh/RefreshDatabaseCommand.java | 149 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../test_nereids_refresh_catalog.out | 24 ++- .../test_nereids_refresh_catalog.groovy | 28 +++- 7 files changed, 218 insertions(+), 13 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/refresh/RefreshDatabaseCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index d30ef144e52cd04..fd5ae1bd5744f97 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -437,6 +437,7 @@ channelDescription supportedRefreshStatement : REFRESH CATALOG name=identifier propertyClause? #refreshCatalog + | REFRESH DATABASE name=multipartIdentifier propertyClause? #refreshDatabase ; supportedCleanStatement @@ -445,7 +446,6 @@ supportedCleanStatement unsupportedRefreshStatement : REFRESH TABLE name=multipartIdentifier #refreshTable - | REFRESH DATABASE name=multipartIdentifier propertyClause? #refreshDatabase | REFRESH LDAP (ALL | (FOR user=identifierOrText)) #refreshLdap ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 1422d807b71bd9f..70c4f61e4033c4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -188,6 +188,7 @@ import org.apache.doris.nereids.DorisParser.RecoverPartitionContext; import org.apache.doris.nereids.DorisParser.RecoverTableContext; import org.apache.doris.nereids.DorisParser.RefreshCatalogContext; +import org.apache.doris.nereids.DorisParser.RefreshDatabaseContext; import org.apache.doris.nereids.DorisParser.RefreshMTMVContext; import org.apache.doris.nereids.DorisParser.RefreshMethodContext; import org.apache.doris.nereids.DorisParser.RefreshScheduleContext; @@ -618,6 +619,7 @@ import org.apache.doris.nereids.trees.plans.commands.load.LoadSequenceClause; import org.apache.doris.nereids.trees.plans.commands.load.LoadWhereClause; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshCatalogCommand; +import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshDatabaseCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; @@ -4351,6 +4353,25 @@ public Object visitRefreshCatalog(RefreshCatalogContext ctx) { } @Override + public RefreshDatabaseCommand visitRefreshDatabase(RefreshDatabaseContext ctx) { + Map properties = visitPropertyClause(ctx.propertyClause()) == null ? Maps.newHashMap() + : visitPropertyClause(ctx.propertyClause()); + List parts = visitMultipartIdentifier(ctx.name); + int size = parts.size(); + if (size == 0) { + throw new ParseException("database name can't be empty"); + } + String dbName = parts.get(size - 1); + + // [db]. + if (size == 1) { + return new RefreshDatabaseCommand(dbName, properties); + } else if (parts.size() == 2) { // [ctl,db]. + return new RefreshDatabaseCommand(parts.get(0), dbName, properties); + } + throw new ParseException("Only one dot can be in the name: " + String.join(".", parts)); + } + public LogicalPlan visitShowLastInsert(ShowLastInsertContext ctx) { return new ShowLastInsertCommand(); } @@ -4379,7 +4400,6 @@ public LogicalPlan visitShowPartitionId(ShowPartitionIdContext ctx) { partitionId = Long.parseLong(ctx.partitionId.getText()); } return new ShowPartitionIdCommand(partitionId); - } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 177e6b4fcff1570..6a8fcadaf520c5b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -187,6 +187,7 @@ public enum PlanType { SET_USER_PROPERTIES_COMMAND, SET_DEFAULT_STORAGE_VAULT_COMMAND, REFRESH_CATALOG_COMMAND, + REFRESH_DATABASE_COMMAND, PREPARED_COMMAND, EXECUTE_COMMAND, DROP_SQL_BLOCK_RULE_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/refresh/RefreshDatabaseCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/refresh/RefreshDatabaseCommand.java new file mode 100644 index 000000000000000..010adbb05d1cba0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/refresh/RefreshDatabaseCommand.java @@ -0,0 +1,149 @@ +// 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.doris.nereids.trees.plans.commands.refresh; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.InfoSchemaDb; +import org.apache.doris.catalog.MysqlDb; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalDatabase; +import org.apache.doris.datasource.ExternalObjectLog; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.ForwardWithSync; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * Refresh database. + */ +public class RefreshDatabaseCommand extends Command implements ForwardWithSync { + private static final String INVALID_CACHE = "invalid_cache"; + + private String catalogName; + private String dbName; + private Map properties; + private boolean invalidCache = false; + + public RefreshDatabaseCommand(String dbName, Map properties) { + super(PlanType.REFRESH_DATABASE_COMMAND); + this.dbName = dbName; + this.properties = properties; + } + + public RefreshDatabaseCommand(String catalogName, String dbName, Map properties) { + super(PlanType.REFRESH_DATABASE_COMMAND); + this.catalogName = catalogName; + this.dbName = dbName; + this.properties = properties; + } + + private void validate(ConnectContext ctx) throws AnalysisException { + if (Strings.isNullOrEmpty(catalogName)) { + catalogName = ConnectContext.get().getCurrentCatalog().getName(); + } + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_DB_NAME, dbName); + } + + // Don't allow dropping 'information_schema' database + if (dbName.equalsIgnoreCase(InfoSchemaDb.DATABASE_NAME)) { + + ErrorReport.reportAnalysisException( + ErrorCode.ERR_DBACCESS_DENIED_ERROR, ctx.getQualifiedUser(), dbName); + } + // Don't allow dropping 'mysql' database + if (dbName.equalsIgnoreCase(MysqlDb.DATABASE_NAME)) { + ErrorReport.reportAnalysisException( + ErrorCode.ERR_DBACCESS_DENIED_ERROR, ctx.getQualifiedUser(), dbName); + } + // check access + if (!Env.getCurrentEnv().getAccessManager().checkDbPriv(ConnectContext.get(), catalogName, + dbName, PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, + PrivPredicate.SHOW.getPrivs().toString(), dbName); + } + String invalidConfig = properties == null ? null : properties.get(INVALID_CACHE); + // Default is to invalid cache. + invalidCache = invalidConfig == null || invalidConfig.equalsIgnoreCase("true"); + } + + /** + * Refresh database + */ + public void handleRefreshDb() throws DdlException { + Env env = Env.getCurrentEnv(); + CatalogIf catalog = catalogName != null ? env.getCatalogMgr().getCatalog(catalogName) : env.getCurrentCatalog(); + if (catalog == null) { + throw new DdlException("Catalog " + catalogName + " doesn't exist."); + } + if (!(catalog instanceof ExternalCatalog)) { + throw new DdlException("Only support refresh database in external catalog"); + } + DatabaseIf db = catalog.getDbOrDdlException(dbName); + ((ExternalDatabase) db).setUnInitialized(invalidCache); + + ExternalObjectLog log = new ExternalObjectLog(); + log.setCatalogId(catalog.getId()); + log.setDbId(db.getId()); + log.setInvalidCache(invalidCache); + Env.getCurrentEnv().getEditLog().logRefreshExternalDb(log); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(ctx); + handleRefreshDb(); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitRefreshDatabaseCommand(this, context); + } + + /** + * refresh database statement. + */ + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("REFRESH DATABASE "); + if (catalogName != null) { + sb.append("`").append(catalogName).append("`."); + } + sb.append("`").append(dbName).append("`"); + return sb.toString(); + } + + @Override + public StmtType stmtType() { + return StmtType.REFRESH; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index de5228f498178b9..fec52fd395f9db2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -119,6 +119,7 @@ import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; import org.apache.doris.nereids.trees.plans.commands.load.CreateRoutineLoadCommand; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshCatalogCommand; +import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshDatabaseCommand; /** CommandVisitor. */ public interface CommandVisitor { @@ -353,6 +354,10 @@ default R visitShowViewCommand(ShowViewCommand showViewCommand, C context) { return visitCommand(showViewCommand, context); } + default R visitRefreshDatabaseCommand(RefreshDatabaseCommand refreshDatabaseCommand, C context) { + return visitCommand(refreshDatabaseCommand, context); + } + default R visitShowBackendsCommand(ShowBackendsCommand showBackendsCommand, C context) { return visitCommand(showBackendsCommand, context); } diff --git a/regression-test/data/external_table_p0/nereids_commands/test_nereids_refresh_catalog.out b/regression-test/data/external_table_p0/nereids_commands/test_nereids_refresh_catalog.out index 89b47bbe4eb1e7e..c3f57d7ba810b82 100644 --- a/regression-test/data/external_table_p0/nereids_commands/test_nereids_refresh_catalog.out +++ b/regression-test/data/external_table_p0/nereids_commands/test_nereids_refresh_catalog.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !sql -- +-- !database -- DORIS Doris doris @@ -19,7 +19,7 @@ show_test_do_not_modify 114 abf 115 abg --- !sql -- +-- !preceding_create_external_database -- DORIS Doris doris @@ -29,7 +29,7 @@ init_db mysql show_test_do_not_modify --- !sql -- +-- !subsequent_create_external_database -- DORIS Doris doris @@ -40,7 +40,21 @@ mysql new_mysql_db show_test_do_not_modify --- !sql -- +-- !sql_show_tables -- + +-- !preceding_refresh_database -- + +-- !subsequent_refresh_database -- +new_mysql_table1 + +-- !preceding_refresh_database -- +new_mysql_table1 + +-- !subsequent_refresh_database -- +new_mysql_table1 +new_mysql_table2 + +-- !preceding_drop_external_database -- DORIS Doris doris @@ -51,7 +65,7 @@ mysql new_mysql_db show_test_do_not_modify --- !sql -- +-- !subsequent_drop_external_database -- DORIS Doris doris diff --git a/regression-test/suites/external_table_p0/nereids_commands/test_nereids_refresh_catalog.groovy b/regression-test/suites/external_table_p0/nereids_commands/test_nereids_refresh_catalog.groovy index f74f78a3709dc2d..87dacd4142501ea 100644 --- a/regression-test/suites/external_table_p0/nereids_commands/test_nereids_refresh_catalog.groovy +++ b/regression-test/suites/external_table_p0/nereids_commands/test_nereids_refresh_catalog.groovy @@ -28,6 +28,8 @@ suite("test_nereids_refresh_catalog", "p0,external,mysql,external_docker,externa String mysql_port = context.config.otherConfigs.get("mysql_57_port"); String ex_tb0 = "ex_tb0"; String new_mysql_db = "new_mysql_db"; + String new_mysql_table1 = "new_mysql_table1"; + String new_mysql_table2 = "new_mysql_table2"; sql """drop catalog if exists ${catalog_name} """ @@ -43,27 +45,41 @@ suite("test_nereids_refresh_catalog", "p0,external,mysql,external_docker,externa "driver_class" = "com.mysql.cj.jdbc.Driver" );""" - sql """switch ${catalog_name}""" sql """CALL EXECUTE_STMT("${catalog_name}", "drop database if exists ${new_mysql_db}");""" + sql """switch ${catalog_name}""" - qt_sql """show databases;""" + qt_database """show databases;""" sql """ use ${ex_db_name}""" qt_ex_tb0_where """select id from ${ex_tb0} where id = 111;""" order_qt_ex_tb0 """ select id, name from ${ex_tb0} order by id; """ // create database in mysql sql """CALL EXECUTE_STMT("${catalog_name}", "create database ${new_mysql_db} ;");""" - qt_sql """show databases;""" + qt_preceding_create_external_database """show databases;""" checkNereidsExecute("refresh catalog ${catalog_name} ;") - qt_sql """show databases;""" + qt_subsequent_create_external_database """show databases;""" checkNereidsExecute("refresh catalog ${catalog_name} properties ('invalid_cache'='true');") + sql """use ${new_mysql_db}""" + qt_sql_show_tables """show tables;""" + + // create table in mysql external database + sql """CALL EXECUTE_STMT("${catalog_name}", "create table ${new_mysql_db}.${new_mysql_table1} (id int, name varchar(20));");""" + + qt_preceding_refresh_database """show tables;""" + checkNereidsExecute("refresh database ${new_mysql_db} ;") + qt_subsequent_refresh_database """show tables;""" + + sql """CALL EXECUTE_STMT("${catalog_name}", "create table ${new_mysql_db}.${new_mysql_table2} (id int, name varchar(20));");""" + qt_preceding_refresh_database """show tables;""" + checkNereidsExecute("refresh database ${catalog_name}.${new_mysql_db} ;") + qt_subsequent_refresh_database """show tables;""" sql """CALL EXECUTE_STMT("${catalog_name}", "drop database if exists ${new_mysql_db} ;");""" - qt_sql """show databases;""" + qt_preceding_drop_external_database """show databases;""" checkNereidsExecute("refresh catalog ${catalog_name} properties ('invalid_cache'='true');") - qt_sql """show databases;""" + qt_subsequent_drop_external_database """show databases;""" sql """ drop catalog if exists ${catalog_name} ;""" } From 2bd293c560afdb7665f05ae513f1354d6dfb6371 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Mon, 2 Dec 2024 08:30:54 +0530 Subject: [PATCH 13/31] [Enhancement] (nereids)implement CreateSqlBlockCommand and AlterSqlBlockCommand in nereids (#44702) Issue Number: close #42592 #42796 --- .../org/apache/doris/nereids/DorisParser.g4 | 8 +- .../doris/blockrule/SqlBlockRuleMgr.java | 12 +- .../nereids/parser/LogicalPlanBuilder.java | 18 +++ .../doris/nereids/trees/plans/PlanType.java | 2 + .../commands/AlterSqlBlockRuleCommand.java | 87 +++++++++++++ .../commands/CreateSqlBlockRuleCommand.java | 99 +++++++++++++++ .../plans/commands/SqlBlockRuleCommand.java | 117 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 10 ++ .../test_sql_block_rule.groovy | 11 +- 9 files changed, 349 insertions(+), 15 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterSqlBlockRuleCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateSqlBlockRuleCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SqlBlockRuleCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index fd5ae1bd5744f97..6e2a518ec209827 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -185,7 +185,9 @@ supportedCreateStatement ON table=multipartIdentifier AS type=(RESTRICTIVE | PERMISSIVE) TO (user=userIdentify | ROLE roleName=identifier) - USING LEFT_PAREN booleanExpression RIGHT_PAREN #createRowPolicy + USING LEFT_PAREN booleanExpression RIGHT_PAREN #createRowPolicy + | CREATE SQL_BLOCK_RULE (IF NOT EXISTS)? + name=identifier properties=propertyClause? #createSqlBlockRule ; supportedAlterStatement @@ -197,6 +199,7 @@ supportedAlterStatement properties=propertyClause? #alterWorkloadGroup | ALTER WORKLOAD POLICY name=identifierOrText properties=propertyClause? #alterWorkloadPolicy + | ALTER SQL_BLOCK_RULE name=identifier properties=propertyClause? #alterSqlBlockRule ; supportedDropStatement @@ -573,7 +576,6 @@ unsupportedAlterStatement SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterColocateGroup | ALTER ROUTINE LOAD FOR name=multipartIdentifier properties=propertyClause? (FROM type=identifier LEFT_PAREN propertyItemList RIGHT_PAREN)? #alterRoutineLoad - | ALTER SQL_BLOCK_RULE name=identifier properties=propertyClause? #alterSqlBlockRule | ALTER TABLE name=multipartIdentifier SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterTableProperties | ALTER STORAGE POLICY name=identifierOrText @@ -765,8 +767,6 @@ unsupportedCreateStatement (ACTIONS LEFT_PAREN workloadPolicyActions RIGHT_PAREN)? properties=propertyClause? #createWorkloadPolicy | CREATE ENCRYPTKEY (IF NOT EXISTS)? multipartIdentifier AS STRING_LITERAL #createEncryptkey - | CREATE SQL_BLOCK_RULE (IF NOT EXISTS)? - name=identifier properties=propertyClause? #createSqlBlockRule | CREATE STORAGE POLICY (IF NOT EXISTS)? name=identifier properties=propertyClause? #createStoragePolicy | BUILD INDEX name=identifier ON tableName=multipartIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java index aa3b844f3e38ef8..e4e288bc14b2bde 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java @@ -116,12 +116,15 @@ private static void verifyLimitations(SqlBlockRule sqlBlockRule) throws DdlExcep * Create SqlBlockRule for create stmt. **/ public void createSqlBlockRule(CreateSqlBlockRuleStmt stmt) throws UserException { + createSqlBlockRule(SqlBlockRule.fromCreateStmt(stmt), stmt.isIfNotExists()); + } + + public void createSqlBlockRule(SqlBlockRule sqlBlockRule, boolean isIfNotExists) throws UserException { writeLock(); try { - SqlBlockRule sqlBlockRule = SqlBlockRule.fromCreateStmt(stmt); String ruleName = sqlBlockRule.getName(); if (existRule(ruleName)) { - if (stmt.isIfNotExists()) { + if (isIfNotExists) { return; } throw new DdlException("the sql block rule " + ruleName + " already create"); @@ -146,9 +149,12 @@ public void replayCreate(SqlBlockRule sqlBlockRule) { * Alter SqlBlockRule for alter stmt. **/ public void alterSqlBlockRule(AlterSqlBlockRuleStmt stmt) throws AnalysisException, DdlException { + alterSqlBlockRule(SqlBlockRule.fromAlterStmt(stmt)); + } + + public void alterSqlBlockRule(SqlBlockRule sqlBlockRule) throws AnalysisException, DdlException { writeLock(); try { - SqlBlockRule sqlBlockRule = SqlBlockRule.fromAlterStmt(stmt); String ruleName = sqlBlockRule.getName(); if (!existRule(ruleName)) { throw new DdlException("the sql block rule " + ruleName + " not exist"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 70c4f61e4033c4e..dfe879e7d44dad9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -57,6 +57,7 @@ import org.apache.doris.nereids.DorisParser.AliasedQueryContext; import org.apache.doris.nereids.DorisParser.AlterMTMVContext; import org.apache.doris.nereids.DorisParser.AlterRoleContext; +import org.apache.doris.nereids.DorisParser.AlterSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.AlterStorageVaultContext; import org.apache.doris.nereids.DorisParser.AlterViewContext; import org.apache.doris.nereids.DorisParser.AlterWorkloadGroupContext; @@ -92,6 +93,7 @@ import org.apache.doris.nereids.DorisParser.CreateProcedureContext; import org.apache.doris.nereids.DorisParser.CreateRoutineLoadContext; import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext; +import org.apache.doris.nereids.DorisParser.CreateSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.CreateTableContext; import org.apache.doris.nereids.DorisParser.CreateTableLikeContext; import org.apache.doris.nereids.DorisParser.CreateViewContext; @@ -460,6 +462,7 @@ import org.apache.doris.nereids.trees.plans.commands.AddConstraintCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; @@ -477,6 +480,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; import org.apache.doris.nereids.trees.plans.commands.CreateViewCommand; @@ -4154,6 +4158,20 @@ public LogicalPlan visitShowCreateProcedure(ShowCreateProcedureContext ctx) { return ParserUtils.withOrigin(ctx, () -> new ShowCreateProcedureCommand(procedureName)); } + @Override + public LogicalPlan visitCreateSqlBlockRule(CreateSqlBlockRuleContext ctx) { + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new CreateSqlBlockRuleCommand(stripQuotes(ctx.name.getText()), ctx.EXISTS() != null, properties); + } + + @Override + public LogicalPlan visitAlterSqlBlockRule(AlterSqlBlockRuleContext ctx) { + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new AlterSqlBlockRuleCommand(stripQuotes(ctx.name.getText()), properties); + } + @Override public LogicalPlan visitDropCatalogRecycleBin(DropCatalogRecycleBinContext ctx) { String idTypeStr = ctx.idType.getText().substring(1, ctx.idType.getText().length() - 1); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 6a8fcadaf520c5b..14a1b13a4667dfc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -134,6 +134,7 @@ public enum PlanType { // commands CREATE_POLICY_COMMAND, CREATE_TABLE_COMMAND, + CREATE_SQL_BLOCK_RULE_COMMAND, DELETE_COMMAND, EXPLAIN_COMMAND, EXPORT_COMMAND, @@ -194,6 +195,7 @@ public enum PlanType { DROP_USER_COMMAND, DROP_WORKLOAD_GROUP_NAME, DROP_WORKLOAD_POLICY_COMMAND, + ALTER_SQL_BLOCK_RULE_COMMAND, SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, SHOW_BROKER_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterSqlBlockRuleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterSqlBlockRuleCommand.java new file mode 100644 index 000000000000000..481eb64a58d6d24 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterSqlBlockRuleCommand.java @@ -0,0 +1,87 @@ +// 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.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.blockrule.SqlBlockRule; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.util.SqlBlockUtil; +import org.apache.doris.common.util.Util; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; + +/** + * alter Sql block rule Commands. + */ +public class AlterSqlBlockRuleCommand extends SqlBlockRuleCommand { + + /** + * constructor + */ + public AlterSqlBlockRuleCommand(String ruleName, Map properties) { + super(ruleName, properties, PlanType.ALTER_SQL_BLOCK_RULE_COMMAND); + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + Env.getCurrentEnv().getSqlBlockRuleMgr().alterSqlBlockRule(new SqlBlockRule(ruleName, + sql, sqlHash, partitionNum, + tabletNum, cardinality, global, enable)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAlterSqlBlockRuleCommand(this, context); + } + + @Override + public StmtType stmtType() { + return StmtType.ALTER; + } + + @Override + public void setProperties(Map properties) throws AnalysisException { + this.sql = properties.getOrDefault(SQL_PROPERTY, SqlBlockRuleCommand.STRING_NOT_SET); + this.sqlHash = properties.getOrDefault(SQL_HASH_PROPERTY, SqlBlockRuleCommand.STRING_NOT_SET); + String partitionNumString = properties.get(SCANNED_PARTITION_NUM); + String tabletNumString = properties.get(SCANNED_TABLET_NUM); + String cardinalityString = properties.get(SCANNED_CARDINALITY); + + SqlBlockUtil.checkSqlAndSqlHashSetBoth(sql, sqlHash); + SqlBlockUtil.checkSqlAndLimitationsSetBoth(sql, sqlHash, + partitionNumString, tabletNumString, cardinalityString); + this.partitionNum = Util.getLongPropertyOrDefault(partitionNumString, SqlBlockRuleCommand.LONG_NOT_SET, null, + SCANNED_PARTITION_NUM + " should be a long"); + this.tabletNum = Util.getLongPropertyOrDefault(tabletNumString, SqlBlockRuleCommand.LONG_NOT_SET, null, + SCANNED_TABLET_NUM + " should be a long"); + this.cardinality = Util.getLongPropertyOrDefault(cardinalityString, SqlBlockRuleCommand.LONG_NOT_SET, null, + SCANNED_CARDINALITY + " should be a long"); + // allow null, represents no modification + String globalStr = properties.get(GLOBAL_PROPERTY); + this.global = StringUtils.isNotEmpty(globalStr) ? Boolean.parseBoolean(globalStr) : null; + String enableStr = properties.get(ENABLE_PROPERTY); + this.enable = StringUtils.isNotEmpty(enableStr) ? Boolean.parseBoolean(enableStr) : null; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateSqlBlockRuleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateSqlBlockRuleCommand.java new file mode 100644 index 000000000000000..ab8e1a1b2689e3c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateSqlBlockRuleCommand.java @@ -0,0 +1,99 @@ +// 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.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.blockrule.SqlBlockRule; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.SqlBlockUtil; +import org.apache.doris.common.util.Util; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; +import java.util.regex.Pattern; + +/** + * create Sql block rule Commands. + */ +public class CreateSqlBlockRuleCommand extends SqlBlockRuleCommand { + private static final String NAME_TYPE = "SQL BLOCK RULE NAME"; + private final boolean ifNotExists; + + /** + * constructor + */ + public CreateSqlBlockRuleCommand(String ruleName, boolean ifNotExists, Map properties) { + super(ruleName, properties, PlanType.CREATE_SQL_BLOCK_RULE_COMMAND); + this.ifNotExists = ifNotExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check name + FeNameFormat.checkCommonName(NAME_TYPE, ruleName); + // avoid a rule block any ddl for itself + if (StringUtils.isNotEmpty(sql) && Pattern.compile(sql).matcher(this.ruleName).find()) { + throw new AnalysisException("sql of SQL_BLOCK_RULE should not match its name"); + } + Env.getCurrentEnv().getSqlBlockRuleMgr().createSqlBlockRule(new SqlBlockRule(ruleName, sql, + sqlHash, partitionNum, + tabletNum, cardinality, global, enable), ifNotExists); + } + + @Override + public void setProperties(Map properties) throws UserException { + this.sql = properties.getOrDefault(SQL_PROPERTY, SqlBlockRuleCommand.STRING_NOT_SET); + this.sqlHash = properties.getOrDefault(SQL_HASH_PROPERTY, SqlBlockRuleCommand.STRING_NOT_SET); + String partitionNumString = properties.get(SCANNED_PARTITION_NUM); + String tabletNumString = properties.get(SCANNED_TABLET_NUM); + String cardinalityString = properties.get(SCANNED_CARDINALITY); + + SqlBlockUtil.checkSqlAndSqlHashSetBoth(sql, sqlHash); + SqlBlockUtil.checkPropertiesValidate(sql, sqlHash, partitionNumString, tabletNumString, cardinalityString); + + this.partitionNum = Util.getLongPropertyOrDefault(partitionNumString, 0L, null, + SCANNED_PARTITION_NUM + " should be a long"); + this.tabletNum = Util.getLongPropertyOrDefault(tabletNumString, 0L, null, + SCANNED_TABLET_NUM + " should be a long"); + this.cardinality = Util.getLongPropertyOrDefault(cardinalityString, 0L, null, + SCANNED_CARDINALITY + " should be a long"); + + this.global = Util.getBooleanPropertyOrDefault(properties.get(GLOBAL_PROPERTY), false, + GLOBAL_PROPERTY + " should be a boolean"); + this.enable = Util.getBooleanPropertyOrDefault(properties.get(ENABLE_PROPERTY), true, + ENABLE_PROPERTY + " should be a boolean"); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateSqlBlockRuleCommand(this, context); + } + + @Override + public StmtType stmtType() { + return StmtType.CREATE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SqlBlockRuleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SqlBlockRuleCommand.java new file mode 100644 index 000000000000000..785da1aeb5861e9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SqlBlockRuleCommand.java @@ -0,0 +1,117 @@ +// 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.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.SqlBlockUtil; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.ImmutableSet; + +import java.util.Map; +import java.util.Optional; + +/** + * Common class for SqlBlockRule Commands. + */ +public abstract class SqlBlockRuleCommand extends Command { + public static final String SQL_PROPERTY = "sql"; + + public static final String SQL_HASH_PROPERTY = "sqlHash"; + + public static final String SCANNED_PARTITION_NUM = "partition_num"; + + public static final String SCANNED_TABLET_NUM = "tablet_num"; + + public static final String SCANNED_CARDINALITY = "cardinality"; + + public static final String GLOBAL_PROPERTY = "global"; + + public static final String ENABLE_PROPERTY = "enable"; + + public static final Long LONG_NOT_SET = SqlBlockUtil.LONG_MINUS_ONE; + + public static final String STRING_NOT_SET = SqlBlockUtil.STRING_DEFAULT; + + private static final ImmutableSet PROPERTIES_SET = new ImmutableSet.Builder().add(SQL_PROPERTY) + .add(SQL_HASH_PROPERTY).add(GLOBAL_PROPERTY).add(ENABLE_PROPERTY).add(SCANNED_PARTITION_NUM) + .add(SCANNED_TABLET_NUM).add(SCANNED_CARDINALITY).build(); + + protected final String ruleName; + + protected String sql; + + protected String sqlHash; + + protected Long partitionNum; + + protected Long tabletNum; + + protected Long cardinality; + + // whether effective global, default is false + protected Boolean global; + + // whether to use the rule, default is true + protected Boolean enable; + + protected final Map properties; + + /** + * constructor + */ + public SqlBlockRuleCommand(String ruleName, Map properties, PlanType planType) { + super(planType); + this.ruleName = ruleName; + this.properties = properties; + } + + private static void checkCommonProperties(Map properties) throws UserException { + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Not set properties"); + } + Optional optional = properties.keySet().stream().filter(entity -> !PROPERTIES_SET.contains(entity)) + .findFirst(); + if (optional.isPresent()) { + throw new AnalysisException(optional.get() + " is invalid property"); + } + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + // check properties + checkCommonProperties(properties); + setProperties(properties); + doRun(ctx, executor); + } + + public abstract void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception; + + public abstract void setProperties(Map properties) throws UserException; +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index fec52fd395f9db2..d386d097a93b49a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterJobStatusCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadPolicyCommand; @@ -36,6 +37,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; import org.apache.doris.nereids.trees.plans.commands.CreateViewCommand; @@ -471,6 +473,14 @@ default R visitShowLoadProfileCommand(ShowLoadProfileCommand showLoadProfileComm return visitCommand(showLoadProfileCommand, context); } + default R visitAlterSqlBlockRuleCommand(AlterSqlBlockRuleCommand dropRoleCommand, C context) { + return visitCommand(dropRoleCommand, context); + } + + default R visitCreateSqlBlockRuleCommand(CreateSqlBlockRuleCommand dropRoleCommand, C context) { + return visitCommand(dropRoleCommand, context); + } + default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } diff --git a/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy b/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy index 001c64fbc0d0ad5..f90b89dfb6e5a72 100644 --- a/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy +++ b/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy @@ -47,10 +47,7 @@ suite("test_sql_block_rule", "nonConcurrent") { """ sql """ INSERT INTO table_2 VALUES ('H220427011909850160918','2022-04-27 16:00:33'),('T220427400109910160949','2022-04-27 16:00:54'),('T220427400123770120058','2022-04-27 16:00:56'),('T220427400126530112854','2022-04-27 16:00:34'),('T220427400127160144672','2022-04-27 16:00:10'),('T220427400127900184511','2022-04-27 16:00:34'),('T220427400129940120380','2022-04-27 16:00:23'),('T220427400139720192986','2022-04-27 16:00:34'),('T220427400140260152375','2022-04-27 16:00:02'),('T220427400153170104281','2022-04-27 16:00:31'),('H220427011909800104411','2022-04-27 16:00:14'),('H220427011909870184823','2022-04-27 16:00:36'),('T220427400115770144416','2022-04-27 16:00:12'),('T220427400126390112736','2022-04-27 16:00:19'),('T220427400128350120717','2022-04-27 16:00:56'),('T220427400129680120838','2022-04-27 16:00:39'),('T220427400136970192083','2022-04-27 16:00:51'),('H220427011909770192580','2022-04-27 16:00:04'),('H220427011909820192943','2022-04-27 16:00:23'),('T220427400109110184990','2022-04-27 16:00:29'),('T220427400109930192249','2022-04-27 16:00:56'),('T220427400123050168464','2022-04-27 16:00:37'),('T220427400124330112931','2022-04-27 16:00:56'),('T220427400124430144718','2022-04-27 16:00:07'),('T220427400130570160488','2022-04-27 16:00:34'),('T220427400130610112671','2022-04-27 16:00:30'),('T220427400137600160704','2022-04-27 16:00:35'),('T220427400144590176969','2022-04-27 16:00:49'),('T220427400146320176530','2022-04-27 16:00:34'),('T220427601780480120027','2022-04-27 16:00:58');""" - sql """ - CREATE SQL_BLOCK_RULE if not exists test_rule_sql - PROPERTIES("sql"="SELECT \\\\* FROM table_2", "global"= "true", "enable"= "true") - """ + checkNereidsExecute("CREATE SQL_BLOCK_RULE if not exists test_rule_sql PROPERTIES(\"sql\"=\"SELECT \\\\* FROM table_2\", \"global\"= \"true\", \"enable\"= \"true\")") test { sql("SELECT * FROM table_2", false) @@ -66,10 +63,8 @@ suite("test_sql_block_rule", "nonConcurrent") { exception "sql match regex sql block rule: test_rule_sql" } - sql """ - ALTER SQL_BLOCK_RULE test_rule_sql PROPERTIES("enable"="false") - """ - + checkNereidsExecute("ALTER SQL_BLOCK_RULE test_rule_sql PROPERTIES(\"enable\"=\"false\")") + sql "SELECT * FROM table_2" sql """ From 7d7f7fec5b3ca0ae9d30e8fd798e11da96c077ee Mon Sep 17 00:00:00 2001 From: lihangyu Date: Mon, 2 Dec 2024 11:08:46 +0800 Subject: [PATCH 14/31] [Opt](SegmentIterator) clear and release iterators memory footprint in advance when EOF (#44768) Otherwise the memory will hold until BetaRowsetReader released --- be/src/olap/rowset/segment_v2/segment_iterator.cpp | 8 ++++++++ be/src/olap/rowset/segment_v2/segment_iterator.h | 2 ++ 2 files changed, 10 insertions(+) diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index 96b0bea2ae82125..4ee73547c117e9d 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -1998,6 +1998,12 @@ Status SegmentIterator::copy_column_data_by_selector(vectorized::IColumn* input_ return input_col_ptr->filter_by_selector(sel_rowid_idx, select_size, output_col); } +void SegmentIterator::_clear_iterators() { + _column_iterators.clear(); + _bitmap_index_iterators.clear(); + _inverted_index_iterators.clear(); +} + Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { bool is_mem_reuse = block->mem_reuse(); DCHECK(is_mem_reuse); @@ -2104,6 +2110,8 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { } } block->clear_column_data(); + // clear and release iterators memory footprint in advance + _clear_iterators(); return Status::EndOfFile("no more data in segment"); } diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index 5588661302dd060..5b4c8f6d73d0cd3 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -383,6 +383,8 @@ class SegmentIterator : public RowwiseIterator { void _calculate_expr_in_remaining_conjunct_root(); + void _clear_iterators(); + class BitmapRangeIterator; class BackwardBitmapRangeIterator; From 1abfd10c34934bd24b3b208975c3441751f0608d Mon Sep 17 00:00:00 2001 From: lihangyu Date: Mon, 2 Dec 2024 11:09:02 +0800 Subject: [PATCH 15/31] [Fix](Variant) fix some nested explode_variant_array bug and add more test (#44533) --- be/src/vec/exprs/table_function/vexplode.cpp | 40 +++++++++++--- be/src/vec/exprs/table_function/vexplode.h | 1 + .../array/function_array_contains_all.cpp | 5 +- .../functions/array/function_array_distance.h | 4 +- .../functions/array/function_array_utils.cpp | 13 ++++- .../functions/array/function_array_utils.h | 6 ++- regression-test/data/variant_p0/nested.out | 20 +++++++ .../suites/variant_p0/nested.groovy | 53 ++++++++++++++++++- 8 files changed, 128 insertions(+), 14 deletions(-) diff --git a/be/src/vec/exprs/table_function/vexplode.cpp b/be/src/vec/exprs/table_function/vexplode.cpp index feef58cd27726c2..5fa378f63517eca 100644 --- a/be/src/vec/exprs/table_function/vexplode.cpp +++ b/be/src/vec/exprs/table_function/vexplode.cpp @@ -23,12 +23,17 @@ #include "common/status.h" #include "vec/columns/column.h" +#include "vec/columns/column_array.h" +#include "vec/columns/column_nothing.h" #include "vec/columns/column_object.h" #include "vec/core/block.h" #include "vec/core/column_with_type_and_name.h" #include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_nothing.h" #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" +#include "vec/functions/function_helpers.h" namespace doris::vectorized { #include "common/compile_check_begin.h" @@ -37,6 +42,34 @@ VExplodeTableFunction::VExplodeTableFunction() { _fn_name = "vexplode"; } +Status VExplodeTableFunction::_process_init_variant(Block* block, int value_column_idx) { + // explode variant array + const auto& variant_column = check_and_get_column( + remove_nullable(block->get_by_position(value_column_idx) + .column->convert_to_full_column_if_const()) + .get()); + _detail.output_as_variant = true; + if (!variant_column->is_null_root()) { + _array_column = variant_column->get_root(); + // We need to wrap the output nested column within a variant column. + // Otherwise the type is missmatched + const auto* array_type = check_and_get_data_type( + remove_nullable(variant_column->get_root_type()).get()); + if (array_type == nullptr) { + return Status::NotSupported("explode not support none array type {}", + variant_column->get_root_type()->get_name()); + } + _detail.nested_type = array_type->get_nested_type(); + } else { + // null root, use nothing type + _array_column = ColumnNullable::create(ColumnArray::create(ColumnNothing::create(0)), + ColumnUInt8::create(0)); + _array_column->assume_mutable()->insert_many_defaults(variant_column->size()); + _detail.nested_type = std::make_shared(); + } + return Status::OK(); +} + Status VExplodeTableFunction::process_init(Block* block, RuntimeState* state) { CHECK(_expr_context->root()->children().size() == 1) << "VExplodeTableFunction only support 1 child but has " @@ -47,12 +80,7 @@ Status VExplodeTableFunction::process_init(Block* block, RuntimeState* state) { &value_column_idx)); if (WhichDataType(remove_nullable(block->get_by_position(value_column_idx).type)) .is_variant_type()) { - // explode variant array - const auto& variant_column = check_and_get_column( - remove_nullable(block->get_by_position(value_column_idx) - .column->convert_to_full_column_if_const()) - .get()); - _array_column = variant_column->get_root(); + RETURN_IF_ERROR(_process_init_variant(block, value_column_idx)); } else { _array_column = block->get_by_position(value_column_idx).column->convert_to_full_column_if_const(); diff --git a/be/src/vec/exprs/table_function/vexplode.h b/be/src/vec/exprs/table_function/vexplode.h index 17b67d07824fa06..7b53926ae2c31e4 100644 --- a/be/src/vec/exprs/table_function/vexplode.h +++ b/be/src/vec/exprs/table_function/vexplode.h @@ -47,6 +47,7 @@ class VExplodeTableFunction : public TableFunction { int get_value(MutableColumnPtr& column, int max_step) override; private: + Status _process_init_variant(Block* block, int value_column_idx); ColumnPtr _array_column; ColumnArrayExecutionData _detail; size_t _array_offset; // start offset of array[row_idx] diff --git a/be/src/vec/functions/array/function_array_contains_all.cpp b/be/src/vec/functions/array/function_array_contains_all.cpp index 67279babd5f49be..c65ec57e3d65720 100644 --- a/be/src/vec/functions/array/function_array_contains_all.cpp +++ b/be/src/vec/functions/array/function_array_contains_all.cpp @@ -231,8 +231,9 @@ class FunctionArrayContainsAll : public IFunction { is_equal_value = false; } else { // all is not null, check the data is equal - const auto* left_column = assert_cast(left_data.nested_col); - const auto* right_column = assert_cast(right_data.nested_col); + const auto* left_column = assert_cast(left_data.nested_col.get()); + const auto* right_column = + assert_cast(right_data.nested_col.get()); auto res = left_column->compare_at(left_nested_loop_pos, right_pos, *right_column, -1); is_equal_value = (res == 0); diff --git a/be/src/vec/functions/array/function_array_distance.h b/be/src/vec/functions/array/function_array_distance.h index e03e52a0ce191fa..0984479ce3404b2 100644 --- a/be/src/vec/functions/array/function_array_distance.h +++ b/be/src/vec/functions/array/function_array_distance.h @@ -128,8 +128,8 @@ class FunctionArrayDistance : public IFunction { const auto& offsets1 = *arr1.offsets_ptr; const auto& offsets2 = *arr2.offsets_ptr; - const auto& nested_col1 = assert_cast(arr1.nested_col); - const auto& nested_col2 = assert_cast(arr2.nested_col); + const auto& nested_col1 = assert_cast(arr1.nested_col.get()); + const auto& nested_col2 = assert_cast(arr2.nested_col.get()); for (ssize_t row = 0; row < offsets1.size(); ++row) { if (arr1.array_nullmap_data && arr1.array_nullmap_data[row]) { dst_null_data[row] = true; diff --git a/be/src/vec/functions/array/function_array_utils.cpp b/be/src/vec/functions/array/function_array_utils.cpp index ab999aa21ccdd1d..d25904baf931f4b 100644 --- a/be/src/vec/functions/array/function_array_utils.cpp +++ b/be/src/vec/functions/array/function_array_utils.cpp @@ -24,7 +24,9 @@ #include "vec/columns/column.h" #include "vec/columns/column_nullable.h" +#include "vec/columns/column_object.h" #include "vec/columns/column_vector.h" +#include "vec/data_types/data_type.h" namespace doris::vectorized { @@ -45,12 +47,19 @@ bool extract_column_array_info(const IColumn& src, ColumnArrayExecutionData& dat // extract array offsets and nested column data.offsets_ptr = &data.array_col->get_offsets(); - data.nested_col = &data.array_col->get_data(); + data.nested_col = data.array_col->get_data_ptr(); // extract nested column is nullable if (data.nested_col->is_nullable()) { const auto& nested_null_col = reinterpret_cast(*data.nested_col); data.nested_nullmap_data = nested_null_col.get_null_map_data().data(); - data.nested_col = nested_null_col.get_nested_column_ptr().get(); + data.nested_col = nested_null_col.get_nested_column_ptr(); + } + if (data.output_as_variant && + !WhichDataType(remove_nullable(data.nested_type)).is_variant_type()) { + // set variant root column/type to from column/type + auto variant = ColumnObject::create(true /*always nullable*/); + variant->create_root(data.nested_type, make_nullable(data.nested_col)->assume_mutable()); + data.nested_col = variant->get_ptr(); } return true; } diff --git a/be/src/vec/functions/array/function_array_utils.h b/be/src/vec/functions/array/function_array_utils.h index dc522ed813760a3..36217c024522142 100644 --- a/be/src/vec/functions/array/function_array_utils.h +++ b/be/src/vec/functions/array/function_array_utils.h @@ -16,6 +16,7 @@ // under the License. #pragma once +#include "vec/columns/column.h" #include "vec/columns/column_array.h" #include "vec/columns/column_nullable.h" #include "vec/columns/columns_number.h" @@ -55,7 +56,10 @@ struct ColumnArrayExecutionData { const ColumnArray* array_col = nullptr; const ColumnArray::Offsets64* offsets_ptr = nullptr; const UInt8* nested_nullmap_data = nullptr; - const IColumn* nested_col = nullptr; + ColumnPtr nested_col = nullptr; + DataTypePtr nested_type = nullptr; + // wrap the nested column as variant column + bool output_as_variant = false; ColumnArrayMutableData to_mutable_data() const { ColumnArrayMutableData dst; diff --git a/regression-test/data/variant_p0/nested.out b/regression-test/data/variant_p0/nested.out index 2c105a68778a0dc..d0e0e9c822ddc2d 100644 --- a/regression-test/data/variant_p0/nested.out +++ b/regression-test/data/variant_p0/nested.out @@ -174,3 +174,23 @@ v.xx tinyint Yes false \N NONE 1 {"callLimit":3,"number":"02124713252","type":"HOME"} 1 {"callLimit":5,"number":"5550219210","type":"GSM"} +-- !sql -- +2 {"nested":[{"ba":"11111"},{"a":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +4 {"nested":[{"baaa":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +5 {"nested":[{"ba":"11111"},{"a":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +6 {"nested":[{"mmm":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +7 {"nested":[{"ba":"11111"},{"a":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +8 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +9 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +11 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +12 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +13 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} + +-- !explode_sql -- + +-- !explode_sql -- +19 10 + +-- !explode_sql -- +2 10 + diff --git a/regression-test/suites/variant_p0/nested.groovy b/regression-test/suites/variant_p0/nested.groovy index 90728df25326686..25bd682d43aca47 100644 --- a/regression-test/suites/variant_p0/nested.groovy +++ b/regression-test/suites/variant_p0/nested.groovy @@ -164,13 +164,64 @@ parallel_pipeline_task_num=7,parallel_fragment_exec_instance_num=4,profile_level sql """insert into var_nested2 select * from var_nested order by k limit 1024""" qt_sql """select /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=true,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=5,parallel_pipeline_task_num=1,profile_level=1,enable_pipeline_engine=false,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=false,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=false,enable_local_exchange=false,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=true,enable_delete_sub_predicate_v2=true,min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_join_spill=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ * from var_nested2 order by k limit 10;""" qt_sql """select v['nested'] from var_nested2 where k < 10 order by k limit 10;""" - // explode variant array + // 0. nomal explode variant array order_qt_explode_sql """select count(),cast(vv['xx'] as int) from var_nested lateral view explode_variant_array(v['nested']) tmp as vv where vv['xx'] = 10 group by cast(vv['xx'] as int)""" sql """truncate table var_nested2""" sql """insert into var_nested2 values(1119111, '{"eventId":1,"firstName":"Name1","lastName":"Surname1","body":{"phoneNumbers":[{"number":"5550219210","type":"GSM","callLimit":5},{"number":"02124713252","type":"HOME","callLimit":3},{"number":"05550219211","callLimit":2,"type":"WORK"}]}} ')""" order_qt_explode_sql """select v['eventId'], phone_numbers from var_nested2 lateral view explode_variant_array(v['body']['phoneNumbers']) tmp1 as phone_numbers where phone_numbers['type'] = 'GSM' OR phone_numbers['type'] = 'HOME' and phone_numbers['callLimit'] > 2;""" + + // test array_function + sql "DROP TABLE IF EXISTS var_nested_array_agg" + sql """ + CREATE TABLE IF NOT EXISTS var_nested_array_agg( + k bigint, + v variant + ) + UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(k) BUCKETS 1 + properties("replication_num" = "1", "disable_auto_compaction" = "false", "enable_unique_key_merge_on_write" = "true", "variant_enable_flatten_nested" = "true"); + """ + sql "insert into var_nested_array_agg select * from var_nested" + // 1. array_contains + qt_sql "select * from var_nested_array_agg where array_contains(cast(v['nested']['xx'] as array), 10) order by k limit 10" + // 2. array_agg scalar + sql "select k, array_agg(cast(v['nested'] as text)) from var_nested_array_agg group by k limit 10" + + // test explode_variant_array with abonomal case + sql "DROP TABLE IF EXISTS var_nested_explode_variant_with_abnomal" + sql """ + CREATE TABLE IF NOT EXISTS var_nested_explode_variant_with_abnomal( + k bigint, + v variant + ) + UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(k) BUCKETS 1 + properties("replication_num" = "1", "disable_auto_compaction" = "false", "enable_unique_key_merge_on_write" = "true", "variant_enable_flatten_nested" = "true"); + """ + sql "insert into var_nested_explode_variant_with_abnomal select * from var_nested" + // 1. v['nested']['x'] is null root + order_qt_explode_sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['nested']['x']) tmp as vv where vv = 10 group by cast(vv as int)""" + // 2. v['nested']['xx'] is normal array + order_qt_explode_sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['nested']['xx']) tmp as vv where vv = 10 group by cast(vv as int)""" + // 3. v['xx'] is none array scalar type + test { + sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['xx']) tmp as vv where vv = 10 group by cast(vv as int)""" + exception("explode not support none array type") + } + // 4. v['k1'] is json scalar type + test { + sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['k1']) tmp as vv where vv = 10 group by cast(vv as int)""" + exception("explode not support none array type") + } + // 5. toplevel nested array + sql "truncate table var_nested_explode_variant_with_abnomal" + sql """insert into var_nested_explode_variant_with_abnomal values(1, '[{"a" : 10}, {"b" : "20", "c" :1024, "a" : 11}]')""" + sql """insert into var_nested_explode_variant_with_abnomal values(2, '[{"a" : 10}, {"b" : "20", "a" : 150}]')""" + order_qt_explode_sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['a']) tmp as vv where vv = 10 group by cast(vv as int)""" + // FIXME after refator + // order_qt_explode_sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v) tmp as vv where vv['a'] = 10 group by cast(vv as int)""" } finally { // reset flags } From 7526b9c3a3e5f4e477cda5615d7943ed0d34f11a Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Mon, 2 Dec 2024 11:27:17 +0800 Subject: [PATCH 16/31] [fix](inverted index) add a sequence number to an enum variable (#44605) Related PR: #36757 Problem Summary: add a sequence number to an enum variable --- gensrc/thrift/Types.thrift | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 235c1cb28378e24..1912f950587dbfb 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -122,9 +122,9 @@ enum TStorageBackendType { // This enum is used to distinguish between different organizational methods // of inverted index data, affecting how the index is stored and accessed. enum TInvertedIndexFileStorageFormat { - DEFAULT, // Default format, unspecified storage method. - V1, // Index per idx: Each index is stored separately based on its identifier. - V2 // Segment id per idx: Indexes are organized based on segment identifiers, grouping indexes by their associated segment. + DEFAULT = 0, // Default format, unspecified storage method. + V1 = 1, // Index per idx: Each index is stored separately based on its identifier. + V2 = 2 // Segment id per idx: Indexes are organized based on segment identifiers, grouping indexes by their associated segment. } struct TScalarType { From ed4f7fb5995b9726bcf05430ecdabd77ff6cbd21 Mon Sep 17 00:00:00 2001 From: feiniaofeiafei Date: Mon, 2 Dec 2024 12:00:06 +0800 Subject: [PATCH 17/31] [feat](nereids) add rewrite rule :EliminateGroupByKeyByUniform (#43391) This PR introduces two main changes: 1. Adds an optional constant value to the uniform attribute in DataTrait. A slot with a constant value that is not null will be considered uniform and not null. 2. Introduces a new transform rule: EliminateGroupByKeyByUniform, which utilizes the newly added part of the uniform attribute. Following is example transformation: from +--aggregate(group by a,b output a,b,max(c)) (a is uniform and not null: e.g. a is projection 2 as a in logicalProject) to +--aggregate(group by b output b,any_value(a) as a,max(c)) --- .../doris/nereids/jobs/executor/Rewriter.java | 2 + .../doris/nereids/properties/DataTrait.java | 211 +++++++++++-- .../apache/doris/nereids/rules/RuleType.java | 7 + .../rules/expression/ExpressionRewrite.java | 2 +- .../rewrite/EliminateGroupByKeyByUniform.java | 148 +++++++++ .../nereids/rules/rewrite/ExprIdRewriter.java | 284 ++++++++++++++++++ .../plans/commands/info/CreateMTMVInfo.java | 3 +- .../trees/plans/logical/LogicalFilter.java | 7 +- .../trees/plans/logical/LogicalHaving.java | 7 +- .../trees/plans/logical/LogicalJoin.java | 34 ++- .../trees/plans/logical/LogicalProject.java | 14 +- .../trees/plans/logical/LogicalQualify.java | 7 +- .../doris/nereids/util/ExpressionUtils.java | 11 +- .../mv/MaterializedViewUtilsTest.java | 2 +- .../EliminateGroupByKeyByUniformTest.java | 250 +++++++++++++++ .../rewrite/EliminateGroupByKeyTest.java | 4 +- .../eliminate_group_by_key_by_uniform.out | 269 +++++++++++++++++ .../eliminate_group_by_key_by_uniform.groovy | 221 ++++++++++++++ .../aggregate_without_roll_up.groovy | 6 +- 19 files changed, 1428 insertions(+), 61 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java create mode 100644 regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out create mode 100644 regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index f4ca9a972a6814c..fa00a4e93433250 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -64,6 +64,7 @@ import org.apache.doris.nereids.rules.rewrite.EliminateFilter; import org.apache.doris.nereids.rules.rewrite.EliminateGroupBy; import org.apache.doris.nereids.rules.rewrite.EliminateGroupByKey; +import org.apache.doris.nereids.rules.rewrite.EliminateGroupByKeyByUniform; import org.apache.doris.nereids.rules.rewrite.EliminateJoinByFK; import org.apache.doris.nereids.rules.rewrite.EliminateJoinByUnique; import org.apache.doris.nereids.rules.rewrite.EliminateJoinCondition; @@ -356,6 +357,7 @@ public class Rewriter extends AbstractBatchJobExecutor { topDown(new EliminateJoinByUnique()) ), topic("eliminate Aggregate according to fd items", + custom(RuleType.ELIMINATE_GROUP_BY_KEY_BY_UNIFORM, EliminateGroupByKeyByUniform::new), topDown(new EliminateGroupByKey()), topDown(new PushDownAggThroughJoinOnPkFk()), topDown(new PullUpJoinFromUnionAll()) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java index e97fad6f4790470..ff4756979e450e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java @@ -17,18 +17,23 @@ package org.apache.doris.nereids.properties; +import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.util.ImmutableEqualSet; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -46,16 +51,16 @@ public class DataTrait { public static final DataTrait EMPTY_TRAIT - = new DataTrait(new NestedSet().toImmutable(), - new NestedSet().toImmutable(), new ImmutableSet.Builder().build(), + = new DataTrait(new UniqueDescription().toImmutable(), + new UniformDescription().toImmutable(), new ImmutableSet.Builder().build(), ImmutableEqualSet.empty(), new FuncDepsDG.Builder().build()); - private final NestedSet uniqueSet; - private final NestedSet uniformSet; + private final UniqueDescription uniqueSet; + private final UniformDescription uniformSet; private final ImmutableSet fdItems; private final ImmutableEqualSet equalSet; private final FuncDepsDG fdDg; - private DataTrait(NestedSet uniqueSet, NestedSet uniformSet, ImmutableSet fdItems, + private DataTrait(UniqueDescription uniqueSet, UniformDescription uniformSet, ImmutableSet fdItems, ImmutableEqualSet equalSet, FuncDepsDG fdDg) { this.uniqueSet = uniqueSet; this.uniformSet = uniformSet; @@ -86,8 +91,7 @@ public boolean isUniform(Slot slot) { } public boolean isUniform(Set slotSet) { - return !slotSet.isEmpty() - && uniformSet.slots.containsAll(slotSet); + return uniformSet.contains(slotSet); } public boolean isUniqueAndNotNull(Slot slot) { @@ -102,11 +106,25 @@ public boolean isUniqueAndNotNull(Set slotSet) { } public boolean isUniformAndNotNull(Slot slot) { - return !slot.nullable() && isUniform(slot); + return uniformSet.isUniformAndNotNull(slot); } + /** isUniformAndNotNull for slot set */ public boolean isUniformAndNotNull(ImmutableSet slotSet) { - return slotSet.stream().noneMatch(Slot::nullable) && isUniform(slotSet); + for (Slot slot : slotSet) { + if (!uniformSet.isUniformAndNotNull(slot)) { + return false; + } + } + return true; + } + + public boolean isUniformAndHasConstValue(Slot slot) { + return uniformSet.isUniformAndHasConstValue(slot); + } + + public Optional getUniformValue(Slot slot) { + return uniformSet.slotUniformValue.get(slot); } public boolean isNullSafeEqual(Slot l, Slot r) { @@ -143,23 +161,23 @@ public String toString() { * Builder of trait */ public static class Builder { - private final NestedSet uniqueSet; - private final NestedSet uniformSet; + private final UniqueDescription uniqueSet; + private final UniformDescription uniformSet; private ImmutableSet fdItems; private final ImmutableEqualSet.Builder equalSetBuilder; private final FuncDepsDG.Builder fdDgBuilder; public Builder() { - uniqueSet = new NestedSet(); - uniformSet = new NestedSet(); + uniqueSet = new UniqueDescription(); + uniformSet = new UniformDescription(); fdItems = new ImmutableSet.Builder().build(); equalSetBuilder = new ImmutableEqualSet.Builder<>(); fdDgBuilder = new FuncDepsDG.Builder(); } public Builder(DataTrait other) { - this.uniformSet = new NestedSet(other.uniformSet); - this.uniqueSet = new NestedSet(other.uniqueSet); + this.uniformSet = new UniformDescription(other.uniformSet); + this.uniqueSet = new UniqueDescription(other.uniqueSet); this.fdItems = ImmutableSet.copyOf(other.fdItems); equalSetBuilder = new ImmutableEqualSet.Builder<>(other.equalSet); fdDgBuilder = new FuncDepsDG.Builder(other.fdDg); @@ -173,6 +191,14 @@ public void addUniformSlot(DataTrait dataTrait) { uniformSet.add(dataTrait.uniformSet); } + public void addUniformSlotForOuterJoinNullableSide(DataTrait dataTrait) { + uniformSet.addUniformSlotForOuterJoinNullableSide(dataTrait.uniformSet); + } + + public void addUniformSlotAndLiteral(Slot slot, Expression literal) { + uniformSet.add(slot, literal); + } + public void addUniqueSlot(Slot slot) { uniqueSet.add(slot); } @@ -261,8 +287,21 @@ public void addUniqueByEqualSet(Set equalSet) { * if there is a uniform slot in the equivalence set, then all slots of an equivalence set are uniform */ public void addUniformByEqualSet(Set equalSet) { - if (uniformSet.isIntersect(uniformSet.slots, equalSet)) { - uniformSet.slots.addAll(equalSet); + List intersectionList = uniformSet.slotUniformValue.keySet().stream() + .filter(equalSet::contains) + .collect(Collectors.toList()); + if (intersectionList.isEmpty()) { + return; + } + Expression expr = null; + for (Slot slot : intersectionList) { + if (uniformSet.slotUniformValue.get(slot).isPresent()) { + expr = uniformSet.slotUniformValue.get(slot).get(); + break; + } + } + for (Slot equal : equalSet) { + uniformSet.add(equal, expr); } } @@ -293,9 +332,11 @@ public List> getAllUniqueAndNotNull() { */ public List> getAllUniformAndNotNull() { List> res = new ArrayList<>(); - for (Slot s : uniformSet.slots) { - if (!s.nullable()) { - res.add(ImmutableSet.of(s)); + for (Map.Entry> entry : uniformSet.slotUniformValue.entrySet()) { + if (!entry.getKey().nullable()) { + res.add(ImmutableSet.of(entry.getKey())); + } else if (entry.getValue().isPresent() && !entry.getValue().get().nullable()) { + res.add(ImmutableSet.of(entry.getKey())); } } return res; @@ -338,21 +379,21 @@ public void replaceFuncDepsBy(Map replaceMap) { } } - static class NestedSet { + static class UniqueDescription { Set slots; Set> slotSets; - NestedSet() { + UniqueDescription() { slots = new HashSet<>(); slotSets = new HashSet<>(); } - NestedSet(NestedSet o) { + UniqueDescription(UniqueDescription o) { this.slots = new HashSet<>(o.slots); this.slotSets = new HashSet<>(o.slotSets); } - NestedSet(Set slots, Set> slotSets) { + UniqueDescription(Set slots, Set> slotSets) { this.slots = slots; this.slotSets = slotSets; } @@ -408,9 +449,9 @@ public void add(ImmutableSet slotSet) { slotSets.add(slotSet); } - public void add(NestedSet nestedSet) { - slots.addAll(nestedSet.slots); - slotSets.addAll(nestedSet.slotSets); + public void add(UniqueDescription uniqueDescription) { + slots.addAll(uniqueDescription.slots); + slotSets.addAll(uniqueDescription.slotSets); } public boolean isIntersect(Set set1, Set set2) { @@ -446,8 +487,120 @@ public void replace(Map replaceMap) { .collect(Collectors.toSet()); } - public NestedSet toImmutable() { - return new NestedSet(ImmutableSet.copyOf(slots), ImmutableSet.copyOf(slotSets)); + public UniqueDescription toImmutable() { + return new UniqueDescription(ImmutableSet.copyOf(slots), ImmutableSet.copyOf(slotSets)); + } + } + + static class UniformDescription { + // slot and its uniform expression(literal or const expression) + // some slot can get uniform values, others can not. + // e.g.select a from t where a=10 group by a, b; + // in LogicalAggregate, a UniformDescription with map {a : 10} can be obtained. + // which means a is uniform and the uniform value is 10. + Map> slotUniformValue; + + public UniformDescription() { + slotUniformValue = new LinkedHashMap<>(); + } + + public UniformDescription(UniformDescription ud) { + slotUniformValue = new LinkedHashMap<>(ud.slotUniformValue); + } + + public UniformDescription(Map> slotUniformValue) { + this.slotUniformValue = slotUniformValue; + } + + public UniformDescription toImmutable() { + return new UniformDescription(ImmutableMap.copyOf(slotUniformValue)); + } + + public boolean isEmpty() { + return slotUniformValue.isEmpty(); + } + + public boolean contains(Slot slot) { + return slotUniformValue.containsKey(slot); + } + + public boolean contains(Set slots) { + return !slots.isEmpty() && slotUniformValue.keySet().containsAll(slots); + } + + public void add(Slot slot) { + slotUniformValue.putIfAbsent(slot, Optional.empty()); + } + + public void add(Set slots) { + for (Slot s : slots) { + slotUniformValue.putIfAbsent(s, Optional.empty()); + } + } + + public void add(UniformDescription ud) { + slotUniformValue.putAll(ud.slotUniformValue); + for (Map.Entry> entry : ud.slotUniformValue.entrySet()) { + add(entry.getKey(), entry.getValue().orElse(null)); + } + } + + public void add(Slot slot, Expression literal) { + if (null == literal) { + slotUniformValue.putIfAbsent(slot, Optional.empty()); + } else { + slotUniformValue.put(slot, Optional.of(literal)); + } + } + + public void addUniformSlotForOuterJoinNullableSide(UniformDescription ud) { + for (Map.Entry> entry : ud.slotUniformValue.entrySet()) { + if ((!entry.getValue().isPresent() && entry.getKey().nullable()) + || (entry.getValue().isPresent() && entry.getValue().get() instanceof NullLiteral)) { + add(entry.getKey(), entry.getValue().orElse(null)); + } + } + } + + public void removeNotContain(Set slotSet) { + if (slotSet.isEmpty()) { + return; + } + Map> newSlotUniformValue = new LinkedHashMap<>(); + for (Map.Entry> entry : slotUniformValue.entrySet()) { + if (slotSet.contains(entry.getKey())) { + newSlotUniformValue.put(entry.getKey(), entry.getValue()); + } + } + this.slotUniformValue = newSlotUniformValue; + } + + public void replace(Map replaceMap) { + Map> newSlotUniformValue = new LinkedHashMap<>(); + for (Map.Entry> entry : slotUniformValue.entrySet()) { + Slot newKey = replaceMap.getOrDefault(entry.getKey(), entry.getKey()); + newSlotUniformValue.put(newKey, entry.getValue()); + } + slotUniformValue = newSlotUniformValue; + } + + // The current implementation logic is: if a slot key exists in map slotUniformValue, + // its value is present and is not nullable, + // or if a slot key exists in map slotUniformValue and the slot is not nullable + // it indicates that this slot is uniform and not null. + public boolean isUniformAndNotNull(Slot slot) { + return slotUniformValue.containsKey(slot) + && (!slot.nullable() || slotUniformValue.get(slot).isPresent() + && !slotUniformValue.get(slot).get().nullable()); + } + + public boolean isUniformAndHasConstValue(Slot slot) { + return slotUniformValue.containsKey(slot) && slotUniformValue.get(slot).isPresent(); + } + + @Override + public String toString() { + return "{" + slotUniformValue + "}"; } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index beb8bd436557434..c81f56f85b6adde 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -220,6 +220,12 @@ public enum RuleType { REWRITE_HAVING_EXPRESSION(RuleTypeClass.REWRITE), REWRITE_REPEAT_EXPRESSION(RuleTypeClass.REWRITE), REWRITE_OLAP_TABLE_SINK_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_SINK_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_WINDOW_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_SET_OPERATION_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_PARTITION_TOPN_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_QUALIFY_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_TOPN_EXPRESSION(RuleTypeClass.REWRITE), EXTRACT_FILTER_FROM_JOIN(RuleTypeClass.REWRITE), REORDER_JOIN(RuleTypeClass.REWRITE), MERGE_PERCENTILE_TO_ARRAY(RuleTypeClass.REWRITE), @@ -249,6 +255,7 @@ public enum RuleType { ELIMINATE_JOIN_BY_UK(RuleTypeClass.REWRITE), ELIMINATE_JOIN_BY_FK(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY_KEY(RuleTypeClass.REWRITE), + ELIMINATE_GROUP_BY_KEY_BY_UNIFORM(RuleTypeClass.REWRITE), ELIMINATE_FILTER_GROUP_BY_KEY(RuleTypeClass.REWRITE), ELIMINATE_DEDUP_JOIN_CONDITION(RuleTypeClass.REWRITE), ELIMINATE_NULL_AWARE_LEFT_ANTI_JOIN(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java index e79dd6705c05130..e5b74ee26bcb02f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java @@ -56,7 +56,7 @@ * expression of plan rewrite rule. */ public class ExpressionRewrite implements RewriteRuleFactory { - private final ExpressionRuleExecutor rewriter; + protected final ExpressionRuleExecutor rewriter; public ExpressionRewrite(ExpressionRewriteRule... rules) { this.rewriter = new ExpressionRuleExecutor(ImmutableList.copyOf(rules)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java new file mode 100644 index 000000000000000..4cb39c2a9341aee --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java @@ -0,0 +1,148 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.properties.DataTrait; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * +--aggregate(group by a, b output a#0 ,b#1, max(c) as max(c)#2) + * (a is uniform and not null: e.g. a is projection 2 as a in logicalProject) + * -> + * +--aggregate(group by b output b#1, any_value(a#0) as a#3, max(c)#2) + * if output any_value(a#0) as a#0, the uniqueness of ExprId #0 is violated, because #0 is both any_value(a#0) and a#0 + * error will occurs in other module(e.g. mv rewrite). + * As a result, new aggregate outputs #3 instead of #0, but upper plan refer slot #0, + * therefore, all references to #0 in the upper plan need to be changed to #3. + * use ExprIdRewriter to do this ExprId rewrite, and use CustomRewriter to rewrite upward。 + * */ +public class EliminateGroupByKeyByUniform extends DefaultPlanRewriter> implements CustomRewriter { + private ExprIdRewriter exprIdReplacer; + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + Optional cteId = jobContext.getCascadesContext().getCurrentTree(); + if (cteId.isPresent()) { + return plan; + } + Map replaceMap = new HashMap<>(); + ExprIdRewriter.ReplaceRule replaceRule = new ExprIdRewriter.ReplaceRule(replaceMap); + exprIdReplacer = new ExprIdRewriter(replaceRule, jobContext); + return plan.accept(this, replaceMap); + } + + @Override + public Plan visit(Plan plan, Map replaceMap) { + plan = visitChildren(this, plan, replaceMap); + plan = exprIdReplacer.rewriteExpr(plan, replaceMap); + return plan; + } + + @Override + public Plan visitLogicalAggregate(LogicalAggregate aggregate, Map replaceMap) { + aggregate = visitChildren(this, aggregate, replaceMap); + aggregate = (LogicalAggregate) exprIdReplacer.rewriteExpr(aggregate, replaceMap); + + if (aggregate.getGroupByExpressions().isEmpty() || aggregate.getSourceRepeat().isPresent()) { + return aggregate; + } + DataTrait aggChildTrait = aggregate.child().getLogicalProperties().getTrait(); + // Get the Group by column of agg. If there is a uniform one, delete the group by key. + Set removedExpression = new LinkedHashSet<>(); + List newGroupBy = new ArrayList<>(); + for (Expression groupBy : aggregate.getGroupByExpressions()) { + if (!(groupBy instanceof Slot)) { + newGroupBy.add(groupBy); + continue; + } + if (aggChildTrait.isUniformAndNotNull((Slot) groupBy)) { + removedExpression.add(groupBy); + } else { + newGroupBy.add(groupBy); + } + } + if (removedExpression.isEmpty()) { + return aggregate; + } + // when newGroupBy is empty, need retain one expr in group by, otherwise the result may be wrong in empty table + if (newGroupBy.isEmpty()) { + Expression expr = removedExpression.iterator().next(); + newGroupBy.add(expr); + removedExpression.remove(expr); + } + if (removedExpression.isEmpty()) { + return aggregate; + } + List newOutputs = new ArrayList<>(); + // If this output appears in the removedExpression column, replace it with any_value + for (NamedExpression output : aggregate.getOutputExpressions()) { + if (output instanceof Slot) { + if (removedExpression.contains(output)) { + Alias alias = new Alias(new AnyValue(false, output), output.getName()); + newOutputs.add(alias); + replaceMap.put(output.getExprId(), alias.getExprId()); + } else { + newOutputs.add(output); + } + } else if (output instanceof Alias) { + if (removedExpression.contains(output.child(0))) { + newOutputs.add(new Alias( + new AnyValue(false, output.child(0)), output.getName())); + } else { + newOutputs.add(output); + } + } else { + newOutputs.add(output); + } + } + + // Adjust the order of this new output so that aggregate functions are placed at the back + // and non-aggregated functions are placed at the front. + List aggFuncs = new ArrayList<>(); + List orderOutput = new ArrayList<>(); + for (NamedExpression output : newOutputs) { + if (output.anyMatch(e -> e instanceof AggregateFunction)) { + aggFuncs.add(output); + } else { + orderOutput.add(output); + } + } + orderOutput.addAll(aggFuncs); + return aggregate.withGroupByAndOutput(newGroupBy, orderOutput); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java new file mode 100644 index 000000000000000..60c9da4bc6eec57 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java @@ -0,0 +1,284 @@ +// 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.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.pattern.MatchingContext; +import org.apache.doris.nereids.pattern.Pattern; +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher; +import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory; +import org.apache.doris.nereids.rules.expression.ExpressionRewrite; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.ExpressionRuleExecutor; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; +import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; + +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** replace SlotReference ExprId in logical plans */ +public class ExprIdRewriter extends ExpressionRewrite { + private final List rules; + private final JobContext jobContext; + + public ExprIdRewriter(ReplaceRule replaceRule, JobContext jobContext) { + super(new ExpressionRuleExecutor(ImmutableList.of(bottomUp(replaceRule)))); + rules = buildRules(); + this.jobContext = jobContext; + } + + @Override + public List buildRules() { + ImmutableList.Builder builder = ImmutableList.builder(); + builder.addAll(super.buildRules()); + builder.addAll(ImmutableList.of( + new LogicalPartitionTopNExpressionRewrite().build(), + new LogicalTopNExpressionRewrite().build(), + new LogicalSetOperationRewrite().build(), + new LogicalWindowRewrite().build(), + new LogicalResultSinkRewrite().build(), + new LogicalFileSinkRewrite().build(), + new LogicalHiveTableSinkRewrite().build(), + new LogicalIcebergTableSinkRewrite().build(), + new LogicalJdbcTableSinkRewrite().build(), + new LogicalOlapTableSinkRewrite().build(), + new LogicalDeferMaterializeResultSinkRewrite().build() + )); + return builder.build(); + } + + /**rewriteExpr*/ + public Plan rewriteExpr(Plan plan, Map replaceMap) { + if (replaceMap.isEmpty()) { + return plan; + } + for (Rule rule : rules) { + Pattern pattern = (Pattern) rule.getPattern(); + if (pattern.matchPlanTree(plan)) { + List newPlans = rule.transform(plan, jobContext.getCascadesContext()); + Plan newPlan = newPlans.get(0); + if (!newPlan.deepEquals(plan)) { + return newPlan; + } + } + } + return plan; + } + + /** + * Iteratively rewrites IDs using the replaceMap: + * 1. For a given SlotReference with initial ID, retrieve the corresponding value ID from the replaceMap. + * 2. If the value ID exists within the replaceMap, continue the lookup process using the value ID + * until it no longer appears in the replaceMap. + * 3. return SlotReference final value ID as the result of the rewrite. + * e.g. replaceMap:{0:3, 1:6, 6:7} + * SlotReference:a#0 -> a#3, a#1 -> a#7 + * */ + public static class ReplaceRule implements ExpressionPatternRuleFactory { + private final Map replaceMap; + + public ReplaceRule(Map replaceMap) { + this.replaceMap = replaceMap; + } + + @Override + public List> buildRules() { + return ImmutableList.of( + matchesType(SlotReference.class).thenApply(ctx -> { + Slot slot = ctx.expr; + if (replaceMap.containsKey(slot.getExprId())) { + ExprId newId = replaceMap.get(slot.getExprId()); + while (replaceMap.containsKey(newId)) { + newId = replaceMap.get(newId); + } + return slot.withExprId(newId); + } + return slot; + }) + ); + } + } + + private class LogicalResultSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalResultSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalFileSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalFileSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalHiveTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalHiveTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalIcebergTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalIcebergTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalJdbcTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalJdbcTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalOlapTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalOlapTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalDeferMaterializeResultSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalDeferMaterializeResultSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalSetOperationRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalSetOperation().thenApply(ctx -> { + LogicalSetOperation setOperation = ctx.root; + List> slotsList = setOperation.getRegularChildrenOutputs(); + List> newSlotsList = new ArrayList<>(); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + for (List slots : slotsList) { + List newSlots = rewriteAll(slots, rewriter, context); + newSlotsList.add(newSlots); + } + if (newSlotsList.equals(slotsList)) { + return setOperation; + } + return setOperation.withChildrenAndTheirOutputs(setOperation.children(), newSlotsList); + }) + .toRule(RuleType.REWRITE_SET_OPERATION_EXPRESSION); + } + } + + private class LogicalWindowRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalWindow().thenApply(ctx -> { + LogicalWindow window = ctx.root; + List windowExpressions = window.getWindowExpressions(); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List newWindowExpressions = rewriteAll(windowExpressions, rewriter, context); + if (newWindowExpressions.equals(windowExpressions)) { + return window; + } + return window.withExpressionsAndChild(newWindowExpressions, window.child()); + }) + .toRule(RuleType.REWRITE_WINDOW_EXPRESSION); + } + } + + private class LogicalTopNExpressionRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalTopN().thenApply(ctx -> { + LogicalTopN topN = ctx.root; + List orderKeys = topN.getOrderKeys(); + ImmutableList.Builder rewrittenOrderKeys + = ImmutableList.builderWithExpectedSize(orderKeys.size()); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + boolean changed = false; + for (OrderKey k : orderKeys) { + Expression expression = rewriter.rewrite(k.getExpr(), context); + changed |= expression != k.getExpr(); + rewrittenOrderKeys.add(new OrderKey(expression, k.isAsc(), k.isNullFirst())); + } + return changed ? topN.withOrderKeys(rewrittenOrderKeys.build()) : topN; + }).toRule(RuleType.REWRITE_TOPN_EXPRESSION); + } + } + + private class LogicalPartitionTopNExpressionRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalPartitionTopN().thenApply(ctx -> { + LogicalPartitionTopN partitionTopN = ctx.root; + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List newOrderExpressions = new ArrayList<>(); + boolean changed = false; + for (OrderExpression orderExpression : partitionTopN.getOrderKeys()) { + OrderKey orderKey = orderExpression.getOrderKey(); + Expression expr = rewriter.rewrite(orderKey.getExpr(), context); + changed |= expr != orderKey.getExpr(); + OrderKey newOrderKey = new OrderKey(expr, orderKey.isAsc(), orderKey.isNullFirst()); + newOrderExpressions.add(new OrderExpression(newOrderKey)); + } + List newPartitionKeys = rewriteAll(partitionTopN.getPartitionKeys(), rewriter, context); + if (!newPartitionKeys.equals(partitionTopN.getPartitionKeys())) { + changed = true; + } + if (!changed) { + return partitionTopN; + } + return partitionTopN.withPartitionKeysAndOrderKeys(newPartitionKeys, newOrderExpressions); + }).toRule(RuleType.REWRITE_PARTITION_TOPN_EXPRESSION); + } + } + + private LogicalSink applyRewrite(MatchingContext> ctx) { + LogicalSink sink = ctx.root; + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List outputExprs = sink.getOutputExprs(); + List newOutputExprs = rewriteAll(outputExprs, rewriter, context); + if (outputExprs.equals(newOutputExprs)) { + return sink; + } + return sink.withOutputExprs(newOutputExprs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 8c44b42a5ccfbdc..5c6c71c92e81119 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -103,7 +103,8 @@ */ public class CreateMTMVInfo { public static final Logger LOG = LogManager.getLogger(CreateMTMVInfo.class); - public static final String MTMV_PLANER_DISABLE_RULES = "OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION"; + public static final String MTMV_PLANER_DISABLE_RULES = "OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION," + + "ELIMINATE_GROUP_BY_KEY_BY_UNIFORM"; private final boolean ifNotExists; private final TableNameInfo mvName; private List keys; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java index d23ea3d2395f051..efd7e90c13615e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java @@ -37,6 +37,7 @@ import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -154,9 +155,9 @@ public void computeUnique(Builder builder) { @Override public void computeUniform(Builder builder) { for (Expression e : getConjuncts()) { - Set uniformSlots = ExpressionUtils.extractUniformSlot(e); - for (Slot slot : uniformSlots) { - builder.addUniformSlot(slot); + Map uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Map.Entry entry : uniformSlots.entrySet()) { + builder.addUniformSlotAndLiteral(entry.getKey(), entry.getValue()); } } builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java index f4f2178840b6ab9..680988b39f6bb1f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java @@ -35,6 +35,7 @@ import com.google.common.collect.ImmutableSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -125,9 +126,9 @@ public void computeUnique(Builder builder) { @Override public void computeUniform(Builder builder) { for (Expression e : getConjuncts()) { - Set uniformSlots = ExpressionUtils.extractUniformSlot(e); - for (Slot slot : uniformSlots) { - builder.addUniformSlot(slot); + Map uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Map.Entry entry : uniformSlots.entrySet()) { + builder.addUniformSlotAndLiteral(entry.getKey(), entry.getValue()); } } builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java index f557b07d3b646ef..c583360c3d8a768 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java @@ -535,11 +535,35 @@ public void computeUniform(Builder builder) { // TODO disable function dependence calculation for mark join, but need re-think this in future. return; } - if (!joinType.isLeftSemiOrAntiJoin()) { - builder.addUniformSlot(right().getLogicalProperties().getTrait()); - } - if (!joinType.isRightSemiOrAntiJoin()) { - builder.addUniformSlot(left().getLogicalProperties().getTrait()); + switch (joinType) { + case INNER_JOIN: + case CROSS_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + break; + case LEFT_SEMI_JOIN: + case LEFT_ANTI_JOIN: + case NULL_AWARE_LEFT_ANTI_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + break; + case RIGHT_SEMI_JOIN: + case RIGHT_ANTI_JOIN: + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + break; + case LEFT_OUTER_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(right().getLogicalProperties().getTrait()); + break; + case RIGHT_OUTER_JOIN: + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(left().getLogicalProperties().getTrait()); + break; + case FULL_OUTER_JOIN: + builder.addUniformSlotForOuterJoinNullableSide(left().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(right().getLogicalProperties().getTrait()); + break; + default: + break; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java index 171a355d8452d82..612231340f16edf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java @@ -238,14 +238,18 @@ public void computeUnique(DataTrait.Builder builder) { public void computeUniform(DataTrait.Builder builder) { builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); for (NamedExpression proj : getProjects()) { - if (proj.children().isEmpty()) { + if (!(proj instanceof Alias)) { continue; } if (proj.child(0).isConstant()) { - builder.addUniformSlot(proj.toSlot()); - } else if (ExpressionUtils.isInjective(proj.child(0))) { - ImmutableSet inputs = ImmutableSet.copyOf(proj.getInputSlots()); - if (child(0).getLogicalProperties().getTrait().isUniform(inputs)) { + builder.addUniformSlotAndLiteral(proj.toSlot(), proj.child(0)); + } else if (proj.child(0) instanceof Slot) { + Slot slot = (Slot) proj.child(0); + DataTrait childTrait = child(0).getLogicalProperties().getTrait(); + if (childTrait.isUniformAndHasConstValue(slot)) { + builder.addUniformSlotAndLiteral(proj.toSlot(), + child(0).getLogicalProperties().getTrait().getUniformValue(slot).get()); + } else if (childTrait.isUniform(slot)) { builder.addUniformSlot(proj.toSlot()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java index ced6730dfb54a20..904c66f6482ee53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java @@ -35,6 +35,7 @@ import com.google.common.collect.ImmutableSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -130,9 +131,9 @@ public void computeUnique(Builder builder) { @Override public void computeUniform(Builder builder) { for (Expression e : getConjuncts()) { - Set uniformSlots = ExpressionUtils.extractUniformSlot(e); - for (Slot slot : uniformSlots) { - builder.addUniformSlot(slot); + Map uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Map.Entry entry : uniformSlots.entrySet()) { + builder.addUniformSlotAndLiteral(entry.getKey(), entry.getValue()); } } builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index e3909b87bd3726f..53ab8a50683ecbd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -73,6 +73,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -744,15 +745,15 @@ public static List collectToList(Collection express /** * extract uniform slot for the given predicate, such as a = 1 and b = 2 */ - public static ImmutableSet extractUniformSlot(Expression expression) { - ImmutableSet.Builder builder = new ImmutableSet.Builder<>(); + public static ImmutableMap extractUniformSlot(Expression expression) { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); if (expression instanceof And) { - builder.addAll(extractUniformSlot(expression.child(0))); - builder.addAll(extractUniformSlot(expression.child(1))); + builder.putAll(extractUniformSlot(expression.child(0))); + builder.putAll(extractUniformSlot(expression.child(1))); } if (expression instanceof EqualTo) { if (isInjective(expression.child(0)) && expression.child(1).isConstant()) { - builder.add((Slot) expression.child(0)); + builder.put((Slot) expression.child(0), expression.child(1)); } } return builder.build(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java index f824a40eda64746..45e1190412d0a4f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java @@ -248,7 +248,7 @@ protected void runBeforeAll() throws Exception { + "\"replication_allocation\" = \"tag.location.default: 1\"\n" + ");\n"); // Should not make scan to empty relation when the table used by materialized view has no data - connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION"); + connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION,ELIMINATE_GROUP_BY_KEY_BY_UNIFORM"); } // Test when join both side are all partition table and partition column name is same diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java new file mode 100644 index 000000000000000..78d8034e3fdfed3 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java @@ -0,0 +1,250 @@ +// 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.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Test; + +public class EliminateGroupByKeyByUniformTest extends TestWithFeService implements MemoPatternMatchSupported { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + createTable("create table test.eli_gbk_by_uniform_t(a int null, b int not null," + + "c varchar(10) null, d date, dt datetime)\n" + + "distributed by hash(a) properties('replication_num' = '1');"); + connectContext.setDatabase("test"); + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + } + + @Test + void testEliminateByFilter() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a),b from eli_gbk_by_uniform_t where a = 1 group by a,b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("b"))); + + } + + @Test + void testNotEliminateWhenOnlyOneGbyKey() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a) from eli_gbk_by_uniform_t where a = 1 group by a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + + } + + @Test + void testEliminateByProjectConst() { + PlanChecker.from(connectContext) + .analyze("select sum(c1), c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 ") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c3"))); + } + + @Test + void testEliminateByProjectUniformSlot() { + PlanChecker.from(connectContext) + .analyze("select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t group by c1,c2,c3") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c3"))); + } + + @Test + void testEliminateDate() { + PlanChecker.from(connectContext) + .analyze("select d, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where d = '2023-01-06' group by d,a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + } + + @Test + void testSaveOneExpr() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=100 group by a, b,'abc'") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + } + + @Test + void testSaveOneExprProjectConst() { + PlanChecker.from(connectContext) + .analyze("select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c2"))); + } + + @Test + void testNotRewriteWhenHasRepeat() { + PlanChecker.from(connectContext) + .analyze("select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by grouping sets((c2),(c3)) order by 1;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testInnerJoin() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testLeftJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testRightJoinOnConditionNullableSideFilterNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testRightJoinOnConditionNonNullableSideFilterNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testRightJoinWhereConditionToInnerRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftSemiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftSemiJoinRetainOneSlotInGroupBy() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightSemiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightSemiJoinRetainOneSlotInGroupBy() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftAntiJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testLeftAntiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t1.c") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightAntiJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testRightAntiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t2.b=100 group by t2.b,t2.c") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java index 5a9e15cf4774d10..103e074c73bfd52 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java @@ -111,7 +111,7 @@ void testProjectAlias() { .rewrite() .printlnTree() .matches(logicalAggregate().when(agg -> - agg.getGroupByExpressions().size() == 2)); + agg.getGroupByExpressions().size() == 1)); PlanChecker.from(connectContext) .analyze("select id as c, name as n from t1 group by name, id") .rewrite() @@ -123,7 +123,7 @@ void testProjectAlias() { .rewrite() .printlnTree() .matches(logicalAggregate().when(agg -> - agg.getGroupByExpressions().size() == 2)); + agg.getGroupByExpressions().size() == 1)); } @Test diff --git a/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out b/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out new file mode 100644 index 000000000000000..94cebafa4e9aad9 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out @@ -0,0 +1,269 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !empty_tranform_not_to_scalar_agg -- + +-- !empty_tranform_multi_column -- + +-- !empty_tranform_multi_column -- +1 1 2 2 + +-- !tranform_to_scalar_agg_not_null_column -- + +-- !project_const -- +\N 1 +\N 1 +2 1 +2 1 +4 1 +6 1 +6 1 +10 1 + +-- !project_slot_uniform -- +2023-01-01 1 1 2023-01-01 + +-- !upper_refer -- + +-- !upper_refer_varchar_alias -- +cherry 3 + +-- !upper_refer_date -- +2023-01-06 + +-- !upper_refer_datetime_not_to_scalar_agg -- +2023-01-06T15:00 + +-- !upper_refer_datetime -- +2023-01-06T15:00 + +-- !project_no_other_agg_func -- +1 +1 +1 +1 +1 +1 +1 +1 + +-- !project_const_not_to_scalar_agg_multi -- +1 + +-- !not_to_scalar_agg_multi -- +1 1 2 2 + +-- !conflict_equal_value -- + +-- !project_slot_uniform_confict_value -- + +-- !inner_join_left_has_filter -- +100 100 + +-- !inner_join_right_has_filter -- +100 100 + +-- !left_join_right_has_filter -- +100 100 +101 \N +102 \N +103 \N +104 \N +105 \N +106 \N +107 \N + +-- !left_join_left_has_filter -- +100 100 +101 \N +102 \N +103 \N +104 \N +105 \N +106 \N +107 \N + +-- !right_join_right_has_filter -- +\N 101 +\N 102 +\N 103 +\N 104 +\N 105 +\N 106 +\N 107 +100 100 + +-- !right_join_left_has_filter -- +\N 101 +\N 102 +\N 103 +\N 104 +\N 105 +\N 106 +\N 107 +100 100 + +-- !left_semi_join_right_has_filter -- +100 + +-- !left_semi_join_left_has_filter -- +100 + +-- !left_anti_join_right_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !left_anti_join_left_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !left_anti_join_left_has_where_filter -- + +-- !right_semi_join_right_has_filter -- +100 + +-- !right_semi_join_left_has_filter -- +100 + +-- !right_anti_join_right_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !right_anti_join_left_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !right_anti_join_right_has_where_filter -- + +-- !cross_join_left_has_filter -- +100 100 +100 101 +100 102 +100 103 +100 104 +100 105 +100 106 +100 107 + +-- !cross_join_right_has_filter -- +100 100 +101 100 +102 100 +103 100 +104 100 +105 100 +106 100 +107 100 + +-- !union -- +1 100 +5 105 + +-- !union_all -- +1 100 +1 100 +5 105 + +-- !intersect -- + +-- !except -- + +-- !set_op_mixed -- +1 100 + +-- !window -- + +-- !partition_topn -- + +-- !partition_topn_qualifiy -- + +-- !cte_producer -- +1 1 100 + +-- !cte_multi_producer -- + +-- !cte_consumer -- + +-- !filter -- +1 100 + +-- !topn -- +1 100 + +-- !sink -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !nest_exprid_replace -- +2023-10-17 2 2023-10-17 2 6 +2023-10-17 2 2023-10-18 2 6 +2023-10-17 2 2023-10-21 2 6 +2023-10-18 2 2023-10-17 2 6 +2023-10-18 2 2023-10-18 2 6 +2023-10-18 2 2023-10-21 2 6 + +-- !full_join_uniform_should_not_eliminate_group_by_key -- +\N 1 +105 1 + +-- !full2 -- +1 \N +1 105 + +-- !left_join_right_side_should_not_eliminate_group_by_key -- +\N 1 +105 1 + +-- !left_join_left_side_should_eliminate_group_by_key -- +\N 1 +105 1 + +-- !right_join_left_side_should_not_eliminate_group_by_key -- +1 \N +1 105 + +-- !right_join_right_side_should_eliminate_group_by_key -- +1 \N +1 105 + +-- !left_semi_left_side -- +1 +1 + +-- !left_anti_left_side -- +1 + +-- !right_semi_right_side -- +105 +105 + +-- !right_anti_right_side -- + diff --git a/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy b/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy new file mode 100644 index 000000000000000..4375aac316d5d19 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy @@ -0,0 +1,221 @@ +// 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. +suite("eliminate_group_by_key_by_uniform") { + sql "set enable_nereids_rules = 'ELIMINATE_GROUP_BY_KEY_BY_UNIFORM'" + sql "drop table if exists eli_gbk_by_uniform_t" + sql """create table eli_gbk_by_uniform_t(a int null, b int not null, c varchar(10) null, d date, dt datetime) + distributed by hash(a) properties("replication_num"="1"); + """ + qt_empty_tranform_not_to_scalar_agg "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 group by a" + qt_empty_tranform_multi_column "select a,b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=2 group by a,b" + + sql """ + INSERT INTO eli_gbk_by_uniform_t (a, b, c, d, dt) VALUES + (1, 100, 'apple', '2023-01-01', '2023-01-01 10:00:00'), + (1, 100, 'apple', '2023-01-01', '2023-01-01 10:00:00'), + (2, 101, 'banana', '2023-01-02', '2023-01-02 11:00:00'), + (3, 102, 'cherry', '2023-01-03', '2023-01-03 12:00:00'), + (3, 102, 'cherry', '2023-01-03', '2023-01-03 12:00:00'), + (NULL, 103, 'date', '2023-01-04', '2023-01-04 13:00:00'), + (4, 104, 'elderberry', '2023-01-05', '2023-01-05 14:00:00'), + (5, 105, NULL, '2023-01-06', '2023-01-06 15:00:00'), + (5, 105, NULL, '2023-01-06', '2023-01-06 15:00:00'), + (6, 106, 'fig', '2023-01-07', '2023-01-07 16:00:00'), + (NULL, 107, 'grape', '2023-01-08', '2023-01-08 17:00:00'); + """ + qt_empty_tranform_multi_column "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 group by a, b,'abc' order by 1,2,3,4" + qt_tranform_to_scalar_agg_not_null_column "select b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where b = 1 group by a, b order by 1,2,3,4" + + qt_project_const "select sum(c1), c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1,2;" + qt_project_slot_uniform "select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t group by c1,c2,c3 order by 1,2,3,4;" + + qt_upper_refer "select b from (select b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where b = 1 group by a, b) t order by b" + qt_upper_refer_varchar_alias "select c1,c2 from (select c as c1, min(a) c2, sum(a), count(a) from eli_gbk_by_uniform_t where c = 'cherry' group by a, b,c) t order by c1,c2" + qt_upper_refer_date "select d from (select d, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where d = '2023-01-06' group by d,a) t order by 1" + qt_upper_refer_datetime_not_to_scalar_agg "select dt from (select dt, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where dt = '2023-01-06 15:00:00' group by dt) t order by 1" + qt_upper_refer_datetime "select dt from (select dt, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where dt = '2023-01-06 15:00:00' group by dt, a) t order by 1" + + qt_project_no_other_agg_func "select c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;" + qt_project_const_not_to_scalar_agg_multi "select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;" + qt_not_to_scalar_agg_multi "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=100 group by a, b,'abc' order by 1,2,3,4" + qt_conflict_equal_value "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and a=2 group by a, b,'abc' order by 1,2,3,4" + qt_project_slot_uniform_confict_value "select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t where c2=2 group by c1,c2,c3 order by 1,2,3,4;" + + // test join + qt_inner_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_inner_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_right_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_right_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_semi_join_right_has_filter "select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a order by 1" + qt_left_semi_join_left_has_filter "select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_right_has_on_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_left_has_on_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_left_has_where_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t1.a order by 1" + qt_right_semi_join_right_has_filter "select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.c order by 1" + qt_right_semi_join_left_has_filter "select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_right_has_on_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_left_has_on_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_right_has_where_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t2.b=100 group by t2.b,t2.c order by 1" + qt_cross_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 cross join eli_gbk_by_uniform_t t2 where t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_cross_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 cross join eli_gbk_by_uniform_t t2 where t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + + //test union + qt_union "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union select a,b from eli_gbk_by_uniform_t where b=100 group by a,b union select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_union_all "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union all select a,b from eli_gbk_by_uniform_t where b=100 group by a,b union all select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_intersect "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b intersect select a,b from eli_gbk_by_uniform_t where b=100 group by a,b intersect select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_except "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b except select a,b from eli_gbk_by_uniform_t where b=100 group by a,b except select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_set_op_mixed "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union select a,b from eli_gbk_by_uniform_t where b=100 group by a,b except select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + + //test window + qt_window "select max(a) over(partition by a order by a) from eli_gbk_by_uniform_t where a=10 group by a,b order by 1" + //test partition topn + qt_partition_topn "select r from (select rank() over(partition by a order by a) r from eli_gbk_by_uniform_t where a=10 group by a,b) t where r<2 order by 1" + qt_partition_topn_qualifiy "select rank() over(partition by a order by a) r from eli_gbk_by_uniform_t where a=10 group by a,b qualify r<2 order by 1" + //test cte + qt_cte_producer "with t as (select a,b,count(*) from eli_gbk_by_uniform_t where a=1 group by a,b) select t1.a,t2.a,t2.b from t t1 inner join t t2 on t1.a=t2.a order by 1,2,3" + qt_cte_multi_producer "with t as (select a,b,count(*) from eli_gbk_by_uniform_t where a=1 group by a,b), tt as (select a,b,count(*) from eli_gbk_by_uniform_t where b=10 group by a,b) select t1.a,t2.a,t2.b from t t1 inner join tt t2 on t1.a=t2.a order by 1,2,3" + qt_cte_consumer "with t as (select * from eli_gbk_by_uniform_t) select t1.a,t2.b from t t1 inner join t t2 on t1.a=t2.a where t1.a=10 group by t1.a,t2.b order by 1,2 " + + //test filter + qt_filter "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b) t where a>0 order by 1,2" + + //test topn + qt_topn "select a,b from eli_gbk_by_uniform_t where a=1 group by a,b order by a limit 10 offset 0" + + //olap table sink + sql "insert into eli_gbk_by_uniform_t select a,b,c,d,dt from eli_gbk_by_uniform_t where a = 1 group by a,b,c,d,dt" + qt_sink "select * from eli_gbk_by_uniform_t order by 1,2,3,4,5" + + sql """ + drop table if exists orders_inner_1 + """ + + sql """CREATE TABLE `orders_inner_1` ( + `o_orderkey` BIGINT not NULL, + `o_custkey` INT NULL, + `o_orderstatus` VARCHAR(1) NULL, + `o_totalprice` DECIMAL(15, 2) NULL, + `o_orderpriority` VARCHAR(15) NULL, + `o_clerk` VARCHAR(15) NULL, + `o_shippriority` INT NULL, + `o_comment` VARCHAR(79) NULL, + `o_orderdate` DATE NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + PARTITION BY list(o_orderkey) ( + PARTITION p1 VALUES in ('1'), + PARTITION p2 VALUES in ('2'), + PARTITION p3 VALUES in ('3'), + PARTITION p4 VALUES in ('4') + ) + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_inner_1 + """ + + sql """CREATE TABLE `lineitem_inner_1` ( + `l_orderkey` BIGINT not NULL, + `l_linenumber` INT NULL, + `l_partkey` INT NULL, + `l_suppkey` INT NULL, + `l_quantity` DECIMAL(15, 2) NULL, + `l_extendedprice` DECIMAL(15, 2) NULL, + `l_discount` DECIMAL(15, 2) NULL, + `l_tax` DECIMAL(15, 2) NULL, + `l_returnflag` VARCHAR(1) NULL, + `l_linestatus` VARCHAR(1) NULL, + `l_commitdate` DATE NULL, + `l_receiptdate` DATE NULL, + `l_shipinstruct` VARCHAR(25) NULL, + `l_shipmode` VARCHAR(10) NULL, + `l_comment` VARCHAR(44) NULL, + `l_shipdate` DATE NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + PARTITION BY list(l_orderkey) ( + PARTITION p1 VALUES in ('1'), + PARTITION p2 VALUES in ('2'), + PARTITION p3 VALUES in ('3') + ) + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_inner_1 values + (2, 1, 'o', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (1, null, 'k', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'o', 99.5, 'a', null, 1, 'yy', '2023-10-22'), + (1, 3, 'k', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (4, 5, 'o', 99.5, 'a', 'b', 1, 'yy', '2023-10-19'); + """ + + sql """ + insert into lineitem_inner_1 values + (2, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (1, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (3, 3, null, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 2, 3, null, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (2, 3, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); + """ + + qt_nest_exprid_replace """ + select l_shipdate, l_orderkey, t.O_ORDERDATE, t.o_orderkey, + count(t.O_ORDERDATE) over (partition by lineitem_inner_1.l_orderkey order by lineitem_inner_1.l_orderkey) as window_count + from lineitem_inner_1 + inner join (select O_ORDERDATE, o_orderkey, count(O_ORDERDATE) over (partition by O_ORDERDATE order by o_orderkey ) from orders_inner_1 where o_orderkey=2 group by O_ORDERDATE, o_orderkey) as t + on lineitem_inner_1.l_orderkey = t.o_orderkey + where t.o_orderkey=2 + group by l_shipdate, l_orderkey, t.O_ORDERDATE, t.o_orderkey + order by 1,2,3,4,5 + """ + sql "drop table if exists test1" + sql "drop table if exists test2" + sql "create table test1(a int, b int) distributed by hash(a) properties('replication_num'='1');" + sql "insert into test1 values(1,1),(2,1),(3,1);" + sql "create table test2(a int, b int) distributed by hash(a) properties('replication_num'='1');" + sql "insert into test2 values(1,105),(2,105);" + qt_full_join_uniform_should_not_eliminate_group_by_key "select t2.b,t1.b from test1 t1 full join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_full2 "select t2.b,t1.b from (select * from test2 where b=105) t1 full join test1 t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + + qt_left_join_right_side_should_not_eliminate_group_by_key "select t2.b,t1.b from test1 t1 left join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_left_join_left_side_should_eliminate_group_by_key "select t2.b,t1.b from test1 t1 left join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t2.b,t1.b order by 1,2;" + + qt_right_join_left_side_should_not_eliminate_group_by_key "select t2.b,t1.b from (select * from test2 where b=105) t1 right join test1 t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_right_join_right_side_should_eliminate_group_by_key "select t2.b,t1.b from (select * from test2 where b=105) t1 right join test1 t2 on t1.a=t2.a where t2.b=1 group by t2.b,t1.b order by 1,2;" + + qt_left_semi_left_side "select t1.b from test1 t1 left semi join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t1.b,t1.a order by 1;" + qt_left_anti_left_side "select t1.b from test1 t1 left anti join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t1.b,t1.a order by 1;" + qt_right_semi_right_side "select t2.b from test1 t1 right semi join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t2.a order by 1;" + qt_right_anti_right_side "select t2.b from test1 t1 right anti join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t2.a order by 1;" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy index c43bde59acaa3d1..92f54f3dd456e84 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy @@ -373,7 +373,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + @@ -571,7 +571,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + @@ -661,7 +661,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + From 8514df82ca87716dd1e5b175817b6beac411fb60 Mon Sep 17 00:00:00 2001 From: Pxl Date: Mon, 2 Dec 2024 14:29:04 +0800 Subject: [PATCH 18/31] [Bug](runtime-filter) check rf dependency is set and fix wrong throw status (#44838) ### What problem does this PR solve? 1. check rf dependency is set 2. fix wrong throw status, introduced by https://github.com/apache/doris/issues/44697 3. catch throw exception on buildsink::close --- be/src/exprs/runtime_filter.h | 5 +++-- be/src/pipeline/exec/hashjoin_build_sink.cpp | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index a1fdfbf07d239dd..63ef3e2dbd46537 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -355,8 +355,9 @@ class IRuntimeFilter { const std::shared_ptr& dependency); int64_t get_synced_size() const { - if (_synced_size == -1) { - throw Status::InternalError("sync filter size meet error, filter: {}", debug_string()); + if (_synced_size == -1 || !_dependency) { + throw Exception(doris::ErrorCode::INTERNAL_ERROR, + "sync filter size meet error, filter: {}", debug_string()); } return _synced_size; } diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index cec0c77da8a61d9..864927e1721923e 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -156,7 +156,8 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu uint64_t hash_table_size = block ? block->rows() : 0; { SCOPED_TIMER(_runtime_filter_init_timer); - RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION( + _runtime_filter_slots->init_filters(state, hash_table_size)); RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); } if (hash_table_size > 1) { From fa13a700770349521c39ac88e75b5da7f313c4a9 Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Mon, 2 Dec 2024 14:49:30 +0800 Subject: [PATCH 19/31] [feat] (inverted index) show index file size (#44120) ### What problem does this PR solve? Problem Summary: 1. Show the data in detail, including the size of the inverted index file and the data file. ``` mysql > show data all; +------------------------+-----------+--------------+-------------+-----------------+-----------------+-----------------+-----------------+ | TableName | ReplicaCount | LocalTotalSize| LocalDataSize | LocalIndexSize | RemoteTotalSize| RemoteDataSize | RemoteIndexSize | +------------------------+-----------+--------------+-------------+-----------------+-----------------+-----------------+-----------------+ | test_show_index_data_p2 | 1 | 291.534 MB | 133.697 MB | 157.837 MB | 0.000 | 0.000 | 0.000 | | Total | 1 | 291.534 MB | 133.697 MB | 157.837 MB | 0.000 | 0.000 | 0.000 | | Quota | 1024.000 TB | 1073741824 | | | | | | | Left | 1024.000 TB | 1073741823 | | | | | | +------------------------+-----------+--------------+-------------+-----------------+-----------------+-----------------+-----------------+ 4 rows in set (0.00 sec) ``` ``` msql> show data all from test_show_index_data_p2; +------------------------+------------------------+-----------+--------------+----------+-------------+-----------------+-----------------+-----------------+-----------------+ | TableName | IndexName | ReplicaCount | RowCount| LocalTotalSize | LocalDataSize | LocalIndexSize | RemoteTotalSize | RemoteDataSize | RemoteIndexSize | +------------------------+------------------------+-----------+--------------+----------+-------------+-----------------+-----------------+-----------------+-----------------+ | test_show_index_data_p2 | test_show_index_data_p2 | 1 | 19697882 | 291.534 MB | 157.837 MB | 133.697 MN | 0.000 | 0.000 | 0.000 | | | Total | 1 | | 291.534 MB | 133.697 MB | 157.837 MB | 0.000 | 0.000 | 0.000 | +------------------------+------------------------+-----------+--------------+----------+-------------+-----------------+-----------------+-----------------+-----------------+ 2 rows in set (0.00 sec) ``` 2. It is possible to obtain the sizes of data and index files by querying the system tables ``` msyql > select * from information_schema.tables where TABLE_NAME = "test_show_index_data_p2"; +-------------------------+-------------------------------+---------------------+----------------+-------------+-------------+-------------+------------+------------+---------------------+---------------------+------------+---------+------------+---------------+----------------+ | TABLE_CATALOG | TABLE_SCHEMA | TABLE_NAME | TABLE_TYPE | ENGINE | VERSION | ROW_FORMAT | TABLE_ROWS | AVG_ROW_LENGTH | DATA_LENGTH | MAX_DATA_LENGTH | INDEX_LENGTH | DATA_FREE | AUTO_INCREMENT | CREATE_TIME | UPDATE_TIME | CHECK_TIME | TABLE_COLLATION | CHECKSUM | CREATE_OPTIONS | TABLE_COMMENT | +-------------------------+-------------------------------+---------------------+----------------+-------------+-------------+-------------+------------+--------------+-------------+----------------+--------------+-----------+---------------+---------------------+---------------------+------------+----------------+---------+---------------+---------------+ | internal | regression_test_inverted_index_p2_show_data | test_show_index_data_p2 | BASE TABLE | Doris | NULL | NULL | 19697882 | 15 | 140191631 | NULL | 165504277 | NULL | NULL | 2024-11-18 15:22:32 | 2024-11-18 15:24:52 | NULL | utf-8 | NULL | NULL | | +-------------------------+-------------------------------+---------------------+----------------+-------------+-------------+-------------+------------+--------------+-------------+----------------+--------------+-----------+---------------+---------------------+---------------------+------------+----------------+---------+---------------+---------------+ 1 row in set (0.02 sec) ``` ### Release note 1. Added `show data all;` to retrieve the detailed file size. 2. Fixed the semantics of `DATA_LENGTH` and `INDEX_LENGTH` in the system `table information_schema.tables`. --- be/src/cloud/cloud_base_compaction.cpp | 5 + be/src/cloud/cloud_cumulative_compaction.cpp | 4 + be/src/cloud/cloud_full_compaction.cpp | 4 + be/src/cloud/cloud_schema_change_job.cpp | 6 + .../schema_scanner/schema_tables_scanner.cpp | 16 +- be/src/olap/tablet.cpp | 4 + be/src/olap/tablet_manager.cpp | 4 + be/src/olap/tablet_meta.h | 46 ++ cloud/src/meta-service/keys.cpp | 8 + cloud/src/meta-service/keys.h | 6 + cloud/src/meta-service/meta_service.cpp | 2 + cloud/src/meta-service/meta_service_job.cpp | 22 +- .../meta_service_tablet_stats.cpp | 111 +++- .../meta-service/meta_service_tablet_stats.h | 2 + cloud/src/meta-service/meta_service_txn.cpp | 26 +- cloud/src/meta-service/txn_lazy_committer.cpp | 4 +- cloud/test/http_encode_key_test.cpp | 32 +- cloud/test/keys_test.cpp | 34 ++ cloud/test/meta_service_http_test.cpp | 14 + cloud/test/meta_service_job_test.cpp | 60 ++- cloud/test/meta_service_test.cpp | 24 + cloud/test/schema_kv_test.cpp | 6 + .../org/apache/doris/nereids/DorisParser.g4 | 3 +- fe/fe-core/src/main/cup/sql_parser.cup | 21 +- .../apache/doris/analysis/ShowDataStmt.java | 484 +++++++++++------- .../doris/catalog/CloudTabletStatMgr.java | 20 +- .../doris/catalog/MaterializedIndex.java | 40 ++ .../org/apache/doris/catalog/OlapTable.java | 55 +- .../org/apache/doris/catalog/Replica.java | 19 + .../java/org/apache/doris/catalog/Table.java | 3 + .../org/apache/doris/catalog/TableIf.java | 2 + .../apache/doris/catalog/TabletStatMgr.java | 17 +- .../doris/datasource/ExternalTable.java | 5 + .../doris/service/FrontendServiceImpl.java | 1 + .../doris/analysis/ShowDataStmtTest.java | 6 +- gensrc/proto/cloud.proto | 10 +- gensrc/thrift/BackendService.thrift | 7 +- gensrc/thrift/FrontendService.thrift | 1 + gensrc/thrift/MasterService.thrift | 4 + .../system/test_query_sys_tables.out | 6 +- .../query_p0/system/test_query_sys_tables.out | 6 +- .../show_data/test_show_index_data.groovy | 399 +++++++++++++++ .../show_data/test_show_index_data_p2.groovy | 397 ++++++++++++++ 43 files changed, 1702 insertions(+), 244 deletions(-) create mode 100644 regression-test/suites/inverted_index_p1/show_data/test_show_index_data.groovy create mode 100644 regression-test/suites/inverted_index_p2/show_data/test_show_index_data_p2.groovy diff --git a/be/src/cloud/cloud_base_compaction.cpp b/be/src/cloud/cloud_base_compaction.cpp index 88d83000e95dfaa..9742e57dcf9d34d 100644 --- a/be/src/cloud/cloud_base_compaction.cpp +++ b/be/src/cloud/cloud_base_compaction.cpp @@ -125,6 +125,7 @@ Status CloudBaseCompaction::prepare_compact() { _input_row_num += rs->num_rows(); _input_segments += rs->num_segments(); _input_rowsets_data_size += rs->data_disk_size(); + _input_rowsets_index_size += rs->index_disk_size(); _input_rowsets_total_size += rs->total_disk_size(); } LOG_INFO("start CloudBaseCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), @@ -320,6 +321,10 @@ Status CloudBaseCompaction::modify_rowsets() { compaction_job->add_output_versions(_output_rowset->end_version()); compaction_job->add_txn_id(_output_rowset->txn_id()); compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + compaction_job->set_index_size_input_rowsets(_input_rowsets_index_size); + compaction_job->set_segment_size_input_rowsets(_input_rowsets_data_size); + compaction_job->set_index_size_output_rowsets(_output_rowset->index_disk_size()); + compaction_job->set_segment_size_output_rowsets(_output_rowset->data_disk_size()); DeleteBitmapPtr output_rowset_delete_bitmap = nullptr; if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp index 2f08082f51b5f32..c466c35e2a2ab69 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -263,6 +263,10 @@ Status CloudCumulativeCompaction::modify_rowsets() { compaction_job->add_output_versions(_output_rowset->end_version()); compaction_job->add_txn_id(_output_rowset->txn_id()); compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + compaction_job->set_index_size_input_rowsets(_input_rowsets_index_size); + compaction_job->set_segment_size_input_rowsets(_input_rowsets_data_size); + compaction_job->set_index_size_output_rowsets(_output_rowset->index_disk_size()); + compaction_job->set_segment_size_output_rowsets(_output_rowset->data_disk_size()); DBUG_EXECUTE_IF("CloudCumulativeCompaction::modify_rowsets.enable_spin_wait", { LOG(INFO) << "CloudCumulativeCompaction::modify_rowsets.enable_spin_wait, start"; diff --git a/be/src/cloud/cloud_full_compaction.cpp b/be/src/cloud/cloud_full_compaction.cpp index c27b728c93d29b1..bce00c9a2e74f67 100644 --- a/be/src/cloud/cloud_full_compaction.cpp +++ b/be/src/cloud/cloud_full_compaction.cpp @@ -216,6 +216,10 @@ Status CloudFullCompaction::modify_rowsets() { compaction_job->add_output_versions(_output_rowset->end_version()); compaction_job->add_txn_id(_output_rowset->txn_id()); compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + compaction_job->set_index_size_input_rowsets(_input_rowsets_index_size); + compaction_job->set_segment_size_input_rowsets(_input_rowsets_data_size); + compaction_job->set_index_size_output_rowsets(_output_rowset->index_disk_size()); + compaction_job->set_segment_size_output_rowsets(_output_rowset->data_disk_size()); DeleteBitmapPtr output_rowset_delete_bitmap = nullptr; if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && diff --git a/be/src/cloud/cloud_schema_change_job.cpp b/be/src/cloud/cloud_schema_change_job.cpp index 0bab742c3ad6e5a..1cc4d052a81d692 100644 --- a/be/src/cloud/cloud_schema_change_job.cpp +++ b/be/src/cloud/cloud_schema_change_job.cpp @@ -340,17 +340,23 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam int64_t num_output_rows = 0; int64_t size_output_rowsets = 0; int64_t num_output_segments = 0; + int64_t index_size_output_rowsets = 0; + int64_t segment_size_output_rowsets = 0; for (auto& rs : _output_rowsets) { sc_job->add_txn_ids(rs->txn_id()); sc_job->add_output_versions(rs->end_version()); num_output_rows += rs->num_rows(); size_output_rowsets += rs->total_disk_size(); num_output_segments += rs->num_segments(); + index_size_output_rowsets += rs->index_disk_size(); + segment_size_output_rowsets += rs->data_disk_size(); } sc_job->set_num_output_rows(num_output_rows); sc_job->set_size_output_rowsets(size_output_rowsets); sc_job->set_num_output_segments(num_output_segments); sc_job->set_num_output_rowsets(_output_rowsets.size()); + sc_job->set_index_size_output_rowsets(index_size_output_rowsets); + sc_job->set_segment_size_output_rowsets(segment_size_output_rowsets); } _output_cumulative_point = std::min(_output_cumulative_point, sc_job->alter_version() + 1); sc_job->set_output_cumulative_point(_output_cumulative_point); diff --git a/be/src/exec/schema_scanner/schema_tables_scanner.cpp b/be/src/exec/schema_scanner/schema_tables_scanner.cpp index 23710b81971c151..3aba0dfcc4f83c0 100644 --- a/be/src/exec/schema_scanner/schema_tables_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_tables_scanner.cpp @@ -236,7 +236,7 @@ Status SchemaTablesScanner::_fill_block_impl(vectorized::Block* block) { std::vector srcs(table_num); for (int i = 0; i < table_num; ++i) { const TTableStatus& tbl_status = _table_result.tables[i]; - if (tbl_status.__isset.avg_row_length) { + if (tbl_status.__isset.data_length) { srcs[i] = tbl_status.data_length; datas[i] = srcs.data() + i; } else { @@ -248,7 +248,19 @@ Status SchemaTablesScanner::_fill_block_impl(vectorized::Block* block) { // max_data_length { RETURN_IF_ERROR(fill_dest_column_for_range(block, 10, null_datas)); } // index_length - { RETURN_IF_ERROR(fill_dest_column_for_range(block, 11, null_datas)); } + { + std::vector srcs(table_num); + for (int i = 0; i < table_num; ++i) { + const TTableStatus& tbl_status = _table_result.tables[i]; + if (tbl_status.__isset.index_length) { + srcs[i] = tbl_status.index_length; + datas[i] = srcs.data() + i; + } else { + datas[i] = nullptr; + } + } + RETURN_IF_ERROR(fill_dest_column_for_range(block, 11, datas)); + } // data_free { RETURN_IF_ERROR(fill_dest_column_for_range(block, 12, null_datas)); } // auto_increment diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 0d04984d0e06bae..379fb6eec3cb43b 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -1692,6 +1692,10 @@ void Tablet::build_tablet_report_info(TTabletInfo* tablet_info, // tablet may not have cooldowned data, but the storage policy is set tablet_info->__set_cooldown_term(_cooldown_conf.term); } + tablet_info->__set_local_index_size(_tablet_meta->tablet_local_index_size()); + tablet_info->__set_local_segment_size(_tablet_meta->tablet_local_segment_size()); + tablet_info->__set_remote_index_size(_tablet_meta->tablet_remote_index_size()); + tablet_info->__set_remote_segment_size(_tablet_meta->tablet_remote_segment_size()); } void Tablet::report_error(const Status& st) { diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index d6a944dbc398539..a18dc0b4a161b69 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -1083,6 +1083,10 @@ void TabletManager::build_all_report_tablets_info(std::map* t_tablet_stat.__set_total_version_count(tablet_info.total_version_count); t_tablet_stat.__set_visible_version_count(tablet_info.visible_version_count); t_tablet_stat.__set_visible_version(tablet_info.version); + t_tablet_stat.__set_local_index_size(tablet_info.local_index_size); + t_tablet_stat.__set_local_segment_size(tablet_info.local_segment_size); + t_tablet_stat.__set_remote_index_size(tablet_info.remote_index_size); + t_tablet_stat.__set_remote_segment_size(tablet_info.remote_segment_size); }; for_each_tablet(handler, filter_all_tablets); diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index fb0895604a19fe0..34794ef6d0a7607 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -170,6 +170,12 @@ class TabletMeta : public MetadataAdder { size_t tablet_local_size() const; // Remote disk space occupied by tablet. size_t tablet_remote_size() const; + + size_t tablet_local_index_size() const; + size_t tablet_local_segment_size() const; + size_t tablet_remote_index_size() const; + size_t tablet_remote_segment_size() const; + size_t version_count() const; size_t stale_version_count() const; size_t version_count_cross_with_range(const Version& range) const; @@ -668,6 +674,46 @@ inline size_t TabletMeta::tablet_remote_size() const { return total_size; } +inline size_t TabletMeta::tablet_local_index_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (rs->is_local()) { + total_size += rs->index_disk_size(); + } + } + return total_size; +} + +inline size_t TabletMeta::tablet_local_segment_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (rs->is_local()) { + total_size += rs->data_disk_size(); + } + } + return total_size; +} + +inline size_t TabletMeta::tablet_remote_index_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (!rs->is_local()) { + total_size += rs->index_disk_size(); + } + } + return total_size; +} + +inline size_t TabletMeta::tablet_remote_segment_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (!rs->is_local()) { + total_size += rs->data_disk_size(); + } + } + return total_size; +} + inline size_t TabletMeta::version_count() const { return _rs_metas.size(); } diff --git a/cloud/src/meta-service/keys.cpp b/cloud/src/meta-service/keys.cpp index 820d349084e35fd..a518b6e264d20d3 100644 --- a/cloud/src/meta-service/keys.cpp +++ b/cloud/src/meta-service/keys.cpp @@ -382,6 +382,14 @@ void stats_tablet_num_segs_key(const StatsTabletKeyInfo& in, std::string* out) { stats_tablet_key(in, out); encode_bytes(STATS_KEY_SUFFIX_NUM_SEGS, out); } +void stats_tablet_index_size_key(const StatsTabletKeyInfo& in, std::string* out) { + stats_tablet_key(in, out); + encode_bytes(STATS_KEY_SUFFIX_INDEX_SIZE, out); +} +void stats_tablet_segment_size_key(const StatsTabletKeyInfo& in, std::string* out) { + stats_tablet_key(in, out); + encode_bytes(STATS_KEY_SUFFIX_SEGMENT_SIZE, out); +} //============================================================================== // Job keys diff --git a/cloud/src/meta-service/keys.h b/cloud/src/meta-service/keys.h index c63af925b8f4de6..855171c1dc648ad 100644 --- a/cloud/src/meta-service/keys.h +++ b/cloud/src/meta-service/keys.h @@ -51,6 +51,8 @@ // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_rows" -> int64 // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_rowsets" -> int64 // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_segs" -> int64 +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "index_size" -> int64 +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "segment_size"-> int64 // // 0x01 "recycle" ${instance_id} "index" ${index_id} -> RecycleIndexPB // 0x01 "recycle" ${instance_id} "partition" ${partition_id} -> RecyclePartitionPB @@ -83,6 +85,8 @@ static constexpr std::string_view STATS_KEY_SUFFIX_DATA_SIZE = "data_size"; static constexpr std::string_view STATS_KEY_SUFFIX_NUM_ROWS = "num_rows"; static constexpr std::string_view STATS_KEY_SUFFIX_NUM_ROWSETS = "num_rowsets"; static constexpr std::string_view STATS_KEY_SUFFIX_NUM_SEGS = "num_segs"; +static constexpr std::string_view STATS_KEY_SUFFIX_INDEX_SIZE = "index_size"; +static constexpr std::string_view STATS_KEY_SUFFIX_SEGMENT_SIZE = "segment_size"; // clang-format off /** @@ -247,6 +251,8 @@ void stats_tablet_data_size_key(const StatsTabletKeyInfo& in, std::string* out); void stats_tablet_num_rows_key(const StatsTabletKeyInfo& in, std::string* out); void stats_tablet_num_rowsets_key(const StatsTabletKeyInfo& in, std::string* out); void stats_tablet_num_segs_key(const StatsTabletKeyInfo& in, std::string* out); +void stats_tablet_index_size_key(const StatsTabletKeyInfo& in, std::string* out); +void stats_tablet_segment_size_key(const StatsTabletKeyInfo& in, std::string* out); static inline std::string stats_tablet_key(const StatsTabletKeyInfo& in) { std::string s; stats_tablet_key(in, &s); return s; } void job_recycle_key(const JobRecycleKeyInfo& in, std::string* out); diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 974b3091a73e95d..ddf4ee3889c00eb 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -1642,6 +1642,8 @@ void MetaServiceImpl::get_tablet_stats(::google::protobuf::RpcController* contro #ifdef NDEBUG // Force data size >= 0 to reduce the losses caused by bugs if (tablet_stats->data_size() < 0) tablet_stats->set_data_size(0); + if (tablet_stats->index_size() < 0) tablet_stats->set_index_size(0); + if (tablet_stats->segment_size() < 0) tablet_stats->set_segment_size(0); #endif } } diff --git a/cloud/src/meta-service/meta_service_job.cpp b/cloud/src/meta-service/meta_service_job.cpp index d1c8df15870de70..cc80d5bd138980e 100644 --- a/cloud/src/meta-service/meta_service_job.cpp +++ b/cloud/src/meta-service/meta_service_job.cpp @@ -701,6 +701,8 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_index_size(stats->index_size() + (compaction.index_size_output_rowsets() - compaction.index_size_input_rowsets())); + stats->set_segment_size(stats->segment_size() + (compaction.segment_size_output_rowsets() - compaction.segment_size_input_rowsets())); stats->set_last_cumu_compaction_time_ms(now * 1000); // clang-format on } else if (compaction.type() == TabletCompactionJobPB::BASE) { @@ -710,6 +712,8 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_index_size(stats->index_size() + (compaction.index_size_output_rowsets() - compaction.index_size_input_rowsets())); + stats->set_segment_size(stats->segment_size() + (compaction.segment_size_output_rowsets() - compaction.segment_size_input_rowsets())); stats->set_last_base_compaction_time_ms(now * 1000); // clang-format on } else if (compaction.type() == TabletCompactionJobPB::FULL) { @@ -724,6 +728,8 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_index_size(stats->index_size() + (compaction.index_size_output_rowsets() - compaction.index_size_input_rowsets())); + stats->set_segment_size(stats->segment_size() + (compaction.segment_size_output_rowsets() - compaction.segment_size_input_rowsets())); stats->set_last_full_compaction_time_ms(now * 1000); // clang-format on } else { @@ -738,10 +744,14 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string << " stats.data_size=" << stats->data_size() << " stats.num_rowsets=" << stats->num_rowsets() << " stats.num_segments=" << stats->num_segments() + << " stats.index_size=" << stats->index_size() + << " stats.segment_size=" << stats->segment_size() << " detached_stats.num_rows=" << detached_stats.num_rows << " detached_stats.data_size=" << detached_stats.data_size << " detached_stats.num_rowset=" << detached_stats.num_rowsets << " detached_stats.num_segments=" << detached_stats.num_segs + << " detached_stats.index_size=" << detached_stats.index_size + << " detached_stats.segment_size=" << detached_stats.segment_size << " compaction.size_output_rowsets=" << compaction.size_output_rowsets() << " compaction.size_input_rowsets=" << compaction.size_input_rowsets(); txn->put(stats_key, stats_val); @@ -752,10 +762,14 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string << " stats.data_size=" << stats->data_size() << " stats.num_rowsets=" << stats->num_rowsets() << " stats.num_segments=" << stats->num_segments() + << " stats.index_size=" << stats->index_size() + << " stats.segment_size=" << stats->segment_size() << " detached_stats.num_rows=" << detached_stats.num_rows << " detached_stats.data_size=" << detached_stats.data_size << " detached_stats.num_rowset=" << detached_stats.num_rowsets << " detached_stats.num_segments=" << detached_stats.num_segs + << " detached_stats.index_size=" << detached_stats.index_size + << " detached_stats.segment_size=" << detached_stats.segment_size << " compaction.size_output_rowsets=" << compaction.size_output_rowsets() << " compaction.size_input_rowsets=" << compaction.size_input_rowsets(); @@ -1133,6 +1147,8 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str int64_t size_remove_rowsets = 0; int64_t num_remove_rowsets = 0; int64_t num_remove_segments = 0; + int64_t index_size_remove_rowsets = 0; + int64_t segment_size_remove_rowsets = 0; auto rs_start = meta_rowset_key({instance_id, new_tablet_id, 2}); auto rs_end = meta_rowset_key({instance_id, new_tablet_id, schema_change.alter_version() + 1}); @@ -1162,9 +1178,11 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str } num_remove_rows += rs.num_rows(); - size_remove_rowsets += rs.data_disk_size(); + size_remove_rowsets += rs.total_disk_size(); ++num_remove_rowsets; num_remove_segments += rs.num_segments(); + index_size_remove_rowsets += rs.index_disk_size(); + segment_size_remove_rowsets += rs.data_disk_size(); auto recycle_key = recycle_rowset_key({instance_id, new_tablet_id, rs.rowset_id_v2()}); RecycleRowsetPB recycle_rowset; @@ -1199,6 +1217,8 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str stats->set_data_size(stats->data_size() + (schema_change.size_output_rowsets() - size_remove_rowsets)); stats->set_num_rowsets(stats->num_rowsets() + (schema_change.num_output_rowsets() - num_remove_rowsets)); stats->set_num_segments(stats->num_segments() + (schema_change.num_output_segments() - num_remove_segments)); + stats->set_index_size(stats->index_size() + (schema_change.index_size_output_rowsets() - index_size_remove_rowsets)); + stats->set_segment_size(stats->segment_size() + (schema_change.segment_size_output_rowsets() - segment_size_remove_rowsets)); // clang-format on auto stats_key = stats_tablet_key( {instance_id, new_table_id, new_index_id, new_partition_id, new_tablet_id}); diff --git a/cloud/src/meta-service/meta_service_tablet_stats.cpp b/cloud/src/meta-service/meta_service_tablet_stats.cpp index cecccbd67673ad1..4cbf629c9a31854 100644 --- a/cloud/src/meta-service/meta_service_tablet_stats.cpp +++ b/cloud/src/meta-service/meta_service_tablet_stats.cpp @@ -45,7 +45,8 @@ void internal_get_tablet_stats(MetaServiceCode& code, std::string& msg, Transact auto end_key = stats_tablet_key({instance_id, idx.table_id(), idx.index_id(), idx.partition_id(), idx.tablet_id() + 1}); // clang-format on std::vector> stats_kvs; - stats_kvs.reserve(5); // aggregate + data_size + num_rows + num_rowsets + num_segments + stats_kvs.reserve( + 7); // aggregate + data_size + num_rows + num_rowsets + num_segments + index_size + segment_size std::unique_ptr it; do { @@ -93,7 +94,7 @@ int get_detached_tablet_stats(const std::vector int64 + std::string tablet_stat_index_size_key; + stats_tablet_index_size_key( + {instance_id, tablet_stat.idx().table_id(), tablet_stat.idx().index_id(), + tablet_stat.idx().partition_id(), tablet_stat.idx().tablet_id()}, + &tablet_stat_index_size_key); + int64_t tablet_stat_index_size = 0; + std::string tablet_stat_index_size_value(sizeof(tablet_stat_index_size), '\0'); + memcpy(tablet_stat_index_size_value.data(), &tablet_stat_index_size, + sizeof(tablet_stat_index_size)); + txn->put(tablet_stat_index_size_key, tablet_stat_index_size_value); + // set tablet stats segment size = 0 + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "segment_size" -> int64 + std::string tablet_stat_segment_size_key; + stats_tablet_segment_size_key( + {instance_id, tablet_stat.idx().table_id(), tablet_stat.idx().index_id(), + tablet_stat.idx().partition_id(), tablet_stat.idx().tablet_id()}, + &tablet_stat_segment_size_key); + int64_t tablet_stat_segment_size = 0; + std::string tablet_stat_segment_size_value(sizeof(tablet_stat_segment_size), '\0'); + memcpy(tablet_stat_segment_size_value.data(), &tablet_stat_segment_size, + sizeof(tablet_stat_segment_size)); + txn->put(tablet_stat_segment_size_key, tablet_stat_segment_size_value); } } @@ -396,6 +433,76 @@ MetaServiceResponseStatus check_new_tablet_stats( .tag("data size", tablet_stat_data_size) .tag("check data size", tablet_stat_data_size_check); } + + // check index size + std::string tablet_stat_index_size_key; + stats_tablet_index_size_key( + {instance_id, tablet_stat_ptr->idx().table_id(), tablet_stat_ptr->idx().index_id(), + tablet_stat_ptr->idx().partition_id(), tablet_stat_ptr->idx().tablet_id()}, + &tablet_stat_index_size_key); + int64_t tablet_stat_index_size = 0; + std::string tablet_stat_index_size_value(sizeof(tablet_stat_index_size), '\0'); + err = txn->get(tablet_stat_index_size_key, &tablet_stat_index_size_value); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + st.set_code(cast_as(err)); + return st; + } + int64_t tablet_stat_index_size_check; + + if (tablet_stat_index_size_value.size() != sizeof(tablet_stat_index_size_check)) + [[unlikely]] { + LOG(WARNING) << " malformed tablet stats value v.index_size=" + << tablet_stat_index_size_value.size() + << " value=" << hex(tablet_stat_index_size_value); + } + std::memcpy(&tablet_stat_index_size_check, tablet_stat_index_size_value.data(), + sizeof(tablet_stat_index_size_check)); + if constexpr (std::endian::native == std::endian::big) { + tablet_stat_index_size_check = bswap_64(tablet_stat_index_size_check); + } + if (tablet_stat_index_size_check != tablet_stat_index_size && + // ditto + (tablet_stat_index_size_check > 2 * tablet_stat_index_size || + tablet_stat_index_size > 2 * tablet_stat_index_size_check)) { + LOG_WARNING("[fix tablet stats]:index size check failed") + .tag("index size", tablet_stat_index_size) + .tag("check index size", tablet_stat_index_size_check); + } + + // check data size + std::string tablet_stat_segment_size_key; + stats_tablet_segment_size_key( + {instance_id, tablet_stat_ptr->idx().table_id(), tablet_stat_ptr->idx().index_id(), + tablet_stat_ptr->idx().partition_id(), tablet_stat_ptr->idx().tablet_id()}, + &tablet_stat_segment_size_key); + int64_t tablet_stat_segment_size = 0; + std::string tablet_stat_segment_size_value(sizeof(tablet_stat_segment_size), '\0'); + err = txn->get(tablet_stat_segment_size_key, &tablet_stat_segment_size_value); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + st.set_code(cast_as(err)); + return st; + } + int64_t tablet_stat_segment_size_check; + + if (tablet_stat_segment_size_value.size() != sizeof(tablet_stat_segment_size_check)) + [[unlikely]] { + LOG(WARNING) << " malformed tablet stats value v.segment_size=" + << tablet_stat_segment_size_value.size() + << " value=" << hex(tablet_stat_segment_size_value); + } + std::memcpy(&tablet_stat_segment_size_check, tablet_stat_segment_size_value.data(), + sizeof(tablet_stat_segment_size_check)); + if constexpr (std::endian::native == std::endian::big) { + tablet_stat_segment_size_check = bswap_64(tablet_stat_segment_size_check); + } + if (tablet_stat_segment_size_check != tablet_stat_segment_size && + // ditto + (tablet_stat_segment_size_check > 2 * tablet_stat_segment_size || + tablet_stat_segment_size > 2 * tablet_stat_segment_size_check)) { + LOG_WARNING("[fix tablet stats]:segment size check failed") + .tag("segment size", tablet_stat_segment_size) + .tag("check segment size", tablet_stat_segment_size_check); + } } return st; diff --git a/cloud/src/meta-service/meta_service_tablet_stats.h b/cloud/src/meta-service/meta_service_tablet_stats.h index a7aea5885a8e1a7..30cbaf56f6dc800 100644 --- a/cloud/src/meta-service/meta_service_tablet_stats.h +++ b/cloud/src/meta-service/meta_service_tablet_stats.h @@ -31,6 +31,8 @@ struct TabletStats { int64_t num_rows = 0; int64_t num_rowsets = 0; int64_t num_segs = 0; + int64_t index_size = 0; + int64_t segment_size = 0; }; // Get tablet stats and detached tablet stats via `txn`. If an error occurs, `code` will be set to non OK. diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 58930f6edfc63ba..5d696220b72dabf 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -880,6 +880,12 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat std::string num_segs_key; stats_tablet_num_segs_key(info, &num_segs_key); txn->atomic_add(num_segs_key, stats.num_segs); + std::string index_size_key; + stats_tablet_index_size_key(info, &index_size_key); + txn->atomic_add(index_size_key, stats.index_size); + std::string segment_size_key; + stats_tablet_segment_size_key(info, &segment_size_key); + txn->atomic_add(segment_size_key, stats.segment_size); } std::string num_rowsets_key; stats_tablet_num_rowsets_key(info, &num_rowsets_key); @@ -906,6 +912,8 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat stats_pb.set_num_rows(stats_pb.num_rows() + stats.num_rows); stats_pb.set_num_rowsets(stats_pb.num_rowsets() + stats.num_rowsets); stats_pb.set_num_segments(stats_pb.num_segments() + stats.num_segs); + stats_pb.set_index_size(stats_pb.index_size() + stats.index_size); + stats_pb.set_segment_size(stats_pb.segment_size() + stats.segment_size); stats_pb.SerializeToString(&val); txn->put(key, val); LOG(INFO) << "put stats_tablet_key key=" << hex(key); @@ -1234,6 +1242,8 @@ void commit_txn_immediately( stats.num_rows += i.num_rows(); ++stats.num_rowsets; stats.num_segs += i.num_segments(); + stats.index_size += i.index_disk_size(); + stats.segment_size += i.data_disk_size(); } // for tmp_rowsets_meta process_mow_when_commit_txn(request, instance_id, code, msg, txn, table_id_tablet_ids); @@ -1864,10 +1874,12 @@ void commit_txn_eventually( for (auto& [_, i] : tmp_rowsets_meta) { // Accumulate affected rows auto& stats = tablet_stats[i.tablet_id()]; - stats.data_size += i.data_disk_size(); + stats.data_size += i.total_disk_size(); stats.num_rows += i.num_rows(); ++stats.num_rowsets; stats.num_segs += i.num_segments(); + stats.index_size += i.index_disk_size(); + stats.segment_size += i.data_disk_size(); } // calculate table stats from tablets stats @@ -2238,10 +2250,12 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* // Accumulate affected rows auto& stats = tablet_stats[tablet_id]; - stats.data_size += i.data_disk_size(); + stats.data_size += i.total_disk_size(); stats.num_rows += i.num_rows(); ++stats.num_rowsets; stats.num_segs += i.num_segments(); + stats.index_size += i.index_disk_size(); + stats.segment_size += i.data_disk_size(); } // for tmp_rowsets_meta } @@ -2351,6 +2365,12 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* auto& num_segs_key = kv_pool.emplace_back(); stats_tablet_num_segs_key(info, &num_segs_key); txn->atomic_add(num_segs_key, stats.num_segs); + auto& index_size_key = kv_pool.emplace_back(); + stats_tablet_index_size_key(info, &index_size_key); + txn->atomic_add(index_size_key, stats.index_size); + auto& segment_size_key = kv_pool.emplace_back(); + stats_tablet_segment_size_key(info, &segment_size_key); + txn->atomic_add(segment_size_key, stats.segment_size); } auto& num_rowsets_key = kv_pool.emplace_back(); stats_tablet_num_rowsets_key(info, &num_rowsets_key); @@ -2379,6 +2399,8 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* stats_pb.set_num_rows(stats_pb.num_rows() + stats.num_rows); stats_pb.set_num_rowsets(stats_pb.num_rowsets() + stats.num_rowsets); stats_pb.set_num_segments(stats_pb.num_segments() + stats.num_segs); + stats_pb.set_index_size(stats_pb.index_size() + stats.index_size); + stats_pb.set_segment_size(stats_pb.segment_size() + stats.segment_size); stats_pb.SerializeToString(&val); txn->put(key, val); LOG(INFO) << "put stats_tablet_key, key=" << hex(key); diff --git a/cloud/src/meta-service/txn_lazy_committer.cpp b/cloud/src/meta-service/txn_lazy_committer.cpp index 25d36faab06796b..9859c2b0ed1ae3a 100644 --- a/cloud/src/meta-service/txn_lazy_committer.cpp +++ b/cloud/src/meta-service/txn_lazy_committer.cpp @@ -174,10 +174,12 @@ void convert_tmp_rowsets( // Accumulate affected rows auto& stats = tablet_stats[tmp_rowset_pb.tablet_id()]; - stats.data_size += tmp_rowset_pb.data_disk_size(); + stats.data_size += tmp_rowset_pb.total_disk_size(); stats.num_rows += tmp_rowset_pb.num_rows(); ++stats.num_rowsets; stats.num_segs += tmp_rowset_pb.num_segments(); + stats.index_size += tmp_rowset_pb.index_disk_size(); + stats.segment_size += tmp_rowset_pb.data_disk_size(); } for (auto& [tablet_id, stats] : tablet_stats) { diff --git a/cloud/test/http_encode_key_test.cpp b/cloud/test/http_encode_key_test.cpp index 77f800846076a95..d245682474349de 100644 --- a/cloud/test/http_encode_key_test.cpp +++ b/cloud/test/http_encode_key_test.cpp @@ -310,7 +310,7 @@ txn_id=126419752960)", }, R"({"creation_time":"12345","label":"label_1"})", }, - Input { // aggregated_stats + full detached_stats, there are 5 KVs in total + Input { // aggregated_stats + full detached_stats, there are 7 KVs in total "StatsTabletKey", "instance_id=gavin-instance&table_id=10086&index_id=10010&part_id=10000&tablet_id=1008601", { @@ -327,15 +327,17 @@ txn_id=126419752960)", idx->set_index_id(100010); idx->set_partition_id(10000); idx->set_tablet_id(1008601); - pb.set_data_size(1); + pb.set_data_size(2); pb.set_num_rows(10); pb.set_num_rowsets(11); pb.set_num_segments(12); + pb.set_index_size(1); + pb.set_segment_size(1); return {pb.SerializeAsString(), {"\x01\x00\x00\x00\x00\x00\x00\x00",8}, {"\x02\x00\x00\x00\x00\x00\x00\x00",8}, {"\x03\x00\x00\x00\x00\x00\x00\x00",8}, {"\x04\x00\x00\x00\x00\x00\x00\x00",8}}; }, - R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"data_size":"1","num_rows":"10","num_rowsets":"11","num_segments":"12"} -detached_stats: {"data_size":"1","num_rows":"2","num_rowsets":"3","num_segments":"4"} -merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"data_size":"2","num_rows":"12","num_rowsets":"14","num_segments":"16"} + R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"data_size":"2","num_rows":"10","num_rowsets":"11","num_segments":"12","index_size":"1","segment_size":"1"} +detached_stats: {"data_size":"1","num_rows":"2","num_rowsets":"3","num_segments":"4","index_size":"0","segment_size":"0"} +merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"data_size":"3","num_rows":"12","num_rowsets":"14","num_segments":"16","index_size":"1","segment_size":"1"} )", }, Input { // aggregated_stats + half detached_stats (num_segs == 0, there is num_rowsets detached stats) @@ -352,15 +354,17 @@ merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"100 idx->set_index_id(100010); idx->set_partition_id(10000); idx->set_tablet_id(1008602); - pb.set_data_size(1); + pb.set_data_size(2); pb.set_num_rows(10); pb.set_num_rowsets(11); pb.set_num_segments(12); + pb.set_index_size(1); + pb.set_segment_size(1); return {pb.SerializeAsString(), {"\x03\x00\x00\x00\x00\x00\x00\x00",8}}; }, - R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"1","num_rows":"10","num_rowsets":"11","num_segments":"12"} -detached_stats: {"data_size":"0","num_rows":"0","num_rowsets":"3","num_segments":"0"} -merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"1","num_rows":"10","num_rowsets":"14","num_segments":"12"} + R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"2","num_rows":"10","num_rowsets":"11","num_segments":"12","index_size":"1","segment_size":"1"} +detached_stats: {"data_size":"0","num_rows":"0","num_rowsets":"3","num_segments":"0","index_size":"0","segment_size":"0"} +merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"2","num_rows":"10","num_rowsets":"14","num_segments":"12","index_size":"1","segment_size":"1"} )", }, Input { // aggregated_stats only, the legacy @@ -376,15 +380,17 @@ merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"100 idx->set_index_id(100010); idx->set_partition_id(10000); idx->set_tablet_id(1008602); - pb.set_data_size(1); + pb.set_data_size(2); pb.set_num_rows(10); pb.set_num_rowsets(11); pb.set_num_segments(12); + pb.set_index_size(1); + pb.set_segment_size(1); return {pb.SerializeAsString()}; }, - R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"1","num_rows":"10","num_rowsets":"11","num_segments":"12"} -detached_stats: {"data_size":"0","num_rows":"0","num_rowsets":"0","num_segments":"0"} -merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"1","num_rows":"10","num_rowsets":"11","num_segments":"12"} + R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"2","num_rows":"10","num_rowsets":"11","num_segments":"12","index_size":"1","segment_size":"1"} +detached_stats: {"data_size":"0","num_rows":"0","num_rowsets":"0","num_segments":"0","index_size":"0","segment_size":"0"} +merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"2","num_rows":"10","num_rowsets":"11","num_segments":"12","index_size":"1","segment_size":"1"} )", }, Input { diff --git a/cloud/test/keys_test.cpp b/cloud/test/keys_test.cpp index a92a685cda45f82..52d840e1f5ce6b2 100644 --- a/cloud/test/keys_test.cpp +++ b/cloud/test/keys_test.cpp @@ -803,6 +803,40 @@ TEST(KeysTest, StatsKeysTest) { EXPECT_EQ("num_segs", dec_stats_suffix); } + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "index_size" -> int64 + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_index_size_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string dec_stats_suffix; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("index_size", dec_stats_suffix); + } + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "segment_size" -> int64 + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_segment_size_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string dec_stats_suffix; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("segment_size", dec_stats_suffix); + } } TEST(KeysTest, JobKeysTest) { diff --git a/cloud/test/meta_service_http_test.cpp b/cloud/test/meta_service_http_test.cpp index 4360efeb4422a9c..81c322303a5a8c8 100644 --- a/cloud/test/meta_service_http_test.cpp +++ b/cloud/test/meta_service_http_test.cpp @@ -1257,6 +1257,8 @@ TEST(MetaServiceHttpTest, GetTabletStatsTest) { EXPECT_EQ(res.tablet_stats(0).num_rows(), 0); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 1); EXPECT_EQ(res.tablet_stats(0).num_segments(), 0); + EXPECT_EQ(res.tablet_stats(0).index_size(), 0); + EXPECT_EQ(res.tablet_stats(0).segment_size(), 0); { GetTabletStatsRequest req; auto idx = req.add_tablet_idx(); @@ -1288,6 +1290,16 @@ TEST(MetaServiceHttpTest, GetTabletStatsTest) { &data_size_key); ASSERT_EQ(txn->get(data_size_key, &data_size_val), TxnErrorCode::TXN_OK); EXPECT_EQ(*(int64_t*)data_size_val.data(), 22000); + std::string index_size_key, index_size_val; + stats_tablet_index_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &index_size_key); + ASSERT_EQ(txn->get(index_size_key, &index_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)index_size_val.data(), 2000); + std::string segment_size_key, segment_size_val; + stats_tablet_segment_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &segment_size_key); + ASSERT_EQ(txn->get(segment_size_key, &segment_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)segment_size_val.data(), 20000); std::string num_rows_key, num_rows_val; stats_tablet_num_rows_key({mock_instance, table_id, index_id, partition_id, tablet_id}, &num_rows_key); @@ -1312,6 +1324,8 @@ TEST(MetaServiceHttpTest, GetTabletStatsTest) { EXPECT_EQ(res.tablet_stats(0).num_rows(), 400); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 5); EXPECT_EQ(res.tablet_stats(0).num_segments(), 4); + EXPECT_EQ(res.tablet_stats(0).index_size(), 4000); + EXPECT_EQ(res.tablet_stats(0).segment_size(), 40000); { GetTabletStatsRequest req; auto idx = req.add_tablet_idx(); diff --git a/cloud/test/meta_service_job_test.cpp b/cloud/test/meta_service_job_test.cpp index f0323eebb790bea..e1dbb0089e2e731 100644 --- a/cloud/test/meta_service_job_test.cpp +++ b/cloud/test/meta_service_job_test.cpp @@ -95,7 +95,9 @@ doris::RowsetMetaCloudPB create_rowset(int64_t tablet_id, int64_t start_version, rowset.set_end_version(end_version); rowset.set_num_segments(1); rowset.set_num_rows(num_rows); - rowset.set_data_disk_size(num_rows * 100); + rowset.set_total_disk_size(num_rows * 100); + rowset.set_data_disk_size(num_rows * 50); + rowset.set_index_disk_size(num_rows * 50); rowset.mutable_tablet_schema()->set_schema_version(0); rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK return rowset; @@ -114,11 +116,13 @@ void insert_rowsets(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int64_t p std::unique_ptr txn; ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK) << tablet_id; std::deque buffer; - int data_size = 0, num_rows = 0, num_seg = 0; + int data_size = 0, num_rows = 0, num_seg = 0, index_size = 0, segment_size = 0; for (auto& rowset : rowsets) { - data_size += rowset.data_disk_size(); + data_size += rowset.total_disk_size(); num_rows += rowset.num_rows(); num_seg += rowset.num_segments(); + index_size += rowset.index_disk_size(); + segment_size += rowset.data_disk_size(); auto& key = buffer.emplace_back(); auto& val = buffer.emplace_back(); meta_rowset_key({instance_id, tablet_id, rowset.end_version()}, &key); @@ -138,6 +142,12 @@ void insert_rowsets(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int64_t p std::string num_segs_key; stats_tablet_num_segs_key(info, &num_segs_key); txn->atomic_add(num_segs_key, num_seg); + std::string index_size_key; + stats_tablet_index_size_key(info, &index_size_key); + txn->atomic_add(index_size_key, index_size); + std::string segment_size_key; + stats_tablet_segment_size_key(info, &segment_size_key); + txn->atomic_add(segment_size_key, segment_size); ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK) << tablet_id; } @@ -241,7 +251,11 @@ void finish_schema_change_job( sc->add_output_versions(rowset.end_version()); sc->set_num_output_rows(sc->num_output_rows() + rowset.num_rows()); sc->set_num_output_segments(sc->num_output_segments() + rowset.num_segments()); - sc->set_size_output_rowsets(sc->size_output_rowsets() + rowset.data_disk_size()); + sc->set_size_output_rowsets(sc->size_output_rowsets() + rowset.total_disk_size()); + sc->set_index_size_output_rowsets(sc->index_size_output_rowsets() + + rowset.index_disk_size()); + sc->set_segment_size_output_rowsets(sc->segment_size_output_rowsets() + + rowset.data_disk_size()); } sc->set_num_output_rowsets(output_rowsets.size()); } @@ -853,6 +867,10 @@ TEST(MetaServiceJobTest, CompactionJobTest) { compaction->set_num_input_segments(dist(rng)); compaction->set_size_input_rowsets(dist(rng)); compaction->set_size_output_rowsets(dist(rng)); + compaction->set_index_size_input_rowsets(dist(rng)); + compaction->set_segment_size_output_rowsets(dist(rng)); + compaction->set_index_size_input_rowsets(dist(rng)); + compaction->set_segment_size_output_rowsets(dist(rng)); compaction->set_type(type); tablet_stats_pb.set_cumulative_compaction_cnt(dist(rng)); @@ -863,6 +881,8 @@ TEST(MetaServiceJobTest, CompactionJobTest) { tablet_stats_pb.set_data_size(dist(rng) + compaction->size_input_rowsets()); tablet_stats_pb.set_num_rowsets(dist(rng) + compaction->num_input_rowsets()); tablet_stats_pb.set_num_segments(dist(rng) + compaction->num_input_segments()); + tablet_stats_pb.set_index_size(dist(rng) + compaction->index_size_input_rowsets()); + tablet_stats_pb.set_segment_size(dist(rng) + compaction->segment_size_input_rowsets()); std::string tablet_stats_val = tablet_stats_pb.SerializeAsString(); ASSERT_FALSE(tablet_stats_val.empty()); @@ -981,6 +1001,8 @@ TEST(MetaServiceJobTest, CompactionJobTest) { EXPECT_EQ(stats.data_size() , tablet_stats_pb.data_size() + (req.job().compaction(0).size_output_rowsets() - req.job().compaction(0).size_input_rowsets())); EXPECT_EQ(stats.num_rowsets() , tablet_stats_pb.num_rowsets() + (req.job().compaction(0).num_output_rowsets() - req.job().compaction(0).num_input_rowsets())); EXPECT_EQ(stats.num_segments() , tablet_stats_pb.num_segments() + (req.job().compaction(0).num_output_segments() - req.job().compaction(0).num_input_segments())); + EXPECT_EQ(stats.index_size() , tablet_stats_pb.index_size() + (req.job().compaction(0).index_size_output_rowsets() - req.job().compaction(0).index_size_input_rowsets())); + EXPECT_EQ(stats.segment_size() , tablet_stats_pb.segment_size() + (req.job().compaction(0).segment_size_output_rowsets() - req.job().compaction(0).segment_size_input_rowsets())); // clang-format on // Check job removed, tablet meta updated @@ -1157,6 +1179,8 @@ TEST(MetaServiceJobTest, CompactionJobWithMoWTest) { tablet_stats_pb.set_data_size(dist(rng) + compaction->size_input_rowsets()); tablet_stats_pb.set_num_rowsets(dist(rng) + compaction->num_input_rowsets()); tablet_stats_pb.set_num_segments(dist(rng) + compaction->num_input_segments()); + tablet_stats_pb.set_index_size(dist(rng) + compaction->index_size_input_rowsets()); + tablet_stats_pb.set_segment_size(dist(rng) + compaction->segment_size_input_rowsets()); std::string tablet_stats_val = tablet_stats_pb.SerializeAsString(); ASSERT_FALSE(tablet_stats_val.empty()); @@ -1391,12 +1415,16 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { EXPECT_EQ(res.stats().num_rowsets(), 6); EXPECT_EQ(res.stats().num_segments(), 5); EXPECT_EQ(res.stats().data_size(), 50000); + EXPECT_EQ(res.stats().index_size(), 25000); + EXPECT_EQ(res.stats().segment_size(), 25000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 500); EXPECT_EQ(tablet_stats.num_rowsets(), 6); EXPECT_EQ(tablet_stats.num_segments(), 5); EXPECT_EQ(tablet_stats.data_size(), 50000); + EXPECT_EQ(tablet_stats.index_size(), 25000); + EXPECT_EQ(tablet_stats.segment_size(), 25000); std::unique_ptr txn; ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -1465,12 +1493,16 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { EXPECT_EQ(res.stats().num_rowsets(), 6); EXPECT_EQ(res.stats().num_segments(), 5); EXPECT_EQ(res.stats().data_size(), 50000); + EXPECT_EQ(res.stats().index_size(), 25000); + EXPECT_EQ(res.stats().segment_size(), 25000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 500); EXPECT_EQ(tablet_stats.num_rowsets(), 6); EXPECT_EQ(tablet_stats.num_segments(), 5); EXPECT_EQ(tablet_stats.data_size(), 50000); + EXPECT_EQ(tablet_stats.index_size(), 25000); + EXPECT_EQ(tablet_stats.segment_size(), 25000); std::unique_ptr txn; ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -1628,12 +1660,16 @@ TEST(MetaServiceJobTest, RetrySchemaChangeJobTest) { EXPECT_EQ(res.stats().num_rowsets(), 6); EXPECT_EQ(res.stats().num_segments(), 5); EXPECT_EQ(res.stats().data_size(), 50000); + EXPECT_EQ(res.stats().index_size(), 25000); + EXPECT_EQ(res.stats().segment_size(), 25000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 500); EXPECT_EQ(tablet_stats.num_rowsets(), 6); EXPECT_EQ(tablet_stats.num_segments(), 5); EXPECT_EQ(tablet_stats.data_size(), 50000); + EXPECT_EQ(tablet_stats.index_size(), 25000); + EXPECT_EQ(tablet_stats.segment_size(), 25000); std::unique_ptr txn; ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -1942,10 +1978,14 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { compaction->add_output_rowset_ids(output_rowset.rowset_id_v2()); compaction->set_output_cumulative_point(11); compaction->set_size_input_rowsets(60000); + compaction->set_index_size_input_rowsets(30000); + compaction->set_segment_size_input_rowsets(30000); compaction->set_num_input_rows(600); compaction->set_num_input_rowsets(6); compaction->set_num_input_segments(6); compaction->set_size_output_rowsets(10000); + compaction->set_index_size_output_rowsets(5000); + compaction->set_segment_size_output_rowsets(5000); compaction->set_num_output_rows(100); compaction->set_num_output_rowsets(1); compaction->set_num_output_segments(1); @@ -1958,12 +1998,16 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { EXPECT_EQ(res.stats().num_rowsets(), 6); EXPECT_EQ(res.stats().num_segments(), 5); EXPECT_EQ(res.stats().data_size(), 50000); + EXPECT_EQ(res.stats().index_size(), 25000); + EXPECT_EQ(res.stats().segment_size(), 25000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 500); EXPECT_EQ(tablet_stats.num_rowsets(), 6); EXPECT_EQ(tablet_stats.num_segments(), 5); EXPECT_EQ(tablet_stats.data_size(), 50000); + EXPECT_EQ(tablet_stats.index_size(), 25000); + EXPECT_EQ(tablet_stats.segment_size(), 25000); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); // Check tmp rowsets @@ -2046,10 +2090,14 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { compaction->add_output_rowset_ids(output_rowset.rowset_id_v2()); compaction->set_output_cumulative_point(5); compaction->set_size_input_rowsets(30000); + compaction->set_index_size_input_rowsets(15000); + compaction->set_segment_size_input_rowsets(15000); compaction->set_num_input_rows(300); compaction->set_num_input_rowsets(3); compaction->set_num_input_segments(3); compaction->set_size_output_rowsets(10000); + compaction->set_index_size_output_rowsets(5000); + compaction->set_segment_size_output_rowsets(5000); compaction->set_num_output_rows(100); compaction->set_num_output_rowsets(1); compaction->set_num_output_segments(1); @@ -2062,12 +2110,16 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { EXPECT_EQ(res.stats().num_rowsets(), 4); EXPECT_EQ(res.stats().num_segments(), 3); EXPECT_EQ(res.stats().data_size(), 30000); + EXPECT_EQ(res.stats().index_size(), 15000); + EXPECT_EQ(res.stats().segment_size(), 15000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 300); EXPECT_EQ(tablet_stats.num_rowsets(), 4); EXPECT_EQ(tablet_stats.num_segments(), 3); EXPECT_EQ(tablet_stats.data_size(), 30000); + EXPECT_EQ(tablet_stats.index_size(), 15000); + EXPECT_EQ(tablet_stats.segment_size(), 15000); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); // Check tmp rowsets diff --git a/cloud/test/meta_service_test.cpp b/cloud/test/meta_service_test.cpp index 1f93b228c6de086..d2dd80f6871e3e1 100644 --- a/cloud/test/meta_service_test.cpp +++ b/cloud/test/meta_service_test.cpp @@ -4429,6 +4429,8 @@ TEST(MetaServiceTest, GetTabletStatsTest) { EXPECT_EQ(res.tablet_stats(0).num_rows(), 0); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 1); EXPECT_EQ(res.tablet_stats(0).num_segments(), 0); + EXPECT_EQ(res.tablet_stats(0).index_size(), 0); + EXPECT_EQ(res.tablet_stats(0).segment_size(), 0); // Insert rowset config::split_tablet_stats = false; ASSERT_NO_FATAL_FAILURE( @@ -4448,6 +4450,16 @@ TEST(MetaServiceTest, GetTabletStatsTest) { &data_size_key); ASSERT_EQ(txn->get(data_size_key, &data_size_val), TxnErrorCode::TXN_OK); EXPECT_EQ(*(int64_t*)data_size_val.data(), 22000); + std::string index_size_key, index_size_val; + stats_tablet_index_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &index_size_key); + ASSERT_EQ(txn->get(index_size_key, &index_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)index_size_val.data(), 2000); + std::string segment_size_key, segment_size_val; + stats_tablet_segment_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &segment_size_key); + ASSERT_EQ(txn->get(segment_size_key, &segment_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)segment_size_val.data(), 20000); std::string num_rows_key, num_rows_val; stats_tablet_num_rows_key({mock_instance, table_id, index_id, partition_id, tablet_id}, &num_rows_key); @@ -4472,6 +4484,8 @@ TEST(MetaServiceTest, GetTabletStatsTest) { EXPECT_EQ(res.tablet_stats(0).num_rows(), 400); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 5); EXPECT_EQ(res.tablet_stats(0).num_segments(), 4); + EXPECT_EQ(res.tablet_stats(0).index_size(), 4000); + EXPECT_EQ(res.tablet_stats(0).segment_size(), 40000); } TEST(MetaServiceTest, GetDeleteBitmapUpdateLock) { @@ -7771,6 +7785,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { // simulate that there are new segments added to this rowset rowset.set_num_segments(rowset.num_segments() + 3); rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_total_disk_size(rowset.total_disk_size() + 11000); + rowset.set_index_disk_size(rowset.index_disk_size() + 1000); rowset.set_data_disk_size(rowset.data_disk_size() + 10000); ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); @@ -7789,6 +7805,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { ASSERT_EQ(doris::BEGIN_PARTIAL_UPDATE, fetchedRowsetMeta.rowset_state()); ASSERT_EQ(rowset.num_segments(), fetchedRowsetMeta.num_segments()); ASSERT_EQ(rowset.num_rows(), fetchedRowsetMeta.num_rows()); + ASSERT_EQ(rowset.total_disk_size(), fetchedRowsetMeta.total_disk_size()); + ASSERT_EQ(rowset.index_disk_size(), fetchedRowsetMeta.index_disk_size()); ASSERT_EQ(rowset.data_disk_size(), fetchedRowsetMeta.data_disk_size()); ASSERT_NO_FATAL_FAILURE(commit_txn(meta_service.get(), db_id, txn_id, label)); @@ -7819,6 +7837,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { // simulate that there are new segments added to this rowset rowset.set_num_segments(rowset.num_segments() + 3); rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_total_disk_size(rowset.total_disk_size() + 11000); + rowset.set_index_disk_size(rowset.index_disk_size() + 1000); rowset.set_data_disk_size(rowset.data_disk_size() + 10000); // repeated calls to update_tmp_rowset will all success @@ -7844,6 +7864,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { ASSERT_EQ(doris::BEGIN_PARTIAL_UPDATE, fetchedRowsetMeta.rowset_state()); ASSERT_EQ(rowset.num_segments(), fetchedRowsetMeta.num_segments()); ASSERT_EQ(rowset.num_rows(), fetchedRowsetMeta.num_rows()); + ASSERT_EQ(rowset.total_disk_size(), fetchedRowsetMeta.total_disk_size()); + ASSERT_EQ(rowset.index_disk_size(), fetchedRowsetMeta.index_disk_size()); ASSERT_EQ(rowset.data_disk_size(), fetchedRowsetMeta.data_disk_size()); ASSERT_NO_FATAL_FAILURE(commit_txn(meta_service.get(), db_id, txn_id, label)); @@ -7869,6 +7891,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { // simulate that there are new segments added to this rowset rowset.set_num_segments(rowset.num_segments() + 3); rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_total_disk_size(rowset.total_disk_size() + 11000); + rowset.set_index_disk_size(rowset.index_disk_size() + 1000); rowset.set_data_disk_size(rowset.data_disk_size() + 10000); ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); diff --git a/cloud/test/schema_kv_test.cpp b/cloud/test/schema_kv_test.cpp index 07f658175c806f1..52e54f5e494b7f5 100644 --- a/cloud/test/schema_kv_test.cpp +++ b/cloud/test/schema_kv_test.cpp @@ -210,6 +210,8 @@ TEST(DetachSchemaKVTest, TabletTest) { EXPECT_EQ(get_rowset_res.stats().num_rowsets(), 1); EXPECT_EQ(get_rowset_res.stats().num_segments(), 0); EXPECT_EQ(get_rowset_res.stats().data_size(), 0); + EXPECT_EQ(get_rowset_res.stats().index_size(), 0); + EXPECT_EQ(get_rowset_res.stats().segment_size(), 0); } // new MS batch create tablets with write_schema_kv=true @@ -481,6 +483,8 @@ TEST(DetachSchemaKVTest, RowsetTest) { EXPECT_EQ(get_rowset_res.stats().num_rowsets(), 2); EXPECT_EQ(get_rowset_res.stats().num_segments(), 1); EXPECT_EQ(get_rowset_res.stats().data_size(), 11000); + EXPECT_EQ(get_rowset_res.stats().index_size(), 1000); + EXPECT_EQ(get_rowset_res.stats().segment_size(), 10000); } // new MS read rowsets committed by both old and new MS @@ -530,6 +534,8 @@ TEST(DetachSchemaKVTest, RowsetTest) { EXPECT_EQ(get_rowset_res->stats().num_rowsets(), 26); EXPECT_EQ(get_rowset_res->stats().num_segments(), 25); EXPECT_EQ(get_rowset_res->stats().data_size(), 275000); + EXPECT_EQ(get_rowset_res->stats().index_size(), 25000); + EXPECT_EQ(get_rowset_res->stats().segment_size(), 250000); if (schema != nullptr) { auto schema_version = get_rowset_res->rowset_meta(10).schema_version(); get_rowset_res->mutable_rowset_meta(10)->mutable_tablet_schema()->set_schema_version(3); diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 6e2a518ec209827..a372a993ac6f228 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -324,7 +324,8 @@ unsupportedShowStatement ((FROM | IN) database=multipartIdentifier)? wildWhere? sortClause? limitClause? #showAlterTable | SHOW DATA SKEW FROM baseTableRef #showDataSkew - | SHOW DATA (FROM tableName=multipartIdentifier)? sortClause? propertyClause? #showData + | SHOW DATA (ALL)? (FROM tableName=multipartIdentifier)? + sortClause? propertyClause? #showData | SHOW TEMPORARY? PARTITIONS FROM tableName=multipartIdentifier wildWhere? sortClause? limitClause? #showPartitions | SHOW TABLET tabletId=INTEGER_VALUE #showTabletId diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 03f1f02f158f841..05804343bc34175 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -1014,6 +1014,8 @@ nonterminal List copy_select_expr_list; //genearted column nonterminal Boolean opt_generated_always; +nonterminal Boolean opt_detailed; + precedence nonassoc COMMA; precedence nonassoc STRING_LITERAL; precedence nonassoc KW_COLUMNS; @@ -4493,13 +4495,13 @@ show_param ::= RESULT = new ShowDataSkewStmt(table_ref); :} /* Show data statement: used to show data size of specified range */ - | KW_DATA order_by_clause:orderByClause opt_properties:prop + | KW_DATA opt_detailed:detailed order_by_clause:orderByClause opt_properties:prop {: - RESULT = new ShowDataStmt(null, orderByClause, prop); + RESULT = new ShowDataStmt(null, orderByClause, prop, detailed); :} - | KW_DATA KW_FROM table_name:dbTblName order_by_clause:orderByClause + | KW_DATA opt_detailed:detailed KW_FROM table_name:dbTblName order_by_clause:orderByClause {: - RESULT = new ShowDataStmt(dbTblName, orderByClause, null); + RESULT = new ShowDataStmt(dbTblName, orderByClause, null, detailed); :} | opt_tmp:tmp KW_PARTITIONS KW_FROM table_name:tblName opt_wild_where order_by_clause:orderByClause limit_clause: limitClause {: @@ -5063,6 +5065,17 @@ cancel_param ::= :} ; +opt_detailed ::= + /* empty */ + {: + RESULT = false; + :} + | KW_ALL + {: + RESULT = true; + :} + ; + // Delete stmt delete_stmt ::= KW_DELETE KW_FROM table_name:table opt_partition_names:partitionNames opt_table_alias:alias opt_using_clause:fromClause where_clause:wherePredicate diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java index 26b77e0b3d5d260..398b0d7ec05d420 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java @@ -100,26 +100,64 @@ public class ShowDataStmt extends ShowStmt implements NotFallbackInParser { new ImmutableList.Builder().add("TableName").add("IndexName").add("Size").add("ReplicaCount") .add("RowCount").add("RemoteSize").build(); + private static final ShowResultSetMetaData SHOW_DETAILED_TABLE_DATA_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("TableName", ScalarType.createVarchar(20))) + .addColumn(new Column("ReplicaCount", ScalarType.createVarchar(20))) + .addColumn(new Column("LocalTotalSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalDataSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalIndexSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteTotalSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteDataSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteIndexSize", ScalarType.createVarchar(30))) + .build(); + + private static final ShowResultSetMetaData SHOW_DETAILED_INDEX_DATA_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("TableName", ScalarType.createVarchar(20))) + .addColumn(new Column("IndexName", ScalarType.createVarchar(20))) + .addColumn(new Column("ReplicaCount", ScalarType.createVarchar(20))) + .addColumn(new Column("RowCount", ScalarType.createVarchar(20))) + .addColumn(new Column("LocalTotalSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalDataSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalIndexSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteTotalSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteDataSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteIndexSize", ScalarType.createVarchar(30))) + .build(); + TableName tableName; String dbName; - List> totalRows; + List> totalRows = Lists.newArrayList(); List> totalRowsObject = Lists.newArrayList(); - private List orderByElements; - private List orderByPairs; + private List orderByElements = null; + private List orderByPairs = null; - private final Map properties; + private Map properties = null; + private boolean detailed = true; private static final String WAREHOUSE = "entire_warehouse"; private static final String DB_LIST = "db_names"; - public ShowDataStmt(TableName tableName, List orderByElements, Map properties) { + private long totalSize = 0; + private long totalReplicaCount = 0; + private long totalRemoteSize = 0; + private long totalLocalInvertedSize = 0; + private long totalLocalSegmentSize = 0; + private long totalRemoteInvertedSize = 0; + private long totalRemoteSegmentSize = 0; + + public ShowDataStmt(TableName tableName, List orderByElements, + Map properties, boolean detailed) { this.tableName = tableName; - this.totalRows = Lists.newArrayList(); this.orderByElements = orderByElements; this.properties = properties; + this.detailed = detailed; } + public ShowDataStmt() {} + @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); @@ -156,208 +194,245 @@ public void analyze(Analyzer analyzer) throws UserException { } } - if (tableName == null) { - db.readLock(); - try { - long totalSize = 0; - long totalReplicaCount = 0; - long totalRemoteSize = 0; - // sort by table name - List tables = db.getTables(); - SortedSet
sortedTables = new TreeSet<>(new Comparator
() { - @Override - public int compare(Table t1, Table t2) { - return t1.getName().compareTo(t2.getName()); - } - }); - - for (Table table : tables) { - if (!Env.getCurrentEnv().getAccessManager() - .checkTblPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, - table.getName(), - PrivPredicate.SHOW)) { - continue; - } - sortedTables.add(table); - } + if (hasTable()) { + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), tableName, + PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW DATA", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), + dbName + ": " + tableName); + } + OlapTable table = (OlapTable) db.getTableOrMetaException(tableName.getTbl(), TableType.OLAP); + getSingleTableStats(table); + } else { + getSingleDbStats(db); + } + return; + } - for (Table table : sortedTables) { - if (!table.isManagedTable()) { - continue; - } + private void collectDbStats(Database db) { + // sort by table name + List
tables = db.getTables(); + SortedSet
sortedTables = new TreeSet<>(new Comparator
() { + @Override + public int compare(Table t1, Table t2) { + return t1.getName().compareTo(t2.getName()); + } + }); - OlapTable olapTable = (OlapTable) table; - long tableSize = 0; - long replicaCount = 0; - long remoteSize = 0; + for (Table table : tables) { + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, + table.getName(), + PrivPredicate.SHOW)) { + continue; + } + sortedTables.add(table); + } - tableSize = olapTable.getDataSize(); - replicaCount = olapTable.getReplicaCount(); - remoteSize = olapTable.getRemoteDataSize(); + for (Table table : sortedTables) { + if (!table.isManagedTable()) { + continue; + } - //|TableName|Size|ReplicaCount|RemoteSize - List row = Arrays.asList(table.getName(), tableSize, replicaCount, remoteSize); - totalRowsObject.add(row); + OlapTable olapTable = (OlapTable) table; + long tableSize = 0; + long replicaCount = 0; + long remoteSize = 0; - totalSize += tableSize; - totalReplicaCount += replicaCount; - totalRemoteSize += remoteSize; - } // end for tables + tableSize = olapTable.getDataSize(); + replicaCount = olapTable.getReplicaCount(); + remoteSize = olapTable.getRemoteDataSize(); - // sort by - if (orderByPairs != null && !orderByPairs.isEmpty()) { - // k-> index, v-> isDesc - Map sortMap = Maps.newLinkedHashMap(); - for (OrderByPair orderByPair : orderByPairs) { - sortMap.put(orderByPair.getIndex(), orderByPair.isDesc()); + if (!detailed) { + totalRowsObject.add(Arrays.asList(table.getName(), tableSize, replicaCount, remoteSize)); + } else { + long localIndexSize = olapTable.getLocalIndexFileSize(); + long localSegmentSize = olapTable.getLocalSegmentSize(); + long remoteIndexSize = olapTable.getRemoteIndexFileSize(); + long remoteSegmentSize = olapTable.getRemoteSegmentSize(); + totalRowsObject.add(Arrays.asList(table.getName(), tableSize, replicaCount, remoteSize, + localIndexSize, localSegmentSize, remoteIndexSize, remoteSegmentSize)); + totalLocalInvertedSize += localIndexSize; + totalLocalSegmentSize += localSegmentSize; + totalRemoteInvertedSize += remoteIndexSize; + totalRemoteSegmentSize += remoteSegmentSize; + } - } - totalRowsObject.sort(sortRows(sortMap)); - } + totalSize += tableSize; + totalReplicaCount += replicaCount; + totalRemoteSize += remoteSize; + } // end for tables + } - // for output - for (List row : totalRowsObject) { - //|TableName|Size|ReplicaCount|RemoteSize - Pair tableSizePair = DebugUtil.getByteUint((long) row.get(1)); - String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(tableSizePair.first) + " " - + tableSizePair.second; - Pair remoteSizePair = DebugUtil.getByteUint((long) row.get(3)); - String remoteReadableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(remoteSizePair.first) + " " - + remoteSizePair.second; - List result = Arrays.asList(String.valueOf(row.get(0)), - readableSize, String.valueOf(row.get(2)), remoteReadableSize); - totalRows.add(result); - } + private void collectTableStats(OlapTable table) { + // sort by index name + Map indexNames = table.getIndexNameToId(); + Map sortedIndexNames = new TreeMap(); + for (Map.Entry entry : indexNames.entrySet()) { + sortedIndexNames.put(entry.getKey(), entry.getValue()); + } - Pair totalSizePair = DebugUtil.getByteUint(totalSize); - String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalSizePair.first) + " " - + totalSizePair.second; - Pair totalRemoteSizePair = DebugUtil.getByteUint(totalRemoteSize); - String remoteReadableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalRemoteSizePair.first) + " " - + totalRemoteSizePair.second; - List total = Arrays.asList("Total", readableSize, String.valueOf(totalReplicaCount), - remoteReadableSize); - totalRows.add(total); - - // quota - long quota = db.getDataQuota(); - long replicaQuota = db.getReplicaQuota(); - Pair quotaPair = DebugUtil.getByteUint(quota); - String readableQuota = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(quotaPair.first) + " " - + quotaPair.second; - - List quotaRow = Arrays.asList("Quota", readableQuota, String.valueOf(replicaQuota), ""); - totalRows.add(quotaRow); - - // left - long left = Math.max(0, quota - totalSize); - long replicaCountLeft = Math.max(0, replicaQuota - totalReplicaCount); - Pair leftPair = DebugUtil.getByteUint(left); - String readableLeft = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(leftPair.first) + " " - + leftPair.second; - List leftRow = Arrays.asList("Left", readableLeft, String.valueOf(replicaCountLeft), ""); - totalRows.add(leftRow); - } finally { - db.readUnlock(); + for (Long indexId : sortedIndexNames.values()) { + long indexSize = 0; + long indexReplicaCount = 0; + long indexRowCount = 0; + long indexRemoteSize = 0; + long localIndexSize = 0; + long localSegmentSize = 0; + long remoteIndexSize = 0; + long remoteSegmentSize = 0; + for (Partition partition : table.getAllPartitions()) { + MaterializedIndex mIndex = partition.getIndex(indexId); + indexSize += mIndex.getDataSize(false); + indexReplicaCount += mIndex.getReplicaCount(); + indexRowCount += mIndex.getRowCount() == -1 ? 0 : mIndex.getRowCount(); + indexRemoteSize += mIndex.getRemoteDataSize(); + localIndexSize += mIndex.getLocalIndexSize(); + localSegmentSize += mIndex.getLocalSegmentSize(); + remoteIndexSize += mIndex.getRemoteIndexSize(); + remoteSegmentSize += mIndex.getRemoteSegmentSize(); } - } else { - if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), tableName, - PrivPredicate.SHOW)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW DATA", - ConnectContext.get().getQualifiedUser(), - ConnectContext.get().getRemoteIP(), - dbName + ": " + tableName); + + String indexName = table.getIndexNameById(indexId); + if (!detailed) { + totalRowsObject.add(Arrays.asList(tableName, indexName, indexSize, indexReplicaCount, + indexRowCount, indexRemoteSize)); + } else { + totalRowsObject.add(Arrays.asList(tableName, indexName, indexSize, indexReplicaCount, indexRowCount, + indexRemoteSize, localIndexSize, localSegmentSize, remoteIndexSize, remoteSegmentSize)); } - OlapTable olapTable = (OlapTable) db - .getTableOrMetaException(tableName.getTbl(), TableType.OLAP); - long totalSize = 0; - long totalReplicaCount = 0; - long totalRemoteSize = 0; - olapTable.readLock(); - try { - // sort by index name - Map indexNames = olapTable.getIndexNameToId(); - Map sortedIndexNames = new TreeMap(); - for (Map.Entry entry : indexNames.entrySet()) { - sortedIndexNames.put(entry.getKey(), entry.getValue()); - } + totalSize += indexSize; + totalReplicaCount += indexReplicaCount; + totalRemoteSize += indexRemoteSize; + totalLocalInvertedSize += localIndexSize; + totalLocalSegmentSize += localSegmentSize; + totalRemoteInvertedSize += remoteIndexSize; + totalRemoteSegmentSize += remoteSegmentSize; + } // end for indices + } - for (Long indexId : sortedIndexNames.values()) { - long indexSize = 0; - long indexReplicaCount = 0; - long indexRowCount = 0; - long indexRemoteSize = 0; - for (Partition partition : olapTable.getAllPartitions()) { - MaterializedIndex mIndex = partition.getIndex(indexId); - indexSize += mIndex.getDataSize(false); - indexReplicaCount += mIndex.getReplicaCount(); - indexRowCount += mIndex.getRowCount() == -1 ? 0 : mIndex.getRowCount(); - indexRemoteSize += mIndex.getRemoteDataSize(); - } + private void sortResult() { + if (orderByPairs != null && !orderByPairs.isEmpty()) { + // k-> index, v-> isDesc + Map sortMap = Maps.newLinkedHashMap(); + for (OrderByPair orderByPair : orderByPairs) { + sortMap.put(orderByPair.getIndex(), orderByPair.isDesc()); - String indexName = olapTable.getIndexNameById(indexId); - // .add("TableName").add("IndexName").add("Size").add("ReplicaCount").add("RowCount") - // .add("RemoteSize") - List row = Arrays.asList(tableName, indexName, indexSize, indexReplicaCount, - indexRowCount, indexRemoteSize); - totalRowsObject.add(row); - - totalSize += indexSize; - totalReplicaCount += indexReplicaCount; - totalRemoteSize += indexRemoteSize; - } // end for indices - - // sort by - if (orderByPairs != null && !orderByPairs.isEmpty()) { - // k-> index, v-> isDesc - Map sortMap = Maps.newLinkedHashMap(); - for (OrderByPair orderByPair : orderByPairs) { - sortMap.put(orderByPair.getIndex(), orderByPair.isDesc()); + } + totalRowsObject.sort(sortRows(sortMap)); + } + } - } - totalRowsObject.sort(sortRows(sortMap)); - } + private void buildDbStatsOutput(long quota, long replicaQuota) { + // for output + for (List row : totalRowsObject) { + if (!detailed) { + totalRows.add(Arrays.asList(String.valueOf(row.get(0)), + DebugUtil.printByteWithUnit((long) row.get(1)), String.valueOf(row.get(2)), + DebugUtil.printByteWithUnit((long) row.get(3)))); + } else { + totalRows.add(Arrays.asList(String.valueOf(row.get(0)), String.valueOf(row.get(2)), + DebugUtil.printByteWithUnit((long) row.get(1)), DebugUtil.printByteWithUnit((long) row.get(5)), + DebugUtil.printByteWithUnit((long) row.get(4)), DebugUtil.printByteWithUnit((long) row.get(3)), + DebugUtil.printByteWithUnit((long) row.get(7)), + DebugUtil.printByteWithUnit((long) row.get(6)))); + } + } - // for output - for (int index = 0; index <= totalRowsObject.size() - 1; index++) { - //| TableName| IndexName | Size | ReplicaCount | RowCount | RemoteSize - List row = totalRowsObject.get(index); - List result; - Pair tableSizePair = DebugUtil.getByteUint((long) row.get(2)); - String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(tableSizePair.first) - + " " + tableSizePair.second; - Pair remoteSizePair = DebugUtil.getByteUint((long) row.get(5)); - String remoteReadableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(remoteSizePair.first) + " " - + remoteSizePair.second; - if (index == 0) { - result = Arrays.asList(tableName.getTbl(), String.valueOf(row.get(1)), - readableSize, String.valueOf(row.get(3)), - String.valueOf(row.get(4)), remoteReadableSize); - } else { - result = Arrays.asList("", String.valueOf(row.get(1)), - readableSize, String.valueOf(row.get(3)), - String.valueOf(row.get(4)), remoteReadableSize); - } - totalRows.add(result); - } + long left = Math.max(0, quota - totalSize); + long replicaCountLeft = Math.max(0, replicaQuota - totalReplicaCount); - Pair totalSizePair = DebugUtil.getByteUint(totalSize); - String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalSizePair.first) + " " - + totalSizePair.second; - Pair totalRemoteSizePair = DebugUtil.getByteUint(totalRemoteSize); - String remoteReadableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalRemoteSizePair.first) + " " - + totalRemoteSizePair.second; - List row = Arrays.asList("", "Total", readableSize, String.valueOf(totalReplicaCount), "", - remoteReadableSize); - totalRows.add(row); - } finally { - olapTable.readUnlock(); + if (!detailed) { + totalRows.add(Arrays.asList("Total", DebugUtil.printByteWithUnit(totalSize), + String.valueOf(totalReplicaCount), DebugUtil.printByteWithUnit(totalRemoteSize))); + totalRows.add(Arrays.asList("Quota", DebugUtil.printByteWithUnit(quota), + String.valueOf(replicaQuota), "")); + totalRows.add(Arrays.asList("Left", DebugUtil.printByteWithUnit(left), + String.valueOf(replicaCountLeft), "")); + } else { + totalRows.add(Arrays.asList("Total", String.valueOf(totalReplicaCount), + DebugUtil.printByteWithUnit(totalSize), + DebugUtil.printByteWithUnit(totalLocalSegmentSize), + DebugUtil.printByteWithUnit(totalLocalInvertedSize), + DebugUtil.printByteWithUnit(totalRemoteSize), + DebugUtil.printByteWithUnit(totalRemoteSegmentSize), + DebugUtil.printByteWithUnit(totalRemoteInvertedSize))); + totalRows.add(Arrays.asList("Quota", String.valueOf(replicaQuota), + DebugUtil.printByteWithUnit(quota), "", "", "", "", "")); + totalRows.add(Arrays.asList("Left", String.valueOf(replicaCountLeft), + DebugUtil.printByteWithUnit(left), "", "", "", "", "")); + } + } + + private void buildTableStatsOutput() { + for (int index = 0; index < totalRowsObject.size(); index++) { + List row = totalRowsObject.get(index); + String indexName = index == 0 ? tableName.getTbl() : ""; + if (!detailed) { + totalRows.add(Arrays.asList(indexName, String.valueOf(row.get(1)), + DebugUtil.printByteWithUnit((long) row.get(2)), String.valueOf(row.get(3)), + String.valueOf(row.get(4)), DebugUtil.printByteWithUnit((long) row.get(5)))); + } else { + totalRows.add(Arrays.asList(indexName, String.valueOf(row.get(1)), + String.valueOf(row.get(3)), String.valueOf(row.get(4)), + DebugUtil.printByteWithUnit((long) row.get(2)), DebugUtil.printByteWithUnit((long) row.get(7)), + DebugUtil.printByteWithUnit((long) row.get(6)), DebugUtil.printByteWithUnit((long) row.get(5)), + DebugUtil.printByteWithUnit((long) row.get(9)), + DebugUtil.printByteWithUnit((long) row.get(8)))); } } + + // Total + if (!detailed) { + totalRows.add(Arrays.asList("", "Total", DebugUtil.printByteWithUnit(totalSize), + String.valueOf(totalReplicaCount), "", DebugUtil.printByteWithUnit(totalRemoteSize))); + } else { + totalRows.add(Arrays.asList("", "Total", String.valueOf(totalReplicaCount), "", + DebugUtil.printByteWithUnit(totalSize), DebugUtil.printByteWithUnit(totalLocalSegmentSize), + DebugUtil.printByteWithUnit(totalLocalInvertedSize), + DebugUtil.printByteWithUnit(totalRemoteSize), + DebugUtil.printByteWithUnit(totalRemoteSegmentSize), + DebugUtil.printByteWithUnit(totalRemoteInvertedSize))); + } + } + + // |TableName|Size|ReplicaCount|RemoteSize| + // |TableName|ReplicaCount|LocalTotalSize|LocalDataSize|LocalIndexSize| + // |RemoteTotalSize|RemoteDataSize|RemoteIndexSize| + private void getSingleDbStats(Database db) { + db.readLock(); + long quota = 0; + long replicaQuota = 0; + try { + collectDbStats(db); + quota = db.getDataQuota(); + replicaQuota = db.getReplicaQuota(); + } finally { + db.readUnlock(); + } + // sort by + sortResult(); + buildDbStatsOutput(quota, replicaQuota); } - public static int analyzeColumn(String columnName, String tableName) throws AnalysisException { + // |TableName|IndexName|Size|ReplicaCount|RowCount|RemoteSize| + // |TableName|IndexName|ReplicaCount||RowCount|LocalTotalSize |LocalDataSize |LocalIndexSize| + // |RemoteTotalSize|RemoteDataSize|RemoteIndexSize| + private void getSingleTableStats(OlapTable table) { + table.readLock(); + try { + collectTableStats(table); + } finally { + table.readUnlock(); + } + // sort by + sortResult(); + buildTableStatsOutput(); + } + + private int analyzeColumn(String columnName, String tableName) throws AnalysisException { ImmutableList titles = SHOW_TABLE_DATA_META_DATA_ORIGIN; if (tableName != null) { titles = SHOW_INDEX_DATA_META_DATA_ORIGIN; @@ -371,7 +446,7 @@ public static int analyzeColumn(String columnName, String tableName) throws Anal throw new AnalysisException("Title name[" + columnName + "] does not exist"); } - private static Comparator> sortRows(Map sortMap) { + private Comparator> sortRows(Map sortMap) { Ordering ordering = Ordering.natural(); return new Comparator>() { @@ -400,28 +475,41 @@ public List> getResultRows() throws AnalysisException { @Override public ShowResultSetMetaData getMetaData() { - String value = null; if (properties != null) { - value = properties.get(WAREHOUSE); - } - if (value != null && value.equals("true")) { - return SHOW_WAREHOUSE_DATA_META_DATA; + String value = properties.get(WAREHOUSE); + if (value != null && value.equals("true")) { + return SHOW_WAREHOUSE_DATA_META_DATA; + } } if (Strings.isNullOrEmpty(dbName)) { return SHOW_DATABASE_DATA_META_DATA; } - if (tableName != null) { - return SHOW_INDEX_DATA_META_DATA; + + if (hasTable()) { + if (!detailed) { + return SHOW_INDEX_DATA_META_DATA; + } else { + return SHOW_DETAILED_INDEX_DATA_META_DATA; + } } else { - return SHOW_TABLE_DATA_META_DATA; + if (!detailed) { + return SHOW_TABLE_DATA_META_DATA; + } else { + return SHOW_DETAILED_TABLE_DATA_META_DATA; + } } } @Override public String toSql() { StringBuilder builder = new StringBuilder(); - builder.append("SHOW DATA"); + builder.append("SHOW "); + if (detailed) { + builder.append("DETAILED DATA"); + } else { + builder.append("DATA"); + } if (tableName != null) { builder.append(" FROM "); @@ -445,6 +533,7 @@ public String toString() { return toSql(); } + // |DBName|DataSize|RecycleSize| private boolean getDbStatsByProperties() { if (properties == null) { return false; @@ -507,6 +596,7 @@ private boolean getDbStatsByProperties() { return false; } + // |DbId|DbName|Size|RemoteSize|RecycleSize|RecycleRemoteSize| private void getAllDbStats() throws AnalysisException { // check auth if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java index 309b433572c942c..3babb0e001a2040 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java @@ -165,6 +165,8 @@ protected void runAfterCatalogReady() { Long tableDataSize = 0L; Long tableTotalReplicaDataSize = 0L; + Long tableTotalLocalIndexSize = 0L; + Long tableTotalLocalSegmentSize = 0L; Long tableReplicaCount = 0L; @@ -185,6 +187,8 @@ protected void runAfterCatalogReady() { long tabletRowsetCount = 0L; long tabletSegmentCount = 0L; long tabletRowCount = 0L; + long tabletIndexSize = 0L; + long tabletSegmentSize = 0L; for (Replica replica : tablet.getReplicas()) { if (replica.getDataSize() > tabletDataSize) { @@ -204,6 +208,13 @@ protected void runAfterCatalogReady() { tabletSegmentCount = replica.getSegmentCount(); } + if (replica.getLocalInvertedIndexSize() > tabletIndexSize) { + tabletIndexSize = replica.getLocalInvertedIndexSize(); + } + if (replica.getLocalSegmentSize() > tabletSegmentSize) { + tabletSegmentSize = replica.getLocalSegmentSize(); + } + tableReplicaCount++; } @@ -214,6 +225,8 @@ protected void runAfterCatalogReady() { tableRowsetCount += tabletRowsetCount; tableSegmentCount += tabletSegmentCount; + tableTotalLocalIndexSize += tabletIndexSize; + tableTotalLocalSegmentSize += tabletSegmentSize; } // end for tablets index.setRowCountReported(true); index.setRowCount(indexRowCount); @@ -223,7 +236,8 @@ protected void runAfterCatalogReady() { // this is only one thread to update table statistics, readLock is enough olapTable.setStatistics(new OlapTable.Statistics(db.getName(), table.getName(), tableDataSize, tableTotalReplicaDataSize, 0L, - tableReplicaCount, tableRowCount, tableRowsetCount, tableSegmentCount)); + tableReplicaCount, tableRowCount, tableRowsetCount, tableSegmentCount, + tableTotalLocalIndexSize, tableTotalLocalSegmentSize, 0L, 0L)); LOG.debug("finished to set row num for table: {} in database: {}", table.getName(), db.getFullName()); } finally { @@ -232,7 +246,7 @@ protected void runAfterCatalogReady() { newCloudTableStatsMap.put(Pair.of(dbId, table.getId()), new OlapTable.Statistics(db.getName(), table.getName(), tableDataSize, tableTotalReplicaDataSize, 0L, - tableReplicaCount, tableRowCount, tableRowsetCount, tableSegmentCount)); + tableReplicaCount, tableRowCount, tableRowsetCount, tableSegmentCount, 0L, 0L, 0L, 0L)); } } this.cloudTableStatsMap = newCloudTableStatsMap; @@ -253,6 +267,8 @@ private void updateTabletStat(GetTabletStatsResponse response) { replica.setRowsetCount(stat.getNumRowsets()); replica.setSegmentCount(stat.getNumSegments()); replica.setRowCount(stat.getNumRows()); + replica.setLocalInvertedIndexSize(stat.getIndexSize()); + replica.setLocalSegmentSize(stat.getSegmentSize()); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java index 6e4f6a2248e719c..5e367b538cd74c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java @@ -197,6 +197,46 @@ public long getReplicaCount() { return replicaCount; } + public long getLocalIndexSize() { + long localIndexSize = 0; + for (Tablet tablet : getTablets()) { + for (Replica replica : tablet.getReplicas()) { + localIndexSize += replica.getLocalInvertedIndexSize(); + } + } + return localIndexSize; + } + + public long getLocalSegmentSize() { + long localSegmentSize = 0; + for (Tablet tablet : getTablets()) { + for (Replica replica : tablet.getReplicas()) { + localSegmentSize += replica.getLocalSegmentSize(); + } + } + return localSegmentSize; + } + + public long getRemoteIndexSize() { + long remoteIndexSize = 0; + for (Tablet tablet : getTablets()) { + for (Replica replica : tablet.getReplicas()) { + remoteIndexSize += replica.getRemoteInvertedIndexSize(); + } + } + return remoteIndexSize; + } + + public long getRemoteSegmentSize() { + long remoteSegmentSize = 0; + for (Tablet tablet : getTablets()) { + for (Replica replica : tablet.getReplicas()) { + remoteSegmentSize += replica.getRemoteSegmentSize(); + } + } + return remoteSegmentSize; + } + public int getTabletOrderIdx(long tabletId) { int idx = 0; for (Tablet tablet : tablets) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 8925e483c29df7f..50b1dc6ab3954f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -1688,11 +1688,22 @@ public long getAvgRowLength() { public long getDataLength() { long dataSize = 0; for (Map.Entry entry : idToPartition.entrySet()) { - dataSize += entry.getValue().getBaseIndex().getDataSize(false); + dataSize += entry.getValue().getBaseIndex().getLocalSegmentSize(); + dataSize += entry.getValue().getBaseIndex().getRemoteSegmentSize(); } return dataSize; } + @Override + public long getIndexLength() { + long indexSize = 0; + for (Map.Entry entry : idToPartition.entrySet()) { + indexSize += entry.getValue().getBaseIndex().getLocalIndexSize(); + indexSize += entry.getValue().getBaseIndex().getRemoteIndexSize(); + } + return indexSize; + } + // Get the signature string of this table with specified partitions. // This method is used to determine whether the tables have the same schema. // Contains: @@ -3387,6 +3398,18 @@ public static class Statistics { @Getter private Long segmentCount; + @Getter + private Long localInvertedIndexSize; // multi replicas + + @Getter + private Long localSegmentSize; // multi replicas + + @Getter + private Long remoteInvertedIndexSize; // single replica + + @Getter + private Long remoteSegmentSize; // single replica + public Statistics() { this.dbName = null; this.tableName = null; @@ -3401,13 +3424,18 @@ public Statistics() { this.rowCount = 0L; this.rowsetCount = 0L; this.segmentCount = 0L; - + this.localInvertedIndexSize = 0L; + this.localSegmentSize = 0L; + this.remoteInvertedIndexSize = 0L; + this.remoteSegmentSize = 0L; } public Statistics(String dbName, String tableName, Long dataSize, Long totalReplicaDataSize, Long remoteDataSize, Long replicaCount, Long rowCount, - Long rowsetCount, Long segmentCount) { + Long rowsetCount, Long segmentCount, + Long localInvertedIndexSize, Long localSegmentSize, + Long remoteInvertedIndexSize, Long remoteSegmentSize) { this.dbName = dbName; this.tableName = tableName; @@ -3422,6 +3450,11 @@ public Statistics(String dbName, String tableName, this.rowCount = rowCount; this.rowsetCount = rowsetCount; this.segmentCount = segmentCount; + + this.localInvertedIndexSize = localInvertedIndexSize; + this.localSegmentSize = localSegmentSize; + this.remoteInvertedIndexSize = remoteInvertedIndexSize; + this.remoteSegmentSize = remoteSegmentSize; } } @@ -3445,6 +3478,22 @@ public long getReplicaCount() { return statistics.getReplicaCount(); } + public long getLocalIndexFileSize() { + return statistics.getLocalInvertedIndexSize(); + } + + public long getLocalSegmentSize() { + return statistics.getLocalSegmentSize(); + } + + public long getRemoteIndexFileSize() { + return statistics.getRemoteInvertedIndexSize(); + } + + public long getRemoteSegmentSize() { + return statistics.getRemoteSegmentSize(); + } + public boolean isShadowIndex(long indexId) { String indexName = getIndexNameById(indexId); if (indexName != null && indexName.startsWith(org.apache.doris.alter.SchemaChangeHandler.SHADOW_NAME_PREFIX)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index e86bd25cb5e926d..d248cc408399880 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -26,6 +26,8 @@ import org.apache.doris.thrift.TUniqueId; import com.google.gson.annotations.SerializedName; +import lombok.Getter; +import lombok.Setter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -122,6 +124,23 @@ public static class ReplicaContext { @SerializedName(value = "lsvh", alternate = {"lastSuccessVersionHash"}) private long lastSuccessVersionHash = 0L; + @Setter + @Getter + @SerializedName(value = "lis", alternate = {"localInvertedIndexSize"}) + private Long localInvertedIndexSize = 0L; + @Setter + @Getter + @SerializedName(value = "lss", alternate = {"localSegmentSize"}) + private Long localSegmentSize = 0L; + @Setter + @Getter + @SerializedName(value = "ris", alternate = {"remoteInvertedIndexSize"}) + private Long remoteInvertedIndexSize = 0L; + @Setter + @Getter + @SerializedName(value = "rss", alternate = {"remoteSegmentSize"}) + private Long remoteSegmentSize = 0L; + private volatile long totalVersionCount = -1; private volatile long visibleVersionCount = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index ed37a936e108196..0f1d91087d2988c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -446,6 +446,9 @@ public long getDataLength() { return 0; } + public long getIndexLength() { + return 0; + } public TTableDescriptor toThrift() { return null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index 8f6e924f44a54d9..4761ac9d86db2aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -174,6 +174,8 @@ default int getBaseColumnIdxByName(String colName) { long getAvgRowLength(); + long getIndexLength(); + long getLastCheckTime(); String getComment(boolean escapeQuota); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java index aa46c362e38ba97..f79ed89215b4d37 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java @@ -114,6 +114,11 @@ protected void runAfterCatalogReady() { Long tableDataSize = 0L; Long tableTotalReplicaDataSize = 0L; + Long tableTotalLocalIndexSize = 0L; + Long tableTotalLocalSegmentSize = 0L; + Long tableTotalRemoteIndexSize = 0L; + Long tableTotalRemoteSegmentSize = 0L; + Long tableRemoteDataSize = 0L; Long tableReplicaCount = 0L; @@ -171,6 +176,10 @@ protected void runAfterCatalogReady() { tabletRemoteDataSize = replica.getRemoteDataSize(); } tableReplicaCount++; + tableTotalLocalIndexSize += replica.getLocalInvertedIndexSize(); + tableTotalLocalSegmentSize += replica.getLocalSegmentSize(); + tableTotalRemoteIndexSize += replica.getRemoteInvertedIndexSize(); + tableTotalRemoteSegmentSize += replica.getRemoteSegmentSize(); } tableDataSize += tabletDataSize; @@ -196,7 +205,9 @@ protected void runAfterCatalogReady() { // this is only one thread to update table statistics, readLock is enough olapTable.setStatistics(new OlapTable.Statistics(db.getName(), table.getName(), tableDataSize, tableTotalReplicaDataSize, - tableRemoteDataSize, tableReplicaCount, tableRowCount, 0L, 0L)); + tableRemoteDataSize, tableReplicaCount, tableRowCount, 0L, 0L, + tableTotalLocalIndexSize, tableTotalLocalSegmentSize, + tableTotalRemoteIndexSize, tableTotalRemoteSegmentSize)); if (LOG.isDebugEnabled()) { LOG.debug("finished to set row num for table: {} in database: {}", @@ -220,6 +231,10 @@ private void updateTabletStat(Long beId, TTabletStatResult result) { if (replica != null) { replica.setDataSize(stat.getDataSize()); replica.setRemoteDataSize(stat.getRemoteDataSize()); + replica.setLocalInvertedIndexSize(stat.getLocalIndexSize()); + replica.setLocalSegmentSize(stat.getLocalSegmentSize()); + replica.setRemoteInvertedIndexSize(stat.getRemoteIndexSize()); + replica.setRemoteSegmentSize(stat.getRemoteSegmentSize()); replica.setRowCount(stat.getRowCount()); replica.setTotalVersionCount(stat.getTotalVersionCount()); replica.setVisibleVersionCount(stat.isSetVisibleVersionCount() ? stat.getVisibleVersionCount() diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index 6c72d0a67d4732c..bd1e36e7bc968b0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -239,6 +239,11 @@ public long getDataLength() { return 0; } + @Override + public long getIndexLength() { + return 0; + } + @Override public long getCreateTime() { return 0; diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index e35fd2dc852322f..cf9e8e82ce112f9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -659,6 +659,7 @@ public TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE status.setRows(table.getCachedRowCount()); status.setDataLength(table.getDataLength()); status.setAvgRowLength(table.getAvgRowLength()); + status.setIndexLength(table.getIndexLength()); tablesResult.add(status); } finally { table.readUnlock(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java index 9b421de0c397bd3..83d230274f6f292 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java @@ -139,7 +139,7 @@ public void setUp() throws UserException { @Test public void testNormal() throws AnalysisException, UserException { - ShowDataStmt stmt = new ShowDataStmt(null, null, null); + ShowDataStmt stmt = new ShowDataStmt(null, null, null, false); stmt.analyze(analyzer); Assert.assertEquals("SHOW DATA", stmt.toString()); Assert.assertEquals(4, stmt.getMetaData().getColumnCount()); @@ -151,7 +151,7 @@ public void testNormal() throws AnalysisException, UserException { OrderByElement orderByElementTwo = new OrderByElement(slotRefTwo, false, false); stmt = new ShowDataStmt(new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "testDb", "test_tbl"), - Arrays.asList(orderByElementOne, orderByElementTwo), null); + Arrays.asList(orderByElementOne, orderByElementTwo), null, false); stmt.analyze(analyzer); Assert.assertEquals( "SHOW DATA FROM `testDb`.`test_tbl` ORDER BY `ReplicaCount` DESC, `Size` DESC", @@ -159,7 +159,7 @@ public void testNormal() throws AnalysisException, UserException { Assert.assertEquals(6, stmt.getMetaData().getColumnCount()); Assert.assertEquals(true, stmt.hasTable()); - stmt = new ShowDataStmt(null, Arrays.asList(orderByElementOne, orderByElementTwo), null); + stmt = new ShowDataStmt(null, Arrays.asList(orderByElementOne, orderByElementTwo), null, false); stmt.analyze(analyzer); Assert.assertEquals("SHOW DATA ORDER BY `ReplicaCount` DESC, `Size` DESC", stmt.toString()); diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index 381075074335c53..c113868a2c32862 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -544,6 +544,10 @@ message TabletCompactionJobPB { optional int64 delete_bitmap_lock_initiator = 24; optional int64 full_compaction_cnt = 25; // prepare optional bool check_input_versions_range = 26; + optional int64 index_size_input_rowsets = 27; + optional int64 segment_size_input_rowsets = 28; + optional int64 index_size_output_rowsets = 29; + optional int64 segment_size_output_rowsets = 30; } message TabletSchemaChangeJobPB { @@ -563,6 +567,8 @@ message TabletSchemaChangeJobPB { optional int64 output_cumulative_point = 12; optional bool is_inverted_index_change = 13 [default = false]; optional int64 delete_bitmap_lock_initiator = 14; + optional int64 index_size_output_rowsets = 15; + optional int64 segment_size_output_rowsets = 16; } message TabletJobInfoPB { @@ -576,7 +582,7 @@ message TabletJobInfoPB { //============================================================================== message TabletStatsPB { optional TabletIndexPB idx = 1; - optional int64 data_size = 2; + optional int64 data_size = 2; // data_size = index_size + segment_size optional int64 num_rows = 3; optional int64 num_rowsets = 4; optional int64 num_segments = 5; @@ -588,6 +594,8 @@ message TabletStatsPB { optional int64 last_cumu_compaction_time_ms = 11; optional int64 full_compaction_cnt = 12; optional int64 last_full_compaction_time_ms = 13; + optional int64 index_size = 14; + optional int64 segment_size = 15; } message ObjectFilePB { diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index 7f073b2b71095ae..aed248adfe891a3 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -32,13 +32,18 @@ struct TExportTaskRequest { struct TTabletStat { 1: required i64 tablet_id - // local data size + // local data size = local inverted index file size + local segment file size 2: optional i64 data_size 3: optional i64 row_count 4: optional i64 total_version_count + // remote data size = remote inverted index file size + remote segment file size 5: optional i64 remote_data_size 6: optional i64 visible_version_count 7: optional i64 visible_version + 8: optional i64 local_index_size = 0 // .idx + 9: optional i64 local_segment_size = 0 // .dat + 10: optional i64 remote_index_size = 0 // .idx + 11: optional i64 remote_segment_size = 0 // .dat } struct TTabletStatResult { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index e2af8937425d0c7..246cc9995629343 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -350,6 +350,7 @@ struct TTableStatus { 11: optional i64 rows; 12: optional i64 avg_row_length 13: optional i64 data_length; + 14: optional i64 index_length; } struct TListTableStatusResult { diff --git a/gensrc/thrift/MasterService.thrift b/gensrc/thrift/MasterService.thrift index 9d8cd9111ba5c1e..a045fa3adcb5bd8 100644 --- a/gensrc/thrift/MasterService.thrift +++ b/gensrc/thrift/MasterService.thrift @@ -47,6 +47,10 @@ struct TTabletInfo { 19: optional i64 cooldown_term 20: optional Types.TUniqueId cooldown_meta_id 21: optional i64 visible_version_count + 22: optional i64 local_index_size = 0 // .idx + 23: optional i64 local_segment_size = 0 // .dat + 24: optional i64 remote_index_size = 0 // .idx + 25: optional i64 remote_segment_size = 0 // .dat // For cloud 1000: optional bool is_persistent diff --git a/regression-test/data/nereids_p0/system/test_query_sys_tables.out b/regression-test/data/nereids_p0/system/test_query_sys_tables.out index a43229a87adeada..16b71543a113025 100644 --- a/regression-test/data/nereids_p0/system/test_query_sys_tables.out +++ b/regression-test/data/nereids_p0/system/test_query_sys_tables.out @@ -25,9 +25,9 @@ internal test_query_sys_db_5 \N internal test_query_sys_db_6 \N -- !tables -- -internal test_query_sys_tb_4 BASE TABLE 0 \N \N -internal test_query_sys_tb_5 BASE TABLE 0 \N \N -internal test_query_sys_tb_6 BASE TABLE 0 \N \N +internal test_query_sys_tb_4 BASE TABLE 0 \N 0 +internal test_query_sys_tb_5 BASE TABLE 0 \N 0 +internal test_query_sys_tb_6 BASE TABLE 0 \N 0 -- !session_variables -- wait_timeout 30000 diff --git a/regression-test/data/query_p0/system/test_query_sys_tables.out b/regression-test/data/query_p0/system/test_query_sys_tables.out index cdd89914994c63d..0100314ef799a1f 100644 --- a/regression-test/data/query_p0/system/test_query_sys_tables.out +++ b/regression-test/data/query_p0/system/test_query_sys_tables.out @@ -158,9 +158,9 @@ internal test_query_sys_db_2 \N internal test_query_sys_db_3 \N -- !tables -- -internal test_query_sys_tb_1 BASE TABLE 0 \N \N -internal test_query_sys_tb_2 BASE TABLE 0 \N \N -internal test_query_sys_tb_3 BASE TABLE 0 \N \N +internal test_query_sys_tb_1 BASE TABLE 0 \N 0 +internal test_query_sys_tb_2 BASE TABLE 0 \N 0 +internal test_query_sys_tb_3 BASE TABLE 0 \N 0 -- !session_variables -- wait_timeout 30000 diff --git a/regression-test/suites/inverted_index_p1/show_data/test_show_index_data.groovy b/regression-test/suites/inverted_index_p1/show_data/test_show_index_data.groovy new file mode 100644 index 000000000000000..671fe907782f2c1 --- /dev/null +++ b/regression-test/suites/inverted_index_p1/show_data/test_show_index_data.groovy @@ -0,0 +1,399 @@ +// 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. + +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility + +enum FileSizeChange { + LARGER, + SMALLER, + UNCHANGED +} + +suite("test_show_index_data", "p1") { + + def show_table_name = "test_show_index_data" + + def load_json_data = { table_name, file_name -> + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'read_json_by_line', 'true' + set 'format', 'json' + set 'max_filter_ratio', '0.1' + file file_name // import json file + time 10000 // limit inflight 10s + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + logger.info("Stream load ${file_name} result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + // assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + + + def convert_size = { str -> + def matcher = str =~ /(\d+(\.\d+)?)\s*(KB|MB|GB|TB|B)/ + if (matcher) { + def value = matcher[0][1] as double + def unit = matcher[0][3] + logger.info("value is: " + value + " unit is: " + unit) + def result = 0.0 + switch (unit) { + case 'KB': + result = value * 1024 + break + case 'MB': + result = value * 1024 * 1024 + break + case 'GB': + result = value * 1024 * 1024 * 1024 + break + case 'B': + result = value + break + default: + throw new IllegalArgumentException("Unknown unit: $unit") + } + + return result + } else { + return 0 + } + } + + sql "DROP TABLE IF EXISTS ${show_table_name}" + sql """ + CREATE TABLE ${show_table_name} ( + `@timestamp` int(11) NULL, + `clientip` varchar(20) NULL, + `request` varchar(500) NULL, + `status` int NULL, + `size` int NULL, + INDEX clientip_idx (`clientip`) USING INVERTED COMMENT '', + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "unicode") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`, `clientip`) + DISTRIBUTED BY HASH(`@timestamp`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "compaction_policy" = "time_series", + "time_series_compaction_file_count_threshold" = "20", + "disable_auto_compaction" = "true" + ); + """ + + def compaction = { + + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + + for (def tablet in tablets) { + int beforeSegmentCount = 0 + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + beforeSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + assertEquals(beforeSegmentCount, 10) + } + + // trigger compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + assertEquals("success", compactJson.status.toLowerCase()) + } + + // wait for all compactions done + for (def tablet in tablets) { + Awaitility.await().atMost(30, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("compaction task for this tablet is not running", compactionStatus.msg.toLowerCase()) + }); + } + + + for (def tablet in tablets) { + int afterSegmentCount = 0 + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + logger.info("rowset is: " + rowset) + afterSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + assertEquals(afterSegmentCount, 1) + } + } + + double localIndexSize = 0 + double localSegmentSize = 0 + + def check_size_equal = { double result1, double result2 -> + double tolerance = 0.1 * Math.max(result1, result2); + return Math.abs(result1 - result2) <= tolerance; + } + + def check_show_data = { FileSizeChange expect_idx, FileSizeChange expect_data -> + Thread.sleep(90000) + Awaitility.await().atMost(5, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(10000) + def result = sql """ show data all;""" + logger.info("show data all; result is: ${result}") + def currentLocalIndexSize = convert_size.call(result[0][4]) + def currentSegmentIndexSize = convert_size.call(result[0][3]) + + if (expect_idx == FileSizeChange.LARGER) { + assertTrue(currentLocalIndexSize > localIndexSize) + } else if (expect_idx == FileSizeChange.SMALLER) { + assertTrue(currentLocalIndexSize < localIndexSize) + } else { + assertTrue(check_size_equal(currentLocalIndexSize, localIndexSize)) + } + + if (expect_data == FileSizeChange.LARGER) { + assertTrue(currentSegmentIndexSize > localSegmentSize) + } else if (expect_data == FileSizeChange.SMALLER) { + assertTrue(currentSegmentIndexSize < localSegmentSize) + } else { + assertTrue(check_size_equal(currentSegmentIndexSize, localSegmentSize)) + } + assertTrue(currentLocalIndexSize != 0) + assertTrue(currentSegmentIndexSize != 0) + localIndexSize = currentLocalIndexSize + localSegmentSize = currentSegmentIndexSize + + def result2 = sql """ select * from information_schema.tables where TABLE_NAME = '${show_table_name}' """ + logger.info("result 2 is: ${result2}") + def currentLocalIndexSize2 = result2[0][11] as double + def currentSegmentIndexSize2 = result2[0][9] as double + logger.info("currentLocalIndexSize2 is: ${currentLocalIndexSize2}, currentSegmentIndexSize2 is: ${currentSegmentIndexSize2}") + assertTrue(check_size_equal(currentLocalIndexSize, currentLocalIndexSize2)) + assertTrue(check_size_equal(currentSegmentIndexSize, currentSegmentIndexSize2)) + logger.info("show data all localIndexSize is: " + localIndexSize) + logger.info("show data all localSegmentSize is: " + localSegmentSize) + }); + } + + def schema_change = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ alter table ${show_table_name} drop column clientip""" + Awaitility.await().atMost(30, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + def build_index = { + + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ ALTER TABLE ${show_table_name} ADD INDEX status_idx (status) using inverted; """ + if (!isCloudMode()) { + sql """ build index status_idx on ${show_table_name}""" + } + Awaitility.await().atMost(30, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + def drop_index = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ DROP INDEX status_idx on ${show_table_name}""" + Awaitility.await().atMost(30, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + // 1. load data + def executor = Executors.newFixedThreadPool(5) + (1..10).each { i -> + executor.submit { + def fileName = "documents-" + i + ".json" + load_json_data.call(show_table_name, """${getS3Url()}/regression/inverted_index_cases/httplogs/${fileName}""") + } + } + executor.shutdown() + executor.awaitTermination(30, TimeUnit.MINUTES) + + // 2. check show data + check_show_data.call(FileSizeChange.LARGER, FileSizeChange.LARGER) + + // 3. compaction + compaction.call() + + // 4. check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.SMALLER) + + // 5. schema change + schema_change.call() + + // 6.check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.SMALLER) + + // 7. build index + build_index.call() + + // 8.check show data + check_show_data.call(FileSizeChange.LARGER, FileSizeChange.UNCHANGED) + + // 9. drop index + drop_index.call() + + // 10.check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.UNCHANGED) +} diff --git a/regression-test/suites/inverted_index_p2/show_data/test_show_index_data_p2.groovy b/regression-test/suites/inverted_index_p2/show_data/test_show_index_data_p2.groovy new file mode 100644 index 000000000000000..2839a8a47b04aad --- /dev/null +++ b/regression-test/suites/inverted_index_p2/show_data/test_show_index_data_p2.groovy @@ -0,0 +1,397 @@ +// 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. + +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility + +enum FileSizeChange { + LARGER, + SMALLER, + UNCHANGED +} + +suite("test_show_index_data_p2", "p2") { + + def show_table_name = "test_show_index_data_p2" + + def load_json_data = { table_name, file_name -> + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'read_json_by_line', 'true' + set 'format', 'json' + set 'max_filter_ratio', '0.1' + file file_name // import json file + time 10000 // limit inflight 10s + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + logger.info("Stream load ${file_name} result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + // assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + + + def convert_size = { str -> + def matcher = str =~ /(\d+(\.\d+)?)\s*(KB|MB|GB|TB|B)/ + if (matcher) { + def value = matcher[0][1] as double + def unit = matcher[0][3] + logger.info("value is: " + value + " unit is: " + unit) + def result = 0.0 + switch (unit) { + case 'KB': + result = value * 1024 + break + case 'MB': + result = value * 1024 * 1024 + break + case 'GB': + result = value * 1024 * 1024 * 1024 + break + case 'B': + result = value + break + default: + throw new IllegalArgumentException("Unknown unit: $unit") + } + + return result + } else { + return 0 + } + } + + sql "DROP TABLE IF EXISTS ${show_table_name}" + sql """ + CREATE TABLE ${show_table_name} ( + `@timestamp` int(11) NULL, + `clientip` varchar(20) NULL, + `request` varchar(500) NULL, + `status` int NULL, + `size` int NULL, + INDEX clientip_idx (`clientip`) USING INVERTED COMMENT '', + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "unicode") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`, `clientip`) + DISTRIBUTED BY HASH(`@timestamp`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "compaction_policy" = "time_series", + "time_series_compaction_file_count_threshold" = "20", + "disable_auto_compaction" = "true" + ); + """ + + def compaction = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + int beforeSegmentCount = 0 + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + beforeSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + assertEquals(beforeSegmentCount, 110) + } + + // trigger compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + assertEquals("success", compactJson.status.toLowerCase()) + } + + // wait for all compactions done + for (def tablet in tablets) { + Awaitility.await().atMost(60, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("compaction task for this tablet is not running", compactionStatus.msg.toLowerCase()) + }); + } + + for (def tablet in tablets) { + int afterSegmentCount = 0 + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + logger.info("rowset is: " + rowset) + afterSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + assertEquals(afterSegmentCount, 1) + } + + } + + double localIndexSize = 0 + double localSegmentSize = 0 + + def check_size_equal = { double result1, double result2 -> + double tolerance = 0.1 * Math.max(result1, result2); + return Math.abs(result1 - result2) <= tolerance; + } + + def check_show_data = { FileSizeChange expect_idx, FileSizeChange expect_data -> + Thread.sleep(90000) + Awaitility.await().atMost(10, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(10000) + def result = sql """ show data all;""" + logger.info("show data all; result is: ${result}") + def currentLocalIndexSize = convert_size.call(result[0][4]) + def currentSegmentIndexSize = convert_size.call(result[0][3]) + + if (expect_idx == FileSizeChange.LARGER) { + assertTrue(currentLocalIndexSize > localIndexSize) + } else if (expect_idx == FileSizeChange.SMALLER) { + assertTrue(currentLocalIndexSize < localIndexSize) + } else { + assertTrue(check_size_equal(currentLocalIndexSize, localIndexSize)) + } + + if (expect_data == FileSizeChange.LARGER) { + assertTrue(currentSegmentIndexSize > localSegmentSize) + } else if (expect_data == FileSizeChange.SMALLER) { + assertTrue(currentSegmentIndexSize < localSegmentSize) + } else { + assertTrue(check_size_equal(currentSegmentIndexSize, localSegmentSize)) + } + + assertTrue(currentLocalIndexSize != 0) + assertTrue(currentSegmentIndexSize != 0) + localIndexSize = currentLocalIndexSize + localSegmentSize = currentSegmentIndexSize + + def result2 = sql """ select * from information_schema.tables where TABLE_NAME = '${show_table_name}' """ + logger.info("result 2 is: ${result2}") + def currentLocalIndexSize2 = result2[0][11] as double + def currentSegmentIndexSize2 = result2[0][9] as double + logger.info("currentLocalIndexSize2 is: ${currentLocalIndexSize2}, currentSegmentIndexSize2 is: ${currentSegmentIndexSize2}") + assertTrue(check_size_equal(currentLocalIndexSize, currentLocalIndexSize2)) + assertTrue(check_size_equal(currentSegmentIndexSize, currentSegmentIndexSize2)) + logger.info("show data all localIndexSize is: " + localIndexSize) + logger.info("show data all localSegmentSize is: " + localSegmentSize) + }); + } + + def schema_change = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ alter table ${show_table_name} drop column clientip""" + Awaitility.await().atMost(60, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + def build_index = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ ALTER TABLE ${show_table_name} ADD INDEX status_idx (status) using inverted; """ + if (!isCloudMode()) { + sql """ build index status_idx on ${show_table_name}""" + } + Awaitility.await().atMost(60, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + def drop_index = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ DROP INDEX status_idx on ${show_table_name}""" + Awaitility.await().atMost(60, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + // 1. load data + def executor = Executors.newFixedThreadPool(5) + (1..110).each { i -> + executor.submit { + def fileName = "documents-" + i + ".json" + load_json_data.call(show_table_name, """${getS3Url()}/regression/inverted_index_cases/httplogs/${fileName}""") + } + } + executor.shutdown() + executor.awaitTermination(60, TimeUnit.MINUTES) + + // 2. check show data + check_show_data.call(FileSizeChange.LARGER, FileSizeChange.LARGER) + + // 3. compaction + compaction.call() + + // 4. check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.LARGER) + + // 5. schema change + schema_change.call() + + // 6.check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.SMALLER) + + // 7. build index + build_index.call() + + // 8.check show data + check_show_data.call(FileSizeChange.LARGER, FileSizeChange.UNCHANGED) + + // 9. drop index + drop_index.call() + + // 10.check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.UNCHANGED) +} From b2b0ac76c6c0e8533673486ba6db7b04b9ec4627 Mon Sep 17 00:00:00 2001 From: airborne12 Date: Mon, 2 Dec 2024 14:54:22 +0800 Subject: [PATCH 20/31] [feat](test) add bloomfilter ut case (#44351) add bloomfilter index UT test case for BE --- be/src/olap/bloom_filter.hpp | 272 ---------- .../segment_v2/block_split_bloom_filter.h | 1 - be/src/olap/rowset/segment_v2/bloom_filter.h | 2 +- .../segment_v2/bloom_filter_index_reader.cpp | 1 + .../segment_v2/bloom_filter_index_writer.cpp | 29 +- be/test/olap/bloom_filter_test.cpp | 160 ------ .../bloom_filter_index_reader_writer_test.cpp | 485 +++++++++++++++++- 7 files changed, 488 insertions(+), 462 deletions(-) delete mode 100644 be/src/olap/bloom_filter.hpp delete mode 100644 be/test/olap/bloom_filter_test.cpp diff --git a/be/src/olap/bloom_filter.hpp b/be/src/olap/bloom_filter.hpp deleted file mode 100644 index 5c7cb5f9e6419f3..000000000000000 --- a/be/src/olap/bloom_filter.hpp +++ /dev/null @@ -1,272 +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. - -#ifndef DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_HPP -#define DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_HPP - -#include - -#include -#include - -#include "olap/olap_define.h" -#include "olap/utils.h" -#include "util/hash_util.hpp" - -namespace doris { - -static const uint64_t DEFAULT_SEED = 104729; -static const uint64_t BLOOM_FILTER_NULL_HASHCODE = 2862933555777941757ULL; - -struct BloomFilterIndexHeader { - uint64_t block_count; - BloomFilterIndexHeader() : block_count(0) {} -} __attribute__((packed)); - -// Bare metal bit set implementation. For performance reasons, this implementation does not -// check for index bounds nor expand the bit set if the specified index is greater than the size. -class BitSet { -public: - BitSet() : _data(nullptr), _data_len(0) {} - - ~BitSet() { SAFE_DELETE_ARRAY(_data); } - - // Init BitSet with given bit_num, which will align up to uint64_t - bool init(uint32_t bit_num) { - if (bit_num <= 0) { - return false; - } - - _data_len = (bit_num + sizeof(uint64_t) * 8 - 1) / (sizeof(uint64_t) * 8); - _data = new (std::nothrow) uint64_t[_data_len]; - if (_data == nullptr) { - return false; - } - - memset(_data, 0, _data_len * sizeof(uint64_t)); - return true; - } - - // Init BitSet with given buffer - bool init(uint64_t* data, uint32_t data_len) { - _data = data; - _data_len = data_len; - return true; - } - - // Set the bit specified by param, note that uint64_t type contains 2^6 bits - void set(uint32_t index) { _data[index >> 6] |= 1L << (index % 64); } - - // Return true if the bit specified by param is set - bool get(uint32_t index) const { return (_data[index >> 6] & (1L << (index % 64))) != 0; } - - // Merge with another BitSet by byte, return false when the length is not equal - bool merge(const BitSet& set) { - if (_data_len != set.data_len()) { - return false; - } - - for (uint32_t i = 0; i < _data_len; ++i) { - _data[i] |= set.data()[i]; - } - - return true; - } - - // Convert BitSet to string to convenient debug and test - std::string to_string() const { - uint32_t bit_num = _data_len * sizeof(uint64_t) * 8; - std::string str(bit_num, '0'); - for (uint32_t i = 0; i < bit_num; ++i) { - if ((_data[i >> 6] & (1L << i)) != 0) { - str[i] = '1'; - } - } - - return str; - } - - uint64_t* data() const { return _data; } - - uint32_t data_len() const { return _data_len; } - - uint32_t bit_num() const { return _data_len * sizeof(uint64_t) * 8; } - - void clear() { memset(_data, 0, _data_len * sizeof(uint64_t)); } - - void reset() { - _data = NULL; - _data_len = 0; - } - -private: - uint64_t* _data; - uint32_t _data_len; -}; - -class BloomFilter { -public: - BloomFilter() : _bit_num(0), _hash_function_num(0) {} - ~BloomFilter() {} - - // Create BloomFilter with given entry num and fpp, which is used for loading data - bool init(int64_t expected_entries, double fpp) { - uint32_t bit_num = _optimal_bit_num(expected_entries, fpp); - if (!_bit_set.init(bit_num)) { - return false; - } - - _bit_num = _bit_set.bit_num(); - _hash_function_num = _optimal_hash_function_num(expected_entries, _bit_num); - return true; - } - - // Create BloomFilter with given entry num and default fpp - bool init(int64_t expected_entries) { - return this->init(expected_entries, BLOOM_FILTER_DEFAULT_FPP); - } - - // Init BloomFilter with given buffer, which is used for query - bool init(uint64_t* data, uint32_t len, uint32_t hash_function_num) { - _bit_num = sizeof(uint64_t) * 8 * len; - _hash_function_num = hash_function_num; - return _bit_set.init(data, len); - } - - // Compute hash value of given buffer and add to BloomFilter - void add_bytes(const char* buf, uint32_t len) { - uint64_t hash = buf == nullptr ? BLOOM_FILTER_NULL_HASHCODE - : HashUtil::hash64(buf, len, DEFAULT_SEED); - add_hash(hash); - } - - // Generate multiple hash value according to following rule: - // new_hash_value = hash_high_part + (i * hash_low_part) - void add_hash(uint64_t hash) { - uint32_t hash1 = (uint32_t)hash; - uint32_t hash2 = (uint32_t)(hash >> 32); - - for (uint32_t i = 0; i < _hash_function_num; ++i) { - uint64_t combine_hash = hash1 + hash2 * i; - uint32_t index = combine_hash % _bit_num; - _bit_set.set(index); - } - } - - // Compute hash value of given buffer and verify whether exist in BloomFilter - bool test_bytes(const char* buf, uint32_t len) const { - uint64_t hash = buf == nullptr ? BLOOM_FILTER_NULL_HASHCODE - : HashUtil::hash64(buf, len, DEFAULT_SEED); - return test_hash(hash); - } - - // Verify whether hash value in BloomFilter - bool test_hash(uint64_t hash) const { - uint32_t hash1 = (uint32_t)hash; - uint32_t hash2 = (uint32_t)(hash >> 32); - - for (uint32_t i = 0; i < _hash_function_num; ++i) { - uint64_t combine_hash = hash1 + hash2 * i; - uint32_t index = combine_hash % _bit_num; - if (!_bit_set.get(index)) { - return false; - } - } - - return true; - } - - // Merge with another BloomFilter, return false when the length - // and hash function number is not equal - bool merge(const BloomFilter& that) { - if (_bit_num == that.bit_num() && _hash_function_num == that.hash_function_num()) { - _bit_set.merge(that.bit_set()); - return true; - } - - return false; - } - - void clear() { _bit_set.clear(); } - - void reset() { - _bit_num = 0; - _hash_function_num = 0; - _bit_set.reset(); - } - - uint32_t bit_num() const { return _bit_num; } - - uint32_t hash_function_num() const { return _hash_function_num; } - - const BitSet& bit_set() const { return _bit_set; } - - uint64_t* bit_set_data() const { return _bit_set.data(); } - - uint32_t bit_set_data_len() const { return _bit_set.data_len(); } - - // Convert BloomFilter to string to convenient debug and test - std::string to_string() const { - std::stringstream bf_stream; - bf_stream << "bit_num:" << _bit_num << " hash_function_num:" << _hash_function_num - << " bit_set:" << _bit_set.to_string(); - return bf_stream.str(); - } - - // Get points which set by given buffer in the BitSet - std::string get_bytes_points_string(const char* buf, uint32_t len) const { - uint64_t hash = buf == nullptr ? BLOOM_FILTER_NULL_HASHCODE - : HashUtil::hash64(buf, len, DEFAULT_SEED); - uint32_t hash1 = (uint32_t)hash; - uint32_t hash2 = (uint32_t)(hash >> 32); - - std::stringstream stream; - for (uint32_t i = 0; i < _hash_function_num; ++i) { - if (i != 0) { - stream << "-"; - } - - uint32_t combine_hash = hash1 + hash2 * i; - uint32_t index = combine_hash % _bit_num; - stream << index; - } - - return stream.str(); - } - -private: - // Compute the optimal bit number according to the following rule: - // m = -n * ln(fpp) / (ln(2) ^ 2) - uint32_t _optimal_bit_num(int64_t n, double fpp) { - return (uint32_t)(-n * log(fpp) / (log(2) * log(2))); - } - - // Compute the optimal hash function number according to the following rule: - // k = round(m * ln(2) / n) - uint32_t _optimal_hash_function_num(int64_t n, uint32_t m) { - uint32_t k = (uint32_t)round(m * log(2) / n); - return k > 1 ? k : 1; - } - - BitSet _bit_set; - uint32_t _bit_num; - uint32_t _hash_function_num; -}; - -} // namespace doris - -#endif // DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_HPP diff --git a/be/src/olap/rowset/segment_v2/block_split_bloom_filter.h b/be/src/olap/rowset/segment_v2/block_split_bloom_filter.h index f68ddd7e74bfc5f..8dc470d9da4f88f 100644 --- a/be/src/olap/rowset/segment_v2/block_split_bloom_filter.h +++ b/be/src/olap/rowset/segment_v2/block_split_bloom_filter.h @@ -34,7 +34,6 @@ class BlockSplitBloomFilter : public BloomFilter { void add_hash(uint64_t hash) override; bool test_hash(uint64_t hash) const override; - bool contains(const BloomFilter&) const override { return true; } private: // Bytes in a tiny Bloom filter block. diff --git a/be/src/olap/rowset/segment_v2/bloom_filter.h b/be/src/olap/rowset/segment_v2/bloom_filter.h index a7845d1ca36704a..4f4adf0fd12283d 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter.h @@ -186,7 +186,7 @@ class BloomFilter { /// Checks if this contains everything from another bloom filter. /// Bloom filters must have equal size and seed. - virtual bool contains(const BloomFilter& bf_) const = 0; + virtual bool contains(const BloomFilter& bf_) const { return true; }; virtual char* data() const { return _data; } diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp index 609d21ce4f5c224..8c63c25d20acee8 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp @@ -70,6 +70,7 @@ Status BloomFilterIndexIterator::read_bloom_filter(rowid_t ordinal, auto column = data_type->create_column(); RETURN_IF_ERROR(_bloom_filter_iter.seek_to_ordinal(ordinal)); + DCHECK(current_bloom_filter_index() == ordinal); size_t num_read = num_to_read; RETURN_IF_ERROR(_bloom_filter_iter.next_batch(&num_read, column)); DCHECK(num_to_read == num_read); diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp index edc6102703f4927..3f9fb94df0a844d 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include #include @@ -68,15 +69,12 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { explicit BloomFilterIndexWriterImpl(const BloomFilterOptions& bf_options, const TypeInfo* type_info) - : _bf_options(bf_options), - _type_info(type_info), - _has_null(false), - _bf_buffer_size(0) {} + : _bf_options(bf_options), _type_info(type_info) {} ~BloomFilterIndexWriterImpl() override = default; Status add_values(const void* values, size_t count) override { - const CppType* v = (const CppType*)values; + const auto* v = (const CppType*)values; for (int i = 0; i < count; ++i) { if (_values.find(*v) == _values.end()) { if constexpr (_is_slice_type()) { @@ -105,7 +103,7 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { bf->set_has_null(_has_null); for (auto& v : _values) { if constexpr (_is_slice_type()) { - Slice* s = (Slice*)&v; + auto* s = (Slice*)&v; bf->add_bytes(s->data, s->size); } else { bf->add_bytes((char*)&v, sizeof(CppType)); @@ -160,11 +158,11 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { static constexpr bool _is_int128() { return field_type == FieldType::OLAP_FIELD_TYPE_LARGEINT; } private: - BloomFilterOptions _bf_options; - const TypeInfo* _type_info; + BloomFilterOptions _bf_options {}; + const TypeInfo* _type_info = nullptr; vectorized::Arena _arena; - bool _has_null; - uint64_t _bf_buffer_size; + bool _has_null = false; + uint64_t _bf_buffer_size = 0; // distinct values ValueDict _values; std::vector> _bfs; @@ -173,7 +171,7 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { } // namespace Status PrimaryKeyBloomFilterIndexWriterImpl::add_values(const void* values, size_t count) { - const Slice* v = (const Slice*)values; + const auto* v = (const Slice*)values; for (int i = 0; i < count; ++i) { Slice new_value; RETURN_IF_CATCH_EXCEPTION(_type_info->deep_copy(&new_value, v, &_arena)); @@ -189,7 +187,7 @@ Status PrimaryKeyBloomFilterIndexWriterImpl::flush() { RETURN_IF_ERROR(bf->init(_values.size(), _bf_options.fpp, _bf_options.strategy)); bf->set_has_null(_has_null); for (auto& v : _values) { - Slice* s = (Slice*)&v; + auto* s = (Slice*)&v; bf->add_bytes(s->data, s->size); } _bf_buffer_size += bf->size(); @@ -205,7 +203,7 @@ Status PrimaryKeyBloomFilterIndexWriterImpl::flush() { Status PrimaryKeyBloomFilterIndexWriterImpl::finish(io::FileWriter* file_writer, ColumnIndexMetaPB* index_meta) { - if (_values.size() > 0) { + if (!_values.empty()) { RETURN_IF_ERROR(flush()); } index_meta->set_type(BLOOM_FILTER_INDEX); @@ -246,7 +244,7 @@ NGramBloomFilterIndexWriterImpl::NGramBloomFilterIndexWriterImpl( } Status NGramBloomFilterIndexWriterImpl::add_values(const void* values, size_t count) { - const Slice* src = reinterpret_cast(values); + const auto* src = reinterpret_cast(values); for (int i = 0; i < count; ++i, ++src) { if (src->size < _gram_size) { continue; @@ -339,7 +337,8 @@ Status NGramBloomFilterIndexWriterImpl::create(const BloomFilterOptions& bf_opti case FieldType::OLAP_FIELD_TYPE_CHAR: case FieldType::OLAP_FIELD_TYPE_VARCHAR: case FieldType::OLAP_FIELD_TYPE_STRING: - res->reset(new NGramBloomFilterIndexWriterImpl(bf_options, gram_size, gram_bf_size)); + *res = std::make_unique(bf_options, gram_size, + gram_bf_size); break; default: return Status::NotSupported("unsupported type for ngram bloom filter index:{}", diff --git a/be/test/olap/bloom_filter_test.cpp b/be/test/olap/bloom_filter_test.cpp deleted file mode 100644 index 32bf88e2c4d6841..000000000000000 --- a/be/test/olap/bloom_filter_test.cpp +++ /dev/null @@ -1,160 +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. - -#include "olap/bloom_filter.hpp" - -#include -#include -#include - -#include - -#include "gtest/gtest_pred_impl.h" - -using std::string; - -namespace doris { - -class TestBloomFilter : public testing::Test { -public: - virtual ~TestBloomFilter() {} - - virtual void SetUp() {} - virtual void TearDown() {} -}; - -// Init BloomFilter with different item number and fpp, -// and verify bit_num and hash_function_num calculated by BloomFilter -TEST_F(TestBloomFilter, init_bloom_filter) { - { - BloomFilter bf; - bf.init(1024); - EXPECT_EQ(6400, bf.bit_num()); - EXPECT_EQ(4, bf.hash_function_num()); - } - - { - BloomFilter bf; - bf.init(1024, 0.01); - EXPECT_EQ(9856, bf.bit_num()); - EXPECT_EQ(7, bf.hash_function_num()); - } - - { - BloomFilter bf; - bf.init(10240, 0.1); - EXPECT_EQ(49088, bf.bit_num()); - EXPECT_EQ(3, bf.hash_function_num()); - } - - { - BloomFilter bf; - uint32_t data_len = 100; - uint32_t hash_function_num = 4; - uint64_t* data = new uint64_t[data_len]; - - bf.init(data, data_len, hash_function_num); - EXPECT_EQ(6400, bf.bit_num()); - EXPECT_EQ(4, bf.hash_function_num()); - EXPECT_EQ(data, bf.bit_set_data()); - - bf.reset(); - EXPECT_EQ(0, bf.bit_num()); - EXPECT_EQ(0, bf.hash_function_num()); - EXPECT_EQ(nullptr, bf.bit_set_data()); - delete[] data; - } -} - -// Add different buffer to BloomFilter and verify existence -TEST_F(TestBloomFilter, add_and_test_bytes) { - string bytes; - BloomFilter bf; - bf.init(1024); - - bf.add_bytes(nullptr, 0); - EXPECT_TRUE(bf.test_bytes(nullptr, 0)); - - bytes = "hello"; - bf.add_bytes(bytes.c_str(), bytes.size()); - EXPECT_TRUE(bf.test_bytes(bytes.c_str(), bytes.size())); - - bytes = "doris"; - bf.add_bytes(bytes.c_str(), bytes.size()); - EXPECT_TRUE(bf.test_bytes(bytes.c_str(), bytes.size())); - - BloomFilter new_bf; - new_bf.init(1024); - - bytes = "world"; - new_bf.add_bytes(bytes.c_str(), bytes.size()); - EXPECT_TRUE(bf.merge(new_bf)); - EXPECT_TRUE(bf.test_bytes(bytes.c_str(), bytes.size())); -} - -// Print bloom filter buffer and points of specified string -TEST_F(TestBloomFilter, bloom_filter_info) { - string bytes; - BloomFilter bf; - bf.init(8, 0.1); - - bytes = "doris"; - bf.add_bytes(bytes.c_str(), bytes.size()); - string buffer_expect = - "bit_num:64 hash_function_num:6 " - "bit_set:0000100000000000100000010000000000010000001000000000000000000100"; - string buffer = bf.to_string(); - EXPECT_TRUE(buffer_expect == buffer); - - string points_expect = "4-23-42-61-16-35"; - string points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - EXPECT_TRUE(points_expect == points); - - bytes = "a"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "ab"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "b"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "ba"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "c"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "bc"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "ac"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "abc"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; -} - -} // namespace doris diff --git a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp index 69cb343f04bf915..813952595efcfd6 100644 --- a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp +++ b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp @@ -32,6 +32,7 @@ #include "io/fs/file_writer.h" #include "io/fs/local_file_system.h" #include "olap/decimal12.h" +#include "olap/itoken_extractor.h" #include "olap/olap_common.h" #include "olap/rowset/segment_v2/bloom_filter.h" #include "olap/rowset/segment_v2/bloom_filter_index_reader.h" @@ -62,18 +63,19 @@ template Status write_bloom_filter_index_file(const std::string& file_name, const void* values, size_t value_count, size_t null_count, ColumnIndexMetaPB* index_meta, - bool use_primary_key_bloom_filter = false) { + bool use_primary_key_bloom_filter = false, double fpp = 0.05) { const auto* type_info = get_scalar_type_info(); using CppType = typename CppTypeTraits::CppType; std::string fname = dname + "/" + file_name; auto fs = io::global_local_filesystem(); { + size_t expect_size = 0; io::FileWriterPtr file_writer; RETURN_IF_ERROR(fs->create_file(fname, &file_writer)); std::unique_ptr bloom_filter_index_writer; BloomFilterOptions bf_options; - + bf_options.fpp = fpp; // Set the expected FPP if (use_primary_key_bloom_filter) { RETURN_IF_ERROR(PrimaryKeyBloomFilterIndexWriterImpl::create( bf_options, type_info, &bloom_filter_index_writer)); @@ -91,12 +93,26 @@ Status write_bloom_filter_index_file(const std::string& file_name, const void* v bloom_filter_index_writer->add_nulls(null_count); } RETURN_IF_ERROR(bloom_filter_index_writer->flush()); + auto bf_size = BloomFilter::optimal_bit_num(num, fpp) / 8; + expect_size += bf_size + 1; i += 1024; } + if (value_count == 3072) { + RETURN_IF_ERROR(bloom_filter_index_writer->add_values(vals + 3071, 1)); + auto bf_size = BloomFilter::optimal_bit_num(1, fpp) / 8; + expect_size += bf_size + 1; + } RETURN_IF_ERROR(bloom_filter_index_writer->finish(file_writer.get(), index_meta)); EXPECT_TRUE(file_writer->close().ok()); EXPECT_EQ(BLOOM_FILTER_INDEX, index_meta->type()); EXPECT_EQ(bf_options.strategy, index_meta->bloom_filter_index().hash_strategy()); + if constexpr (!field_is_slice_type(type)) { + EXPECT_EQ(expect_size, bloom_filter_index_writer->size()); + } + if (use_primary_key_bloom_filter) { + std::cout << "primary key bf size is " << bloom_filter_index_writer->size() + << std::endl; + } } return Status::OK(); } @@ -128,7 +144,7 @@ Status test_bloom_filter_index_reader_writer_template( BloomFilterIndexReader* reader = nullptr; std::unique_ptr iter; get_bloom_filter_reader_iter(file_name, meta, &reader, &iter); - + EXPECT_EQ(reader->algorithm(), BloomFilterAlgorithmPB::BLOCK_BLOOM_FILTER); // page 0 std::unique_ptr bf; RETURN_IF_ERROR(iter->read_bloom_filter(0, &bf)); @@ -171,7 +187,7 @@ Status test_bloom_filter_index_reader_writer_template( } TEST_F(BloomFilterIndexReaderWriterTest, test_int) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int* val = new int[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -186,8 +202,58 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_int) { delete[] val; } +TEST_F(BloomFilterIndexReaderWriterTest, test_string) { + size_t num = 1024 * 3; + std::vector val_strings(num); + for (size_t i = 0; i < num; ++i) { + val_strings[i] = "string_test_" + std::to_string(i + 1); + } + Slice* val = new Slice[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = Slice(val_strings[i]); + } + + std::string file_name = "bloom_filter_string"; + Slice not_exist_value("string_test_not_exist"); + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value, true); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_unsigned_int) { + size_t num = 1024 * 3; + uint32_t* val = new uint32_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = static_cast(i + 1); + } + + std::string file_name = "bloom_filter_unsigned_int"; + uint32_t not_exist_value = 0xFFFFFFFF; + auto st = + test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_smallint) { + size_t num = 1024 * 3; + int16_t* val = new int16_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = static_cast(i + 1); + } + + std::string file_name = "bloom_filter_smallint"; + int16_t not_exist_value = -1; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + TEST_F(BloomFilterIndexReaderWriterTest, test_bigint) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int64_t* val = new int64_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -203,7 +269,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_bigint) { } TEST_F(BloomFilterIndexReaderWriterTest, test_largeint) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int128_t* val = new int128_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -219,7 +285,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_largeint) { } TEST_F(BloomFilterIndexReaderWriterTest, test_varchar_type) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; std::string* val = new std::string[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -240,7 +306,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_varchar_type) { } TEST_F(BloomFilterIndexReaderWriterTest, test_char) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; std::string* val = new std::string[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -261,7 +327,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_char) { } TEST_F(BloomFilterIndexReaderWriterTest, test_date) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; uint24_t* val = new uint24_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -277,7 +343,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_date) { } TEST_F(BloomFilterIndexReaderWriterTest, test_datetime) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int64_t* val = new int64_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -293,7 +359,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_datetime) { } TEST_F(BloomFilterIndexReaderWriterTest, test_decimal) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; decimal12_t* val = new decimal12_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -308,8 +374,29 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_decimal) { delete[] val; } +TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index_char) { + size_t num = 1024 * 3; + std::string* val = new std::string[num]; + for (int i = 0; i < num; ++i) { + // there will be 3 bloom filter pages + val[i] = "primary_key_" + std::to_string(10000 + i); + } + Slice* slices = new Slice[num]; + for (int i = 0; i < num; ++i) { + // there will be 3 bloom filter pages + slices[i] = Slice(val[i].c_str(), val[i].size()); + } + std::string file_name = "primary_key_bloom_filter_index_char"; + Slice not_exist_value("primary_key_not_exist_char"); + auto st = test_bloom_filter_index_reader_writer_template( + file_name, slices, num, 1, ¬_exist_value, true, true); + EXPECT_TRUE(st.ok()); + delete[] val; + delete[] slices; +} + TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; std::vector val_strings(num); for (size_t i = 0; i < num; ++i) { val_strings[i] = "primary_key_" + std::to_string(i); @@ -328,7 +415,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index) { } TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index_int) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int* val = new int[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -344,5 +431,377 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index_int delete[] val; } +TEST_F(BloomFilterIndexReaderWriterTest, test_datev2) { + size_t num = 1024 * 3; + uint32_t* val = new uint32_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = 20210101 + i; // YYYYMMDD + } + + std::string file_name = "bloom_filter_datev2"; + uint32_t not_exist_value = 20211231; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_datetimev2) { + size_t num = 1024 * 3; + uint64_t* val = new uint64_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = 20210101000000 + i; // YYYYMMDDHHMMSS + } + + std::string file_name = "bloom_filter_datetimev2"; + uint64_t not_exist_value = 20211231235959; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_decimal32) { + size_t num = 1024 * 3; + int32_t* val = new int32_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = static_cast(i * 100 + 1); + } + + std::string file_name = "bloom_filter_decimal32"; + int32_t not_exist_value = 99999; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_decimal64) { + size_t num = 1024 * 3; + ; + int64_t* val = new int64_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = static_cast(i * 1000 + 123); + } + + std::string file_name = "bloom_filter_decimal64"; + int64_t not_exist_value = 9999999; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_ipv4) { + size_t num = 1024 * 3; // 3072 + uint32_t* val = new uint32_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = (192 << 24) | (168 << 16) | (i & 0xFFFF); + } + + std::string file_name = "bloom_filter_ipv4"; + uint32_t not_exist_value = (10 << 24) | (0 << 16) | (0 << 8) | 1; // 10.0.0.1 + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_decimal128i) { + size_t num = 1024 * 3; + int128_t* val = new int128_t[num]; + + int128_t base_value = int128_t(1000000000ULL) * int128_t(1000000000ULL); + + for (size_t i = 0; i < num; ++i) { + val[i] = base_value + int128_t(i); + } + + std::string file_name = "bloom_filter_decimal128i"; + int128_t not_exist_value = int128_t(9999999999999999999ULL); + + auto st = + test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_decimal256) { + size_t num = 1024 * 3; + using Decimal256Type = wide::Int256; + + Decimal256Type* val = new Decimal256Type[num]; + + Decimal256Type base_value = Decimal256Type(1000000000ULL); // 1e9 + base_value *= Decimal256Type(1000000000ULL); // base_value = 1e18 + base_value *= Decimal256Type(100000000ULL); // base_value = 1e26 + base_value *= Decimal256Type(100000000ULL); // base_value = 1e34 + base_value *= Decimal256Type(10000ULL); // base_value = 1e38 + + for (size_t i = 0; i < num; ++i) { + val[i] = base_value + Decimal256Type(i); + } + + std::string file_name = "bloom_filter_decimal256"; + + Decimal256Type not_exist_value = base_value + Decimal256Type(9999999ULL); + + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_ipv6) { + size_t num = 1024 * 3; + uint128_t* val = new uint128_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = (uint128_t(0x20010DB800000000) << 64) | uint128_t(i); + } + + std::string file_name = "bloom_filter_ipv6"; + uint128_t not_exist_value = (uint128_t(0x20010DB800000000) << 64) | uint128_t(999999); + + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +template +Status write_ngram_bloom_filter_index_file(const std::string& file_name, Slice* values, + size_t num_values, const TypeInfo* type_info, + BloomFilterIndexWriter* bf_index_writer, + ColumnIndexMetaPB* meta) { + auto fs = io::global_local_filesystem(); + std::string fname = dname + "/" + file_name; + io::FileWriterPtr file_writer; + Status st = fs->create_file(fname, &file_writer); + EXPECT_TRUE(st.ok()) << st.to_string(); + + size_t i = 0; + while (i < num_values) { + size_t num = std::min(static_cast(1024), num_values - i); + st = bf_index_writer->add_values(values + i, num); + EXPECT_TRUE(st.ok()); + st = bf_index_writer->flush(); + EXPECT_TRUE(st.ok()); + i += num; + } + bf_index_writer->add_nulls(1); + st = bf_index_writer->finish(file_writer.get(), meta); + EXPECT_TRUE(st.ok()) << "Writer finish status: " << st.to_string(); + EXPECT_TRUE(file_writer->close().ok()); + + return Status::OK(); +} + +Status read_and_test_ngram_bloom_filter_index_file(const std::string& file_name, size_t num_values, + uint8_t gram_size, uint16_t bf_size, + const ColumnIndexMetaPB& meta, + const std::vector& test_patterns) { + BloomFilterIndexReader* reader = nullptr; + std::unique_ptr iter; + get_bloom_filter_reader_iter(file_name, meta, &reader, &iter); + EXPECT_EQ(reader->algorithm(), BloomFilterAlgorithmPB::NGRAM_BLOOM_FILTER); + + NgramTokenExtractor extractor(gram_size); + uint16_t gram_bf_size = bf_size; + + size_t total_pages = (num_values + 1023) / 1024; + for (size_t page = 0; page < total_pages; ++page) { + std::unique_ptr bf; + auto st = iter->read_bloom_filter(page, &bf); + EXPECT_TRUE(st.ok()); + + for (const auto& pattern : test_patterns) { + std::unique_ptr query_bf; + st = BloomFilter::create(NGRAM_BLOOM_FILTER, &query_bf, gram_bf_size); + EXPECT_TRUE(st.ok()); + + if (extractor.string_like_to_bloom_filter(pattern.data(), pattern.size(), *query_bf)) { + bool contains = bf->contains(*query_bf); + bool expected = false; + if ((page == 0 && (pattern == "ngram15" || pattern == "ngram1000")) || + (page == 1 && pattern == "ngram1499")) { + expected = true; + } + EXPECT_EQ(contains, expected) << "Pattern: " << pattern << ", Page: " << page; + } + } + } + + delete reader; + return Status::OK(); +} + +template +Status test_ngram_bloom_filter_index_reader_writer(const std::string& file_name, Slice* values, + size_t num_values, uint8_t gram_size, + uint16_t bf_size) { + const auto* type_info = get_scalar_type_info(); + ColumnIndexMetaPB meta; + + BloomFilterOptions bf_options; + std::unique_ptr bf_index_writer; + RETURN_IF_ERROR(NGramBloomFilterIndexWriterImpl::create(bf_options, type_info, gram_size, + bf_size, &bf_index_writer)); + + RETURN_IF_ERROR(write_ngram_bloom_filter_index_file( + file_name, values, num_values, type_info, bf_index_writer.get(), &meta)); + + std::vector test_patterns = {"ngram15", "ngram1000", "ngram1499", + "non-existent-string"}; + + RETURN_IF_ERROR(read_and_test_ngram_bloom_filter_index_file(file_name, num_values, gram_size, + bf_size, meta, test_patterns)); + + return Status::OK(); +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_ngram_bloom_filter) { + size_t num = 1500; + std::vector val(num); + for (size_t i = 0; i < num; ++i) { + val[i] = "ngram" + std::to_string(i); + } + std::vector slices(num); + for (size_t i = 0; i < num; ++i) { + slices[i] = Slice(val[i].data(), val[i].size()); + } + + uint8_t gram_size = 5; + uint16_t bf_size = 65535; + + auto st = test_ngram_bloom_filter_index_reader_writer( + "bloom_filter_ngram_varchar", slices.data(), num, gram_size, bf_size); + EXPECT_TRUE(st.ok()); + st = test_ngram_bloom_filter_index_reader_writer( + "bloom_filter_ngram_char", slices.data(), num, gram_size, bf_size); + EXPECT_TRUE(st.ok()); + st = test_ngram_bloom_filter_index_reader_writer( + "bloom_filter_ngram_string", slices.data(), num, gram_size, bf_size); + EXPECT_TRUE(st.ok()); + st = test_ngram_bloom_filter_index_reader_writer( + "bloom_filter_ngram_string", slices.data(), num, gram_size, bf_size); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), TStatusCode::NOT_IMPLEMENTED_ERROR); +} +void test_ngram_bloom_filter_with_size(uint16_t bf_size) { + const auto* type_info = get_scalar_type_info(); + ColumnIndexMetaPB meta; + + BloomFilterOptions bf_options; + size_t num = 1500; + std::vector val(num); + for (size_t i = 0; i < num; ++i) { + val[i] = "ngram" + std::to_string(i); + } + std::vector slices(num); + for (size_t i = 0; i < num; ++i) { + slices[i] = Slice(val[i].data(), val[i].size()); + } + size_t total_pages = (num + 1024 - 1) / 1024; + uint8_t gram_size = 5; + + std::unique_ptr bf_index_writer; + auto st = NGramBloomFilterIndexWriterImpl::create(bf_options, type_info, gram_size, bf_size, + &bf_index_writer); + EXPECT_TRUE(st.ok()); + + std::string file_name = "bloom_filter_ngram_varchar_size_" + std::to_string(bf_size); + st = write_ngram_bloom_filter_index_file( + file_name, slices.data(), num, type_info, bf_index_writer.get(), &meta); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(bf_index_writer->size(), static_cast(bf_size) * total_pages); +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_ngram_bloom_filter_size) { + std::vector bf_sizes = {1024, 2048, 4096, 8192, 16384, 32768, 65535}; + for (uint16_t bf_size : bf_sizes) { + test_ngram_bloom_filter_with_size(bf_size); + } +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_unsupported_type) { + auto type_info = get_scalar_type_info(); + BloomFilterOptions bf_options; + std::unique_ptr bloom_filter_index_writer; + auto st = BloomFilterIndexWriter::create(bf_options, type_info, &bloom_filter_index_writer); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), TStatusCode::NOT_IMPLEMENTED_ERROR); +} + +// Test function for verifying Bloom Filter FPP +void test_bloom_filter_fpp(double expected_fpp) { + size_t n = 10000; // Number of elements to insert into the Bloom Filter + size_t m = 100000; // Number of non-existent elements to test for false positives + + // Generate and insert elements into the Bloom Filter index + std::vector insert_values; + for (size_t i = 0; i < n; ++i) { + int64_t val = static_cast(i); + insert_values.push_back(val); + } + + // Write the Bloom Filter index to file + std::string file_name = "bloom_filter_fpp_test"; + ColumnIndexMetaPB index_meta; + Status st = write_bloom_filter_index_file( + file_name, insert_values.data(), n, 0, &index_meta, false, expected_fpp); + EXPECT_TRUE(st.ok()); + + // Read the Bloom Filter index + BloomFilterIndexReader* reader = nullptr; + std::unique_ptr iter; + get_bloom_filter_reader_iter(file_name, index_meta, &reader, &iter); + + // Read the Bloom Filter (only one page since we flushed once) + std::unique_ptr bf; + st = iter->read_bloom_filter(0, &bf); + EXPECT_TRUE(st.ok()); + + // Generate non-existent elements for testing false positive rate + std::unordered_set inserted_elements(insert_values.begin(), insert_values.end()); + std::unordered_set non_exist_elements; + std::vector test_values; + size_t max_value = n + m * 10; // Ensure test values are not in the inserted range + boost::mt19937_64 rng(12345); // Seed the random number generator for reproducibility + std::uniform_int_distribution dist(static_cast(n + 1), + static_cast(max_value)); + while (non_exist_elements.size() < m) { + int64_t val = dist(rng); + if (inserted_elements.find(val) == inserted_elements.end()) { + non_exist_elements.insert(val); + test_values.push_back(val); + } + } + + // Test non-existent elements and count false positives + size_t fp_count = 0; + for (const auto& val : test_values) { + if (bf->test_bytes(reinterpret_cast(&val), sizeof(int64_t))) { + fp_count++; + } + } + + // Compute actual false positive probability + double actual_fpp = static_cast(fp_count) / static_cast(m); + std::cout << "Expected FPP: " << expected_fpp << ", Actual FPP: " << actual_fpp << std::endl; + + // Verify that actual FPP is within the allowable error range + EXPECT_LE(actual_fpp, expected_fpp); + + delete reader; +} + +// Test case to run FPP tests with multiple expected FPP values +TEST_F(BloomFilterIndexReaderWriterTest, test_bloom_filter_fpp_multiple) { + std::vector fpp_values = {0.01, 0.02, 0.05}; + for (double fpp : fpp_values) { + test_bloom_filter_fpp(fpp); + } +} } // namespace segment_v2 } // namespace doris From 334b3435bc998e5e2405d8fe7b25ba16d0267f03 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Mon, 2 Dec 2024 13:05:44 +0530 Subject: [PATCH 21/31] [Enhancement] (nereids)implement CreateEncryptKeyCommand in nereids (#44807) Issue Number: close #42591 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../doris/catalog/EncryptKeyHelper.java | 10 ++- .../nereids/parser/LogicalPlanBuilder.java | 9 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/CreateEncryptkeyCommand.java | 73 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../test_nereids_encrypt_test.groovy | 2 +- 7 files changed, 98 insertions(+), 4 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateEncryptkeyCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index a372a993ac6f228..5200f714a090ce2 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -188,6 +188,7 @@ supportedCreateStatement USING LEFT_PAREN booleanExpression RIGHT_PAREN #createRowPolicy | CREATE SQL_BLOCK_RULE (IF NOT EXISTS)? name=identifier properties=propertyClause? #createSqlBlockRule + | CREATE ENCRYPTKEY (IF NOT EXISTS)? multipartIdentifier AS STRING_LITERAL #createEncryptkey ; supportedAlterStatement @@ -767,7 +768,6 @@ unsupportedCreateStatement (CONDITIONS LEFT_PAREN workloadPolicyConditions RIGHT_PAREN)? (ACTIONS LEFT_PAREN workloadPolicyActions RIGHT_PAREN)? properties=propertyClause? #createWorkloadPolicy - | CREATE ENCRYPTKEY (IF NOT EXISTS)? multipartIdentifier AS STRING_LITERAL #createEncryptkey | CREATE STORAGE POLICY (IF NOT EXISTS)? name=identifier properties=propertyClause? #createStoragePolicy | BUILD INDEX name=identifier ON tableName=multipartIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java index 2e30f0ba86f326a..934dd22f3a5b89a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java @@ -32,10 +32,16 @@ public class EncryptKeyHelper { public static void createEncryptKey(CreateEncryptKeyStmt stmt) throws UserException { EncryptKeyName name = stmt.getEncryptKeyName(); - Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(name.getDb()); - db.addEncryptKey(stmt.getEncryptKey(), stmt.isIfNotExists()); + createEncryptKey(name.getDb(), stmt.getEncryptKey(), stmt.isIfNotExists()); + } + + public static void createEncryptKey(String dbName, EncryptKey encryptKey, + boolean isIfNotExists) throws UserException { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); + db.addEncryptKey(encryptKey, isIfNotExists); } + public static void replayCreateEncryptKey(EncryptKey encryptKey) throws MetaNotFoundException { String dbName = encryptKey.getEncryptKeyName().getDb(); Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(dbName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index dfe879e7d44dad9..f94284145d5320b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -89,6 +89,7 @@ import org.apache.doris.nereids.DorisParser.ComplexColTypeListContext; import org.apache.doris.nereids.DorisParser.ComplexDataTypeContext; import org.apache.doris.nereids.DorisParser.ConstantContext; +import org.apache.doris.nereids.DorisParser.CreateEncryptkeyContext; import org.apache.doris.nereids.DorisParser.CreateMTMVContext; import org.apache.doris.nereids.DorisParser.CreateProcedureContext; import org.apache.doris.nereids.DorisParser.CreateRoutineLoadContext; @@ -476,6 +477,7 @@ import org.apache.doris.nereids.trees.plans.commands.CleanAllProfileCommand; import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.Constraint; +import org.apache.doris.nereids.trees.plans.commands.CreateEncryptkeyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateJobCommand; import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; @@ -4749,6 +4751,13 @@ public LogicalPlan visitDropRole(DropRoleContext ctx) { return new DropRoleCommand(ctx.name.getText(), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitCreateEncryptkey(CreateEncryptkeyContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.multipartIdentifier()); + return new CreateEncryptkeyCommand(new EncryptKeyName(nameParts), ctx.EXISTS() != null, + stripQuotes(ctx.STRING_LITERAL().getText())); + } + @Override public LogicalPlan visitDropEncryptkey(DropEncryptkeyContext ctx) { List nameParts = visitMultipartIdentifier(ctx.name); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 14a1b13a4667dfc..d7331d5a65d7383 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -237,5 +237,6 @@ public enum PlanType { RECOVER_TABLE_COMMAND, RECOVER_PARTITION_COMMAND, REPLAY_COMMAND, + CREATE_ENCRYPTKEY_COMMAND, CREATE_ROUTINE_LOAD_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateEncryptkeyCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateEncryptkeyCommand.java new file mode 100644 index 000000000000000..f8fc7c3cc12c5e0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateEncryptkeyCommand.java @@ -0,0 +1,73 @@ +// 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.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.EncryptKeyName; +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.EncryptKey; +import org.apache.doris.catalog.EncryptKeyHelper; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +/** CreateEncryptkeyCommand */ +public class CreateEncryptkeyCommand extends Command implements ForwardWithSync { + private final boolean ifNotExists; + private final EncryptKeyName encryptKeyName; + private final String keyString; + + public CreateEncryptkeyCommand(EncryptKeyName encryptKeyName, boolean ifNotExists, String keyString) { + super(PlanType.CREATE_ENCRYPTKEY_COMMAND); + this.ifNotExists = ifNotExists; + this.encryptKeyName = encryptKeyName; + this.keyString = keyString; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check operation privilege + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + encryptKeyName.analyze(ctx); + if (Strings.isNullOrEmpty(keyString)) { + throw new AnalysisException("keyString can not be null or empty string."); + } + EncryptKeyHelper.createEncryptKey(encryptKeyName.getDb(), + new EncryptKey(encryptKeyName, keyString), ifNotExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateEncryptKeyCommand(this, context); + } + + @Override + public StmtType stmtType() { + return StmtType.CREATE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index d386d097a93b49a..033290d37132e8d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -33,6 +33,7 @@ import org.apache.doris.nereids.trees.plans.commands.CancelWarmUpJobCommand; import org.apache.doris.nereids.trees.plans.commands.CleanAllProfileCommand; import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.CreateEncryptkeyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateJobCommand; import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; @@ -175,6 +176,10 @@ default R visitExportCommand(ExportCommand exportCommand, C context) { return visitCommand(exportCommand, context); } + default R visitCreateEncryptKeyCommand(CreateEncryptkeyCommand createEncryptKeyCommand, C context) { + return visitCommand(createEncryptKeyCommand, context); + } + default R visitCreateTableCommand(CreateTableCommand createTableCommand, C context) { return visitCommand(createTableCommand, context); } diff --git a/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy b/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy index 2fab616580faaad..8954090bb4c31dd 100644 --- a/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy +++ b/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy @@ -20,7 +20,7 @@ suite("test_nereids_encrypt_test") { sql """ create database IF NOT EXISTS ${dbName}; """ sql """ use ${dbName}; """ checkNereidsExecute("drop encryptkey if exists ${encryptkeyName}") - sql """CREATE ENCRYPTKEY ${encryptkeyName} AS "ABCD123456789";""" + checkNereidsExecute("""CREATE ENCRYPTKEY ${encryptkeyName} AS "ABCD123456789";""") qt_check_encrypt_1("SHOW ENCRYPTKEYS FROM ${dbName}") checkNereidsExecute("drop encryptkey ${encryptkeyName}") qt_check_encrypt_2("SHOW ENCRYPTKEYS FROM ${dbName}") From 54fe0b463a9dd29a562539bdc351929aebb0b064 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Mon, 2 Dec 2024 13:06:03 +0530 Subject: [PATCH 22/31] [Enhancement] (nereids)implement CreateRoleCommand in nereids (#44814) Issue Number: close #42597 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../apache/doris/mysql/privilege/Auth.java | 4 + .../nereids/parser/LogicalPlanBuilder.java | 9 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/CreateRoleCommand.java | 75 +++++++++++++++++++ .../trees/plans/commands/DropRoleCommand.java | 4 - .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../ddl/account/test_nereids_role.groovy | 4 +- .../ddl/alter/test_nereids_role.groovy | 66 ---------------- 9 files changed, 97 insertions(+), 73 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRoleCommand.java delete mode 100644 regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 5200f714a090ce2..3a377ba05bc7078 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -181,6 +181,7 @@ supportedCreateStatement | CREATE (EXTERNAL)? TABLE (IF NOT EXISTS)? name=multipartIdentifier LIKE existedTable=multipartIdentifier (WITH ROLLUP (rollupNames=identifierList)?)? #createTableLike + | CREATE ROLE (IF NOT EXISTS)? name=identifier (COMMENT STRING_LITERAL)? #createRole | CREATE ROW POLICY (IF NOT EXISTS)? name=identifier ON table=multipartIdentifier AS type=(RESTRICTIVE | PERMISSIVE) @@ -751,7 +752,6 @@ unsupportedCreateStatement (SUPERUSER | DEFAULT ROLE role=STRING_LITERAL)? passwordOption (COMMENT STRING_LITERAL)? #createUser | CREATE (READ ONLY)? REPOSITORY name=identifier WITH storageBackend #createRepository - | CREATE ROLE (IF NOT EXISTS)? name=identifier (COMMENT STRING_LITERAL)? #createRole | CREATE FILE name=STRING_LITERAL ((FROM | IN) database=identifier)? properties=propertyClause #createFile | CREATE INDEX (IF NOT EXISTS)? name=identifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java index f1277a1cd0a3b56..8a13c6dd125548c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java @@ -1019,6 +1019,10 @@ public void createRole(CreateRoleStmt stmt) throws DdlException { createRoleInternal(stmt.getRole(), stmt.isSetIfNotExists(), stmt.getComment(), false); } + public void createRole(String role, boolean ignoreIfExists, String comment) throws DdlException { + createRoleInternal(role, ignoreIfExists, comment, false); + } + public void alterRole(AlterRoleStmt stmt) throws DdlException { alterRoleInternal(stmt.getRole(), stmt.getComment(), false); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index f94284145d5320b..2d55baffb1ebd91 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -92,6 +92,7 @@ import org.apache.doris.nereids.DorisParser.CreateEncryptkeyContext; import org.apache.doris.nereids.DorisParser.CreateMTMVContext; import org.apache.doris.nereids.DorisParser.CreateProcedureContext; +import org.apache.doris.nereids.DorisParser.CreateRoleContext; import org.apache.doris.nereids.DorisParser.CreateRoutineLoadContext; import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext; import org.apache.doris.nereids.DorisParser.CreateSqlBlockRuleContext; @@ -482,6 +483,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateRoleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; @@ -4671,6 +4673,13 @@ public LogicalPlan visitAlterRole(AlterRoleContext ctx) { return new AlterRoleCommand(ctx.role.getText(), comment); } + @Override + public LogicalPlan visitCreateRole(CreateRoleContext ctx) { + String comment = ctx.STRING_LITERAL() == null ? "" : LogicalPlanBuilderAssistant.escapeBackSlash( + ctx.STRING_LITERAL().getText().substring(1, ctx.STRING_LITERAL().getText().length() - 1)); + return new CreateRoleCommand(ctx.EXISTS() != null, ctx.name.getText(), comment); + } + @Override public LogicalPlan visitShowFrontends(ShowFrontendsContext ctx) { String detail = (ctx.name != null) ? ctx.name.getText() : null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index d7331d5a65d7383..0cc0e5594075b44 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -171,6 +171,7 @@ public enum PlanType { SHOW_CREATE_PROCEDURE_COMMAND, CREATE_VIEW_COMMAND, CLEAN_ALL_PROFILE_COMMAND, + CREATE_ROLE_COMMAND, ALTER_ROLE_COMMAND, ALTER_VIEW_COMMAND, ALTER_STORAGE_VAULT, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRoleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRoleCommand.java new file mode 100644 index 000000000000000..1596993ba669f27 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRoleCommand.java @@ -0,0 +1,75 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +/** + * Create role command + */ +public class CreateRoleCommand extends Command implements ForwardWithSync { + private boolean ifNotExists; + private String role; + private String comment; + + /** + * ctor of this command. + */ + public CreateRoleCommand(boolean ifNotExists, String role, String comment) { + super(PlanType.CREATE_ROLE_COMMAND); + this.ifNotExists = ifNotExists; + this.role = role; + this.comment = Strings.nullToEmpty(comment); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateRoleCommand(this, context); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (Config.access_controller_type.equalsIgnoreCase("ranger-doris")) { + throw new AnalysisException("Create role is prohibited when Ranger is enabled."); + } + FeNameFormat.checkRoleName(role, false /* can not be admin */, "Can not create role"); + // check if current user has GRANT priv on GLOBAL level. + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.GRANT)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "CREATE ROLE"); + } + Env.getCurrentEnv().getAuth().createRole(role, ifNotExists, comment); + } + + @Override + public StmtType stmtType() { + return StmtType.CREATE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropRoleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropRoleCommand.java index 2f7858aa6ae1800..5b74d32cf6a4dfd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropRoleCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropRoleCommand.java @@ -29,14 +29,10 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - /** * drop roles command */ public class DropRoleCommand extends DropCommand { - public static final Logger LOG = LogManager.getLogger(DropRoleCommand.class); private final boolean ifExists; private final String role; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 033290d37132e8d..d1d424738f49245 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -38,6 +38,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateRoleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; @@ -486,6 +487,10 @@ default R visitCreateSqlBlockRuleCommand(CreateSqlBlockRuleCommand dropRoleComma return visitCommand(dropRoleCommand, context); } + default R visitCreateRoleCommand(CreateRoleCommand createRoleCommand, C context) { + return visitCommand(createRoleCommand, context); + } + default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } diff --git a/regression-test/suites/nereids_p0/ddl/account/test_nereids_role.groovy b/regression-test/suites/nereids_p0/ddl/account/test_nereids_role.groovy index 4e2cd4ca6375d2c..a3bee25cf5822a4 100644 --- a/regression-test/suites/nereids_p0/ddl/account/test_nereids_role.groovy +++ b/regression-test/suites/nereids_p0/ddl/account/test_nereids_role.groovy @@ -19,7 +19,7 @@ import org.junit.Assert; suite("test_nereids_role") { def role= 'nereids_account_role_test' - def user = 'acount_role_user_test' + def user = 'nereids_acount_role_user_test' def dbName = 'nereids_account_role_test_db' def pwd = 'C123_567p' @@ -28,7 +28,7 @@ suite("test_nereids_role") { sql """DROP DATABASE IF EXISTS ${dbName}""" sql """CREATE DATABASE ${dbName}""" - sql """CREATE ROLE ${role}""" + checkNereidsExecute("CREATE ROLE ${role}") sql """GRANT SELECT_PRIV ON ${context.config.defaultDb} TO ROLE '${role}'""" sql """GRANT SELECT_PRIV ON ${dbName} TO ROLE '${role}'""" sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}' DEFAULT ROLE '${role}'""" diff --git a/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy b/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy deleted file mode 100644 index f23982188269b7e..000000000000000 --- a/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy +++ /dev/null @@ -1,66 +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. - -import org.junit.Assert; - -suite("test_nereids_role_x", "account") { - def role= 'nereids_account_role_test_x' - def user = 'nereids_acount_role_user_test_x' - def dbName = 'nereids_account_role_test_db_x' - def pwd = 'C123_567p' - - try_sql("DROP ROLE ${role}") - try_sql("DROP USER ${user}") - sql """DROP DATABASE IF EXISTS ${dbName}""" - sql """CREATE DATABASE IF NOT EXISTS ${dbName}""" - sql """CREATE ROLE ${role}""" - sql """GRANT SELECT_PRIV ON internal.${context.config.defaultDb}.* TO ROLE '${role}'""" - sql """GRANT SELECT_PRIV ON internal.${dbName}.* TO ROLE '${role}'""" - sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}' DEFAULT ROLE '${role}'""" - def result1 = connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { - sql "show databases like '${dbName}'" - } - assertEquals(result1.size(), 1) - - sql """REVOKE SELECT_PRIV ON ${dbName} FROM ROLE '${role}'""" - def result2 = connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { - sql "show databases like '${dbName}'" - } - assertEquals(result2.size(), 0) - - sql """DROP USER ${user}""" - sql """DROP ROLE ${role}""" - sql """DROP DATABASE ${dbName}""" - - // test comment - // create role with comment - sql """CREATE ROLE ${role} comment 'account_p0_account_role_test_comment_create'""" - def roles_create = sql """show roles""" - logger.info("roles_create: " + roles_create.toString()) - assertTrue(roles_create.toString().contains("account_p0_account_role_test_comment_create")) - // alter role with comment - checkNereidsExecute("ALTER ROLE ${role} comment 'account_p0_account_role_test_comment_alter';"); - def roles_alter = sql """show roles""" - logger.info("roles_alter: " + roles_alter.toString()) - assertTrue(roles_alter.toString().contains("account_p0_account_role_test_comment_alter")) - // drop role - checkNereidsExecute("""DROP ROLE ${role}""") - def roles_drop = sql """show roles""" - logger.info("roles_drop: " + roles_drop.toString()) - assertFalse(roles_drop.toString().contains("account_p0_account_role_test_comment_alter")) -} - From 53ea2a0a1d18900a3ad5bca8d1b1d5fe1a98111c Mon Sep 17 00:00:00 2001 From: 924060929 Date: Mon, 2 Dec 2024 16:40:38 +0800 Subject: [PATCH 23/31] [fix](coordinator) fix NereidsCoordinator can not interrupt query in fe (#44795) fix NereidsCoordinator can not interrupt query in fe, the QueryProcessor should set the status to failed --- .../apache/doris/qe/AbstractJobProcessor.java | 118 ++++++++++++++++++ .../org/apache/doris/qe/JobProcessor.java | 5 +- .../apache/doris/qe/NereidsCoordinator.java | 5 +- .../doris/qe/runtime/LoadProcessor.java | 74 ++--------- .../qe/runtime/PipelineExecutionTask.java | 2 +- .../runtime/PipelineExecutionTaskBuilder.java | 2 +- .../doris/qe/runtime/QueryProcessor.java | 18 ++- 7 files changed, 142 insertions(+), 82 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/qe/AbstractJobProcessor.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/AbstractJobProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/AbstractJobProcessor.java new file mode 100644 index 000000000000000..2858de25d578fc0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/AbstractJobProcessor.java @@ -0,0 +1,118 @@ +// 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.doris.qe; + +import org.apache.doris.common.Status; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.qe.runtime.BackendFragmentId; +import org.apache.doris.qe.runtime.MultiFragmentsPipelineTask; +import org.apache.doris.qe.runtime.PipelineExecutionTask; +import org.apache.doris.qe.runtime.SingleFragmentPipelineTask; +import org.apache.doris.thrift.TReportExecStatusParams; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; + +/** AbstractJobProcessor */ +public abstract class AbstractJobProcessor implements JobProcessor { + private final Logger logger = LogManager.getLogger(getClass()); + + protected final CoordinatorContext coordinatorContext; + protected volatile Optional executionTask; + protected volatile Optional> backendFragmentTasks; + + public AbstractJobProcessor(CoordinatorContext coordinatorContext) { + this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + this.executionTask = Optional.empty(); + this.backendFragmentTasks = Optional.empty(); + } + + protected abstract void doProcessReportExecStatus( + TReportExecStatusParams params, SingleFragmentPipelineTask fragmentTask); + + @Override + public final void setPipelineExecutionTask(PipelineExecutionTask pipelineExecutionTask) { + Preconditions.checkArgument(pipelineExecutionTask != null, "sqlPipelineTask can not be null"); + + this.executionTask = Optional.of(pipelineExecutionTask); + Map backendFragmentTasks + = buildBackendFragmentTasks(pipelineExecutionTask); + this.backendFragmentTasks = Optional.of(backendFragmentTasks); + + afterSetPipelineExecutionTask(pipelineExecutionTask); + } + + protected void afterSetPipelineExecutionTask(PipelineExecutionTask pipelineExecutionTask) {} + + @Override + public final void updateFragmentExecStatus(TReportExecStatusParams params) { + SingleFragmentPipelineTask fragmentTask = backendFragmentTasks.get().get( + new BackendFragmentId(params.getBackendId(), params.getFragmentId())); + if (fragmentTask == null || !fragmentTask.processReportExecStatus(params)) { + return; + } + + TUniqueId queryId = coordinatorContext.queryId; + Status status = new Status(params.status); + // for now, abort the query if we see any error except if the error is cancelled + // and returned_all_results_ is true. + // (UpdateStatus() initiates cancellation, if it hasn't already been initiated) + if (!status.ok()) { + if (coordinatorContext.isEos() && status.isCancelled()) { + logger.warn("Query {} has returned all results, fragment_id={} instance_id={}, be={}" + + " is reporting failed status {}", + DebugUtil.printId(queryId), params.getFragmentId(), + DebugUtil.printId(params.getFragmentInstanceId()), + params.getBackendId(), + status.toString()); + } else { + logger.warn("one instance report fail, query_id={} fragment_id={} instance_id={}, be={}," + + " error message: {}", + DebugUtil.printId(queryId), params.getFragmentId(), + DebugUtil.printId(params.getFragmentInstanceId()), + params.getBackendId(), status.toString()); + coordinatorContext.updateStatusIfOk(status); + } + } + doProcessReportExecStatus(params, fragmentTask); + } + + private Map buildBackendFragmentTasks( + PipelineExecutionTask executionTask) { + ImmutableMap.Builder backendFragmentTasks + = ImmutableMap.builder(); + for (Entry backendTask : executionTask.getChildrenTasks().entrySet()) { + Long backendId = backendTask.getKey(); + for (Entry fragmentIdToTask : backendTask.getValue() + .getChildrenTasks().entrySet()) { + Integer fragmentId = fragmentIdToTask.getKey(); + SingleFragmentPipelineTask fragmentTask = fragmentIdToTask.getValue(); + backendFragmentTasks.put(new BackendFragmentId(backendId, fragmentId), fragmentTask); + } + } + return backendFragmentTasks.build(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java index ede218848c72212..7e4042dde3c3bdd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java @@ -19,9 +19,12 @@ import org.apache.doris.common.Status; import org.apache.doris.qe.runtime.PipelineExecutionTask; +import org.apache.doris.thrift.TReportExecStatusParams; public interface JobProcessor { - void setSqlPipelineTask(PipelineExecutionTask pipelineExecutionTask); + void setPipelineExecutionTask(PipelineExecutionTask pipelineExecutionTask); void cancel(Status cancelReason); + + void updateFragmentExecStatus(TReportExecStatusParams params); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java index a9d6becc7fafe3f..a6f24806ed74aa1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java @@ -233,10 +233,7 @@ public boolean isDone() { @Override public void updateFragmentExecStatus(TReportExecStatusParams params) { - JobProcessor jobProcessor = coordinatorContext.getJobProcessor(); - if (jobProcessor instanceof LoadProcessor) { - coordinatorContext.asLoadProcessor().updateFragmentExecStatus(params); - } + coordinatorContext.getJobProcessor().updateFragmentExecStatus(params); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java index 3a448521fca0bf7..fb32919d834a542 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java @@ -24,46 +24,39 @@ import org.apache.doris.datasource.hive.HMSTransaction; import org.apache.doris.datasource.iceberg.IcebergTransaction; import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.AbstractJobProcessor; import org.apache.doris.qe.CoordinatorContext; -import org.apache.doris.qe.JobProcessor; import org.apache.doris.qe.LoadContext; import org.apache.doris.thrift.TFragmentInstanceReport; import org.apache.doris.thrift.TReportExecStatusParams; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TUniqueId; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; import java.util.Optional; import java.util.concurrent.TimeUnit; -public class LoadProcessor implements JobProcessor { +public class LoadProcessor extends AbstractJobProcessor { private static final Logger LOG = LogManager.getLogger(LoadProcessor.class); - public final CoordinatorContext coordinatorContext; public final LoadContext loadContext; public final long jobId; // this latch is used to wait finish for load, for example, insert into statement // MarkedCountDownLatch: // key: fragmentId, value: backendId - private volatile Optional executionTask; private volatile Optional> latch; - private volatile Optional> backendFragmentTasks; private volatile List topFragmentTasks; public LoadProcessor(CoordinatorContext coordinatorContext, long jobId) { - this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + super(coordinatorContext); + this.loadContext = new LoadContext(); - this.executionTask = Optional.empty(); this.latch = Optional.empty(); this.backendFragmentTasks = Optional.empty(); @@ -87,14 +80,8 @@ public LoadProcessor(CoordinatorContext coordinatorContext, long jobId) { } @Override - public void setSqlPipelineTask(PipelineExecutionTask pipelineExecutionTask) { - Preconditions.checkArgument(pipelineExecutionTask != null, "sqlPipelineTask can not be null"); - - this.executionTask = Optional.of(pipelineExecutionTask); - Map backendFragmentTasks - = buildBackendFragmentTasks(pipelineExecutionTask); - this.backendFragmentTasks = Optional.of(backendFragmentTasks); - + protected void afterSetPipelineExecutionTask(PipelineExecutionTask pipelineExecutionTask) { + Map backendFragmentTasks = this.backendFragmentTasks.get(); MarkedCountDownLatch latch = new MarkedCountDownLatch<>(backendFragmentTasks.size()); for (BackendFragmentId backendFragmentId : backendFragmentTasks.keySet()) { latch.addMark(backendFragmentId.fragmentId, backendFragmentId.backendId); @@ -168,34 +155,9 @@ public boolean await(long timeout, TimeUnit unit) throws InterruptedException { return latch.get().await(timeout, unit); } - public void updateFragmentExecStatus(TReportExecStatusParams params) { - SingleFragmentPipelineTask fragmentTask = backendFragmentTasks.get().get( - new BackendFragmentId(params.getBackendId(), params.getFragmentId())); - if (fragmentTask == null || !fragmentTask.processReportExecStatus(params)) { - return; - } - TUniqueId queryId = coordinatorContext.queryId; - Status status = new Status(params.status); - // for now, abort the query if we see any error except if the error is cancelled - // and returned_all_results_ is true. - // (UpdateStatus() initiates cancellation, if it hasn't already been initiated) - if (!status.ok()) { - if (coordinatorContext.isEos() && status.isCancelled()) { - LOG.warn("Query {} has returned all results, fragment_id={} instance_id={}, be={}" - + " is reporting failed status {}", - DebugUtil.printId(queryId), params.getFragmentId(), - DebugUtil.printId(params.getFragmentInstanceId()), - params.getBackendId(), - status.toString()); - } else { - LOG.warn("one instance report fail, query_id={} fragment_id={} instance_id={}, be={}," - + " error message: {}", - DebugUtil.printId(queryId), params.getFragmentId(), - DebugUtil.printId(params.getFragmentInstanceId()), - params.getBackendId(), status.toString()); - coordinatorContext.updateStatusIfOk(status); - } - } + + @Override + protected void doProcessReportExecStatus(TReportExecStatusParams params, SingleFragmentPipelineTask fragmentTask) { LoadContext loadContext = coordinatorContext.asLoadProcessor().loadContext; if (params.isSetDeltaUrls()) { loadContext.updateDeltaUrls(params.getDeltaUrls()); @@ -234,7 +196,7 @@ public void updateFragmentExecStatus(TReportExecStatusParams params) { if (fragmentTask.isDone()) { if (LOG.isDebugEnabled()) { LOG.debug("Query {} fragment {} is marked done", - DebugUtil.printId(queryId), params.getFragmentId()); + DebugUtil.printId(coordinatorContext.queryId), params.getFragmentId()); } latch.get().markedCountDown(params.getFragmentId(), params.getBackendId()); } @@ -258,22 +220,6 @@ public void updateFragmentExecStatus(TReportExecStatusParams params) { } } - private Map buildBackendFragmentTasks( - PipelineExecutionTask executionTask) { - ImmutableMap.Builder backendFragmentTasks - = ImmutableMap.builder(); - for (Entry backendTask : executionTask.getChildrenTasks().entrySet()) { - Long backendId = backendTask.getKey(); - for (Entry fragmentIdToTask : backendTask.getValue() - .getChildrenTasks().entrySet()) { - Integer fragmentId = fragmentIdToTask.getKey(); - SingleFragmentPipelineTask fragmentTask = fragmentIdToTask.getValue(); - backendFragmentTasks.put(new BackendFragmentId(backendId, fragmentId), fragmentTask); - } - } - return backendFragmentTasks.build(); - } - /* * Check the state of backends in needCheckBackendExecStates. * return true if all of them are OK. Otherwise, return false. diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTask.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTask.java index 8c1b9714c35db81..ae87d59d075d122 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTask.java @@ -102,7 +102,7 @@ public void execute() throws Exception { @Override public String toString() { - return "SqlPipelineTask(\n" + return "PipelineExecutionTask(\n" + childrenTasks.allTasks() .stream() .map(multiFragmentsPipelineTask -> " " + multiFragmentsPipelineTask) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTaskBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTaskBuilder.java index fd00bf0e3e85364..0da6f4a5fe2e43c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTaskBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTaskBuilder.java @@ -61,7 +61,7 @@ private PipelineExecutionTask buildTask(CoordinatorContext coordinatorContext, backendServiceProxy, buildMultiFragmentTasks(coordinatorContext, backendServiceProxy, workerToFragmentsParam) ); - coordinatorContext.getJobProcessor().setSqlPipelineTask(pipelineExecutionTask); + coordinatorContext.getJobProcessor().setPipelineExecutionTask(pipelineExecutionTask); return pipelineExecutionTask; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java index 2ec38e8cc8e3ea0..a5a5100faece1a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java @@ -25,13 +25,14 @@ import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; import org.apache.doris.planner.DataSink; import org.apache.doris.planner.ResultSink; +import org.apache.doris.qe.AbstractJobProcessor; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.CoordinatorContext; -import org.apache.doris.qe.JobProcessor; import org.apache.doris.qe.ResultReceiver; import org.apache.doris.qe.RowBatch; import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TReportExecStatusParams; import org.apache.doris.thrift.TStatusCode; import com.google.common.base.Strings; @@ -44,24 +45,21 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.CopyOnWriteArrayList; -public class QueryProcessor implements JobProcessor { +public class QueryProcessor extends AbstractJobProcessor { private static final Logger LOG = LogManager.getLogger(QueryProcessor.class); // constant fields private final long limitRows; // mutable field - private Optional sqlPipelineTask; - private final CoordinatorContext coordinatorContext; private final List runningReceivers; private int receiverOffset; private long numReceivedRows; public QueryProcessor(CoordinatorContext coordinatorContext, List runningReceivers) { - this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + super(coordinatorContext); this.runningReceivers = new CopyOnWriteArrayList<>( Objects.requireNonNull(runningReceivers, "runningReceivers can not be null") ); @@ -69,8 +67,6 @@ public QueryProcessor(CoordinatorContext coordinatorContext, List { + this.executionTask.ifPresent(sqlPipelineTask -> { for (MultiFragmentsPipelineTask fragmentsTask : sqlPipelineTask.getChildrenTasks().values()) { fragmentsTask.cancelExecute(cancelReason); } From 2cde7b0839d1607ef6dd619b7d9db95bc4c211ee Mon Sep 17 00:00:00 2001 From: 924060929 Date: Mon, 2 Dec 2024 16:41:17 +0800 Subject: [PATCH 24/31] [fix](nereids) fix NereidsCoordinator compute wrong result when exists CTE (#44753) fix NereidsCoordinator compute wrong result when exists CTE, introduced by #41730 --- .../doris/qe/runtime/ThriftPlansBuilder.java | 11 +++- .../distribute/test_multicast_sink.groovy | 60 +++++++++++++++++++ 2 files changed, 68 insertions(+), 3 deletions(-) create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/test_multicast_sink.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java index a02ee90e901cd52..54bc0b24d3e8217 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java @@ -240,14 +240,19 @@ private static Map computeExchangeSenderNum(PipelineDistribute return senderNum; } - private static void setMultiCastDestinationThrift(PipelineDistributedPlan fragmentPlan) { + private static void setMultiCastDestinationThriftIfNotSet(PipelineDistributedPlan fragmentPlan) { MultiCastDataSink multiCastDataSink = (MultiCastDataSink) fragmentPlan.getFragmentJob().getFragment().getSink(); List> destinationList = multiCastDataSink.getDestinations(); List dataStreamSinks = multiCastDataSink.getDataStreamSinks(); for (int i = 0; i < dataStreamSinks.size(); i++) { - DataStreamSink realSink = dataStreamSinks.get(i); List destinations = destinationList.get(i); + if (!destinations.isEmpty()) { + // we should only set destination only once, + // because all backends share the same MultiCastDataSink object + continue; + } + DataStreamSink realSink = dataStreamSinks.get(i); for (Entry> kv : fragmentPlan.getDestinations().entrySet()) { DataSink sink = kv.getKey(); if (sink == realSink) { @@ -318,7 +323,7 @@ private static TPipelineFragmentParams fragmentToThriftIfAbsent( List nonMultiCastDestinations; if (fragment.getSink() instanceof MultiCastDataSink) { nonMultiCastDestinations = Lists.newArrayList(); - setMultiCastDestinationThrift(fragmentPlan); + setMultiCastDestinationThriftIfNotSet(fragmentPlan); } else { nonMultiCastDestinations = nonMultiCastDestinationToThrift(fragmentPlan); } diff --git a/regression-test/suites/nereids_syntax_p0/distribute/test_multicast_sink.groovy b/regression-test/suites/nereids_syntax_p0/distribute/test_multicast_sink.groovy new file mode 100644 index 000000000000000..eeeaad06d5ee225 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/distribute/test_multicast_sink.groovy @@ -0,0 +1,60 @@ +// 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. + +suite("test_multicast_sink") { + multi_sql """ + drop table if exists table_1_undef_partitions2_keys3_properties4_distributed_by5; + CREATE TABLE `table_1_undef_partitions2_keys3_properties4_distributed_by5` ( + `col_int_undef_signed` int NULL, + `col_int_undef_signed_not_null` int NOT NULL, + `col_date_undef_signed` date NULL, + `col_date_undef_signed_not_null` date NOT NULL, + `col_varchar_10__undef_signed` varchar(10) NULL, + `col_varchar_10__undef_signed_not_null` varchar(10) NOT NULL, + `col_varchar_1024__undef_signed` varchar(1024) NULL, + `col_varchar_1024__undef_signed_not_null` varchar(1024) NOT NULL, + `pk` int NULL + ) ENGINE=OLAP + DUPLICATE KEY(`col_int_undef_signed`, `col_int_undef_signed_not_null`, `col_date_undef_signed`) + DISTRIBUTED BY HASH(`pk`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + insert into table_1_undef_partitions2_keys3_properties4_distributed_by5 values(3, 6, '2023-12-17', '2023-12-17', 'ok', 'v', 'want', 'z', 0); + set enable_nereids_distribute_planner=true; + set parallel_pipeline_task_num = 1; + """ + + for (def i in 0..<100) { + test { + sql """ + WITH cte1 AS( + SELECT t1.`pk` + FROM table_1_undef_partitions2_keys3_properties4_distributed_by5 AS t1 + ORDER BY t1.pk + ) + SELECT cte1.`pk` AS pk1 + FROM cte1 + LEFT OUTER JOIN cte1 AS alias1 + ON cte1 . `pk` = alias1 . `pk` + WHERE cte1.`pk` < 3 + LIMIT 66666666 + """ + result([[0]]) + } + } +} From 30b4df9d4f6d3c1f719775b7104c6105af95872d Mon Sep 17 00:00:00 2001 From: xyf <116467462+xyfsjq@users.noreply.github.com> Date: Mon, 2 Dec 2024 17:30:37 +0800 Subject: [PATCH 25/31] [fix](profile) Change the check of whether the username is admin and root to check whether the user specifically corresponds to admin permissions (#41714) --- .../rest/manager/QueryProfileAction.java | 2 +- .../test_http_permissions_check_auth.groovy | 66 +++++++++++++++++++ 2 files changed, 67 insertions(+), 1 deletion(-) create mode 100644 regression-test/suites/http_auth_check/test_http_permissions_check_auth.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/QueryProfileAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/QueryProfileAction.java index 8c3dfa73aea7599..0886edb56fb2f67 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/QueryProfileAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/QueryProfileAction.java @@ -419,7 +419,7 @@ private ResponseEntity getProfileFromAllFrontends(HttpServletRequest request, St private void checkAuthByUserAndQueryId(String queryId) throws AuthenticationException { String user = ConnectContext.get().getCurrentUserIdentity().getQualifiedUser(); - if (!user.equalsIgnoreCase(Auth.ADMIN_USER) && !user.equalsIgnoreCase(Auth.ROOT_USER)) { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { ProfileManager.getInstance().checkAuthByUserAndQueryId(user, queryId); } } diff --git a/regression-test/suites/http_auth_check/test_http_permissions_check_auth.groovy b/regression-test/suites/http_auth_check/test_http_permissions_check_auth.groovy new file mode 100644 index 000000000000000..36c6241a244db7f --- /dev/null +++ b/regression-test/suites/http_auth_check/test_http_permissions_check_auth.groovy @@ -0,0 +1,66 @@ +// 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. + +import org.junit.Assert; + +suite("test_http_permissions_check_auth","p0,auth") { + String suiteName = "test_http_permissions_check_auth" + String tableName = "${suiteName}_table" + String user = "${suiteName}_user" + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """drop table if exists `${tableName}`""" + sql """ + CREATE TABLE `${tableName}` ( + `k1` int, + `k2` int + ) ENGINE=OLAP + DISTRIBUTED BY random BUCKETS auto + PROPERTIES ('replication_num' = '1') ; + """ + sql """insert into ${tableName} values(1,1)""" + sql """set session_context = 'trace_id:mmn9';""" + sql """select * from ${tableName};""" + + def get_queryid_by_traceid = { check_func -> + httpTest { + basicAuthorization "${user}","${pwd}" + endpoint "${context.config.feHttpAddress}" + uri "/rest/v2/manager/query/trace_id/mmn9" + op "get" + check check_func + } + } + + get_queryid_by_traceid.call() { + respCode, body -> + log.info("body:${body}") + assertTrue("${body}".contains("Bad Request")) + } + + sql """grant 'admin' to ${user}""" + + get_queryid_by_traceid.call() { + respCode, body -> + log.info("body:${body}") + assertTrue("${body}".contains("success")) + } + + sql """drop table if exists `${tableName}`""" + try_sql("DROP USER ${user}") +} From 0995618169d807d815d0abb023ff22eb7b3ce656 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Mon, 2 Dec 2024 17:44:17 +0800 Subject: [PATCH 26/31] [opt](nereids) remove enable nereids rules usage (#44769) Related PR: (@35925) Problem Summary: (1) Remove enable_nereids_rules usage, and replaced with use_cbo_rule hint. (2) Current case with use_cbo_rule which has deleted the enable_nereids_rules can't really tracking the rule's impact since the enable_nereids_rules related checking can't be passed. This pr activates these part of plan change. --- .../rules/rewrite/PushDownAggThroughJoin.java | 10 - .../PushDownAggThroughJoinOneSide.java | 10 - .../rewrite/PushDownDistinctThroughJoin.java | 7 - .../eager_aggregate/basic.out | 46 +- .../eager_aggregate/basic_one_side.out | 46 +- .../push_down_count_through_join_one_side.out | 472 ++++++++++++------ .../push_down_max_through_join.out | 168 ++++--- .../push_down_min_through_join.out | 166 +++--- .../push_down_sum_through_join_one_side.out | 254 +++++++--- .../push_down_distinct_through_join.out | 11 +- ...sh_down_count_through_join_one_side.groovy | 1 - ...sh_down_limit_distinct_through_join.groovy | 1 - .../constraints/query23.groovy | 1 - .../tpcds_sf100/constraints/query23.groovy | 1 - 14 files changed, 780 insertions(+), 414 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoin.java index f003d2ac2cc918b..665e3294a93a72e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoin.java @@ -85,11 +85,6 @@ public List buildRules() { ); }) .thenApply(ctx -> { - Set enableNereidsRules = ctx.cascadesContext.getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN.type())) { - return null; - } LogicalAggregate> agg = ctx.root; return pushAgg(agg, agg.child(), ImmutableList.of()); }) @@ -109,11 +104,6 @@ public List buildRules() { ); }) .thenApply(ctx -> { - Set enableNereidsRules = ctx.cascadesContext.getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN.type())) { - return null; - } LogicalAggregate>> agg = ctx.root; return pushAgg(agg, agg.child().child(), agg.child().getProjects()); }) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java index c5d3d0fb49a0a52..9319b09cc304702 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java @@ -79,11 +79,6 @@ public List buildRules() { && (f.children().isEmpty() || f.child(0) instanceof Slot))); }) .thenApply(ctx -> { - Set enableNereidsRules = ctx.cascadesContext.getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE.type())) { - return null; - } LogicalAggregate> agg = ctx.root; return pushMinMaxSumCount(agg, agg.child(), ImmutableList.of()); }) @@ -101,11 +96,6 @@ public List buildRules() { && (f.children().isEmpty() || f.child(0) instanceof Slot)); }) .thenApply(ctx -> { - Set enableNereidsRules = ctx.cascadesContext.getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE.type())) { - return null; - } LogicalAggregate>> agg = ctx.root; return pushMinMaxSumCount(agg, agg.child().child(), agg.child().getProjects()); }) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownDistinctThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownDistinctThroughJoin.java index fa705a08d8db8a9..533452c2f28f61d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownDistinctThroughJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownDistinctThroughJoin.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.rules.rewrite; import org.apache.doris.nereids.jobs.JobContext; -import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.Relation; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; @@ -30,7 +29,6 @@ import com.google.common.collect.ImmutableList; -import java.util.Set; import java.util.function.Function; /** @@ -39,11 +37,6 @@ public class PushDownDistinctThroughJoin extends DefaultPlanRewriter implements CustomRewriter { @Override public Plan rewriteRoot(Plan plan, JobContext context) { - Set enableNereidsRules = context.getCascadesContext().getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_DISTINCT_THROUGH_JOIN.type())) { - return null; - } return plan.accept(this, context); } diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out b/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out index ba18189efcad82a..19b1b3cdb855907 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out @@ -40,14 +40,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------filter((a.event_id = 'ad_click')) -----------PhysicalOlapScan[com_dd_library] ---------filter((cast(experiment_id as DOUBLE) = 37.0)) -----------PhysicalOlapScan[shunt_log_com_dd_library] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------filter((a.event_id = 'ad_click')) +--------------PhysicalOlapScan[com_dd_library] +--------hashAgg[LOCAL] +----------filter((cast(experiment_id as DOUBLE) = 37.0)) +------------PhysicalOlapScan[shunt_log_com_dd_library] Hint log: -Used: -UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: use_push_down_agg_through_join SyntaxError: -- !with_hint_2 -- @@ -55,13 +58,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library] ---------filter((cast(experiment_id as DOUBLE) = 73.0)) -----------PhysicalOlapScan[shunt_log_com_dd_library] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[com_dd_library] +--------hashAgg[LOCAL] +----------filter((cast(experiment_id as DOUBLE) = 73.0)) +------------PhysicalOlapScan[shunt_log_com_dd_library] Hint log: -Used: -UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: use_push_down_agg_through_join SyntaxError: -- !with_hint_3 -- @@ -69,13 +75,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[com_dd_library] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library] Hint log: -Used: -UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: use_push_down_agg_through_join SyntaxError: -- !with_hint_4 -- @@ -83,11 +90,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library] ---------PhysicalOlapScan[shunt_log_com_dd_library] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[com_dd_library] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[shunt_log_com_dd_library] Hint log: -Used: -UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: use_push_down_agg_through_join SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out index aaf6afeca1e9f6e..66f1729e2e38160 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out @@ -40,14 +40,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------filter((a.event_id = 'ad_click')) -----------PhysicalOlapScan[com_dd_library_one_side] ---------filter((cast(experiment_id as DOUBLE) = 37.0)) -----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------filter((a.event_id = 'ad_click')) +--------------PhysicalOlapScan[com_dd_library_one_side] +--------hashAgg[LOCAL] +----------filter((cast(experiment_id as DOUBLE) = 37.0)) +------------PhysicalOlapScan[shunt_log_com_dd_library_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_2 -- @@ -55,13 +58,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library_one_side] ---------filter((cast(experiment_id as DOUBLE) = 73.0)) -----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[com_dd_library_one_side] +--------hashAgg[LOCAL] +----------filter((cast(experiment_id as DOUBLE) = 73.0)) +------------PhysicalOlapScan[shunt_log_com_dd_library_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_3 -- @@ -69,13 +75,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library_one_side] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[com_dd_library_one_side] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_4 -- @@ -83,11 +90,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library_one_side] ---------PhysicalOlapScan[shunt_log_com_dd_library_one_side] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[com_dd_library_one_side] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out index 8267eb3e38ff914..251bc580d3279dd 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out @@ -404,12 +404,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -456,12 +462,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -482,12 +494,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -508,13 +522,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 10)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 10)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -535,13 +555,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 10)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 10)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -550,12 +576,18 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[count_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_mixed_aggregates -- @@ -563,12 +595,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -576,14 +610,26 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] +------------------------hashAgg[LOCAL] +--------------------------PhysicalOlapScan[count_t_one_side] +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -593,12 +639,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -606,12 +658,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- @@ -619,12 +677,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -632,12 +694,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -645,12 +713,20 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -658,13 +734,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((t1.score > 50)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -687,12 +769,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- @@ -700,12 +788,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -713,12 +807,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_function_processed_columns -- @@ -739,14 +835,20 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------filter((count_t_one_side.id < 100)) -----------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((count_t_one_side.id < 100)) +--------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 20) and (t1.id < 100)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_basic -- @@ -754,12 +856,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -806,12 +914,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -832,12 +946,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -845,12 +961,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_subquery -- @@ -858,13 +980,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 10)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 10)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -885,13 +1013,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 10)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 10)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -900,12 +1034,18 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[count_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -913,14 +1053,26 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] +------------------------hashAgg[LOCAL] +--------------------------PhysicalOlapScan[count_t_one_side] +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -930,12 +1082,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -943,12 +1101,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -956,12 +1120,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -969,13 +1139,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((t1.score > 50)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -998,12 +1174,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -1011,12 +1193,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_nested_queries -- @@ -1024,14 +1208,20 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------filter((count_t_one_side.id < 100)) -----------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((count_t_one_side.id < 100)) +--------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 20) and (t1.id < 100)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !shape -- diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out index 79b4ed890ded953..7cf9813e5b71815 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out @@ -224,12 +224,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -276,12 +278,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -302,12 +306,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -329,12 +335,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------filter((max_t.score > 10)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((max_t.score > 10)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -356,12 +364,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------filter((max_t.score > 10)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((max_t.score > 10)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -370,12 +380,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------filter((t1.score > 100)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((t1.score > 100)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_mixed_aggregates -- @@ -383,12 +395,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -396,14 +410,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[max_t] -----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[max_t] +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -413,12 +431,14 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[max_t] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[max_t] ------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -426,12 +446,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- @@ -439,12 +461,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[max_t] ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -452,12 +478,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -465,12 +493,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[max_t] ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -479,12 +511,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((t1.score > 50)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -507,12 +541,14 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[max_t] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[max_t] ------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- @@ -520,12 +556,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -533,12 +571,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_function_processed_columns -- @@ -561,11 +601,13 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------filter((max_t.id < 100)) ----------PhysicalOlapScan[max_t] ---------filter((max_t.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((max_t.score > 20) and (t1.id < 100)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out index d4efe7df0935039..e435bcd4efd3357 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out @@ -224,12 +224,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -276,12 +278,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -302,12 +306,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -329,12 +335,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------filter((min_t.score > 10)) -----------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((min_t.score > 10)) +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -356,12 +364,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------filter((min_t.score > 10)) -----------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((min_t.score > 10)) +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -370,12 +380,14 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[min_t] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[min_t] ----------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_mixed_aggregates -- @@ -383,12 +395,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -396,14 +410,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[min_t] -----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[min_t] +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -413,12 +431,14 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[min_t] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[min_t] ------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -426,12 +446,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- @@ -439,12 +461,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[min_t] ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -452,12 +478,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -465,12 +493,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[min_t] ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -479,12 +511,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((t1.score > 50)) +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -507,12 +541,14 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[min_t] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[min_t] ------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- @@ -520,12 +556,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -533,12 +571,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_function_processed_columns -- @@ -561,11 +601,13 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------filter((min_t.id < 100)) ----------PhysicalOlapScan[min_t] ---------filter((min_t.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((min_t.score > 20) and (t1.id < 100)) +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out index 5013dc7dbdfb93d..3a945f58e29e3c6 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out @@ -224,12 +224,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -276,12 +282,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -302,12 +314,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -328,13 +342,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------filter((sum_t_one_side.score > 10)) -----------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((sum_t_one_side.score > 10)) +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -355,13 +375,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------filter((sum_t_one_side.score > 10)) -----------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((sum_t_one_side.score > 10)) +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -370,12 +396,18 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[sum_t_one_side] -----------PhysicalOlapScan[sum_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[sum_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_mixed_aggregates -- @@ -383,12 +415,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -396,14 +430,26 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[sum_t_one_side] -----------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------PhysicalOlapScan[sum_t_one_side] +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] +------------------------hashAgg[LOCAL] +--------------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -413,12 +459,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[sum_t_one_side] -------------PhysicalOlapScan[sum_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -426,12 +478,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- @@ -439,12 +497,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -452,12 +514,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -465,12 +533,20 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -478,13 +554,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((t1.score > 50)) +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -507,12 +589,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[sum_t_one_side] -------------PhysicalOlapScan[sum_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- @@ -520,12 +608,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -533,12 +627,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_function_processed_columns -- @@ -559,13 +655,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------filter((sum_t_one_side.id < 100)) -----------PhysicalOlapScan[sum_t_one_side] ---------filter((sum_t_one_side.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((sum_t_one_side.id < 100)) +--------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((sum_t_one_side.score > 20) and (t1.id < 100)) +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out b/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out index 65baed4f1194a04..4c5c60acd31a8ee 100644 --- a/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out +++ b/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out @@ -15,14 +15,15 @@ SyntaxError: PhysicalResultSink --hashAgg[LOCAL] ----hashJoin[INNER_JOIN] hashCondition=((t1.id = t3.id)) otherCondition=() -------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[t] ---------PhysicalOlapScan[t] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[t] +----------PhysicalOlapScan[t] ------PhysicalOlapScan[t] Hint log: -Used: -UnUsed: use_push_down_distinct_through_join +Used: use_push_down_distinct_through_join +UnUsed: SyntaxError: -- !basic_sql -- diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy index e551fa04c9110a8..031430a48265ae2 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy @@ -453,7 +453,6 @@ suite("push_down_count_through_join_one_side") { insert into dwd_tracking_sensor_init_tmp_ymd values(1, '2024-08-19', 'click'), (1, '2024-08-19', 'click'); """ sql """ - set ENABLE_NEREIDS_RULES = "PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE"; set disable_join_reorder=true; """ diff --git a/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy b/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy index f53b100c29aff2c..22a59dd1cbdf46f 100644 --- a/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy @@ -21,7 +21,6 @@ suite("push_down_limit_distinct_through_join") { SET enable_fallback_to_original_planner=false; set runtime_filter_mode=OFF; SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'; - SET ENABLE_NEREIDS_RULES=push_down_limit_distinct_through_join; set push_topn_to_agg = true; SET disable_join_reorder=true; set topn_opt_limit_threshold=1024; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy index 96e49d61bfa89f8..9010034d49273e6 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy @@ -37,7 +37,6 @@ suite("query23") { set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; set push_topn_to_agg = true; set topn_opt_limit_threshold=1024; - set enable_nereids_rules='ELIMINATE_GROUP_BY_KEY'; """ def ds = """with frequent_ss_items as diff --git a/regression-test/suites/new_shapes_p0/tpcds_sf100/constraints/query23.groovy b/regression-test/suites/new_shapes_p0/tpcds_sf100/constraints/query23.groovy index b17e9c6d3d0e7f4..82db5a725ce44a6 100644 --- a/regression-test/suites/new_shapes_p0/tpcds_sf100/constraints/query23.groovy +++ b/regression-test/suites/new_shapes_p0/tpcds_sf100/constraints/query23.groovy @@ -40,7 +40,6 @@ suite("query23") { set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; set push_topn_to_agg = true; set topn_opt_limit_threshold=1024; - set enable_nereids_rules='ELIMINATE_GROUP_BY_KEY'; """ def ds = """with frequent_ss_items as From 6f87e35bced5be8f6aec0f6b4a8eb84b496c2efa Mon Sep 17 00:00:00 2001 From: seawinde Date: Mon, 2 Dec 2024 19:14:32 +0800 Subject: [PATCH 27/31] [fix](mtmv) Fix data wrong when query with table operator such as TABLESAMPLE or tablet and so on (#43030) Related PR: #28064 Materialized view is as following: CREATE MATERIALIZED VIEW mv1 BUILD IMMEDIATE REFRESH AUTO ON MANUAL DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') AS select * from orders If run query as following, should rewrite fail by materialized view above to make sure data correctness select * from orders TABLET(110); select * from orders index query_index_test; select * from orders TABLESAMPLE(20 percent); select * from orders_partition PARTITION (day_2); At before, this would rewrite by materialized view succesfully and the result data is wrong, This pr fix this. --- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 6 + .../nereids/rules/analysis/BindRelation.java | 4 + .../mv/InitMaterializationContextHook.java | 5 + .../exploration/mv/MaterializedViewUtils.java | 20 +- .../rules/exploration/mv/StructInfo.java | 7 + .../rules/rewrite/PruneEmptyPartition.java | 7 +- .../rules/rewrite/PruneOlapScanTablet.java | 6 +- .../plans/commands/info/CreateMTMVInfo.java | 3 + .../trees/plans/logical/LogicalOlapScan.java | 55 +++-- .../rewrite/PruneOlapScanTabletTest.java | 21 +- .../doris/nereids/sqltest/SqlTestBase.java | 3 + .../doris/regression/suite/Suite.groovy | 16 ++ .../is_in_debug_mode/is_in_debug_mode.groovy | 156 ++++++++++++++ .../with_table_operator.groovy | 195 ++++++++++++++++++ 14 files changed, 482 insertions(+), 22 deletions(-) create mode 100644 regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy create mode 100644 regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index c0cd47bd5a0f2fb..576e87b44f8d3e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -58,6 +58,12 @@ public static ConnectContext createMTMVContext(MTMV mtmv) { ctx.setCurrentUserIdentity(UserIdentity.ADMIN); ctx.getState().reset(); ctx.setThreadLocalInfo(); + // Debug session variable should be disabled when refreshed + ctx.getSessionVariable().skipDeletePredicate = false; + ctx.getSessionVariable().skipDeleteBitmap = false; + ctx.getSessionVariable().skipDeleteSign = false; + ctx.getSessionVariable().skipStorageEngineMerge = false; + ctx.getSessionVariable().showHiddenColumns = false; ctx.getSessionVariable().allowModifyMaterializedViewData = true; // Disable add default limit rule to avoid refresh data wrong ctx.getSessionVariable().setDisableNereidsRules( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index cba3afca6f08d3b..c62dda5a539df7e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -243,6 +243,10 @@ private LogicalPlan makeOlapScan(TableIf table, UnboundRelation unboundRelation, unboundRelation.getTableSample()); } } + if (!tabletIds.isEmpty()) { + // This tabletIds is set manually, so need to set specifiedTabletIds + scan = scan.withManuallySpecifiedTabletIds(tabletIds); + } if (needGenerateLogicalAggForRandomDistAggTable(scan)) { // it's a random distribution agg table // add agg on olap scan diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index 2e8baecf1656390..f9ea00e178b5b72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -79,6 +79,11 @@ public void initMaterializationContext(CascadesContext cascadesContext) { * @param cascadesContext current cascadesContext in the planner */ protected void doInitMaterializationContext(CascadesContext cascadesContext) { + if (cascadesContext.getConnectContext().getSessionVariable().isInDebugMode()) { + LOG.info(String.format("MaterializationContext init return because is in debug mode, current queryId is %s", + cascadesContext.getConnectContext().getQueryIdentifier())); + return; + } // Only collect the table or mv which query use directly, to avoid useless mv partition in rewrite TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), false); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index 484abd11f01e726..ee4b002007ea053 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -347,7 +347,15 @@ public Plan visitLogicalResultSink(LogicalResultSink logicalResu planner.getCascadesContext().getMemo().getRoot().getStatistics(), null); } - private static final class TableQueryOperatorChecker extends DefaultPlanVisitor { + /** + * Check the query if Contains query operator + * Such sql as following should return true + * select * from orders TABLET(10098) because TABLET(10098) should return true + * select * from orders_partition PARTITION (day_2) because PARTITION (day_2) + * select * from orders index query_index_test because index query_index_test + * select * from orders TABLESAMPLE(20 percent) because TABLESAMPLE(20 percent) + * */ + public static final class TableQueryOperatorChecker extends DefaultPlanVisitor { public static final TableQueryOperatorChecker INSTANCE = new TableQueryOperatorChecker(); @Override @@ -358,12 +366,20 @@ public Boolean visitLogicalRelation(LogicalRelation relation, Void context) { if (relation instanceof LogicalOlapScan) { LogicalOlapScan logicalOlapScan = (LogicalOlapScan) relation; if (logicalOlapScan.getTableSample().isPresent()) { + // Contain sample, select * from orders TABLESAMPLE(20 percent) return true; } - if (!logicalOlapScan.getSelectedTabletIds().isEmpty()) { + if (!logicalOlapScan.getManuallySpecifiedTabletIds().isEmpty()) { + // Contain tablets, select * from orders TABLET(10098) because TABLET(10098) return true; } if (!logicalOlapScan.getManuallySpecifiedPartitions().isEmpty()) { + // Contain specified partitions, select * from orders_partition PARTITION (day_2) + return true; + } + if (logicalOlapScan.getSelectedIndexId() != logicalOlapScan.getTable().getBaseIndexId()) { + // Contains select index or use sync mv in rbo rewrite + // select * from orders index query_index_test return true; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java index 526ec7030d2db59..5a84ab787d7bd51 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils.TableQueryOperatorChecker; import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate; import org.apache.doris.nereids.trees.copier.DeepCopierContext; import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; @@ -36,6 +37,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.plans.AbstractPlan; import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.ObjectId; @@ -323,6 +325,11 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable cascadesContext); valid = valid && hyperGraph.getNodes().stream().allMatch(n -> ((StructInfoNode) n).getExpressions() != null); + // if relationList has any relation which contains table operator, + // such as query with sample, index, table, is invalid + boolean invalid = relationList.stream().anyMatch(relation -> + ((AbstractPlan) relation).accept(TableQueryOperatorChecker.INSTANCE, null)); + valid = valid && !invalid; // collect predicate from top plan which not in hyper graph Set topPlanPredicates = new LinkedHashSet<>(); topPlan.accept(PREDICATE_COLLECTOR, topPlanPredicates); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java index 5465211c73e7931..c7b8f452afbe667 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java @@ -36,11 +36,16 @@ public Rule build() { return logicalOlapScan().thenApply(ctx -> { LogicalOlapScan scan = ctx.root; OlapTable table = scan.getTable(); - List ids = table.selectNonEmptyPartitionIds(scan.getSelectedPartitionIds()); + List partitionIdsToPrune = scan.getSelectedPartitionIds(); + List ids = table.selectNonEmptyPartitionIds(partitionIdsToPrune); if (ids.isEmpty()) { return new LogicalEmptyRelation(ConnectContext.get().getStatementContext().getNextRelationId(), scan.getOutput()); } + if (partitionIdsToPrune.equals(ids)) { + // Not Prune actually, return directly + return null; + } return scan.withSelectedPartitionIds(ids); }).toRule(RuleType.PRUNE_EMPTY_PARTITION); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java index 4b86b7a3706f0ea..fc212550b6f3645 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java @@ -53,7 +53,7 @@ public Rule build() { LogicalOlapScan olapScan = filter.child(); OlapTable table = olapScan.getTable(); Builder selectedTabletIdsBuilder = ImmutableList.builder(); - if (olapScan.getSelectedTabletIds().isEmpty()) { + if (olapScan.getManuallySpecifiedTabletIds().isEmpty()) { for (Long id : olapScan.getSelectedPartitionIds()) { Partition partition = table.getPartition(id); MaterializedIndex index = partition.getIndex(olapScan.getSelectedIndexId()); @@ -64,10 +64,10 @@ public Rule build() { partition.getDistributionInfo())); } } else { - selectedTabletIdsBuilder.addAll(olapScan.getSelectedTabletIds()); + selectedTabletIdsBuilder.addAll(olapScan.getManuallySpecifiedTabletIds()); } List selectedTabletIds = selectedTabletIdsBuilder.build(); - if (new HashSet<>(selectedTabletIds).equals(new HashSet<>(olapScan.getSelectedTabletIds()))) { + if (new HashSet<>(selectedTabletIds).equals(new HashSet<>(olapScan.getManuallySpecifiedTabletIds()))) { return null; } return filter.withChildren(olapScan.withSelectedTabletIds(selectedTabletIds)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 5c6c71c92e81119..f9e298fd9d40d49 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -157,6 +157,9 @@ public void analyze(ConnectContext ctx) throws Exception { if (!InternalCatalog.INTERNAL_CATALOG_NAME.equals(mvName.getCtl())) { throw new AnalysisException("Only support creating asynchronous materialized views in internal catalog"); } + if (ctx.getSessionVariable().isInDebugMode()) { + throw new AnalysisException("Create materialized view fail, because is in debug mode"); + } try { FeNameFormat.checkTableName(mvName.getTbl()); } catch (org.apache.doris.common.AnalysisException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java index 90ceb24231340c7..2216e58c4fa3b86 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java @@ -99,6 +99,12 @@ public class LogicalOlapScan extends LogicalCatalogRelation implements OlapScan */ private final List selectedTabletIds; + /** + * Selected tablet ids to read data from, this would be set if user query with tablets manually + * Such as select * from orders TABLET(100); + */ + private final List manuallySpecifiedTabletIds; + /////////////////////////////////////////////////////////////////////////// // Members for partition ids. /////////////////////////////////////////////////////////////////////////// @@ -127,12 +133,16 @@ public LogicalOlapScan(RelationId id, OlapTable table) { this(id, table, ImmutableList.of()); } + /** + * LogicalOlapScan construct method + */ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier) { this(id, table, qualifier, Optional.empty(), Optional.empty(), table.getPartitionIds(), false, ImmutableList.of(), -1, false, PreAggStatus.unset(), ImmutableList.of(), ImmutableList.of(), - Maps.newHashMap(), Optional.empty(), false, ImmutableMap.of()); + Maps.newHashMap(), Optional.empty(), false, ImmutableMap.of(), + ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List tabletIds, @@ -140,7 +150,7 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L this(id, table, qualifier, Optional.empty(), Optional.empty(), table.getPartitionIds(), false, tabletIds, -1, false, PreAggStatus.unset(), ImmutableList.of(), hints, Maps.newHashMap(), - tableSample, false, ImmutableMap.of()); + tableSample, false, ImmutableMap.of(), ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List specifiedPartitions, @@ -149,7 +159,7 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L // must use specifiedPartitions here for prune partition by sql like 'select * from t partition p1' specifiedPartitions, false, tabletIds, -1, false, PreAggStatus.unset(), specifiedPartitions, hints, Maps.newHashMap(), - tableSample, false, ImmutableMap.of()); + tableSample, false, ImmutableMap.of(), ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List tabletIds, @@ -158,7 +168,8 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L this(id, table, qualifier, Optional.empty(), Optional.empty(), selectedPartitionIds, false, tabletIds, selectedIndexId, true, preAggStatus, - specifiedPartitions, hints, Maps.newHashMap(), tableSample, true, ImmutableMap.of()); + specifiedPartitions, hints, Maps.newHashMap(), tableSample, true, ImmutableMap.of(), + ImmutableList.of()); } /** @@ -171,7 +182,7 @@ public LogicalOlapScan(RelationId id, Table table, List qualifier, PreAggStatus preAggStatus, List specifiedPartitions, List hints, Map, Slot> cacheSlotWithSlotName, Optional tableSample, boolean directMvScan, - Map>> colToSubPathsMap) { + Map>> colToSubPathsMap, List specifiedTabletIds) { super(id, PlanType.LOGICAL_OLAP_SCAN, table, qualifier, groupExpression, logicalProperties); Preconditions.checkArgument(selectedPartitionIds != null, @@ -182,6 +193,7 @@ public LogicalOlapScan(RelationId id, Table table, List qualifier, this.indexSelected = indexSelected; this.preAggStatus = preAggStatus; this.manuallySpecifiedPartitions = ImmutableList.copyOf(specifiedPartitions); + this.manuallySpecifiedTabletIds = ImmutableList.copyOf(specifiedTabletIds); if (selectedPartitionIds.isEmpty()) { this.selectedPartitionIds = ImmutableList.of(); @@ -240,6 +252,7 @@ public boolean equals(Object o) { && partitionPruned == that.partitionPruned && Objects.equals(preAggStatus, that.preAggStatus) && Objects.equals(selectedTabletIds, that.selectedTabletIds) && Objects.equals(manuallySpecifiedPartitions, that.manuallySpecifiedPartitions) + && Objects.equals(manuallySpecifiedTabletIds, that.manuallySpecifiedTabletIds) && Objects.equals(selectedPartitionIds, that.selectedPartitionIds) && Objects.equals(hints, that.hints) && Objects.equals(tableSample, that.tableSample); @@ -248,8 +261,8 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash(super.hashCode(), selectedIndexId, indexSelected, preAggStatus, cacheSlotWithSlotName, - selectedTabletIds, partitionPruned, manuallySpecifiedPartitions, selectedPartitionIds, hints, - tableSample); + selectedTabletIds, partitionPruned, manuallySpecifiedTabletIds, manuallySpecifiedPartitions, + selectedPartitionIds, hints, tableSample); } @Override @@ -258,7 +271,7 @@ public LogicalOlapScan withGroupExpression(Optional groupExpres groupExpression, Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } @Override @@ -267,7 +280,7 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr return new LogicalOlapScan(relationId, (Table) table, qualifier, groupExpression, logicalProperties, selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withSelectedPartitionIds(List selectedPartitionIds) { @@ -275,7 +288,7 @@ public LogicalOlapScan withSelectedPartitionIds(List selectedPartitionIds) Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, true, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withMaterializedIndexSelected(long indexId) { @@ -283,7 +296,7 @@ public LogicalOlapScan withMaterializedIndexSelected(long indexId) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, indexId, true, PreAggStatus.unset(), manuallySpecifiedPartitions, hints, cacheSlotWithSlotName, - tableSample, directMvScan, colToSubPathsMap); + tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withSelectedTabletIds(List selectedTabletIds) { @@ -291,7 +304,7 @@ public LogicalOlapScan withSelectedTabletIds(List selectedTabletIds) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withPreAggStatus(PreAggStatus preAggStatus) { @@ -299,7 +312,7 @@ public LogicalOlapScan withPreAggStatus(PreAggStatus preAggStatus) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withColToSubPathsMap(Map>> colToSubPathsMap) { @@ -307,7 +320,15 @@ public LogicalOlapScan withColToSubPathsMap(Map>> colTo Optional.empty(), Optional.empty(), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); + } + + public LogicalOlapScan withManuallySpecifiedTabletIds(List manuallySpecifiedTabletIds) { + return new LogicalOlapScan(relationId, (Table) table, qualifier, + Optional.empty(), Optional.of(getLogicalProperties()), + selectedPartitionIds, partitionPruned, selectedTabletIds, + selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } @Override @@ -317,7 +338,7 @@ public LogicalOlapScan withRelationId(RelationId relationId) { Optional.empty(), Optional.empty(), selectedPartitionIds, false, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, Maps.newHashMap(), tableSample, directMvScan, colToSubPathsMap); + hints, Maps.newHashMap(), tableSample, directMvScan, colToSubPathsMap, selectedTabletIds); } @Override @@ -333,6 +354,10 @@ public List getSelectedTabletIds() { return selectedTabletIds; } + public List getManuallySpecifiedTabletIds() { + return manuallySpecifiedTabletIds; + } + @Override public long getSelectedIndexId() { return selectedIndexId; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java index e7940cdfb9b519f..385657cd3f229ef 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java @@ -28,6 +28,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.nereids.sqltest.SqlTestBase; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; import org.apache.doris.nereids.trees.expressions.InPredicate; @@ -51,8 +52,9 @@ import org.junit.jupiter.api.Test; import java.util.List; +import java.util.Objects; -class PruneOlapScanTabletTest implements MemoPatternMatchSupported { +class PruneOlapScanTabletTest extends SqlTestBase implements MemoPatternMatchSupported { @Test void testPruneOlapScanTablet(@Mocked OlapTable olapTable, @@ -154,4 +156,21 @@ void testPruneOlapScanTablet(@Mocked OlapTable olapTable, ) ); } + + @Test + void testPruneOlapScanTabletWithManually() { + String sql = "select * from T4 TABLET(110) where id > 8"; + PlanChecker.from(connectContext) + .analyze(sql) + .applyTopDown(new PruneOlapScanTablet()) + .matches( + logicalFilter( + logicalOlapScan().when(s -> + Objects.equals(s.getSelectedTabletIds(), Lists.newArrayList(110L)) + && Objects.equals(s.getManuallySpecifiedTabletIds(), + Lists.newArrayList(110L)) + ) + ) + ); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java index f5b301dcc49b25e..9ebbc22a85c8261 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java @@ -87,6 +87,9 @@ protected void runBeforeAll() throws Exception { + " score bigint\n" + ")\n" + "DUPLICATE KEY(id)\n" + + "AUTO PARTITION BY LIST(`id`)\n" + + "(\n" + + ")\n" + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + "PROPERTIES (\n" + " \"replication_num\" = \"1\"\n" diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 62e6032578fd6a7..744edfe54cbc24d 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -1876,6 +1876,22 @@ class Suite implements GroovyInterceptable { sql "analyze table ${db}.${mv_name} with sync;" } + def create_async_partition_mv = { db, mv_name, mv_sql, partition_col -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${db}.${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${db}.${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + PARTITION BY ${partition_col} + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + sql "analyze table ${db}.${mv_name} with sync;" + } + // mv not part in rewrite process void mv_not_part_in(query_sql, mv_name, sync_cbo_rewrite = enable_sync_mv_cost_based_rewrite()) { logger.info("query_sql = " + query_sql + ", mv_names = " + mv_name + ", sync_cbo_rewrite = " + sync_cbo_rewrite) diff --git a/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy b/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy new file mode 100644 index 000000000000000..15d93e32f65dc2b --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy @@ -0,0 +1,156 @@ +// 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. + +import org.junit.Assert + +suite("is_in_debug_mode") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists orders; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + UNIQUE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + create_async_mv(db, "basic_mv", """ + select * from orders where o_orderkey > 1; + """) + + sql """set skip_delete_sign = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_sign = false;""" + + + sql """set skip_storage_engine_merge = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_storage_engine_merge = false;""" + + + sql """set skip_delete_bitmap = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_bitmap = false;""" + + + sql """set skip_delete_predicate = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_predicate = false;""" + + + sql """set show_hidden_columns = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set show_hidden_columns = false;""" + + sql """drop materialized view if exists basic_mv""" +} + diff --git a/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy b/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy new file mode 100644 index 000000000000000..5ba2c05053e2a29 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy @@ -0,0 +1,195 @@ +// 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. + +suite("with_table_operator") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists orders; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + drop table if exists orders_partition; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders_partition ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) ( + PARTITION `day_2` VALUES LESS THAN ('2023-12-9'), + PARTITION `day_3` VALUES LESS THAN ("2023-12-11"), + PARTITION `day_4` VALUES LESS THAN ("2023-12-30") + ) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders_partition values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + create_async_mv(db, "complete_agg_mv", """ + select count(*),o_orderdate + from orders + group by o_orderdate; + """) + + create_async_mv(db, "complete_mv", """ + select * from orders; + """) + + create_async_partition_mv(db, "partition_agg_mv", """ + select count(*),o_orderdate + from orders_partition + group by o_orderdate; + """, "(o_orderdate)") + + create_async_partition_mv(db, "partition_mv", """ + select * from orders_partition; + """, "(o_orderdate)") + + // query rewrite with tablet should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders TABLET(110) + group by o_orderdate; + """, ["complete_mv", "complete_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders TABLET(110);""", ["complete_mv", "complete_agg_mv"]) + + // query rewrite with index should fail + createMV(""" + create materialized view query_index_test + as + select o_orderdate, count(*) + from orders + group by o_orderdate; + """) + mv_rewrite_all_fail(""" + select * from orders index query_index_test; + """, ["complete_mv", "complete_agg_mv"]) + + // query rewrite with sample should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders TABLESAMPLE(20 percent) + group by o_orderdate; + """, ["complete_mv", "complete_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders TABLESAMPLE(20 percent);""", ["complete_mv", "complete_agg_mv"]) + + // query rewrite with partition should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders_partition PARTITION (day_2) + group by o_orderdate; + """, ["partition_mv", "partition_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders_partition PARTITION (day_2);""", ["partition_mv", "partition_agg_mv"]) + + + sql """ + DROP MATERIALIZED VIEW IF EXISTS complete_agg_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS partition_agg_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS complete_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS partition_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS query_index_test on orders; + """ +} + From ce9c7174fb5ea04781d042d72b69a677793bb965 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 2 Dec 2024 19:45:27 +0800 Subject: [PATCH 28/31] [fix](auth) Prohibit deleting admin user (#44751) --- .../apache/doris/analysis/DropUserStmt.java | 4 +- .../apache/doris/analysis/UserIdentity.java | 4 ++ .../trees/plans/commands/DropUserCommand.java | 4 +- .../suites/account_p0/test_system_user.groovy | 39 +++++++++++++++++++ 4 files changed, 47 insertions(+), 4 deletions(-) create mode 100644 regression-test/suites/account_p0/test_system_user.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java index f9bb218744fc85a..5f9872e42ad7cc8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java @@ -63,8 +63,8 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { userIdent.analyze(); - if (userIdent.isRootUser()) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop root user"); + if (userIdent.isSystemUser()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop system user"); } // only user with GLOBAL level's GRANT_PRIV can drop user. diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java index 122f28ca45092bd..6898915dab73cec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java @@ -189,6 +189,10 @@ public boolean isAdminUser() { return user.equals(Auth.ADMIN_USER); } + public boolean isSystemUser() { + return isRootUser() || isAdminUser(); + } + public TUserIdentity toThrift() { Preconditions.checkState(isAnalyzed); TUserIdentity tUserIdent = new TUserIdentity(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java index 094ce06c36143e5..5c4fb407e402133 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java @@ -55,8 +55,8 @@ public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { userIdent.analyze(); - if (userIdent.isRootUser()) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop root user"); + if (userIdent.isSystemUser()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop system user"); } // only user with GLOBAL level's GRANT_PRIV can drop user. diff --git a/regression-test/suites/account_p0/test_system_user.groovy b/regression-test/suites/account_p0/test_system_user.groovy new file mode 100644 index 000000000000000..1805f1669ea570a --- /dev/null +++ b/regression-test/suites/account_p0/test_system_user.groovy @@ -0,0 +1,39 @@ +// 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. + +import org.junit.Assert; + +suite("test_system_user") { + test { + sql """ + create user `root`; + """ + exception "root" + } + test { + sql """ + drop user `root`; + """ + exception "system" + } + test { + sql """ + drop user `admin`; + """ + exception "system" + } +} From 70b0a08815b1ea4e39a8e39f8f067ea27c795abb Mon Sep 17 00:00:00 2001 From: feiniaofeiafei Date: Mon, 2 Dec 2024 20:04:14 +0800 Subject: [PATCH 29/31] [fix](nereids) fix merge_percentile_to_array when has same agg function (#44783) Related PR: #34313 Problem Summary The original PR did not handle the following scenario: ```sql SELECT SUM(a), PERCENTILE(pk, 0.1) AS c1, PERCENTILE(pk, 0.1) AS c2, PERCENTILE(pk, 0.4) AS c3 FROM test_merge_percentile; ``` In this case, the aggregate outputs include two identical functions (PERCENTILE(pk, 0.1)). When constructing the LogicalProject, a map was used where the key is the child of an Alias and the value is the Alias itself. However, this approach loses information when two Aliases share the same child. This PR modifies the map structure to use the child of an Alias as the key and a list of Alias objects as the value. This ensures that all Alias instances with the same child are preserved, resolving the issue of lost information in such cases. --- .../rules/rewrite/MergePercentileToArray.java | 26 +++++++++---------- .../merge_percentile_to_array.out | 12 +++++++++ .../merge_percentile_to_array.groovy | 4 +++ 3 files changed, 28 insertions(+), 14 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergePercentileToArray.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergePercentileToArray.java index f92ad84bde85257..fe81adf13bf29d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergePercentileToArray.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergePercentileToArray.java @@ -152,10 +152,10 @@ private Plan doMerge(LogicalAggregate aggregate) { (List) (List) newPercentileArrays); ImmutableList.Builder newProjectOutputExpressions = ImmutableList.builder(); newProjectOutputExpressions.addAll((List) (List) notChangeForProject); - Map existsAliasMap = Maps.newHashMap(); + Map> existsAliasMap = Maps.newHashMap(); // existsAliasMap is used to keep upper plan refer the same expr for (Alias alias : existsAliases) { - existsAliasMap.put(alias.child(), alias); + existsAliasMap.computeIfAbsent(alias.child(), k -> new ArrayList<>()).add(alias); } Map slotMap = Maps.newHashMap(); // slotMap is used to find the correspondence @@ -169,20 +169,22 @@ private Plan doMerge(LogicalAggregate aggregate) { for (Map.Entry> entry : funcMap.entrySet()) { for (int i = 0; i < entry.getValue().size(); i++) { AggregateFunction aggFunc = entry.getValue().get(i); - Alias originAlias = existsAliasMap.get(aggFunc); - DistinctAndExpr distinctAndExpr = new DistinctAndExpr(aggFunc.child(0), aggFunc.isDistinct()); - Alias newAlias = new Alias(originAlias.getExprId(), new ElementAt(slotMap.get(distinctAndExpr), - new IntegerLiteral(i + 1)), originAlias.getName()); - newProjectOutputExpressions.add(newAlias); + List originAliases = existsAliasMap.get(aggFunc); + for (Alias originAlias : originAliases) { + DistinctAndExpr distinctAndExpr = new DistinctAndExpr(aggFunc.child(0), aggFunc.isDistinct()); + Alias newAlias = new Alias(originAlias.getExprId(), new ElementAt(slotMap.get(distinctAndExpr), + new IntegerLiteral(i + 1)), originAlias.getName()); + newProjectOutputExpressions.add(newAlias); + } } } newProjectOutputExpressions.addAll(groupBySlots); - return new LogicalProject(newProjectOutputExpressions.build(), newAggregate); + return new LogicalProject<>(newProjectOutputExpressions.build(), newAggregate); } private static class DistinctAndExpr { - private Expression expression; - private boolean isDistinct; + private final Expression expression; + private final boolean isDistinct; public DistinctAndExpr(Expression expression, boolean isDistinct) { this.expression = expression; @@ -193,10 +195,6 @@ public Expression getExpression() { return expression; } - public boolean isDistinct() { - return isDistinct; - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/regression-test/data/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.out b/regression-test/data/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.out index b495302e80d3c85..1b2f876cfba50a8 100644 --- a/regression-test/data/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.out +++ b/regression-test/data/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.out @@ -41,3 +41,15 @@ 7.0 \N \N 7.0 7.0 7 +-- !same_percentile -- +52 1.0 1.0 2.0 + +-- !same_percentile_group_by -- +\N 6.0 6.0 6.0 +2 3.0 3.0 3.0 +25 3.0 3.0 3.0 +4 2.0 2.0 2.0 +5 1.0 1.0 1.6 +7 6.0 6.0 6.0 +9 1.2 1.2 1.8 + diff --git a/regression-test/suites/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.groovy b/regression-test/suites/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.groovy index 2071d75ae85d4ef..5bb13c6336c2640 100644 --- a/regression-test/suites/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.groovy +++ b/regression-test/suites/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.groovy @@ -57,4 +57,8 @@ suite("merge_percentile_to_array") { percentile(abs(a), 0.55) as c2 from test_merge_percentile group by a) t; """ + order_qt_same_percentile """select sum(a),percentile(pk, 0.1) as c1 , percentile(pk, 0.1) as c2 , + percentile(pk, 0.4) as c2 from test_merge_percentile;""" + order_qt_same_percentile_group_by """select sum(a),percentile(pk, 0.1) as c1 , percentile(pk, 0.1) as c2 , + percentile(pk, 0.4) as c2 from test_merge_percentile group by a;""" } \ No newline at end of file From 21e1d6d8a7ccb08848e7be61ae574c7ec4b27e69 Mon Sep 17 00:00:00 2001 From: Mingxi <71588583+joker-star-l@users.noreply.github.com> Date: Mon, 2 Dec 2024 20:27:10 +0800 Subject: [PATCH 30/31] [fix](logstash) remove ShortNameResolver to solve thread race problem (#44598) remove ShortNameResolver to solve thread race problem --- .../logstash/lib/logstash/outputs/doris.rb | 50 +--------------- .../lib/logstash/util/shortname_resolver.rb | 58 ------------------- .../logstash/logstash-output-doris.gemspec | 1 - 3 files changed, 2 insertions(+), 107 deletions(-) delete mode 100644 extension/logstash/lib/logstash/util/shortname_resolver.rb diff --git a/extension/logstash/lib/logstash/outputs/doris.rb b/extension/logstash/lib/logstash/outputs/doris.rb index 02e7591b0a354bc..21d3ee6e752b08e 100644 --- a/extension/logstash/lib/logstash/outputs/doris.rb +++ b/extension/logstash/lib/logstash/outputs/doris.rb @@ -21,7 +21,6 @@ require "logstash/outputs/base" require "logstash/namespace" require "logstash/json" -require "logstash/util/shortname_resolver" require 'logstash/util/formater' require "uri" require "securerandom" @@ -67,8 +66,6 @@ class LogStash::Outputs::Doris < LogStash::Outputs::Base config :save_file, :validate => :string, :default => "failed.data" - config :host_resolve_ttl_sec, :validate => :number, :default => 120 - config :max_retries, :validate => :number, :default => -1 config :log_request, :validate => :boolean, :default => true @@ -92,10 +89,6 @@ def print_plugin_info() def register @http_query = "/api/#{@db}/#{@table}/_stream_load" - @hostnames_pool = - parse_http_hosts(@http_hosts, - ShortNameResolver.new(ttl: @host_resolve_ttl_sec, logger: @logger)) - @request_headers = make_request_headers @logger.info("request headers: ", @request_headers) @@ -141,39 +134,6 @@ def register print_plugin_info() end # def register - private - - def parse_http_hosts(hosts, resolver) - ip_re = /^[\d]+\.[\d]+\.[\d]+\.[\d]+$/ - - lambda { - hosts.flat_map { |h| - scheme = URI(h).scheme - host = URI(h).host - port = URI(h).port - path = URI(h).path - - if ip_re !~ host - resolver.get_addresses(host).map { |ip| - "#{scheme}://#{ip}:#{port}#{path}" - } - else - [h] - end - } - } - end - - private - - def get_host_addresses() - begin - @hostnames_pool.call - rescue Exception => ex - @logger.error('Error while resolving host', :error => ex.to_s) - end - end - def multi_receive(events) return if events.empty? send_events(events) @@ -191,8 +151,6 @@ def send_events(events) # @logger.info("get event num: #{event_num}") @logger.debug("get documents: #{documents}") - hosts = get_host_addresses() - http_headers = @request_headers.dup if !@group_commit # only set label if group_commit is off_mode or not set, since lable can not be used with group_commit @@ -202,7 +160,7 @@ def send_events(events) req_count = 0 sleep_for = 1 while true - response = make_request(documents, http_headers, hosts, @http_query, hosts.sample) + response = make_request(documents, http_headers, @http_query, @http_hosts.sample) req_count += 1 response_json = {} @@ -246,11 +204,7 @@ def send_events(events) end private - def make_request(documents, http_headers, hosts, query, host = "") - if host == "" - host = hosts.pop - end - + def make_request(documents, http_headers, query, host) url = host + query if @log_request or @logger.debug? diff --git a/extension/logstash/lib/logstash/util/shortname_resolver.rb b/extension/logstash/lib/logstash/util/shortname_resolver.rb deleted file mode 100644 index 1437ccba0079889..000000000000000 --- a/extension/logstash/lib/logstash/util/shortname_resolver.rb +++ /dev/null @@ -1,58 +0,0 @@ -=begin -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. -=end -require 'resolv' -require 'mini_cache' - -class ShortNameResolver - def initialize(ttl:, logger:) - @ttl = ttl - @store = MiniCache::Store.new - @logger = logger - end - - private - def resolve_cached(shortname) - @store.get_or_set(shortname) do - addresses = resolve(shortname) - raise "Bad shortname '#{shortname}'" if addresses.empty? - MiniCache::Data.new(addresses, expires_in: @ttl) - end - end - - private - def resolve(shortname) - addresses = Resolv::DNS.open do |dns| - dns.getaddresses(shortname).map { |r| r.to_s } - end - - @logger.info("Resolved shortname '#{shortname}' to addresses #{addresses}") - - return addresses - end - - public - def get_address(shortname) - return resolve_cached(shortname).sample - end - - public - def get_addresses(shortname) - return resolve_cached(shortname) - end -end diff --git a/extension/logstash/logstash-output-doris.gemspec b/extension/logstash/logstash-output-doris.gemspec index 689b93503f6a8b2..30341b83156ed4d 100644 --- a/extension/logstash/logstash-output-doris.gemspec +++ b/extension/logstash/logstash-output-doris.gemspec @@ -38,7 +38,6 @@ Gem::Specification.new do |s| # Gem dependencies s.add_runtime_dependency "logstash-core-plugin-api", ">= 1.60", "<= 2.99" - s.add_runtime_dependency 'mini_cache', ">= 1.0.0", "< 2.0.0" s.add_runtime_dependency "rest-client", '~> 2.1' s.add_development_dependency 'logstash-devutils', '~> 1.3' From 303e27597e3f6c2ee82033ab90593ae8d2599628 Mon Sep 17 00:00:00 2001 From: bobhan1 Date: Mon, 2 Dec 2024 21:42:18 +0800 Subject: [PATCH 31/31] [Fix](TPartitionVersionInfo) Fix duplicate `TPartitionVersionInfo` in `PublishVersionTask.partitionVersionInfos` (#44846) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What problem does this PR solve? Problem Summary: When FE handles BEs' tablet report and finds that there exists some expired txns on BE, it will generate publish version task. `TPartitionVersionInfo` with same values may be added to `transactionsToPublish` under same txn id many times when partitions involved in this failed txn involves many tablets on this BE. Because it uses `ArrayListMultimap`, these duplicate values may occupy a lot of memories when the number of tablets is large. ### Release note Fixed the issue of FE's memory occupation growing too fast in cases of persistent load and clone failures on merge-on-write tables. 修复了在merge-on-write表上有持续的导入失败和clone失败的情况下,FE使用内存增长过快的问题。 --- .../apache/doris/catalog/TabletInvertedIndex.java | 13 +++++++------ .../java/org/apache/doris/master/ReportHandler.java | 9 +++++---- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java index 4a421dc7b2b1efd..a51d1f55014fcce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java @@ -37,13 +37,14 @@ import org.apache.doris.transaction.TransactionStatus; import com.google.common.base.Preconditions; -import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.LinkedHashMultimap; import com.google.common.collect.ListMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; import com.google.common.collect.Table; import com.google.common.collect.TreeMultimap; @@ -135,7 +136,7 @@ public void tabletReport(long backendId, Map backendTablets, Set tabletFoundInMeta, ListMultimap tabletMigrationMap, Map partitionVersionSyncMap, - Map> transactionsToPublish, + Map> transactionsToPublish, ListMultimap transactionsToClear, ListMultimap tabletRecoveryMap, List tabletToUpdate, @@ -314,7 +315,7 @@ && isLocal(tabletMeta.getStorageMedium())) { } private void handleBackendTransactions(long backendId, List transactionIds, long tabletId, - TabletMeta tabletMeta, Map> transactionsToPublish, + TabletMeta tabletMeta, Map> transactionsToPublish, ListMultimap transactionsToClear) { GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); long partitionId = tabletMeta.getPartitionId(); @@ -376,15 +377,15 @@ private TPartitionVersionInfo generatePartitionVersionInfoWhenReport(Transaction } private void publishPartition(TransactionState transactionState, long transactionId, TabletMeta tabletMeta, - long partitionId, Map> transactionsToPublish) { + long partitionId, Map> transactionsToPublish) { TPartitionVersionInfo versionInfo = generatePartitionVersionInfoWhenReport(transactionState, transactionId, tabletMeta, partitionId); if (versionInfo != null) { synchronized (transactionsToPublish) { - ListMultimap map = transactionsToPublish.get( + SetMultimap map = transactionsToPublish.get( transactionState.getDbId()); if (map == null) { - map = ArrayListMultimap.create(); + map = LinkedHashMultimap.create(); transactionsToPublish.put(transactionState.getDbId(), map); } map.put(transactionId, versionInfo); diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index c5c72eae3c5a8ba..06047e2cf16682e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -95,6 +95,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Queues; +import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -503,7 +504,7 @@ public void tabletReport(long backendId, Map backendTablets, Map partitionVersionSyncMap = Maps.newConcurrentMap(); // dbid -> txn id -> [partition info] - Map> transactionsToPublish = Maps.newHashMap(); + Map> transactionsToPublish = Maps.newHashMap(); ListMultimap transactionsToClear = LinkedListMultimap.create(); // db id -> tablet id @@ -1148,14 +1149,14 @@ private static void handleMigration(ListMultimap tabletMet } private static void handleRepublishVersionInfo( - Map> transactionsToPublish, long backendId) { + Map> transactionsToPublish, long backendId) { AgentBatchTask batchTask = new AgentBatchTask(); long createPublishVersionTaskTime = System.currentTimeMillis(); for (Long dbId : transactionsToPublish.keySet()) { - ListMultimap map = transactionsToPublish.get(dbId); + SetMultimap map = transactionsToPublish.get(dbId); for (long txnId : map.keySet()) { PublishVersionTask task = new PublishVersionTask(backendId, txnId, dbId, - map.get(txnId), createPublishVersionTaskTime); + Lists.newArrayList(map.get(txnId)), createPublishVersionTaskTime); batchTask.addTask(task); // add to AgentTaskQueue for handling finish report. AgentTaskQueue.addTask(task);