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 2021/12/14 07:54:07 UTC

[incubator-doris] branch master updated: [refactor][fix](constants-fold) Refactor the code of fold constant mgr and fix some undefined behavior and mem leak (#7373)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 4e02109  [refactor][fix](constants-fold) Refactor the code of fold constant mgr and fix some undefined behavior and mem leak (#7373)
4e02109 is described below

commit 4e021099263b58a9d0e6cfc776b185bc664d5dbe
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Tue Dec 14 01:53:56 2021 -0600

    [refactor][fix](constants-fold) Refactor the code of fold constant mgr and fix some undefined behavior and mem leak (#7373)
    
    1. Fix some memory leaks
    2. Remove redundant and invalid code
    3. Fix some buggy writes to reduce extra memory copies and return null pointers to string
    4. Reframing the naming to make the structure clearer
---
 be/src/runtime/CMakeLists.txt                      |  2 +-
 be/src/runtime/exec_env.h                          |  5 --
 be/src/runtime/exec_env_init.cpp                   |  4 +-
 ...constant_mgr.cpp => fold_constant_executor.cpp} | 72 ++++++++++------------
 ...old_constant_mgr.h => fold_constant_executor.h} | 15 +++--
 be/src/service/internal_service.cpp                |  5 +-
 gensrc/thrift/PaloInternalService.thrift           |  1 +
 7 files changed, 44 insertions(+), 60 deletions(-)

diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt
index 72e8711..d426559 100644
--- a/be/src/runtime/CMakeLists.txt
+++ b/be/src/runtime/CMakeLists.txt
@@ -108,7 +108,7 @@ set(RUNTIME_FILES
     mysql_result_writer.cpp
     memory/system_allocator.cpp
     memory/chunk_allocator.cpp
-    fold_constant_mgr.cpp
+    fold_constant_executor.cpp
     cache/result_node.cpp
     cache/result_cache.cpp
     odbc_table_sink.cpp	
diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h
index 766ca5c..3cc2def 100644
--- a/be/src/runtime/exec_env.h
+++ b/be/src/runtime/exec_env.h
@@ -49,16 +49,13 @@ class ResultBufferMgr;
 class ResultQueueMgr;
 class TMasterInfo;
 class LoadChannelMgr;
-class TestExecEnv;
 class ThreadResourceMgr;
 class TmpFileMgr;
 class WebPageHandler;
 class StreamLoadExecutor;
 class RoutineLoadTaskExecutor;
 class SmallFileMgr;
-class FileBlockManager;
 class PluginMgr;
-class FoldConstantMgr;
 
 class BackendServiceClient;
 class FrontendServiceClient;
@@ -134,7 +131,6 @@ public:
     LoadChannelMgr* load_channel_mgr() { return _load_channel_mgr; }
     LoadStreamMgr* load_stream_mgr() { return _load_stream_mgr; }
     SmallFileMgr* small_file_mgr() { return _small_file_mgr; }
-    FoldConstantMgr* fold_constant_mgr() { return _fold_constant_mgr; }
 
     const std::vector<StorePath>& store_paths() const { return _store_paths; }
     void set_store_paths(const std::vector<StorePath>& paths) { _store_paths = paths; }
@@ -200,7 +196,6 @@ private:
     LoadPathMgr* _load_path_mgr = nullptr;
     DiskIoMgr* _disk_io_mgr = nullptr;
     TmpFileMgr* _tmp_file_mgr = nullptr;
-    FoldConstantMgr* _fold_constant_mgr = nullptr;
 
     BfdParser* _bfd_parser = nullptr;
     BrokerMgr* _broker_mgr = nullptr;
diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp
index a79112a..a172bb6 100644
--- a/be/src/runtime/exec_env_init.cpp
+++ b/be/src/runtime/exec_env_init.cpp
@@ -39,7 +39,7 @@
 #include "runtime/etl_job_mgr.h"
 #include "runtime/exec_env.h"
 #include "runtime/external_scan_context_mgr.h"
-#include "runtime/fold_constant_mgr.h"
+#include "runtime/fold_constant_executor.h"
 #include "runtime/fragment_mgr.h"
 #include "runtime/heartbeat_flags.h"
 #include "runtime/load_channel_mgr.h"
@@ -115,7 +115,6 @@ Status ExecEnv::_init(const std::vector<StorePath>& store_paths) {
     _fragment_mgr = new FragmentMgr(this);
     _result_cache = new ResultCache(config::query_cache_max_size_mb,
                                     config::query_cache_elasticity_size_mb);
-    _fold_constant_mgr = new FoldConstantMgr(this);
     _master_info = new TMasterInfo();
     _etl_job_mgr = new EtlJobMgr(this);
     _load_path_mgr = new LoadPathMgr(this);
@@ -295,7 +294,6 @@ void ExecEnv::_destroy() {
     SAFE_DELETE(_etl_job_mgr);
     SAFE_DELETE(_master_info);
     SAFE_DELETE(_fragment_mgr);
-    SAFE_DELETE(_fold_constant_mgr);
     SAFE_DELETE(_cgroups_mgr);
     SAFE_DELETE(_etl_thread_pool);
     SAFE_DELETE(_scan_thread_pool);
diff --git a/be/src/runtime/fold_constant_mgr.cpp b/be/src/runtime/fold_constant_executor.cpp
similarity index 80%
rename from be/src/runtime/fold_constant_mgr.cpp
rename to be/src/runtime/fold_constant_executor.cpp
index 9c377ee..94d3d9e 100644
--- a/be/src/runtime/fold_constant_mgr.cpp
+++ b/be/src/runtime/fold_constant_executor.cpp
@@ -14,55 +14,52 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
-#include "runtime/fold_constant_mgr.h"
+#include "runtime/fold_constant_executor.h"
 
 #include <map>
 #include <string>
 
+#include "runtime/tuple_row.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "runtime/mem_tracker.h"
+#include "exprs/expr_context.h"
+#include "exprs/expr.h"
 #include "common/object_pool.h"
 #include "common/status.h"
-#include "exprs/expr.h"
-#include "exprs/expr_context.h"
-#include "gen_cpp/PaloInternalService_types.h"
+
 #include "gen_cpp/internal_service.pb.h"
-#include "runtime/exec_env.h"
-#include "runtime/mem_tracker.h"
-#include "runtime/runtime_state.h"
-#include "runtime/tuple_row.h"
+#include "gen_cpp/PaloInternalService_types.h"
 
 using std::string;
 using std::map;
 
 namespace doris {
 
-TUniqueId FoldConstantMgr::_dummy_id;
-
-FoldConstantMgr::FoldConstantMgr(ExecEnv* exec_env) : _exec_env(exec_env), _pool() {}
+TUniqueId FoldConstantExecutor::_dummy_id;
 
-Status FoldConstantMgr::fold_constant_expr(const TFoldConstantParams& params,
-                                           PConstantExprResult* response) {
-    auto expr_map = params.expr_map;
+Status FoldConstantExecutor::fold_constant_expr(
+        const TFoldConstantParams& params, PConstantExprResult* response) {
+    const auto& expr_map = params.expr_map;
     auto expr_result_map = response->mutable_expr_result_map();
 
     TQueryGlobals query_globals = params.query_globals;
-
     // init
-    Status status = init(query_globals);
+    Status status = _init(query_globals);
     if (UNLIKELY(!status.ok())) {
         LOG(WARNING) << "Failed to init mem trackers, msg: " << status.get_error_msg();
         return status;
     }
 
-    for (auto m : expr_map) {
+    for (const auto& m : expr_map) {
         PExprResultMap pexpr_result_map;
-        for (auto n : m.second) {
+        for (const auto& n : m.second) {
             ExprContext* ctx = nullptr;
-            TExpr& texpr = n.second;
+            const TExpr& texpr = n.second;
             // create expr tree from TExpr
             RETURN_IF_ERROR(Expr::create_expr_tree(&_pool, texpr, &ctx));
             // prepare and open context
-            status = prepare_and_open(ctx);
+            status = _prepare_and_open(ctx);
             if (UNLIKELY(!status.ok())) {
                 LOG(WARNING) << "Failed to init mem trackers, msg: " << status.get_error_msg();
                 return status;
@@ -82,12 +79,11 @@ Status FoldConstantMgr::fold_constant_expr(const TFoldConstantParams& params,
                 expr_result.set_success(false);
             } else {
                 expr_result.set_success(true);
-                result = get_result(src, ctx->root()->type().type);
+                result = _get_result(src, ctx->root()->type().type);
             }
 
             expr_result.set_content(std::move(result));
             expr_result.mutable_type()->set_type(t_type);
-
             pexpr_result_map.mutable_map()->insert({n.first, expr_result});
 
             // close context expr
@@ -100,11 +96,11 @@ Status FoldConstantMgr::fold_constant_expr(const TFoldConstantParams& params,
     return Status::OK();
 }
 
-Status FoldConstantMgr::init(TQueryGlobals query_globals) {
+Status FoldConstantExecutor::_init(const TQueryGlobals& query_globals) {
     // init runtime state, runtime profile
     TPlanFragmentExecParams params;
-    params.fragment_instance_id = FoldConstantMgr::_dummy_id;
-    params.query_id = FoldConstantMgr::_dummy_id;
+    params.fragment_instance_id = FoldConstantExecutor::_dummy_id;
+    params.query_id = FoldConstantExecutor::_dummy_id;
     TExecPlanFragmentParams fragment_params;
     fragment_params.params = params;
     fragment_params.protocol_version = PaloInternalServiceVersion::V1;
@@ -112,14 +108,13 @@ Status FoldConstantMgr::init(TQueryGlobals query_globals) {
     _runtime_state.reset(new RuntimeState(fragment_params.params, query_options, query_globals,
                                           ExecEnv::GetInstance()));
     DescriptorTbl* desc_tbl = nullptr;
-    TDescriptorTable* t_desc_tbl = new TDescriptorTable();
-    Status status = DescriptorTbl::create(_runtime_state->obj_pool(), *t_desc_tbl, &desc_tbl);
+    Status status = DescriptorTbl::create(_runtime_state->obj_pool(), TDescriptorTable(), &desc_tbl);
     if (UNLIKELY(!status.ok())) {
         LOG(WARNING) << "Failed to create descriptor table, msg: " << status.get_error_msg();
         return Status::Uninitialized(status.get_error_msg());
     }
     _runtime_state->set_desc_tbl(desc_tbl);
-    status = _runtime_state->init_mem_trackers(FoldConstantMgr::_dummy_id);
+    status = _runtime_state->init_mem_trackers(FoldConstantExecutor::_dummy_id);
     if (UNLIKELY(!status.ok())) {
         LOG(WARNING) << "Failed to init mem trackers, msg: " << status.get_error_msg();
         return Status::Uninitialized(status.get_error_msg());
@@ -127,24 +122,19 @@ Status FoldConstantMgr::init(TQueryGlobals query_globals) {
 
     _runtime_profile = _runtime_state->runtime_profile();
     _runtime_profile->set_name("FoldConstantExpr");
-    _mem_tracker = MemTracker::CreateTracker(-1, "FoldConstantExpr",
-                                             _runtime_state->instance_mem_tracker());
+    _mem_tracker = MemTracker::CreateTracker(-1, "FoldConstantExpr", _runtime_state->instance_mem_tracker());
     _mem_pool.reset(new MemPool(_mem_tracker.get()));
 
     return Status::OK();
 }
 
-Status FoldConstantMgr::prepare_and_open(ExprContext* ctx) {
-    RowDescriptor* desc = new RowDescriptor();
-    ctx->prepare(_runtime_state.get(), *desc, _mem_tracker);
+Status FoldConstantExecutor::_prepare_and_open(ExprContext* ctx) {
+    ctx->prepare(_runtime_state.get(), RowDescriptor(), _mem_tracker);
     return ctx->open(_runtime_state.get());
 }
 
-string FoldConstantMgr::get_result(void* src, PrimitiveType slot_type) {
+string FoldConstantExecutor::_get_result(void* src, PrimitiveType slot_type){
     switch (slot_type) {
-    case TYPE_NULL: {
-        return nullptr;
-    }
     case TYPE_BOOLEAN: {
         bool val = *reinterpret_cast<const bool*>(src);
         return val ? "true" : "false";
@@ -196,8 +186,10 @@ string FoldConstantMgr::get_result(void* src, PrimitiveType slot_type) {
     }
     default:
         DCHECK(false) << "Type not implemented: " << slot_type;
-        return nullptr;
+        return "";
     }
 }
 
-} // namespace doris
+
+}
+
diff --git a/be/src/runtime/fold_constant_mgr.h b/be/src/runtime/fold_constant_executor.h
similarity index 85%
rename from be/src/runtime/fold_constant_mgr.h
rename to be/src/runtime/fold_constant_executor.h
index 353dda9..93ee988 100644
--- a/be/src/runtime/fold_constant_mgr.h
+++ b/be/src/runtime/fold_constant_executor.h
@@ -32,22 +32,21 @@ class TExpr;
 class TQueryGlobals;
 
 // This class used to fold constant expr from fe
-class FoldConstantMgr {
+class FoldConstantExecutor {
 public:
-    FoldConstantMgr(ExecEnv* exec_env);
-    // fold constant expr 
+    // fold constant expr
     Status fold_constant_expr(const TFoldConstantParams& params, PConstantExprResult* response);
+private:
     // init runtime_state and mem_tracker
-    Status init(TQueryGlobals query_globals);
+    Status _init(const TQueryGlobals& query_globals);
     // prepare expr
-    Status prepare_and_open(ExprContext* ctx);
+    Status _prepare_and_open(ExprContext* ctx);
 
-    std::string get_result(void* src, PrimitiveType slot_type);
+    std::string _get_result(void* src, PrimitiveType slot_type);
 
-private:
     std::unique_ptr<RuntimeState> _runtime_state;
     std::shared_ptr<MemTracker> _mem_tracker;
-    RuntimeProfile* _runtime_profile;
+    RuntimeProfile* _runtime_profile = nullptr;
     std::unique_ptr<MemPool> _mem_pool;
     ExecEnv* _exec_env;
     ObjectPool _pool;
diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp
index 25cad50..90d1488 100644
--- a/be/src/service/internal_service.cpp
+++ b/be/src/service/internal_service.cpp
@@ -23,7 +23,7 @@
 #include "runtime/buffer_control_block.h"
 #include "runtime/data_stream_mgr.h"
 #include "runtime/exec_env.h"
-#include "runtime/fold_constant_mgr.h"
+#include "runtime/fold_constant_executor.h"
 #include "runtime/fragment_mgr.h"
 #include "runtime/load_channel_mgr.h"
 #include "runtime/result_buffer_mgr.h"
@@ -418,8 +418,7 @@ Status PInternalServiceImpl<T>::_fold_constant_expr(const std::string& ser_reque
         uint32_t len = ser_request.size();
         RETURN_IF_ERROR(deserialize_thrift_msg(buf, &len, false, &t_request));
     }
-    FoldConstantMgr mgr(_exec_env);
-    return mgr.fold_constant_expr(t_request, response);
+    return FoldConstantExecutor().fold_constant_expr(t_request, response);
 }
 
 template <typename T>
diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift
index 4ded4b1..f347dec 100644
--- a/gensrc/thrift/PaloInternalService.thrift
+++ b/gensrc/thrift/PaloInternalService.thrift
@@ -351,6 +351,7 @@ struct TExprMap {
 struct TFoldConstantParams {
   1: required map<string, map<string, Exprs.TExpr>> expr_map
   2: required TQueryGlobals query_globals
+  3: optional bool vec_exec
 }
 
 // TransmitData

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