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 2022/12/24 09:09:55 UTC

[doris] 03/15: [Bug](Agg) fix crash when encountering not supported agg function like last_value(bitmap) (#15257)

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 e8bb4a3bcc810e0c6fa3c4e3194630ed9c346a17
Author: AlexYue <yj...@gmail.com>
AuthorDate: Fri Dec 23 14:23:21 2022 +0800

    [Bug](Agg) fix crash when encountering not supported agg function like last_value(bitmap) (#15257)
    
    The former logic inside aggregate_function_window.cpp would shutdown BE once encountering agg function with complex type like BITMAP. This pr makes it don't crash and would return one more concrete error message which tells the unsupported function signature to user.
---
 .../aggregate_function_window.cpp                  |  4 +-
 be/src/vec/exec/vanalytic_eval_node.cpp            |  4 ++
 be/src/vec/exprs/vectorized_agg_fn.cpp             |  2 +-
 .../java/org/apache/doris/catalog/FunctionSet.java |  2 +-
 .../function/test_not_supported_agg_fn.groovy      | 43 ++++++++++++++++++++++
 5 files changed, 51 insertions(+), 4 deletions(-)

diff --git a/be/src/vec/aggregate_functions/aggregate_function_window.cpp b/be/src/vec/aggregate_functions/aggregate_function_window.cpp
index 02b283ab2d..4714bd2609 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_window.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_window.cpp
@@ -78,8 +78,8 @@ static IAggregateFunction* create_function_lead_lag_first_last(const String& nam
     TYPE_TO_BASIC_COLUMN_TYPE(DISPATCH)
 #undef DISPATCH
 
-    LOG(FATAL) << "with unknowed type, failed in  create_aggregate_function_" << name
-               << " and type is: " << argument_types[0]->get_name();
+    LOG(WARNING) << "with unknowed type, failed in  create_aggregate_function_" << name
+                 << " and type is: " << argument_types[0]->get_name();
     return nullptr;
 }
 
diff --git a/be/src/vec/exec/vanalytic_eval_node.cpp b/be/src/vec/exec/vanalytic_eval_node.cpp
index 8aa1f8708a..74625c14a6 100644
--- a/be/src/vec/exec/vanalytic_eval_node.cpp
+++ b/be/src/vec/exec/vanalytic_eval_node.cpp
@@ -30,6 +30,7 @@ namespace doris::vectorized {
 VAnalyticEvalNode::VAnalyticEvalNode(ObjectPool* pool, const TPlanNode& tnode,
                                      const DescriptorTbl& descs)
         : ExecNode(pool, tnode, descs),
+          _fn_place_ptr(nullptr),
           _intermediate_tuple_id(tnode.analytic_node.intermediate_tuple_id),
           _output_tuple_id(tnode.analytic_node.output_tuple_id),
           _window(tnode.analytic_node.window) {
@@ -649,6 +650,9 @@ Status VAnalyticEvalNode::_create_agg_status() {
 }
 
 Status VAnalyticEvalNode::_destroy_agg_status() {
+    if (UNLIKELY(_fn_place_ptr == nullptr)) {
+        return Status::OK();
+    }
     for (size_t i = 0; i < _agg_functions_size; ++i) {
         _agg_functions[i]->destroy(_fn_place_ptr + _offsets_of_aggregate_states[i]);
     }
diff --git a/be/src/vec/exprs/vectorized_agg_fn.cpp b/be/src/vec/exprs/vectorized_agg_fn.cpp
index 8f7527a15e..0895eae775 100644
--- a/be/src/vec/exprs/vectorized_agg_fn.cpp
+++ b/be/src/vec/exprs/vectorized_agg_fn.cpp
@@ -129,7 +129,7 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const RowDescriptor& desc, M
                 _fn.name.function_name, argument_types, {}, _data_type->is_nullable());
     }
     if (_function == nullptr) {
-        return Status::InternalError("Agg Function {} is not implemented", _fn.name.function_name);
+        return Status::InternalError("Agg Function {} is not implemented", _fn.signature);
     }
 
     if (!_sort_description.empty()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
index 532635cf36..af72fd6df8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
@@ -2775,7 +2775,7 @@ public class FunctionSet<T> {
         addBuiltin(AggregateFunction.createAnalyticBuiltin("ntile",
                 Collections.singletonList(Type.BIGINT), Type.BIGINT, Type.BIGINT, null, null, null, null, null, true));
 
-        for (Type t : Type.getSupportedTypes()) {
+        for (Type t : Type.getTrivialTypes()) {
             if (t.isNull()) {
                 continue; // NULL is handled through type promotion.
             }
diff --git a/regression-test/suites/tpch_sf1_p1/function/test_not_supported_agg_fn.groovy b/regression-test/suites/tpch_sf1_p1/function/test_not_supported_agg_fn.groovy
new file mode 100644
index 0000000000..a7a6d89686
--- /dev/null
+++ b/regression-test/suites/tpch_sf1_p1/function/test_not_supported_agg_fn.groovy
@@ -0,0 +1,43 @@
+// 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.
+
+// There once exists a bug that when passing a too long string
+// along with an empty one to find_in_set, BE 
+
+suite("test_not_supported_agg_fn") {
+
+    try {
+        test {
+            sql """
+            SELECT ref_0.`PS_SUPPKEY` AS c0,
+            ref_0.`PS_SUPPLYCOST` AS c1,
+            last_value(cast(bitmap_empty() AS BITMAP)) OVER (PARTITION BY ref_0.`PS_AVAILQTY`
+                                                        ORDER BY ref_0.`PS_PARTKEY` DESC) AS c2,
+                                                       CASE
+                                                           WHEN FALSE THEN ref_0.`PS_COMMENT`
+                                                           ELSE ref_0.`PS_COMMENT`
+                                                       END AS c3
+            FROM regression_test_tpch_sf0_1_p1.partsupp AS ref_0
+            WHERE ref_0.`PS_PARTKEY` IS NULL
+            ORDER BY ref_0.`PS_COMMENT`
+            """
+            exception "errCode = 2, detailMessage = No matching function with signature: last_value(bitmap)"            
+        }
+    } finally {
+    }
+
+}
\ No newline at end of file


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