You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2021/11/23 07:46:08 UTC

[GitHub] [incubator-doris] xinyiZzz opened a new pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

xinyiZzz opened a new pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198


   ## Proposed changes
   
   1. Add thread local variable ThreadStatus, save queryID, MemTracker, etc.
   2. Use TCMalloc Hook to consume and release MemTracker in ThreadStatus.
   3. Add real BE process level MemTracker, consume and release through TCMalloc Hook.
   4. Add the real Query MemTracker, a queryID is unique in a BE process, consume and release through TCMalloc Hook.
   
   ## Types of changes
   
   What types of changes does your code introduce to Doris?
   _Put an `x` in the boxes that apply_
   
   - [ ] Bugfix (non-breaking change which fixes an issue)
   - [x] New feature (non-breaking change which adds functionality)
   - [ ] Breaking change (fix or feature that would cause existing functionality to not work as expected)
   - [ ] Documentation Update (if none of the other choices apply)
   - [ ] Code refactor (Modify the code structure, format the code, etc...)
   - [ ] Optimization. Including functional usability improvements and performance improvements.
   - [ ] Dependency. Such as changes related to third-party components.
   - [ ] Other.
   
   ## Checklist
   
   _Put an `x` in the boxes that apply. You can also fill these out after creating the PR. If you're unsure about any of them, don't hesitate to ask. We're here to help! This is simply a reminder of what we are going to look for before merging your code._
   
   - [x] I have created an issue on (Fix #7196) and described the bug/feature there in detail
   - [x] Compiling and unit tests pass locally with my changes
   - [ ] I have added tests that prove my fix is effective or that my feature works
   - [ ] If these changes need document changes, I have updated the document
   - [ ] Any dependent changes have been merged
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at dev@doris.apache.org by explaining why you chose the solution you did and what alternatives you considered, etc...
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755735814



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -242,26 +243,38 @@ int64_t MemTracker::GetPoolMemReserved() {
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    if (mem_limit != -1) {
+        if (mem_limit > MemInfo::physical_mem()) {

Review comment:
       Can check when memtracker is created, I try to change it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764702791



##########
File path: be/src/runtime/thread_mem_tracker.cpp
##########
@@ -0,0 +1,129 @@
+// 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.
+
+#include "runtime/thread_mem_tracker.h"
+
+#include "service/backend_options.h"
+
+namespace doris {
+
+void ThreadMemTracker::attach_query(const std::string& query_id,
+                                    const TUniqueId& fragment_instance_id) {
+#ifdef BE_TEST
+    if (ExecEnv::GetInstance()->query_mem_tracker_registry() == nullptr) {
+        return;
+    }
+#endif
+    update_query_mem_tracker(
+            ExecEnv::GetInstance()->query_mem_tracker_registry()->get_query_mem_tracker(query_id));
+    _fragment_instance_id = fragment_instance_id;
+}
+
+void ThreadMemTracker::detach_query() {
+    update_query_mem_tracker(std::weak_ptr<MemTracker>());
+    _fragment_instance_id = TUniqueId();
+}
+
+void ThreadMemTracker::update_query_mem_tracker(std::weak_ptr<MemTracker> mem_tracker) {
+    if (_untracked_mem != 0) {
+        consume();
+        _untracked_mem = 0;
+    }
+    _query_mem_tracker = mem_tracker;
+}
+
+void ThreadMemTracker::query_mem_limit_exceeded(int64_t mem_usage) {
+    if (_fragment_instance_id != TUniqueId() && ExecEnv::GetInstance()->initialized() &&
+        ExecEnv::GetInstance()->fragment_mgr()->is_canceling(_fragment_instance_id).ok()) {
+        std::string detail = "Query Memory exceed limit in TCMalloc Hook New.";
+        auto st = _query_mem_tracker.lock()->MemLimitExceeded(nullptr, detail, mem_usage);
+
+        detail +=
+                " Query Memory exceed limit in TCMalloc Hook New, Backend: {}, Fragment: {}, Used: "
+                "{}, Limit: {}. You can change the limit by session variable exec_mem_limit.";
+        fmt::format(detail, BackendOptions::get_localhost(), print_id(_fragment_instance_id),
+                    std::to_string(_query_mem_tracker.lock()->consumption()),
+                    std::to_string(_query_mem_tracker.lock()->limit()));
+        ExecEnv::GetInstance()->fragment_mgr()->cancel(

Review comment:
       Here loses the thread context and cannot cancel immediately. 
   The operator will continue to run until the flag is checked.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] morningman commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755108387



##########
File path: be/src/service/doris_main.cpp
##########
@@ -275,6 +276,9 @@ int main(int argc, char** argv) {
 #if defined(LEAK_SANITIZER)
         __lsan_do_leak_check();
 #endif
+        LOG(INFO) << "Tcmalloc New number: " << new_hook_calls

Review comment:
       Better not print log here, use metrics

##########
File path: be/src/exec/olap_scan_node.cpp
##########
@@ -1652,6 +1654,7 @@ void OlapScanNode::scanner_thread(OlapScanner* scanner) {
     // and transfer thread
     _scan_batch_added_cv.notify_one();
     _scan_thread_exit_cv.notify_one();
+    current_thread.update_mem_tracker(nullptr);

Review comment:
       The thread may be returned early.

##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {

Review comment:
       Maybe `TheadContext` is better?

##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -242,26 +243,38 @@ int64_t MemTracker::GetPoolMemReserved() {
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    if (mem_limit != -1) {
+        if (mem_limit > MemInfo::physical_mem()) {
+            LOG(WARNING) << "Memory limit " << PrettyPrinter::print(mem_limit, TUnit::BYTES)
+                         << " exceeds physical memory of "
+                         << PrettyPrinter::print(MemInfo::physical_mem(), TUnit::BYTES);
+        }
+        VLOG(2) << "Using query memory limit: " << PrettyPrinter::print(mem_limit, TUnit::BYTES);
     }
-    if (!create_if_not_present) return nullptr;
-    // First time this pool_name registered, make a new object.
-    std::shared_ptr<MemTracker> tracker = MemTracker::CreateTracker(
-            -1, strings::Substitute(REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT, pool_name),
-            ExecEnv::GetInstance()->process_mem_tracker());
-    tracker->pool_name_ = pool_name;
-    pool_to_mem_trackers_.emplace(pool_name, std::shared_ptr<MemTracker>(tracker));
+
+    // First time this query_id registered, make a new object, otherwise do nothing.
+    _query_mem_trackers.try_emplace_l(
+            query_id, [](std::shared_ptr<MemTracker>) {},
+            MemTracker::CreateTracker(mem_limit,
+                                      strings::Substitute(QUERY_MEM_TRACKER_LABEL_FORMAT, query_id),
+                                      ExecEnv::GetInstance()->hook_process_mem_tracker(), false,
+                                      false, MemTrackerLevel::OVERVIEW, query_id));
+
+    std::shared_ptr<MemTracker> tracker = nullptr;
+    _query_mem_trackers.if_contains(query_id,
+                                    [&tracker](std::shared_ptr<MemTracker> v) { tracker = v; });
     return tracker;
 }
 
+void QueryMemTrackerRegistry::DeregisterQueryMemTracker(const std::string& query_id) {
+    DCHECK(!query_id.empty());
+    _query_mem_trackers.erase_if(query_id, [](std::shared_ptr<MemTracker>) { return true; });
+    LOG(WARNING) << "DeregisterQueryMemTracker " << query_id << " len " << _query_mem_trackers.size();

Review comment:
       Why there is a warning log?

##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -105,12 +105,14 @@ std::shared_ptr<MemTracker> MemTracker::CreateTracker(RuntimeProfile* profile, i
             level > real_parent->_level ? level : real_parent->_level));
     real_parent->AddChildTracker(tracker);
     tracker->Init();
+    tracker->set_query_id(query_id);
 
     return tracker;
 }
 
 std::shared_ptr<MemTracker> MemTracker::CreateTracker(int64_t byte_limit, const std::string& label,
-        std::shared_ptr<MemTracker> parent, bool log_usage_if_zero, bool reset_label_name, MemTrackerLevel level) {
+        std::shared_ptr<MemTracker> parent, bool log_usage_if_zero, bool reset_label_name,
+        MemTrackerLevel level, std::string query_id) {

Review comment:
       ```suggestion
           MemTrackerLevel level, const std::string& query_id) {
   ```

##########
File path: be/src/runtime/exec_env.h
##########
@@ -174,7 +176,8 @@ class ExecEnv {
     ClientCache<TPaloBrokerServiceClient>* _broker_client_cache = nullptr;
     ClientCache<TExtDataSourceServiceClient>* _extdatasource_client_cache = nullptr;
     std::shared_ptr<MemTracker> _mem_tracker;

Review comment:
       Add comments to explain these trackers.

##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -78,7 +78,7 @@ void MemTracker::CreateRootTracker() {
 
 std::shared_ptr<MemTracker> MemTracker::CreateTracker(RuntimeProfile* profile, int64_t byte_limit,
                                                       const std::string& label, const std::shared_ptr<MemTracker>& parent,
-                                                      bool reset_label_name, MemTrackerLevel level) {
+                                                      bool reset_label_name, MemTrackerLevel level, std::string query_id) {

Review comment:
       ```suggestion
                                                         bool reset_label_name, MemTrackerLevel level, const std::string& query_id) {
   ```

##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -242,26 +243,38 @@ int64_t MemTracker::GetPoolMemReserved() {
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    if (mem_limit != -1) {
+        if (mem_limit > MemInfo::physical_mem()) {

Review comment:
       Is this check necessary? I think this can be done when memtracker is created?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r757446999



##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {
+public:
+    ThreadStatus() : _thread_id(std::this_thread::get_id()) {}
+    ~ThreadStatus() { update_mem_tracker(nullptr); }
+
+    void attach_query(const doris::TUniqueId& query_id) {
+        _query_id = doris::print_id(query_id);
+        update_mem_tracker(ExecEnv::GetInstance()->query_mem_trackers()->RegisterQueryMemTracker(
+                doris::print_id(query_id)));
+    }
+
+    void update_mem_tracker(std::shared_ptr<MemTracker> mem_tracker) {
+        if (_untracked_mem != 0 && _mem_tracker != nullptr) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+        _mem_tracker = mem_tracker;
+    }
+
+    void consume(int64_t size) {
+        if (_mem_tracker == nullptr && ExecEnv::GetInstance()->is_init()) {
+            _mem_tracker = ExecEnv::GetInstance()->hook_process_mem_tracker();
+        }
+        _untracked_mem += size;

Review comment:
       `_untracked_mem` only represents untracked memory for a short period of time.
   When there are more releases than consume during this period of time, `_untracked_mem` may be less than 0.
   
   The statistical value of MemTacker is not expected to be less than 0, which is DCHECK in MemTacke consume/release




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762803716



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -17,11 +17,9 @@
 
 #include "runtime/mem_tracker.h"
 
-#include <cstdint>
-
-#include <boost/algorithm/string/join.hpp>
 #include <boost/algorithm/string.hpp>
-
+#include <boost/algorithm/string/join.hpp>

Review comment:
       OK




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764779660



##########
File path: be/src/common/config.h
##########
@@ -601,6 +604,12 @@ CONF_mInt32(remote_storage_read_buffer_mb, "16");
 // the level equal or lower than mem_tracker_level will show in web page
 CONF_Int16(mem_tracker_level, "0");
 
+// The minimum length when TCMalloc Hook consumes/releases MemTracker, consume size
+// smaller than this value will continue to accumulate. specified as number of bytes.
+// Decreasing this value will increase the frequency of consume/release.
+// Increasing this value will cause MemTracker statistics to be inaccurate.
+CONF_mInt32(mem_tracker_consume_min_size_mbytes, "1048576");

Review comment:
       1048576 bytes, 1M, see the last digit of the variable name and the comment.
   The document is wrong, I will revise it, thx~




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762820686



##########
File path: be/src/runtime/mem_tracker.h
##########
@@ -573,33 +611,33 @@ class MemTracker : public std::enable_shared_from_this<MemTracker> {
     IntGauge* limit_metric_;
 };
 
-/// Global registry for query and pool MemTrackers. Owned by ExecEnv.
-class PoolMemTrackerRegistry {
+// Global registry for query MemTrackers. Owned by ExecEnv.
+class QueryMemTrackerRegistry {
 public:
-    /// Returns a MemTracker object for request pool 'pool_name'. Calling this with the same
-    /// 'pool_name' will return the same MemTracker object. This is used to track the local
-    /// memory usage of all requests executing in this pool. If 'create_if_not_present' is
-    /// true, the first time this is called for a pool, a new MemTracker object is created
-    /// with the process tracker as its parent. There is no explicit per-pool byte_limit
-    /// set at any particular impalad, so newly created trackers will always have a limit
-    /// of -1.
-    /// TODO(cmy): this function is not used for now. the memtracker returned from here is
-    ///            got from a shared_ptr in `pool_to_mem_trackers_`.
-    ///            This funtion is from
-    ///            https://github.com/cloudera/Impala/blob/495397101e5807c701df71ea288f4815d69c2c8a/be/src/runtime/mem-tracker.h#L497
-    ///            And in impala this function will return a raw pointer.
-    std::shared_ptr<MemTracker> GetRequestPoolMemTracker(const std::string& pool_name,
-                                                         bool create_if_not_present);
+    // Construct a MemTracker object for 'query_id' with 'mem_limit' as the memory limit.
+    // The MemTracker is a child of the process MemTracker, Calling this with the same
+    // 'query_id' will return the same MemTracker object. This is used to track the local
+    // memory usage of all querys executing. The first time this is called for a query,
+    // a new MemTracker object is created with the process tracker as its parent.
+    // Newly created trackers will always have a limit of -1.
+    std::shared_ptr<MemTracker> RegisterQueryMemTracker(const std::string& query_id,

Review comment:
       done,
   The naming format of this file still follows the impala ==, I will rewrite it later




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Refactor MemTracker and new memory statistics framework based on TCMalloc Hook

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r781732804



##########
File path: be/src/common/config.h
##########
@@ -594,12 +594,28 @@ CONF_Int32(aws_log_level, "3");
 // the buffer size when read data from remote storage like s3
 CONF_mInt32(remote_storage_read_buffer_mb, "16");
 
+// Whether to initialize TCmalloc new/delete Hook, MemTracker is currently counted in Hook.
+CONF_mBool(use_tc_hook, "true");

Review comment:
       track_new_delete maybe better




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755740745



##########
File path: be/src/runtime/tcmalloc_hook.h
##########
@@ -0,0 +1,45 @@
+// 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.
+
+#include <gperftools/malloc_hook.h>
+#include <gperftools/nallocx.h>
+#include <gperftools/tcmalloc.h>
+
+#include "runtime/thread_status.h"
+
+static int new_hook_calls = 0;

Review comment:
       Thats good~




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r767212879



##########
File path: be/src/common/config.h
##########
@@ -601,6 +604,12 @@ CONF_mInt32(remote_storage_read_buffer_mb, "16");
 // the level equal or lower than mem_tracker_level will show in web page
 CONF_Int16(mem_tracker_level, "0");
 
+// The minimum length when TCMalloc Hook consumes/releases MemTracker, consume size
+// smaller than this value will continue to accumulate. specified as number of bytes.
+// Decreasing this value will increase the frequency of consume/release.
+// Increasing this value will cause MemTracker statistics to be inaccurate.
+CONF_mInt32(mem_tracker_consume_min_size_mbytes, "1048576");

Review comment:
       Thats right, done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764634245



##########
File path: be/src/runtime/thread_mem_tracker.cpp
##########
@@ -0,0 +1,129 @@
+// 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.
+
+#include "runtime/thread_mem_tracker.h"
+
+#include "service/backend_options.h"
+
+namespace doris {
+
+void ThreadMemTracker::attach_query(const std::string& query_id,
+                                    const TUniqueId& fragment_instance_id) {
+#ifdef BE_TEST
+    if (ExecEnv::GetInstance()->query_mem_tracker_registry() == nullptr) {
+        return;
+    }
+#endif
+    update_query_mem_tracker(
+            ExecEnv::GetInstance()->query_mem_tracker_registry()->get_query_mem_tracker(query_id));
+    _fragment_instance_id = fragment_instance_id;
+}
+
+void ThreadMemTracker::detach_query() {
+    update_query_mem_tracker(std::weak_ptr<MemTracker>());
+    _fragment_instance_id = TUniqueId();
+}
+
+void ThreadMemTracker::update_query_mem_tracker(std::weak_ptr<MemTracker> mem_tracker) {
+    if (_untracked_mem != 0) {
+        consume();
+        _untracked_mem = 0;
+    }
+    _query_mem_tracker = mem_tracker;
+}
+
+void ThreadMemTracker::query_mem_limit_exceeded(int64_t mem_usage) {
+    if (_fragment_instance_id != TUniqueId() && ExecEnv::GetInstance()->initialized() &&
+        ExecEnv::GetInstance()->fragment_mgr()->is_canceling(_fragment_instance_id).ok()) {
+        std::string detail = "Query Memory exceed limit in TCMalloc Hook New.";
+        auto st = _query_mem_tracker.lock()->MemLimitExceeded(nullptr, detail, mem_usage);
+
+        detail +=
+                " Query Memory exceed limit in TCMalloc Hook New, Backend: {}, Fragment: {}, Used: "
+                "{}, Limit: {}. You can change the limit by session variable exec_mem_limit.";
+        fmt::format(detail, BackendOptions::get_localhost(), print_id(_fragment_instance_id),

Review comment:
       Is it better to print query id?

##########
File path: be/src/runtime/tcmalloc_hook.h
##########
@@ -0,0 +1,46 @@
+// 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.
+
+#include <gperftools/malloc_hook.h>
+#include <gperftools/nallocx.h>
+#include <gperftools/tcmalloc.h>
+
+#include "runtime/thread_context.h"
+
+void new_hook(const void* ptr, size_t size) {
+    doris::thread_local_ctx.consume_mem(tc_nallocx(size, 0));
+}
+
+void delete_hook(const void* ptr) {
+    doris::thread_local_ctx.release_mem(tc_malloc_size(const_cast<void*>(ptr)));
+}
+
+// Notice: modify the command in New/Delete Hook should be careful enough,
+// and should be as simple as possible, otherwise it may cause weird errors. E.g:
+//  1. The first New Hook call of the process may be before some variables of
+//  the process are initialized.
+//  2. Allocating memory in the Hook command causes the Hook to be entered again,
+//  infinite recursion.
+void init_hook() {
+    MallocHook::AddNewHook(&new_hook);
+    MallocHook::AddDeleteHook(&delete_hook);
+}
+
+void destroy_hook() {

Review comment:
       Didn't use?

##########
File path: be/src/runtime/thread_mem_tracker.h
##########
@@ -0,0 +1,80 @@
+// 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.
+
+#pragma once
+
+#include <fmt/format.h>
+
+#include "runtime/exec_env.h"
+#include "runtime/fragment_mgr.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+// TCMalloc new/delete Hook is counted in the memory_tracker of the current thread
+class ThreadMemTracker {
+public:
+    ThreadMemTracker() : _global_hook_tracker(MemTracker::GetGlobalHookTracker()) {}
+    ~ThreadMemTracker() { detach_query(); }
+
+    // After attach, the current thread TCMalloc Hook starts to consume/release query mem_tracker
+    void attach_query(const std::string& query_id, const TUniqueId& fragment_instance_id);
+
+    void detach_query();
+
+    void update_query_mem_tracker(std::weak_ptr<MemTracker> mem_tracker);
+
+    void query_mem_limit_exceeded(int64_t mem_usage);
+
+    void global_mem_limit_exceeded(int64_t mem_usage);
+
+    // Note that, If call the memory allocation operation in TCMalloc new/delete Hook,
+    // such as calling LOG/iostream/sstream/stringstream/etc. related methods,
+    // must increase the control to avoid entering infinite recursion, otherwise it may cause crash or stuck,
+    void consume();

Review comment:
       ```suggestion
       private void consume();
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#issuecomment-988713461


   > Why does this parameter 'untracked_mem_limit_mbytes' affect performance? What is your data and table meta?
   
   Because compared to the previous only consume/release MemTracker for large memory, the consume/release frequency in TCMalloc Hook will be much higher.
   This parameter has been renamed to `mem_tracker_consume_min_size_mbytes`, the comment is clearer


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764793130



##########
File path: be/src/runtime/tcmalloc_hook.h
##########
@@ -0,0 +1,46 @@
+// 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.
+
+#include <gperftools/malloc_hook.h>
+#include <gperftools/nallocx.h>
+#include <gperftools/tcmalloc.h>
+
+#include "runtime/thread_context.h"
+
+void new_hook(const void* ptr, size_t size) {
+    doris::thread_local_ctx.consume_mem(tc_nallocx(size, 0));
+}
+
+void delete_hook(const void* ptr) {
+    doris::thread_local_ctx.release_mem(tc_malloc_size(const_cast<void*>(ptr)));
+}
+
+// Notice: modify the command in New/Delete Hook should be careful enough,
+// and should be as simple as possible, otherwise it may cause weird errors. E.g:
+//  1. The first New Hook call of the process may be before some variables of
+//  the process are initialized.
+//  2. Allocating memory in the Hook command causes the Hook to be entered again,
+//  infinite recursion.
+void init_hook() {
+    MallocHook::AddNewHook(&new_hook);
+    MallocHook::AddDeleteHook(&delete_hook);
+}
+
+void destroy_hook() {

Review comment:
       Yes, the reserved method is for the future.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764785774



##########
File path: be/src/runtime/thread_mem_tracker.h
##########
@@ -0,0 +1,80 @@
+// 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.
+
+#pragma once
+
+#include <fmt/format.h>
+
+#include "runtime/exec_env.h"
+#include "runtime/fragment_mgr.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+// TCMalloc new/delete Hook is counted in the memory_tracker of the current thread
+class ThreadMemTracker {
+public:
+    ThreadMemTracker() : _global_hook_tracker(MemTracker::GetGlobalHookTracker()) {}
+    ~ThreadMemTracker() { detach_query(); }
+
+    // After attach, the current thread TCMalloc Hook starts to consume/release query mem_tracker
+    void attach_query(const std::string& query_id, const TUniqueId& fragment_instance_id);
+
+    void detach_query();
+
+    void update_query_mem_tracker(std::weak_ptr<MemTracker> mem_tracker);
+
+    void query_mem_limit_exceeded(int64_t mem_usage);
+
+    void global_mem_limit_exceeded(int64_t mem_usage);
+
+    // Note that, If call the memory allocation operation in TCMalloc new/delete Hook,
+    // such as calling LOG/iostream/sstream/stringstream/etc. related methods,
+    // must increase the control to avoid entering infinite recursion, otherwise it may cause crash or stuck,
+    void consume();
+
+    void try_consume(int64_t size);
+
+    void stop_mem_tracker() { _stop_mem_tracker = true; }
+
+private:
+    TUniqueId _fragment_instance_id;
+
+    std::weak_ptr<MemTracker> _query_mem_tracker;
+    std::shared_ptr<MemTracker> _global_hook_tracker = nullptr;
+
+    // Consume size smaller than _tracker_consume_min_size will continue to accumulate
+    // to avoid frequent calls to consume/release of MemTracker.
+    int64_t _untracked_mem = 0;
+    int64_t _tracker_consume_min_size = config::mem_tracker_consume_min_size_mbytes;

Review comment:
       Thats right, I think `_tracker_consume_cache_size` seems better~




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762685427



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -233,41 +260,71 @@ int64_t MemTracker::GetPoolMemReserved() {
                 // Make sure we don't overflow if the query limits are set to ridiculous values.
                 mem_reserved += std::min(child_limit, MemInfo::physical_mem());
             } else {
-                DCHECK(child_limit == -1)
-                        << child->LogUsage(UNLIMITED_DEPTH);
+                DCHECK(child_limit == -1) << child->LogUsage(UNLIMITED_DEPTH);
                 mem_reserved += child->consumption();
             }
         }
     }
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    VLOG(2) << "Register query memory tracker, query id: " << query_id

Review comment:
       Using macros defined in `be/src/common/logging.h`, not use the number directly in VLOG




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762818533



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -17,11 +17,9 @@
 
 #include "runtime/mem_tracker.h"
 
-#include <cstdint>
-
-#include <boost/algorithm/string/join.hpp>
 #include <boost/algorithm/string.hpp>
-
+#include <boost/algorithm/string/join.hpp>

Review comment:
       Replace with "util/string_util.h"

##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -17,11 +17,9 @@
 
 #include "runtime/mem_tracker.h"
 
-#include <cstdint>
-
-#include <boost/algorithm/string/join.hpp>
 #include <boost/algorithm/string.hpp>
-
+#include <boost/algorithm/string/join.hpp>

Review comment:
       OK




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] zuochunwei commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
zuochunwei commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755679625



##########
File path: be/src/runtime/exec_env.h
##########
@@ -93,6 +93,7 @@ class ExecEnv {
     // declarations for classes in scoped_ptrs.
     ~ExecEnv();
 
+    bool is_init() { return _is_init; }

Review comment:
       please add const to make is_init() read-only




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755730939



##########
File path: be/src/service/doris_main.cpp
##########
@@ -275,6 +276,9 @@ int main(int argc, char** argv) {
 #if defined(LEAK_SANITIZER)
         __lsan_do_leak_check();
 #endif
+        LOG(INFO) << "Tcmalloc New number: " << new_hook_calls

Review comment:
       This is temp log, forgot to delete it  =_=




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764787810



##########
File path: be/src/runtime/thread_mem_tracker.h
##########
@@ -0,0 +1,80 @@
+// 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.
+
+#pragma once
+
+#include <fmt/format.h>
+
+#include "runtime/exec_env.h"
+#include "runtime/fragment_mgr.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+// TCMalloc new/delete Hook is counted in the memory_tracker of the current thread
+class ThreadMemTracker {
+public:
+    ThreadMemTracker() : _global_hook_tracker(MemTracker::GetGlobalHookTracker()) {}
+    ~ThreadMemTracker() { detach_query(); }
+
+    // After attach, the current thread TCMalloc Hook starts to consume/release query mem_tracker
+    void attach_query(const std::string& query_id, const TUniqueId& fragment_instance_id);
+
+    void detach_query();
+
+    void update_query_mem_tracker(std::weak_ptr<MemTracker> mem_tracker);
+
+    void query_mem_limit_exceeded(int64_t mem_usage);
+
+    void global_mem_limit_exceeded(int64_t mem_usage);
+
+    // Note that, If call the memory allocation operation in TCMalloc new/delete Hook,
+    // such as calling LOG/iostream/sstream/stringstream/etc. related methods,
+    // must increase the control to avoid entering infinite recursion, otherwise it may cause crash or stuck,
+    void consume();

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762795781



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -233,41 +260,71 @@ int64_t MemTracker::GetPoolMemReserved() {
                 // Make sure we don't overflow if the query limits are set to ridiculous values.
                 mem_reserved += std::min(child_limit, MemInfo::physical_mem());
             } else {
-                DCHECK(child_limit == -1)
-                        << child->LogUsage(UNLIMITED_DEPTH);
+                DCHECK(child_limit == -1) << child->LogUsage(UNLIMITED_DEPTH);
                 mem_reserved += child->consumption();
             }
         }
     }
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    VLOG(2) << "Register query memory tracker, query id: " << query_id

Review comment:
       Thats great




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] EmmyMiao87 commented on pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#issuecomment-987850986


   Why does this parameter 'untracked_mem_limit_mbytes' affect performance? 
   What is your data and table meta?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r763937542



##########
File path: be/src/common/config.h
##########
@@ -601,6 +604,12 @@ CONF_mInt32(remote_storage_read_buffer_mb, "16");
 // the level equal or lower than mem_tracker_level will show in web page
 CONF_Int16(mem_tracker_level, "0");
 
+// The minimum length when TCMalloc Hook consumes/releases MemTracker, consume size
+// smaller than this value will continue to accumulate. specified as number of bytes.
+// Decreasing this value will increase the frequency of consume/release.
+// Increasing this value will cause MemTracker statistics to be inaccurate.
+CONF_mInt32(mem_tracker_consume_min_size_mbytes, "1048576");

Review comment:
       1048576 mb ??
   And the default 4M is written in your document.
   Which is the correct number ? 

##########
File path: be/src/runtime/thread_mem_tracker.h
##########
@@ -0,0 +1,80 @@
+// 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.
+
+#pragma once
+
+#include <fmt/format.h>
+
+#include "runtime/exec_env.h"
+#include "runtime/fragment_mgr.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+// TCMalloc new/delete Hook is counted in the memory_tracker of the current thread
+class ThreadMemTracker {
+public:
+    ThreadMemTracker() : _global_hook_tracker(MemTracker::GetGlobalHookTracker()) {}
+    ~ThreadMemTracker() { detach_query(); }
+
+    // After attach, the current thread TCMalloc Hook starts to consume/release query mem_tracker
+    void attach_query(const std::string& query_id, const TUniqueId& fragment_instance_id);
+
+    void detach_query();
+
+    void update_query_mem_tracker(std::weak_ptr<MemTracker> mem_tracker);
+
+    void query_mem_limit_exceeded(int64_t mem_usage);
+
+    void global_mem_limit_exceeded(int64_t mem_usage);
+
+    // Note that, If call the memory allocation operation in TCMalloc new/delete Hook,
+    // such as calling LOG/iostream/sstream/stringstream/etc. related methods,
+    // must increase the control to avoid entering infinite recursion, otherwise it may cause crash or stuck,
+    void consume();
+
+    void try_consume(int64_t size);
+
+    void stop_mem_tracker() { _stop_mem_tracker = true; }
+
+private:
+    TUniqueId _fragment_instance_id;
+
+    std::weak_ptr<MemTracker> _query_mem_tracker;
+    std::shared_ptr<MemTracker> _global_hook_tracker = nullptr;
+
+    // Consume size smaller than _tracker_consume_min_size will continue to accumulate
+    // to avoid frequent calls to consume/release of MemTracker.
+    int64_t _untracked_mem = 0;
+    int64_t _tracker_consume_min_size = config::mem_tracker_consume_min_size_mbytes;

Review comment:
       the `_tracker_consume_min_size` may be consume or release.
   So the name is '_untracked_mem_min_size'. Is it better ?

##########
File path: be/src/runtime/thread_context.h
##########
@@ -0,0 +1,118 @@
+// 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.
+
+#pragma once
+
+#include <string>
+#include <thread>
+
+#include "gen_cpp/Types_types.h"
+#include "runtime/thread_mem_tracker.h"
+
+namespace doris {
+
+// The thread context saves some info about a working thread.
+// 2 requried info:
+//   1. thread_id:   Current thread id, Auto generated.
+//   2. type:        The type is a enum value indicating which type of task current thread is running.
+//                   For example: QUERY, LOAD, COMPACTION, ...
+//   3. task id:     A unique id to identify this task. maybe query id, load job id, etc.
+//
+// There may be other optional info to be added later.
+class ThreadContext {
+public:
+    enum TaskType {
+        UNKNOWN = 0,
+        QUERY = 1,
+        LOAD = 2,
+        COMPACTION = 3
+        // to be added ...
+    };
+
+public:
+    ThreadContext() : _thread_id(std::this_thread::get_id()), _type(TaskType::UNKNOWN) {
+        _thread_mem_tracker.reset(new ThreadMemTracker());
+    }
+    ~ThreadContext() {}
+
+    void attach(const TaskType& type, const std::string& task_id,
+                const TUniqueId& fragment_instance_id = TUniqueId()) {

Review comment:
       Like the compaction thread, there is no fragment instance id. Do I need to add a fake?
   Isn't this value an optional?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz closed pull request #7198: [Memory] Refactor MemTracker and new memory statistics framework based on TCMalloc Hook

Posted by GitBox <gi...@apache.org>.
xinyiZzz closed pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r767212879



##########
File path: be/src/common/config.h
##########
@@ -601,6 +604,12 @@ CONF_mInt32(remote_storage_read_buffer_mb, "16");
 // the level equal or lower than mem_tracker_level will show in web page
 CONF_Int16(mem_tracker_level, "0");
 
+// The minimum length when TCMalloc Hook consumes/releases MemTracker, consume size
+// smaller than this value will continue to accumulate. specified as number of bytes.
+// Decreasing this value will increase the frequency of consume/release.
+// Increasing this value will cause MemTracker statistics to be inaccurate.
+CONF_mInt32(mem_tracker_consume_min_size_mbytes, "1048576");

Review comment:
       Thats right

##########
File path: be/src/common/config.h
##########
@@ -601,6 +604,12 @@ CONF_mInt32(remote_storage_read_buffer_mb, "16");
 // the level equal or lower than mem_tracker_level will show in web page
 CONF_Int16(mem_tracker_level, "0");
 
+// The minimum length when TCMalloc Hook consumes/releases MemTracker, consume size
+// smaller than this value will continue to accumulate. specified as number of bytes.
+// Decreasing this value will increase the frequency of consume/release.
+// Increasing this value will cause MemTracker statistics to be inaccurate.
+CONF_mInt32(mem_tracker_consume_min_size_mbytes, "1048576");

Review comment:
       Thats right, done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762680469



##########
File path: be/src/exec/olap_scan_node.cpp
##########
@@ -1652,6 +1655,7 @@ void OlapScanNode::scanner_thread(OlapScanner* scanner) {
     // and transfer thread
     _scan_batch_added_cv.notify_one();
     _scan_thread_exit_cv.notify_one();
+    thread_local_ctx.unattach_query();

Review comment:
       ```suggestion
       thread_local_ctx. detach_query();
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764777101



##########
File path: be/src/runtime/thread_context.h
##########
@@ -0,0 +1,118 @@
+// 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.
+
+#pragma once
+
+#include <string>
+#include <thread>
+
+#include "gen_cpp/Types_types.h"
+#include "runtime/thread_mem_tracker.h"
+
+namespace doris {
+
+// The thread context saves some info about a working thread.
+// 2 requried info:
+//   1. thread_id:   Current thread id, Auto generated.
+//   2. type:        The type is a enum value indicating which type of task current thread is running.
+//                   For example: QUERY, LOAD, COMPACTION, ...
+//   3. task id:     A unique id to identify this task. maybe query id, load job id, etc.
+//
+// There may be other optional info to be added later.
+class ThreadContext {
+public:
+    enum TaskType {
+        UNKNOWN = 0,
+        QUERY = 1,
+        LOAD = 2,
+        COMPACTION = 3
+        // to be added ...
+    };
+
+public:
+    ThreadContext() : _thread_id(std::this_thread::get_id()), _type(TaskType::UNKNOWN) {
+        _thread_mem_tracker.reset(new ThreadMemTracker());
+    }
+    ~ThreadContext() {}
+
+    void attach(const TaskType& type, const std::string& task_id,
+                const TUniqueId& fragment_instance_id = TUniqueId()) {

Review comment:
       Thats right, currently only query thread needs fragment instance id, other threads don’t need to pass this parameter
   I will modify the location where the fragment instance id is assigned.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r757444063



##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {
+public:
+    ThreadStatus() : _thread_id(std::this_thread::get_id()) {}
+    ~ThreadStatus() { update_mem_tracker(nullptr); }
+
+    void attach_query(const doris::TUniqueId& query_id) {
+        _query_id = doris::print_id(query_id);
+        update_mem_tracker(ExecEnv::GetInstance()->query_mem_trackers()->RegisterQueryMemTracker(
+                doris::print_id(query_id)));
+    }
+
+    void update_mem_tracker(std::shared_ptr<MemTracker> mem_tracker) {
+        if (_untracked_mem != 0 && _mem_tracker != nullptr) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+        _mem_tracker = mem_tracker;
+    }
+
+    void consume(int64_t size) {
+        if (_mem_tracker == nullptr && ExecEnv::GetInstance()->is_init()) {
+            _mem_tracker = ExecEnv::GetInstance()->hook_process_mem_tracker();
+        }
+        _untracked_mem += size;
+        if (_mem_tracker != nullptr && (_untracked_mem >= _s_untracked_mem_limit ||
+                                        _untracked_mem <= -_s_untracked_mem_limit)) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+    }
+
+    void consume_mem(int64_t size) { consume(size); }
+
+    void release_mem(int64_t size) { consume(-size); }
+
+    const std::string& query_id() { return _query_id; }
+    const std::thread::id& thread_id() { return _thread_id; }
+
+private:
+    std::thread::id _thread_id;
+    std::string _query_id;
+    std::shared_ptr<MemTracker> _mem_tracker = nullptr;
+    int64_t _untracked_mem = 0;
+    int64_t _s_untracked_mem_limit = 1 * 1024 * 1024;
+};
+
+inline thread_local ThreadStatus current_thread;

Review comment:
       yes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Refactor MemTracker and new memory statistics framework based on TCMalloc Hook

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r781732804



##########
File path: be/src/common/config.h
##########
@@ -594,12 +594,28 @@ CONF_Int32(aws_log_level, "3");
 // the buffer size when read data from remote storage like s3
 CONF_mInt32(remote_storage_read_buffer_mb, "16");
 
+// Whether to initialize TCmalloc new/delete Hook, MemTracker is currently counted in Hook.
+CONF_mBool(use_tc_hook, "true");

Review comment:
       track_new_delete maybe better, is this can be mutable




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r765548980



##########
File path: be/src/common/config.h
##########
@@ -601,6 +604,12 @@ CONF_mInt32(remote_storage_read_buffer_mb, "16");
 // the level equal or lower than mem_tracker_level will show in web page
 CONF_Int16(mem_tracker_level, "0");
 
+// The minimum length when TCMalloc Hook consumes/releases MemTracker, consume size
+// smaller than this value will continue to accumulate. specified as number of bytes.
+// Decreasing this value will increase the frequency of consume/release.
+// Increasing this value will cause MemTracker statistics to be inaccurate.
+CONF_mInt32(mem_tracker_consume_min_size_mbytes, "1048576");

Review comment:
       Then your variable name should be called 'xxx_bytes' not 'xxx_mbytes'




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755736706



##########
File path: be/src/runtime/exec_env.h
##########
@@ -93,6 +93,7 @@ class ExecEnv {
     // declarations for classes in scoped_ptrs.
     ~ExecEnv();
 
+    bool is_init() { return _is_init; }

Review comment:
       OK




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] zuochunwei commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
zuochunwei commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755683145



##########
File path: be/src/runtime/tcmalloc_hook.h
##########
@@ -0,0 +1,45 @@
+// 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.
+
+#include <gperftools/malloc_hook.h>
+#include <gperftools/nallocx.h>
+#include <gperftools/tcmalloc.h>
+
+#include "runtime/thread_status.h"
+
+static int new_hook_calls = 0;

Review comment:
       why not use uint64_t, or atomic_uint64?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762681382



##########
File path: gensrc/proto/internal_service.proto
##########
@@ -131,6 +131,7 @@ enum PPlanFragmentCancelReason {
     USER_CANCEL = 2;
     INTERNAL_ERROR = 3;
     TIMEOUT = 4;
+    MEMORY_EXCEED_LIMIT = 5;

Review comment:
       ```suggestion
       MEMORY_LIMIT_EXCEED = 5;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762685844



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -233,41 +260,71 @@ int64_t MemTracker::GetPoolMemReserved() {
                 // Make sure we don't overflow if the query limits are set to ridiculous values.
                 mem_reserved += std::min(child_limit, MemInfo::physical_mem());
             } else {
-                DCHECK(child_limit == -1)
-                        << child->LogUsage(UNLIMITED_DEPTH);
+                DCHECK(child_limit == -1) << child->LogUsage(UNLIMITED_DEPTH);
                 mem_reserved += child->consumption();
             }
         }
     }
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    VLOG(2) << "Register query memory tracker, query id: " << query_id
+            << " limit: " << PrettyPrinter::print(mem_limit, TUnit::BYTES);
+
+    // First time this query_id registered, make a new object, otherwise do nothing.
+    // Combine CreateTracker and emplace into one operation to avoid the use of locks
+    _query_mem_trackers.try_emplace_l(
+            query_id, [](std::shared_ptr<MemTracker>) {},
+            MemTracker::CreateTracker(mem_limit,
+                                      strings::Substitute(QUERY_MEM_TRACKER_LABEL_FORMAT, query_id),
+                                      ExecEnv::GetInstance()->all_query_mem_tracker(), false, false,

Review comment:
       better use fmt instead of gutils




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762685427



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -233,41 +260,71 @@ int64_t MemTracker::GetPoolMemReserved() {
                 // Make sure we don't overflow if the query limits are set to ridiculous values.
                 mem_reserved += std::min(child_limit, MemInfo::physical_mem());
             } else {
-                DCHECK(child_limit == -1)
-                        << child->LogUsage(UNLIMITED_DEPTH);
+                DCHECK(child_limit == -1) << child->LogUsage(UNLIMITED_DEPTH);
                 mem_reserved += child->consumption();
             }
         }
     }
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    VLOG(2) << "Register query memory tracker, query id: " << query_id

Review comment:
       Using macros defined in `be/src/common/logging.h`, not use the number directly




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762831425



##########
File path: be/src/runtime/exec_env.h
##########
@@ -93,6 +93,7 @@ class ExecEnv {
     // declarations for classes in scoped_ptrs.
     ~ExecEnv();
 
+    const bool is_init() { return _is_init; }

Review comment:
       Thats great




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755732039



##########
File path: be/src/exec/olap_scan_node.cpp
##########
@@ -1652,6 +1654,7 @@ void OlapScanNode::scanner_thread(OlapScanner* scanner) {
     // and transfer thread
     _scan_batch_added_cv.notify_one();
     _scan_thread_exit_cv.notify_one();
+    current_thread.update_mem_tracker(nullptr);

Review comment:
       OK, Is combing where the threads start and end.
   There is currently a problem that query memtracker cannot be deleted in time.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762684059



##########
File path: be/src/runtime/exec_env.h
##########
@@ -173,16 +175,20 @@ class ExecEnv {
     ClientCache<FrontendServiceClient>* _frontend_client_cache = nullptr;
     ClientCache<TPaloBrokerServiceClient>* _broker_client_cache = nullptr;
     ClientCache<TExtDataSourceServiceClient>* _extdatasource_client_cache = nullptr;
-    std::shared_ptr<MemTracker> _mem_tracker;
-    PoolMemTrackerRegistry* _pool_mem_trackers = nullptr;
+    // The ancestor of all trackers in the process. It is the only child of the root tracker.
+    // All manually created trackers should specify the process tracker as the parent.
+    std::shared_ptr<MemTracker> _process_mem_tracker = nullptr;
+    // The ancestor for all querys tracker.
+    std::shared_ptr<MemTracker> _all_query_mem_tracker = nullptr;
+    QueryMemTrackerRegistry* _query_mem_tracker_registry = nullptr;

Review comment:
       Why not use unique_ptr but use a raw pointer




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762686367



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -17,11 +17,9 @@
 
 #include "runtime/mem_tracker.h"
 
-#include <cstdint>
-
-#include <boost/algorithm/string/join.hpp>
 #include <boost/algorithm/string.hpp>
-
+#include <boost/algorithm/string/join.hpp>

Review comment:
       We should avoid use boost libs, we are working to remove all boost lib




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762680407



##########
File path: be/src/exec/olap_scan_node.cpp
##########
@@ -1503,6 +1505,7 @@ void OlapScanNode::scanner_thread(OlapScanner* scanner) {
         _scan_batch_added_cv.notify_one();
         _scan_thread_exit_cv.notify_one();
         LOG(INFO) << "Scan thread cancelled, cause query done, scan thread started to exit";
+        thread_local_ctx.unattach_query();

Review comment:
       ```suggestion
           thread_local_ctx.detach_query();
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762691775



##########
File path: be/src/runtime/thread_context.h
##########
@@ -0,0 +1,187 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "gen_cpp/Types_types.h"
+#include "runtime/exec_env.h"
+#include "runtime/fragment_mgr.h"
+#include "runtime/mem_tracker.h"
+#include "service/backend_options.h"
+
+namespace doris {
+
+class TheadContext {
+public:
+    TheadContext()
+            : _thread_id(std::this_thread::get_id()),
+              _global_hook_tracker(MemTracker::GetGlobalHookTracker()) {}
+    ~TheadContext() { update_query_mem_tracker(); }
+
+    void attach_query(const TUniqueId& query_id,
+                      const TUniqueId& fragment_instance_id = TUniqueId()) {
+        _query_id = query_id;
+        _fragment_instance_id = fragment_instance_id;
+#ifdef BE_TEST
+        if (ExecEnv::GetInstance()->query_mem_tracker_registry() == nullptr) {
+            return;
+        }
+#endif
+        update_query_mem_tracker(
+                ExecEnv::GetInstance()->query_mem_tracker_registry()->GetQueryMemTracker(
+                        print_id(query_id)));
+    }
+
+    void unattach_query() {
+        _query_id = TUniqueId();
+        _fragment_instance_id = TUniqueId();
+        update_query_mem_tracker();
+    }
+
+    void update_query_mem_tracker(
+            std::weak_ptr<MemTracker> mem_tracker = std::weak_ptr<MemTracker>()) {
+        if (_untracked_mem != 0) {
+            consume();
+            _untracked_mem = 0;
+        }
+        _query_mem_tracker = mem_tracker;
+    }
+
+    void query_mem_limit_exceeded(int64_t mem_usage) {
+        if (_query_id != TUniqueId() && _fragment_instance_id != TUniqueId() &&
+            ExecEnv::GetInstance()->is_init() &&
+            ExecEnv::GetInstance()->fragment_mgr()->is_canceling(_fragment_instance_id).ok()) {
+            std::string detail = "Query Memory exceed limit in TCMalloc Hook New.";
+            auto st = _query_mem_tracker.lock()->MemLimitExceeded(nullptr, detail, mem_usage);

Review comment:
       use fmt::




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762692151



##########
File path: be/src/runtime/exec_env.h
##########
@@ -93,6 +93,7 @@ class ExecEnv {
     // declarations for classes in scoped_ptrs.
     ~ExecEnv();
 
+    const bool is_init() { return _is_init; }

Review comment:
       initialized may be better




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764799713



##########
File path: be/src/runtime/thread_mem_tracker.cpp
##########
@@ -0,0 +1,129 @@
+// 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.
+
+#include "runtime/thread_mem_tracker.h"
+
+#include "service/backend_options.h"
+
+namespace doris {
+
+void ThreadMemTracker::attach_query(const std::string& query_id,
+                                    const TUniqueId& fragment_instance_id) {
+#ifdef BE_TEST
+    if (ExecEnv::GetInstance()->query_mem_tracker_registry() == nullptr) {
+        return;
+    }
+#endif
+    update_query_mem_tracker(
+            ExecEnv::GetInstance()->query_mem_tracker_registry()->get_query_mem_tracker(query_id));
+    _fragment_instance_id = fragment_instance_id;
+}
+
+void ThreadMemTracker::detach_query() {
+    update_query_mem_tracker(std::weak_ptr<MemTracker>());
+    _fragment_instance_id = TUniqueId();
+}
+
+void ThreadMemTracker::update_query_mem_tracker(std::weak_ptr<MemTracker> mem_tracker) {
+    if (_untracked_mem != 0) {
+        consume();
+        _untracked_mem = 0;
+    }
+    _query_mem_tracker = mem_tracker;
+}
+
+void ThreadMemTracker::query_mem_limit_exceeded(int64_t mem_usage) {
+    if (_fragment_instance_id != TUniqueId() && ExecEnv::GetInstance()->initialized() &&
+        ExecEnv::GetInstance()->fragment_mgr()->is_canceling(_fragment_instance_id).ok()) {
+        std::string detail = "Query Memory exceed limit in TCMalloc Hook New.";
+        auto st = _query_mem_tracker.lock()->MemLimitExceeded(nullptr, detail, mem_usage);
+
+        detail +=
+                " Query Memory exceed limit in TCMalloc Hook New, Backend: {}, Fragment: {}, Used: "
+                "{}, Limit: {}. You can change the limit by session variable exec_mem_limit.";
+        fmt::format(detail, BackendOptions::get_localhost(), print_id(_fragment_instance_id),
+                    std::to_string(_query_mem_tracker.lock()->consumption()),
+                    std::to_string(_query_mem_tracker.lock()->limit()));
+        ExecEnv::GetInstance()->fragment_mgr()->cancel(

Review comment:
       I will pay attention to the problem that `PlanFragmentExecutor::cancel` cannot cancel instance in time.
   
   After that, can consider printing the thread context recorded in ThreadContext, but it may be useless, because ThreadMemTracker will accumulate a batch of memory requests to call consume, and the memory request mem_limit_exceeded may be small. 
   It may be more effective to print the status of the query and all instance trackers.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r764792325



##########
File path: be/src/runtime/thread_mem_tracker.cpp
##########
@@ -0,0 +1,129 @@
+// 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.
+
+#include "runtime/thread_mem_tracker.h"
+
+#include "service/backend_options.h"
+
+namespace doris {
+
+void ThreadMemTracker::attach_query(const std::string& query_id,
+                                    const TUniqueId& fragment_instance_id) {
+#ifdef BE_TEST
+    if (ExecEnv::GetInstance()->query_mem_tracker_registry() == nullptr) {
+        return;
+    }
+#endif
+    update_query_mem_tracker(
+            ExecEnv::GetInstance()->query_mem_tracker_registry()->get_query_mem_tracker(query_id));
+    _fragment_instance_id = fragment_instance_id;
+}
+
+void ThreadMemTracker::detach_query() {
+    update_query_mem_tracker(std::weak_ptr<MemTracker>());
+    _fragment_instance_id = TUniqueId();
+}
+
+void ThreadMemTracker::update_query_mem_tracker(std::weak_ptr<MemTracker> mem_tracker) {
+    if (_untracked_mem != 0) {
+        consume();
+        _untracked_mem = 0;
+    }
+    _query_mem_tracker = mem_tracker;
+}
+
+void ThreadMemTracker::query_mem_limit_exceeded(int64_t mem_usage) {
+    if (_fragment_instance_id != TUniqueId() && ExecEnv::GetInstance()->initialized() &&
+        ExecEnv::GetInstance()->fragment_mgr()->is_canceling(_fragment_instance_id).ok()) {
+        std::string detail = "Query Memory exceed limit in TCMalloc Hook New.";
+        auto st = _query_mem_tracker.lock()->MemLimitExceeded(nullptr, detail, mem_usage);
+
+        detail +=
+                " Query Memory exceed limit in TCMalloc Hook New, Backend: {}, Fragment: {}, Used: "
+                "{}, Limit: {}. You can change the limit by session variable exec_mem_limit.";
+        fmt::format(detail, BackendOptions::get_localhost(), print_id(_fragment_instance_id),

Review comment:
       Thats right, All fragment instance memory trackers will be printed in the future.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r757444557



##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {
+public:
+    ThreadStatus() : _thread_id(std::this_thread::get_id()) {}
+    ~ThreadStatus() { update_mem_tracker(nullptr); }
+
+    void attach_query(const doris::TUniqueId& query_id) {
+        _query_id = doris::print_id(query_id);
+        update_mem_tracker(ExecEnv::GetInstance()->query_mem_trackers()->RegisterQueryMemTracker(
+                doris::print_id(query_id)));
+    }
+
+    void update_mem_tracker(std::shared_ptr<MemTracker> mem_tracker) {
+        if (_untracked_mem != 0 && _mem_tracker != nullptr) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+        _mem_tracker = mem_tracker;
+    }
+
+    void consume(int64_t size) {
+        if (_mem_tracker == nullptr && ExecEnv::GetInstance()->is_init()) {
+            _mem_tracker = ExecEnv::GetInstance()->hook_process_mem_tracker();
+        }
+        _untracked_mem += size;
+        if (_mem_tracker != nullptr && (_untracked_mem >= _s_untracked_mem_limit ||
+                                        _untracked_mem <= -_s_untracked_mem_limit)) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+    }
+
+    void consume_mem(int64_t size) { consume(size); }
+
+    void release_mem(int64_t size) { consume(-size); }
+
+    const std::string& query_id() { return _query_id; }
+    const std::thread::id& thread_id() { return _thread_id; }
+
+private:
+    std::thread::id _thread_id;
+    std::string _query_id;
+    std::shared_ptr<MemTracker> _mem_tracker = nullptr;
+    int64_t _untracked_mem = 0;
+    int64_t _s_untracked_mem_limit = 1 * 1024 * 1024;

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755969683



##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {
+public:
+    ThreadStatus() : _thread_id(std::this_thread::get_id()) {}
+    ~ThreadStatus() { update_mem_tracker(nullptr); }
+
+    void attach_query(const doris::TUniqueId& query_id) {
+        _query_id = doris::print_id(query_id);
+        update_mem_tracker(ExecEnv::GetInstance()->query_mem_trackers()->RegisterQueryMemTracker(
+                doris::print_id(query_id)));
+    }
+
+    void update_mem_tracker(std::shared_ptr<MemTracker> mem_tracker) {
+        if (_untracked_mem != 0 && _mem_tracker != nullptr) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+        _mem_tracker = mem_tracker;
+    }
+
+    void consume(int64_t size) {
+        if (_mem_tracker == nullptr && ExecEnv::GetInstance()->is_init()) {
+            _mem_tracker = ExecEnv::GetInstance()->hook_process_mem_tracker();
+        }
+        _untracked_mem += size;
+        if (_mem_tracker != nullptr && (_untracked_mem >= _s_untracked_mem_limit ||
+                                        _untracked_mem <= -_s_untracked_mem_limit)) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+    }
+
+    void consume_mem(int64_t size) { consume(size); }
+
+    void release_mem(int64_t size) { consume(-size); }
+
+    const std::string& query_id() { return _query_id; }
+    const std::thread::id& thread_id() { return _thread_id; }
+
+private:
+    std::thread::id _thread_id;
+    std::string _query_id;
+    std::shared_ptr<MemTracker> _mem_tracker = nullptr;
+    int64_t _untracked_mem = 0;

Review comment:
       maybe uint64 is better?

##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {
+public:
+    ThreadStatus() : _thread_id(std::this_thread::get_id()) {}
+    ~ThreadStatus() { update_mem_tracker(nullptr); }
+
+    void attach_query(const doris::TUniqueId& query_id) {
+        _query_id = doris::print_id(query_id);
+        update_mem_tracker(ExecEnv::GetInstance()->query_mem_trackers()->RegisterQueryMemTracker(
+                doris::print_id(query_id)));
+    }
+
+    void update_mem_tracker(std::shared_ptr<MemTracker> mem_tracker) {
+        if (_untracked_mem != 0 && _mem_tracker != nullptr) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+        _mem_tracker = mem_tracker;
+    }
+
+    void consume(int64_t size) {
+        if (_mem_tracker == nullptr && ExecEnv::GetInstance()->is_init()) {
+            _mem_tracker = ExecEnv::GetInstance()->hook_process_mem_tracker();
+        }
+        _untracked_mem += size;
+        if (_mem_tracker != nullptr && (_untracked_mem >= _s_untracked_mem_limit ||
+                                        _untracked_mem <= -_s_untracked_mem_limit)) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+    }
+
+    void consume_mem(int64_t size) { consume(size); }
+
+    void release_mem(int64_t size) { consume(-size); }
+
+    const std::string& query_id() { return _query_id; }
+    const std::thread::id& thread_id() { return _thread_id; }
+
+private:
+    std::thread::id _thread_id;
+    std::string _query_id;
+    std::shared_ptr<MemTracker> _mem_tracker = nullptr;
+    int64_t _untracked_mem = 0;
+    int64_t _s_untracked_mem_limit = 1 * 1024 * 1024;
+};
+
+inline thread_local ThreadStatus current_thread;

Review comment:
       thread_local_ctx?

##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {
+public:
+    ThreadStatus() : _thread_id(std::this_thread::get_id()) {}
+    ~ThreadStatus() { update_mem_tracker(nullptr); }
+
+    void attach_query(const doris::TUniqueId& query_id) {
+        _query_id = doris::print_id(query_id);
+        update_mem_tracker(ExecEnv::GetInstance()->query_mem_trackers()->RegisterQueryMemTracker(
+                doris::print_id(query_id)));
+    }
+
+    void update_mem_tracker(std::shared_ptr<MemTracker> mem_tracker) {
+        if (_untracked_mem != 0 && _mem_tracker != nullptr) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+        _mem_tracker = mem_tracker;
+    }
+
+    void consume(int64_t size) {
+        if (_mem_tracker == nullptr && ExecEnv::GetInstance()->is_init()) {
+            _mem_tracker = ExecEnv::GetInstance()->hook_process_mem_tracker();
+        }
+        _untracked_mem += size;
+        if (_mem_tracker != nullptr && (_untracked_mem >= _s_untracked_mem_limit ||
+                                        _untracked_mem <= -_s_untracked_mem_limit)) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+    }
+
+    void consume_mem(int64_t size) { consume(size); }
+
+    void release_mem(int64_t size) { consume(-size); }
+
+    const std::string& query_id() { return _query_id; }
+    const std::thread::id& thread_id() { return _thread_id; }
+
+private:
+    std::thread::id _thread_id;
+    std::string _query_id;
+    std::shared_ptr<MemTracker> _mem_tracker = nullptr;
+    int64_t _untracked_mem = 0;
+    int64_t _s_untracked_mem_limit = 1 * 1024 * 1024;

Review comment:
       use const instead 
   btw , What is the _s mean?

##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {
+public:
+    ThreadStatus() : _thread_id(std::this_thread::get_id()) {}
+    ~ThreadStatus() { update_mem_tracker(nullptr); }
+
+    void attach_query(const doris::TUniqueId& query_id) {
+        _query_id = doris::print_id(query_id);
+        update_mem_tracker(ExecEnv::GetInstance()->query_mem_trackers()->RegisterQueryMemTracker(
+                doris::print_id(query_id)));
+    }
+
+    void update_mem_tracker(std::shared_ptr<MemTracker> mem_tracker) {
+        if (_untracked_mem != 0 && _mem_tracker != nullptr) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+        _mem_tracker = mem_tracker;
+    }
+
+    void consume(int64_t size) {
+        if (_mem_tracker == nullptr && ExecEnv::GetInstance()->is_init()) {
+            _mem_tracker = ExecEnv::GetInstance()->hook_process_mem_tracker();
+        }
+        _untracked_mem += size;
+        if (_mem_tracker != nullptr && (_untracked_mem >= _s_untracked_mem_limit ||
+                                        _untracked_mem <= -_s_untracked_mem_limit)) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+    }
+
+    void consume_mem(int64_t size) { consume(size); }
+
+    void release_mem(int64_t size) { consume(-size); }
+
+    const std::string& query_id() { return _query_id; }
+    const std::thread::id& thread_id() { return _thread_id; }
+
+private:
+    std::thread::id _thread_id;
+    std::string _query_id;
+    std::shared_ptr<MemTracker> _mem_tracker = nullptr;
+    int64_t _untracked_mem = 0;
+    int64_t _s_untracked_mem_limit = 1 * 1024 * 1024;

Review comment:
       Please add comment or unit

##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {
+public:
+    ThreadStatus() : _thread_id(std::this_thread::get_id()) {}
+    ~ThreadStatus() { update_mem_tracker(nullptr); }
+
+    void attach_query(const doris::TUniqueId& query_id) {
+        _query_id = doris::print_id(query_id);
+        update_mem_tracker(ExecEnv::GetInstance()->query_mem_trackers()->RegisterQueryMemTracker(
+                doris::print_id(query_id)));
+    }
+
+    void update_mem_tracker(std::shared_ptr<MemTracker> mem_tracker) {
+        if (_untracked_mem != 0 && _mem_tracker != nullptr) {
+            if (!_mem_tracker->TryConsume(_untracked_mem)) {
+                return; // add call back
+            }
+            _untracked_mem = 0;
+        }
+        _mem_tracker = mem_tracker;
+    }
+
+    void consume(int64_t size) {
+        if (_mem_tracker == nullptr && ExecEnv::GetInstance()->is_init()) {
+            _mem_tracker = ExecEnv::GetInstance()->hook_process_mem_tracker();
+        }
+        _untracked_mem += size;

Review comment:
       Add a dcheck(_untracked_mem>=0)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755733076



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -242,26 +243,38 @@ int64_t MemTracker::GetPoolMemReserved() {
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    if (mem_limit != -1) {
+        if (mem_limit > MemInfo::physical_mem()) {
+            LOG(WARNING) << "Memory limit " << PrettyPrinter::print(mem_limit, TUnit::BYTES)
+                         << " exceeds physical memory of "
+                         << PrettyPrinter::print(MemInfo::physical_mem(), TUnit::BYTES);
+        }
+        VLOG(2) << "Using query memory limit: " << PrettyPrinter::print(mem_limit, TUnit::BYTES);
     }
-    if (!create_if_not_present) return nullptr;
-    // First time this pool_name registered, make a new object.
-    std::shared_ptr<MemTracker> tracker = MemTracker::CreateTracker(
-            -1, strings::Substitute(REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT, pool_name),
-            ExecEnv::GetInstance()->process_mem_tracker());
-    tracker->pool_name_ = pool_name;
-    pool_to_mem_trackers_.emplace(pool_name, std::shared_ptr<MemTracker>(tracker));
+
+    // First time this query_id registered, make a new object, otherwise do nothing.
+    _query_mem_trackers.try_emplace_l(
+            query_id, [](std::shared_ptr<MemTracker>) {},
+            MemTracker::CreateTracker(mem_limit,
+                                      strings::Substitute(QUERY_MEM_TRACKER_LABEL_FORMAT, query_id),
+                                      ExecEnv::GetInstance()->hook_process_mem_tracker(), false,
+                                      false, MemTrackerLevel::OVERVIEW, query_id));
+
+    std::shared_ptr<MemTracker> tracker = nullptr;
+    _query_mem_trackers.if_contains(query_id,
+                                    [&tracker](std::shared_ptr<MemTracker> v) { tracker = v; });
     return tracker;
 }
 
+void QueryMemTrackerRegistry::DeregisterQueryMemTracker(const std::string& query_id) {
+    DCHECK(!query_id.empty());
+    _query_mem_trackers.erase_if(query_id, [](std::shared_ptr<MemTracker>) { return true; });
+    LOG(WARNING) << "DeregisterQueryMemTracker " << query_id << " len " << _query_mem_trackers.size();

Review comment:
       sorry, This is temp log, forgot to delete it =_=




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762831425



##########
File path: be/src/runtime/exec_env.h
##########
@@ -93,6 +93,7 @@ class ExecEnv {
     // declarations for classes in scoped_ptrs.
     ~ExecEnv();
 
+    const bool is_init() { return _is_init; }

Review comment:
       Thats great




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762800416



##########
File path: be/src/runtime/mem_tracker.cpp
##########
@@ -233,41 +260,71 @@ int64_t MemTracker::GetPoolMemReserved() {
                 // Make sure we don't overflow if the query limits are set to ridiculous values.
                 mem_reserved += std::min(child_limit, MemInfo::physical_mem());
             } else {
-                DCHECK(child_limit == -1)
-                        << child->LogUsage(UNLIMITED_DEPTH);
+                DCHECK(child_limit == -1) << child->LogUsage(UNLIMITED_DEPTH);
                 mem_reserved += child->consumption();
             }
         }
     }
     return mem_reserved;
 }
 
-std::shared_ptr<MemTracker> PoolMemTrackerRegistry::GetRequestPoolMemTracker(
-        const string& pool_name, bool create_if_not_present) {
-    DCHECK(!pool_name.empty());
-    lock_guard<SpinLock> l(pool_to_mem_trackers_lock_);
-    PoolTrackersMap::iterator it = pool_to_mem_trackers_.find(pool_name);
-    if (it != pool_to_mem_trackers_.end()) {
-        MemTracker* tracker = it->second.get();
-        DCHECK(pool_name == tracker->pool_name_);
-        return it->second;
+std::shared_ptr<MemTracker> QueryMemTrackerRegistry::RegisterQueryMemTracker(
+        const std::string& query_id, int64_t mem_limit) {
+    DCHECK(!query_id.empty());
+    VLOG(2) << "Register query memory tracker, query id: " << query_id
+            << " limit: " << PrettyPrinter::print(mem_limit, TUnit::BYTES);
+
+    // First time this query_id registered, make a new object, otherwise do nothing.
+    // Combine CreateTracker and emplace into one operation to avoid the use of locks
+    _query_mem_trackers.try_emplace_l(
+            query_id, [](std::shared_ptr<MemTracker>) {},
+            MemTracker::CreateTracker(mem_limit,
+                                      strings::Substitute(QUERY_MEM_TRACKER_LABEL_FORMAT, query_id),
+                                      ExecEnv::GetInstance()->all_query_mem_tracker(), false, false,

Review comment:
       Thats great




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r767212879



##########
File path: be/src/common/config.h
##########
@@ -601,6 +604,12 @@ CONF_mInt32(remote_storage_read_buffer_mb, "16");
 // the level equal or lower than mem_tracker_level will show in web page
 CONF_Int16(mem_tracker_level, "0");
 
+// The minimum length when TCMalloc Hook consumes/releases MemTracker, consume size
+// smaller than this value will continue to accumulate. specified as number of bytes.
+// Decreasing this value will increase the frequency of consume/release.
+// Increasing this value will cause MemTracker statistics to be inaccurate.
+CONF_mInt32(mem_tracker_consume_min_size_mbytes, "1048576");

Review comment:
       Thats right




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Refactor MemTracker and new memory statistics framework based on TCMalloc Hook

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r794177942



##########
File path: be/src/common/config.h
##########
@@ -594,12 +594,28 @@ CONF_Int32(aws_log_level, "3");
 // the buffer size when read data from remote storage like s3
 CONF_mInt32(remote_storage_read_buffer_mb, "16");
 
+// Whether to initialize TCmalloc new/delete Hook, MemTracker is currently counted in Hook.
+CONF_mBool(use_tc_hook, "true");

Review comment:
       done, thats sounds good




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755730056



##########
File path: be/src/runtime/thread_status.h
##########
@@ -0,0 +1,78 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+
+namespace doris {
+
+class ThreadStatus {

Review comment:
       Thats good




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r755732306



##########
File path: be/src/runtime/exec_env.h
##########
@@ -174,7 +176,8 @@ class ExecEnv {
     ClientCache<TPaloBrokerServiceClient>* _broker_client_cache = nullptr;
     ClientCache<TExtDataSourceServiceClient>* _extdatasource_client_cache = nullptr;
     std::shared_ptr<MemTracker> _mem_tracker;

Review comment:
       OK, After the code is modified




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Refactor MemTracker and new memory statistics framework based on TCMalloc Hook

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r781732804



##########
File path: be/src/common/config.h
##########
@@ -594,12 +594,28 @@ CONF_Int32(aws_log_level, "3");
 // the buffer size when read data from remote storage like s3
 CONF_mInt32(remote_storage_read_buffer_mb, "16");
 
+// Whether to initialize TCmalloc new/delete Hook, MemTracker is currently counted in Hook.
+CONF_mBool(use_tc_hook, "true");

Review comment:
       track_new_delete maybe better, is this can be mutable?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762793396



##########
File path: be/src/runtime/exec_env.h
##########
@@ -173,16 +175,20 @@ class ExecEnv {
     ClientCache<FrontendServiceClient>* _frontend_client_cache = nullptr;
     ClientCache<TPaloBrokerServiceClient>* _broker_client_cache = nullptr;
     ClientCache<TExtDataSourceServiceClient>* _extdatasource_client_cache = nullptr;
-    std::shared_ptr<MemTracker> _mem_tracker;
-    PoolMemTrackerRegistry* _pool_mem_trackers = nullptr;
+    // The ancestor of all trackers in the process. It is the only child of the root tracker.
+    // All manually created trackers should specify the process tracker as the parent.
+    std::shared_ptr<MemTracker> _process_mem_tracker = nullptr;
+    // The ancestor for all querys tracker.
+    std::shared_ptr<MemTracker> _all_query_mem_tracker = nullptr;
+    QueryMemTrackerRegistry* _query_mem_tracker_registry = nullptr;

Review comment:
       Thats great

##########
File path: gensrc/proto/internal_service.proto
##########
@@ -131,6 +131,7 @@ enum PPlanFragmentCancelReason {
     USER_CANCEL = 2;
     INTERNAL_ERROR = 3;
     TIMEOUT = 4;
+    MEMORY_EXCEED_LIMIT = 5;

Review comment:
       Thats great




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762830604



##########
File path: be/src/runtime/thread_context.h
##########
@@ -0,0 +1,187 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "gen_cpp/Types_types.h"
+#include "runtime/exec_env.h"
+#include "runtime/fragment_mgr.h"
+#include "runtime/mem_tracker.h"
+#include "service/backend_options.h"
+
+namespace doris {
+
+class TheadContext {
+public:
+    TheadContext()
+            : _thread_id(std::this_thread::get_id()),
+              _global_hook_tracker(MemTracker::GetGlobalHookTracker()) {}
+    ~TheadContext() { update_query_mem_tracker(); }
+
+    void attach_query(const TUniqueId& query_id,
+                      const TUniqueId& fragment_instance_id = TUniqueId()) {
+        _query_id = query_id;
+        _fragment_instance_id = fragment_instance_id;
+#ifdef BE_TEST
+        if (ExecEnv::GetInstance()->query_mem_tracker_registry() == nullptr) {
+            return;
+        }
+#endif
+        update_query_mem_tracker(
+                ExecEnv::GetInstance()->query_mem_tracker_registry()->GetQueryMemTracker(
+                        print_id(query_id)));
+    }
+
+    void unattach_query() {
+        _query_id = TUniqueId();
+        _fragment_instance_id = TUniqueId();
+        update_query_mem_tracker();
+    }
+
+    void update_query_mem_tracker(
+            std::weak_ptr<MemTracker> mem_tracker = std::weak_ptr<MemTracker>()) {
+        if (_untracked_mem != 0) {
+            consume();
+            _untracked_mem = 0;
+        }
+        _query_mem_tracker = mem_tracker;
+    }
+
+    void query_mem_limit_exceeded(int64_t mem_usage) {
+        if (_query_id != TUniqueId() && _fragment_instance_id != TUniqueId() &&
+            ExecEnv::GetInstance()->is_init() &&
+            ExecEnv::GetInstance()->fragment_mgr()->is_canceling(_fragment_instance_id).ok()) {
+            std::string detail = "Query Memory exceed limit in TCMalloc Hook New.";
+            auto st = _query_mem_tracker.lock()->MemLimitExceeded(nullptr, detail, mem_usage);

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] xinyiZzz commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762831347



##########
File path: be/src/runtime/exec_env.h
##########
@@ -93,6 +93,7 @@ class ExecEnv {
     // declarations for classes in scoped_ptrs.
     ~ExecEnv();
 
+    const bool is_init() { return _is_init; }

Review comment:
       Thats great




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #7198: [Memory] Use TCMalloc Hook to count the real Process and Query MemTracker

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #7198:
URL: https://github.com/apache/incubator-doris/pull/7198#discussion_r762686880



##########
File path: be/src/runtime/mem_tracker.h
##########
@@ -573,33 +611,33 @@ class MemTracker : public std::enable_shared_from_this<MemTracker> {
     IntGauge* limit_metric_;
 };
 
-/// Global registry for query and pool MemTrackers. Owned by ExecEnv.
-class PoolMemTrackerRegistry {
+// Global registry for query MemTrackers. Owned by ExecEnv.
+class QueryMemTrackerRegistry {
 public:
-    /// Returns a MemTracker object for request pool 'pool_name'. Calling this with the same
-    /// 'pool_name' will return the same MemTracker object. This is used to track the local
-    /// memory usage of all requests executing in this pool. If 'create_if_not_present' is
-    /// true, the first time this is called for a pool, a new MemTracker object is created
-    /// with the process tracker as its parent. There is no explicit per-pool byte_limit
-    /// set at any particular impalad, so newly created trackers will always have a limit
-    /// of -1.
-    /// TODO(cmy): this function is not used for now. the memtracker returned from here is
-    ///            got from a shared_ptr in `pool_to_mem_trackers_`.
-    ///            This funtion is from
-    ///            https://github.com/cloudera/Impala/blob/495397101e5807c701df71ea288f4815d69c2c8a/be/src/runtime/mem-tracker.h#L497
-    ///            And in impala this function will return a raw pointer.
-    std::shared_ptr<MemTracker> GetRequestPoolMemTracker(const std::string& pool_name,
-                                                         bool create_if_not_present);
+    // Construct a MemTracker object for 'query_id' with 'mem_limit' as the memory limit.
+    // The MemTracker is a child of the process MemTracker, Calling this with the same
+    // 'query_id' will return the same MemTracker object. This is used to track the local
+    // memory usage of all querys executing. The first time this is called for a query,
+    // a new MemTracker object is created with the process tracker as its parent.
+    // Newly created trackers will always have a limit of -1.
+    std::shared_ptr<MemTracker> RegisterQueryMemTracker(const std::string& query_id,

Review comment:
       Make it underscores instead of camel case




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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