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/12/07 12:30:20 UTC

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

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