You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by kx...@apache.org on 2023/06/06 15:15:08 UTC

[doris] 06/36: [fix](nereids) select with specified partition name is not work as expected (#20269)

This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 5c4c653aac2e0abb8427266ab11010a883a31755
Author: AKIRA <33...@users.noreply.github.com>
AuthorDate: Mon Jun 5 13:48:54 2023 +0900

    [fix](nereids) select with specified partition name is not work as expected (#20269)
    
    This PR is to fix the select specific partition issue, certain codes related to this feature were accidentally deleted.
---
 .../rewrite/logical/PruneOlapScanPartition.java    | 15 ++++++++----
 .../data/nereids_syntax_p0/select_partition.out    |  3 +++
 .../nereids_syntax_p0/select_partition.groovy      | 28 ++++++++++++++++++++++
 3 files changed, 42 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PruneOlapScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PruneOlapScanPartition.java
index 96c7dc94c2..a148273991 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PruneOlapScanPartition.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PruneOlapScanPartition.java
@@ -28,6 +28,10 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
 import org.apache.doris.nereids.util.Utils;
 
+import com.google.common.collect.ImmutableList;
+import org.apache.commons.collections.CollectionUtils;
+
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -61,10 +65,13 @@ public class PruneOlapScanPartition extends OneRewriteRuleFactory {
                     .map(column -> scanOutput.get(column.getName().toLowerCase()))
                     .collect(Collectors.toList());
 
-            List<Long> prunedPartitions = PartitionPruner.prune(
-                    partitionSlots, filter.getPredicate(), partitionInfo, ctx.cascadesContext);
-
-            LogicalOlapScan rewrittenScan = scan.withSelectedPartitionIds(prunedPartitions);
+            List<Long> prunedPartitions = new ArrayList<>(PartitionPruner.prune(
+                    partitionSlots, filter.getPredicate(), partitionInfo, ctx.cascadesContext));
+            List<Long> manuallySpecifiedPartitions = scan.getManuallySpecifiedPartitions();
+            if (!CollectionUtils.isEmpty(manuallySpecifiedPartitions)) {
+                prunedPartitions.retainAll(manuallySpecifiedPartitions);
+            }
+            LogicalOlapScan rewrittenScan = scan.withSelectedPartitionIds(ImmutableList.copyOf(prunedPartitions));
             return new LogicalFilter<>(filter.getConjuncts(), rewrittenScan);
         }).toRule(RuleType.OLAP_SCAN_PARTITION_PRUNE);
     }
diff --git a/regression-test/data/nereids_syntax_p0/select_partition.out b/regression-test/data/nereids_syntax_p0/select_partition.out
index 5383ef8afb..db982e24c0 100644
--- a/regression-test/data/nereids_syntax_p0/select_partition.out
+++ b/regression-test/data/nereids_syntax_p0/select_partition.out
@@ -21,3 +21,6 @@
 -- !sql --
 16	1234	t
 
+-- !sql --
+1	aaa	aaa
+
diff --git a/regression-test/suites/nereids_syntax_p0/select_partition.groovy b/regression-test/suites/nereids_syntax_p0/select_partition.groovy
index 4a61471d2b..de90c7d5c2 100644
--- a/regression-test/suites/nereids_syntax_p0/select_partition.groovy
+++ b/regression-test/suites/nereids_syntax_p0/select_partition.groovy
@@ -60,4 +60,32 @@ suite("query_on_specific_partition") {
     qt_sql """select * from t_p temporary partitions(tp1);"""
 
     qt_sql """select * from t_p temporary partition tp1;"""
+
+    sql """
+        CREATE TABLE IF NOT EXISTS test_iot (
+                `test_int` int NOT NULL,
+                `test_varchar` varchar(150) NULL,
+        `test_text` text NULL
+        ) ENGINE=OLAP
+        UNIQUE KEY(`test_int`)
+        PARTITION BY LIST (`test_int`)
+        (
+                PARTITION p1 VALUES IN ("1","2","3"),
+                        PARTITION p2 VALUES IN ("4","5","6")
+        )
+        DISTRIBUTED BY HASH(`test_int`) BUCKETS 3
+        PROPERTIES (
+                "replication_allocation" = "tag.location.default: 1",
+                "in_memory" = "false",
+                "storage_format" = "V2"
+        )
+    """
+
+    sql """
+        INSERT INTO test_iot VALUES(1,'aaa','aaa'),(4,'ccc','ccc');
+    """
+
+    qt_sql """
+        SELECT * FROM test_iot PARTITION p1;
+    """
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org