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

[doris] branch branch-1.2-lts updated (8e39a8b11b -> 75c1bd146f)

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

morningman pushed a change to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git


    from 8e39a8b11b [fix](vec) VMergeIterator add key same label for agg table (#14722)
     new 0b7257b551 [vectorized](udaf) fix udaf result is null when has multiple aggs (#15554)
     new 08e51f670c [Bug](filter) fix not in(null) return true (#15466)
     new f174636392 [fix](inbitmap) forbid having clause to include in bitmap. (#15494)
     new 75c1bd146f [fix](bitmapfilter) fix bitmap filter not pushing down (#15532)

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/exprs/runtime_filter.cpp                          |  7 +------
 .../aggregate_functions/aggregate_function_java_udaf.h   |  4 ++--
 be/src/vec/exec/scan/vscan_node.cpp                      |  5 ++++-
 be/src/vec/functions/in.h                                | 16 ++++------------
 .../main/java/org/apache/doris/analysis/SelectStmt.java  | 11 +++++++++++
 .../java/org/apache/doris/planner/RuntimeFilter.java     |  4 ++++
 .../data/javaudf_p0/test_javaudaf_mysum_int.out          | 12 ++++++++++++
 .../data/query_p0/sql_functions/test_in_expr.out         | 10 ++++++++++
 .../suites/javaudf_p0/test_javaudaf_mysum_int.groovy     |  4 ++++
 .../suites/query_p0/join/test_bitmap_filter.groovy       |  5 +++++
 .../suites/query_p0/sql_functions/test_in_expr.groovy    |  8 ++++++++
 11 files changed, 65 insertions(+), 21 deletions(-)


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


[doris] 01/04: [vectorized](udaf) fix udaf result is null when has multiple aggs (#15554)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 0b7257b5519329d59e13ab1a88e477460c475ab3
Author: zhangstar333 <87...@users.noreply.github.com>
AuthorDate: Tue Jan 3 16:03:43 2023 +0800

    [vectorized](udaf) fix udaf result is null when has multiple aggs (#15554)
---
 .../vec/aggregate_functions/aggregate_function_java_udaf.h   |  4 ++--
 regression-test/data/javaudf_p0/test_javaudaf_mysum_int.out  | 12 ++++++++++++
 .../suites/javaudf_p0/test_javaudaf_mysum_int.groovy         |  4 ++++
 3 files changed, 18 insertions(+), 2 deletions(-)

diff --git a/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h b/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h
index 113277498c..0671f716e1 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h
@@ -348,11 +348,11 @@ public:
                      << _fn.name.function_name;
     }
 
-    void add_batch(size_t batch_size, AggregateDataPtr* places, size_t /*place_offset*/,
+    void add_batch(size_t batch_size, AggregateDataPtr* places, size_t place_offset,
                    const IColumn** columns, Arena* /*arena*/, bool /*agg_many*/) const override {
         int64_t places_address[batch_size];
         for (size_t i = 0; i < batch_size; ++i) {
-            places_address[i] = reinterpret_cast<int64_t>(places[i]);
+            places_address[i] = reinterpret_cast<int64_t>(places[i] + place_offset);
         }
         this->data(_exec_place).add(places_address, false, columns, 0, batch_size, argument_types);
     }
diff --git a/regression-test/data/javaudf_p0/test_javaudaf_mysum_int.out b/regression-test/data/javaudf_p0/test_javaudaf_mysum_int.out
index 11092feaae..47c14ac114 100644
--- a/regression-test/data/javaudf_p0/test_javaudaf_mysum_int.out
+++ b/regression-test/data/javaudf_p0/test_javaudaf_mysum_int.out
@@ -19,3 +19,15 @@
 2	6
 9	9
 
+-- !select3 --
+0	0	0
+1	3	3
+2	6	6
+9	9	9
+
+-- !select4 --
+0	0	0
+1	3	3
+2	6	6
+9	9	9
+
diff --git a/regression-test/suites/javaudf_p0/test_javaudaf_mysum_int.groovy b/regression-test/suites/javaudf_p0/test_javaudaf_mysum_int.groovy
index fd9af7eb61..cd7ee3779f 100644
--- a/regression-test/suites/javaudf_p0/test_javaudaf_mysum_int.groovy
+++ b/regression-test/suites/javaudf_p0/test_javaudaf_mysum_int.groovy
@@ -67,6 +67,10 @@ suite("test_javaudaf_mysum_int") {
         qt_select1 """ SELECT udaf_my_sum_int(user_id) result FROM ${tableName}; """
 
         qt_select2 """ select user_id, udaf_my_sum_int(user_id) from ${tableName} group by user_id order by user_id; """
+
+        qt_select3 """ select user_id, sum(user_id), udaf_my_sum_int(user_id) from ${tableName} group by user_id order by user_id; """
+
+        qt_select4 """ select user_id, udaf_my_sum_int(user_id), sum(user_id) from ${tableName} group by user_id order by user_id; """
         
         sql """ DROP FUNCTION udaf_my_sum_int(int); """
     } finally {


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


[doris] 03/04: [fix](inbitmap) forbid having clause to include in bitmap. (#15494)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit f1746363928c1ad63dcaf65a2aef57221de05409
Author: luozenglin <37...@users.noreply.github.com>
AuthorDate: Wed Jan 4 14:33:18 2023 +0800

    [fix](inbitmap) forbid having clause to include in bitmap. (#15494)
---
 .../src/main/java/org/apache/doris/analysis/SelectStmt.java   | 11 +++++++++++
 .../suites/query_p0/join/test_bitmap_filter.groovy            |  5 +++++
 2 files changed, 16 insertions(+)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
index 1af417841a..0950b3d839 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
@@ -957,6 +957,13 @@ public class SelectStmt extends QueryStmt {
         }
     }
 
+    private boolean isContainInBitmap(Expr expr) {
+        List<Expr> inPredicates = Lists.newArrayList();
+        expr.collect(InPredicate.class, inPredicates);
+        return inPredicates.stream().anyMatch(e -> e.getChild(1) instanceof Subquery
+                && ((Subquery) e.getChild(1)).getStatement().getResultExprs().get(0).getType().isBitmapType());
+    }
+
     /**
      * Analyze aggregation-relevant components of the select block (Group By clause,
      * select list, Order By clause),
@@ -1017,6 +1024,10 @@ public class SelectStmt extends QueryStmt {
                         "HAVING clause must not contain analytic expressions: "
                                 + analyticExpr.toSql());
             }
+            if (isContainInBitmap(havingClauseAfterAnaylzed)) {
+                throw new AnalysisException(
+                        "HAVING clause dose not support in bitmap syntax: " + havingClauseAfterAnaylzed.toSql());
+            }
         }
 
         if (groupByClause == null && !selectList.isDistinct()
diff --git a/regression-test/suites/query_p0/join/test_bitmap_filter.groovy b/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
index 5caf5f7ac6..78b33040b4 100644
--- a/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
+++ b/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
@@ -70,4 +70,9 @@ suite("test_bitmap_filter", "query_p0") {
         sql "select k1, k2 from ${tbl1} b1 where k1 in (select k2 from ${tbl2} b2 where b1.k2 = b2.k1) order by k1;"
         exception "In bitmap does not support correlated subquery"
     }
+
+    test {
+        sql "select k1, count(*) from ${tbl1} b1 group by k1 having k1 in (select k2 from ${tbl2} b2) order by k1;"
+        exception "HAVING clause dose not support in bitmap"
+    }
 }


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


[doris] 02/04: [Bug](filter) fix not in(null) return true (#15466)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 08e51f670c08b0ecb5a70747bc786af1452f5625
Author: Pxl <px...@qq.com>
AuthorDate: Tue Jan 3 21:14:50 2023 +0800

    [Bug](filter) fix not in(null) return true (#15466)
    
    fix not in(null) return true
---
 be/src/vec/exec/scan/vscan_node.cpp                      |  3 +++
 be/src/vec/functions/in.h                                | 16 ++++------------
 .../data/query_p0/sql_functions/test_in_expr.out         | 10 ++++++++++
 .../suites/query_p0/sql_functions/test_in_expr.groovy    |  8 ++++++++
 4 files changed, 25 insertions(+), 12 deletions(-)

diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp
index 56f9aec91a..d849c67edd 100644
--- a/be/src/vec/exec/scan/vscan_node.cpp
+++ b/be/src/vec/exec/scan/vscan_node.cpp
@@ -721,6 +721,9 @@ Status VScanNode::_normalize_not_in_and_not_eq_predicate(VExpr* expr, VExprConte
                         ->get_function_state(FunctionContext::FRAGMENT_LOCAL));
         HybridSetBase::IteratorBase* iter = state->hybrid_set->begin();
         auto fn_name = std::string("");
+        if (!is_fixed_range && state->null_in_set) {
+            _eos = true;
+        }
         while (iter->has_next()) {
             // column not in (nullptr) is always true
             if (nullptr == iter->get_value()) {
diff --git a/be/src/vec/functions/in.h b/be/src/vec/functions/in.h
index 79187abb2b..626060e399 100644
--- a/be/src/vec/functions/in.h
+++ b/be/src/vec/functions/in.h
@@ -23,6 +23,7 @@
 #include "exprs/create_predicate_function.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/columns_number.h"
+#include "vec/data_types/data_type.h"
 #include "vec/data_types/data_type_nullable.h"
 #include "vec/data_types/data_type_number.h"
 #include "vec/functions/function.h"
@@ -108,7 +109,7 @@ public:
         vec_res.resize(input_rows_count);
 
         ColumnUInt8::MutablePtr col_null_map_to;
-        col_null_map_to = ColumnUInt8::create(input_rows_count);
+        col_null_map_to = ColumnUInt8::create(input_rows_count, false);
         auto& vec_null_map_to = col_null_map_to->get_data();
 
         /// First argument may be a single column.
@@ -150,11 +151,12 @@ public:
                     }
                 } else {
                     for (size_t i = 0; i < input_rows_count; ++i) {
-                        vec_null_map_to[i] = null_bitmap[i] || (negative == vec_res[i]);
+                        vec_null_map_to[i] = null_bitmap[i] || negative == vec_res[i];
                     }
                 }
 
             } else { // non-nullable
+                DCHECK(!in_state->null_in_set);
 
                 auto search_hash_set = [&](auto* col_ptr) {
                     for (size_t i = 0; i < input_rows_count; ++i) {
@@ -175,16 +177,6 @@ public:
                 } else {
                     search_hash_set(materialized_column.get());
                 }
-
-                if (in_state->null_in_set) {
-                    for (size_t i = 0; i < input_rows_count; ++i) {
-                        vec_null_map_to[i] = negative == vec_res[i];
-                    }
-                } else {
-                    for (size_t i = 0; i < input_rows_count; ++i) {
-                        vec_null_map_to[i] = false;
-                    }
-                }
             }
         } else {
             std::vector<ColumnPtr> set_columns;
diff --git a/regression-test/data/query_p0/sql_functions/test_in_expr.out b/regression-test/data/query_p0/sql_functions/test_in_expr.out
index 5006d062dc..31d6bb5b1a 100644
--- a/regression-test/data/query_p0/sql_functions/test_in_expr.out
+++ b/regression-test/data/query_p0/sql_functions/test_in_expr.out
@@ -5,6 +5,16 @@
 -- !select --
 4
 
+-- !select --
+
+-- !select --
+103	4	d
+
+-- !select --
+103	4	d
+
+-- !select --
+
 -- !select --
 c
 
diff --git a/regression-test/suites/query_p0/sql_functions/test_in_expr.groovy b/regression-test/suites/query_p0/sql_functions/test_in_expr.groovy
index ea744f04ed..01cacef28e 100644
--- a/regression-test/suites/query_p0/sql_functions/test_in_expr.groovy
+++ b/regression-test/suites/query_p0/sql_functions/test_in_expr.groovy
@@ -67,6 +67,14 @@ suite("test_in_expr", "query") {
     // 1.1.2 string + null_in_set
     qt_select "select t1.number from ${nullTableName} t1 left join ${nullTableName} t2 on t1.cid=t2.cid where t2.addr in ('d', null)"
 
+    qt_select "select * from ${nullTableName} where addr not in ('d', null)"
+
+    qt_select "select * from ${nullTableName} where not(addr not in ('d', null))"
+
+    qt_select "select * from ${nullTableName} where addr in ('d', null)"
+
+    qt_select "select * from ${nullTableName} where not(addr in ('d', null))"
+
     // 1.1.3 non-string
     qt_select "select t1.addr from ${nullTableName} t1 left join ${nullTableName} t2 on t1.cid=t2.cid where t2.number in (3)"
 


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


[doris] 04/04: [fix](bitmapfilter) fix bitmap filter not pushing down (#15532)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 75c1bd146fd0a3f3a08912d2e0610d440c4c5f3d
Author: luozenglin <37...@users.noreply.github.com>
AuthorDate: Wed Jan 4 14:33:53 2023 +0800

    [fix](bitmapfilter) fix bitmap filter not pushing down (#15532)
---
 be/src/exprs/runtime_filter.cpp                                    | 7 +------
 be/src/vec/exec/scan/vscan_node.cpp                                | 2 +-
 .../src/main/java/org/apache/doris/planner/RuntimeFilter.java      | 4 ++++
 3 files changed, 6 insertions(+), 7 deletions(-)

diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp
index 0444d3cc42..89e0181c0c 100644
--- a/be/src/exprs/runtime_filter.cpp
+++ b/be/src/exprs/runtime_filter.cpp
@@ -1325,12 +1325,7 @@ Status IRuntimeFilter::init_with_desc(const TRuntimeFilterDesc* desc, const TQue
         doris::vectorized::VExprContext* bitmap_target_ctx = nullptr;
         RETURN_IF_ERROR(doris::vectorized::VExpr::create_expr_tree(_pool, desc->bitmap_target_expr,
                                                                    &bitmap_target_ctx));
-        auto type = const_cast<vectorized::VExpr*>(
-                            vectorized::VExpr::expr_without_cast(bitmap_target_ctx->root()))
-                            ->type();
-        // The bitmap filter evaluates only integers.
-        params.column_return_type =
-                type.is_integer_type() ? type.type : bitmap_target_ctx->root()->type().type;
+        params.column_return_type = bitmap_target_ctx->root()->type().type;
 
         if (desc->__isset.bitmap_filter_not_in) {
             params.bitmap_filter_not_in = desc->bitmap_filter_not_in;
diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp
index d849c67edd..bbe0a75523 100644
--- a/be/src/vec/exec/scan/vscan_node.cpp
+++ b/be/src/vec/exec/scan/vscan_node.cpp
@@ -504,7 +504,7 @@ Status VScanNode::_normalize_bloom_filter(VExpr* expr, VExprContext* expr_ctx, S
 
 Status VScanNode::_normalize_bitmap_filter(VExpr* expr, VExprContext* expr_ctx,
                                            SlotDescriptor* slot, PushDownType* pdt) {
-    if (TExprNodeType::BITMAP_PRED == expr->node_type() && expr->type().is_integer_type()) {
+    if (TExprNodeType::BITMAP_PRED == expr->node_type()) {
         DCHECK(expr->children().size() == 1);
         PushDownType temp_pdt = _should_push_down_bitmap_filter();
         if (temp_pdt != PushDownType::UNACCEPTABLE) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/RuntimeFilter.java b/fe/fe-core/src/main/java/org/apache/doris/planner/RuntimeFilter.java
index e68e71f6ad..ad92f2245f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/RuntimeFilter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/RuntimeFilter.java
@@ -20,6 +20,7 @@ package org.apache.doris.planner;
 import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.BinaryPredicate;
 import org.apache.doris.analysis.BitmapFilterPredicate;
+import org.apache.doris.analysis.CastExpr;
 import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.Predicate;
 import org.apache.doris.analysis.SlotId;
@@ -336,6 +337,9 @@ public final class RuntimeFilter {
             if (targetSlots.isEmpty()) {
                 return null;
             }
+            while (targetExpr instanceof CastExpr && targetExpr.getChild(0).getType().isIntegerType()) {
+                targetExpr = targetExpr.getChild(0);
+            }
 
             RuntimeFilter runtimeFilter =
                     new RuntimeFilter(idGen.getNextId(), filterSrcNode, srcExpr, exprOrder, targetExpr, targetSlots,


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