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/06/01 15:29:14 UTC

[incubator-doris] 19/22: fix core dump on vcase_expr::close (#9875)

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

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

commit 9094188ba6f9b22b88488fd1b782f9543266a1ba
Author: Pxl <px...@qq.com>
AuthorDate: Tue May 31 15:45:39 2022 +0800

    fix core dump on vcase_expr::close (#9875)
---
 be/src/vec/exprs/vcase_expr.cpp      | 10 +++-------
 be/src/vec/exprs/vexpr.cpp           |  8 ++++----
 be/src/vec/exprs/vexpr.h             |  6 +++---
 be/src/vec/functions/function_case.h | 12 +++++++++++-
 4 files changed, 21 insertions(+), 15 deletions(-)

diff --git a/be/src/vec/exprs/vcase_expr.cpp b/be/src/vec/exprs/vcase_expr.cpp
index 7ecf092574..20ad1fe340 100644
--- a/be/src/vec/exprs/vcase_expr.cpp
+++ b/be/src/vec/exprs/vcase_expr.cpp
@@ -68,19 +68,15 @@ Status VCaseExpr::open(RuntimeState* state, VExprContext* context,
                        FunctionContext::FunctionStateScope scope) {
     RETURN_IF_ERROR(VExpr::open(state, context, scope));
     RETURN_IF_ERROR(VExpr::init_function_context(context, scope, _function));
+
     CaseState* case_state = new CaseState {_data_type};
-    context->fn_context(_fn_context_index)
-            ->set_function_state(FunctionContext::FRAGMENT_LOCAL, case_state);
+    context->fn_context(_fn_context_index)->set_function_state(scope, case_state);
+
     return Status::OK();
 }
 
 void VCaseExpr::close(RuntimeState* state, VExprContext* context,
                       FunctionContext::FunctionStateScope scope) {
-    CaseState* case_state = reinterpret_cast<CaseState*>(
-            context->fn_context(_fn_context_index)
-                    ->get_function_state(FunctionContext::FRAGMENT_LOCAL));
-    delete case_state;
-
     VExpr::close_function_context(context, scope, _function);
     VExpr::close(state, context, scope);
 }
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index 066808cb7d..a8a04d8d0d 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -181,7 +181,7 @@ Status VExpr::create_expr_tree(doris::ObjectPool* pool, const doris::TExpr& texp
     }
     int node_idx = 0;
     VExpr* e = nullptr;
-    Status status = create_tree_from_thrift(pool, texpr.nodes, NULL, &node_idx, &e, ctx);
+    Status status = create_tree_from_thrift(pool, texpr.nodes, nullptr, &node_idx, &e, ctx);
     if (status.ok() && node_idx + 1 != texpr.nodes.size()) {
         status = Status::InternalError(
                 "Expression tree only partially reconstructed. Not all thrift nodes were used.");
@@ -228,7 +228,7 @@ Status VExpr::open(const std::vector<VExprContext*>& ctxs, RuntimeState* state)
 
 Status VExpr::clone_if_not_exists(const std::vector<VExprContext*>& ctxs, RuntimeState* state,
                                   std::vector<VExprContext*>* new_ctxs) {
-    DCHECK(new_ctxs != NULL);
+    DCHECK(new_ctxs != nullptr);
     if (!new_ctxs->empty()) {
         // 'ctxs' was already cloned into '*new_ctxs', nothing to do.
         DCHECK_EQ(new_ctxs->size(), ctxs.size());
@@ -317,7 +317,7 @@ void VExpr::register_function_context(doris::RuntimeState* state, VExprContext*
 
 Status VExpr::init_function_context(VExprContext* context,
                                     FunctionContext::FunctionStateScope scope,
-                                    const FunctionBasePtr& function) {
+                                    const FunctionBasePtr& function) const {
     FunctionContext* fn_ctx = context->fn_context(_fn_context_index);
     if (scope == FunctionContext::FRAGMENT_LOCAL) {
         std::vector<ColumnPtrWrapper*> constant_cols;
@@ -335,7 +335,7 @@ Status VExpr::init_function_context(VExprContext* context,
 }
 
 void VExpr::close_function_context(VExprContext* context, FunctionContext::FunctionStateScope scope,
-                                   const FunctionBasePtr& function) {
+                                   const FunctionBasePtr& function) const {
     if (_fn_context_index != -1) {
         FunctionContext* fn_ctx = context->fn_context(_fn_context_index);
         function->close(fn_ctx, FunctionContext::THREAD_LOCAL);
diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h
index 8958b91e00..31ccc8efba 100644
--- a/be/src/vec/exprs/vexpr.h
+++ b/be/src/vec/exprs/vexpr.h
@@ -125,7 +125,7 @@ public:
     static std::string debug_string(const std::vector<VExpr*>& exprs);
     static std::string debug_string(const std::vector<VExprContext*>& ctxs);
 
-    bool is_and_expr() { return _fn.name.function_name == "and"; }
+    bool is_and_expr() const { return _fn.name.function_name == "and"; }
 
     const TFunction& fn() const { return _fn; }
 
@@ -157,12 +157,12 @@ protected:
     /// 2. Call function's prepare() to initialize function state, fragment-local or
     /// thread-local according the input `FunctionStateScope` argument.
     Status init_function_context(VExprContext* context, FunctionContext::FunctionStateScope scope,
-                                 const FunctionBasePtr& function);
+                                 const FunctionBasePtr& function) const;
 
     /// Helper function to close function context, fragment-local or thread-local according
     /// the input `FunctionStateScope` argument. Called in `close` phase of VExpr.
     void close_function_context(VExprContext* context, FunctionContext::FunctionStateScope scope,
-                                const FunctionBasePtr& function);
+                                const FunctionBasePtr& function) const;
 
     TExprNodeType::type _node_type;
     TypeDescriptor _type;
diff --git a/be/src/vec/functions/function_case.h b/be/src/vec/functions/function_case.h
index 7cea64ac9e..5210ac80ad 100644
--- a/be/src/vec/functions/function_case.h
+++ b/be/src/vec/functions/function_case.h
@@ -222,7 +222,8 @@ public:
     }
 
     template <typename ColumnType>
-    void update_result_auto_simd(MutableColumnPtr& result_column_ptr, uint8* __restrict then_idx,
+    void update_result_auto_simd(MutableColumnPtr& result_column_ptr,
+                                 const uint8* __restrict then_idx,
                                  CaseWhenColumnHolder& column_holder) {
         size_t rows_count = column_holder.rows_count;
         result_column_ptr->resize(rows_count);
@@ -332,6 +333,15 @@ public:
         return execute_get_type(case_state->result_type, block, arguments, result,
                                 input_rows_count);
     }
+
+    Status close(FunctionContext* context, FunctionContext::FunctionStateScope scope) override {
+        if (scope == FunctionContext::THREAD_LOCAL) {
+            auto* state = reinterpret_cast<CaseState*>(
+                    context->get_function_state(FunctionContext::THREAD_LOCAL));
+            delete state;
+        }
+        return Status::OK();
+    }
 };
 
 } // namespace doris::vectorized


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