You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/01/07 16:12:07 UTC

[doris] 02/03: [Enhancement](jemalloc):support heap dump by http request at runtime (#15429)

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

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

commit bd59ec2b8a930d222b1260357b9ea16fecf67a60
Author: spaces-x <we...@gmail.com>
AuthorDate: Wed Dec 28 20:10:50 2022 +0800

    [Enhancement](jemalloc):support heap dump by http request at runtime (#15429)
---
 be/src/common/config.h                             |  2 +
 be/src/http/CMakeLists.txt                         |  1 +
 be/src/http/action/jeprofile_actions.cpp           | 81 ++++++++++++++++++++++
 be/src/http/action/jeprofile_actions.h             | 31 +++++++++
 be/src/service/http_service.cpp                    |  4 ++
 docs/en/community/developer-guide/debug-tool.md    | 53 ++++++++++++++
 docs/zh-CN/community/developer-guide/debug-tool.md | 53 ++++++++++++++
 7 files changed, 225 insertions(+)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index 3aa219f522..a94fbe6c92 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -456,6 +456,8 @@ CONF_Bool(enable_quadratic_probing, "false");
 
 // for pprof
 CONF_String(pprof_profile_dir, "${DORIS_HOME}/log");
+// for jeprofile in jemalloc
+CONF_mString(jeprofile_dir, "${DORIS_HOME}/log");
 
 // to forward compatibility, will be removed later
 CONF_mBool(enable_token_check, "true");
diff --git a/be/src/http/CMakeLists.txt b/be/src/http/CMakeLists.txt
index e38ff4d1ee..a2e1c3eb46 100644
--- a/be/src/http/CMakeLists.txt
+++ b/be/src/http/CMakeLists.txt
@@ -55,4 +55,5 @@ add_library(Webserver STATIC
   action/reset_rpc_channel_action.cpp
   action/check_tablet_segment_action.cpp
   action/version_action.cpp
+  action/jeprofile_actions.cpp
 )
diff --git a/be/src/http/action/jeprofile_actions.cpp b/be/src/http/action/jeprofile_actions.cpp
new file mode 100644
index 0000000000..27c3d07962
--- /dev/null
+++ b/be/src/http/action/jeprofile_actions.cpp
@@ -0,0 +1,81 @@
+// 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 "http/action/jeprofile_actions.h"
+
+#include <jemalloc/jemalloc.h>
+
+#include <ctime>
+#include <fstream>
+#include <mutex>
+#include <sstream>
+
+#include "common/config.h"
+#include "common/object_pool.h"
+#include "http/ev_http_server.h"
+#include "http/http_channel.h"
+#include "http/http_handler.h"
+#include "http/http_headers.h"
+#include "http/http_request.h"
+#include "util/file_utils.h"
+
+namespace doris {
+
+static std::mutex kJeprofileActionMutex;
+class JeHeapAction : public HttpHandler {
+public:
+    JeHeapAction() = default;
+    virtual ~JeHeapAction() = default;
+
+    virtual void handle(HttpRequest* req) override;
+};
+
+void JeHeapAction::handle(HttpRequest* req) {
+    std::lock_guard<std::mutex> lock(kJeprofileActionMutex);
+#ifndef USE_JEMALLOC
+    std::string str = "jemalloc heap dump is not available without setting USE_JEMALLOC";
+    HttpChannel::send_reply(req, str);
+#else
+    std::stringstream tmp_jeprof_file_name;
+    std::time_t now = std::time(nullptr);
+    // Build a temporary file name that is hopefully unique.
+    tmp_jeprof_file_name << config::jeprofile_dir << "/jeheap_dump." << now << "." << getpid()
+                         << "." << rand() << ".heap";
+    const std::string& tmp_file_name_str = tmp_jeprof_file_name.str();
+    const char* file_name_ptr = tmp_file_name_str.c_str();
+    int result = je_mallctl("prof.dump", nullptr, nullptr, &file_name_ptr, sizeof(const char*));
+    std::stringstream response;
+    if (result == 0) {
+        response << "Jemalloc heap dump success, dump file path: " << tmp_jeprof_file_name.str()
+                 << "\n";
+    } else {
+        response << "Jemalloc heap dump failed, je_mallctl return: " << result << "\n";
+    }
+    HttpChannel::send_reply(req, response.str());
+#endif
+}
+
+Status JeprofileActions::setup(doris::ExecEnv* exec_env, doris::EvHttpServer* http_server,
+                               doris::ObjectPool& pool) {
+    if (!config::jeprofile_dir.empty()) {
+        FileUtils::create_dir(config::jeprofile_dir);
+    }
+    http_server->register_handler(HttpMethod::GET, "/jeheap/dump", pool.add(new JeHeapAction()));
+    return Status::OK();
+}
+
+} // namespace doris
diff --git a/be/src/http/action/jeprofile_actions.h b/be/src/http/action/jeprofile_actions.h
new file mode 100644
index 0000000000..2ebeb3c9ff
--- /dev/null
+++ b/be/src/http/action/jeprofile_actions.h
@@ -0,0 +1,31 @@
+// 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.
+
+#ifndef DORIS_JEPROFILE_ACTIONS_H
+#define DORIS_JEPROFILE_ACTIONS_H
+#include "common/status.h"
+namespace doris {
+class EvHttpServer;
+class ExecEnv;
+class ObjectPool;
+class JeprofileActions {
+public:
+    static Status setup(ExecEnv* exec_env, EvHttpServer* http_server, ObjectPool& pool);
+};
+
+} // namespace doris
+#endif //DORIS_JEPROFILE_ACTIONS_H
diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp
index 06334bde29..b7acb1fa6a 100644
--- a/be/src/service/http_service.cpp
+++ b/be/src/service/http_service.cpp
@@ -24,6 +24,7 @@
 #include "http/action/config_action.h"
 #include "http/action/download_action.h"
 #include "http/action/health_action.h"
+#include "http/action/jeprofile_actions.h"
 #include "http/action/meta_action.h"
 #include "http/action/metrics_action.h"
 #include "http/action/pad_rowset_action.h"
@@ -118,6 +119,9 @@ Status HttpService::start() {
     // register pprof actions
     PprofActions::setup(_env, _ev_http_server.get(), _pool);
 
+    // register jeprof actions
+    JeprofileActions::setup(_env, _ev_http_server.get(), _pool);
+
     // register metrics
     {
         auto action = _pool.add(new MetricsAction(DorisMetrics::instance()->metric_registry()));
diff --git a/docs/en/community/developer-guide/debug-tool.md b/docs/en/community/developer-guide/debug-tool.md
index 4e7f50e9ba..96c705b454 100644
--- a/docs/en/community/developer-guide/debug-tool.md
+++ b/docs/en/community/developer-guide/debug-tool.md
@@ -235,6 +235,59 @@ From the above output, we can see that 1024 bytes have been leaked, and the stac
 
 **NOTE: if the LSAN switch is turned on, the TCMalloc will be automatically turned off**
 
+#### JEMALLOC HEAP PROFILE
+
+##### 1. runtime heap dump by http
+No need to restart BE, use jemalloc heap dump http interface, jemalloc generates heap dump file on the corresponding BE machine according to the current memory usage.
+
+The directory where the heap dump file is located can be configured through the ``jeprofile_dir`` variable in ``be.conf``, and the default is ``${DORIS_HOME}/log``
+
+```shell
+curl http://be_host:be_webport/jeheap/dump
+```
+
+##### 2. heap dump by JEMALLOC_CONF
+Perform heap dump by restarting BE after changing the `JEMALLOC_CONF` variable in `start_be.sh`
+
+1. Dump every 1MB:
+
+   Two new variable settings `prof:true,lg_prof_interval:20` have been added to the `JEMALLOC_CONF` variable, where `prof:true` is to enable profiling, and `lg_prof_interval:20` means that a dump is generated every 1MB (2^20)
+2. Dump each time a new high is reached:
+
+   Added two variable settings `prof:true,prof_gdump:true` in the `JEMALLOC_CONF` variable, where `prof:true` is to enable profiling, and `prof_gdump:true` means to generate a dump when the memory usage reaches a new high
+3. Memory leak dump when the program exits:
+
+   Added three new variable settings `prof_leak: true, lg_prof_sample: 0, prof_final: true` in the `JEMALLOC_CONF` variable
+
+
+#### 3. jemalloc heap dump profiling
+
+3.1 Generating plain text analysis results
+```shell
+jeprof lib/doris_be --base=heap_dump_file_1 heap_dump_file_2
+```
+
+3.2 Generate call relationship picture
+
+ Install dependencies required for plotting
+ ```shell
+ yum install ghostscript graphviz
+ ```
+ Multiple dump files can be generated by running the above command multiple times in a short period of time, and the first dump file can be selected as the baseline for diff comparison analysis
+
+ ```shell
+ jeprof --dot lib/doris_be --base=heap_dump_file_1 heap_dump_file_2
+ ```
+ After executing the above command, the terminal will output a diagram of dot syntax, and paste it to [online dot drawing website](http://www.webgraphviz.com/), generate a memory allocation diagram, and then analyze it. This method can Drawing directly through the terminal output results is more suitable for servers where file transfer is not very convenient.
+
+ You can also use the following command to directly generate the call relationship result.pdf file and transfer it to the local for viewing
+ ```shell
+ jeprof --pdf lib/doris_be --base=heap_dump_file_1 heap_dump_file_2 > result.pdf
+ ```
+
+In the above jeprof related commands, remove the `--base` option to analyze only a single heap dump file
+
+
 #### ASAN
 
 Except for the unreasonable use and leakage of memory. Sometimes there will be memory access illegal address and other errors. At this time, we can use [ASAN](https://github.com/google/sanitizers/wiki/addresssanitizer) to help us find the cause of the problem. Like LSAN, ASAN is integrated into GCC. Doris can open this function by compiling as follows
diff --git a/docs/zh-CN/community/developer-guide/debug-tool.md b/docs/zh-CN/community/developer-guide/debug-tool.md
index bf56879c28..b4952f9a13 100644
--- a/docs/zh-CN/community/developer-guide/debug-tool.md
+++ b/docs/zh-CN/community/developer-guide/debug-tool.md
@@ -199,6 +199,59 @@ Total: 1296.4 MB
 
 这个命令的输出与HEAP PROFILE的输出及查看方式一样,这里就不再详细说明。这个命令只有在执行的过程中才会开启统计,相比HEAP PROFILE对于进程性能的影响有限。
 
+#### JEMALLOC HEAP PROFILE
+
+##### 1. runtime heap dump by http 
+无需重启BE, 使用jemalloc heap dump http接口,jemalloc根据当前内存使用情况,在对应的BE机器上生成heap dump文件。
+
+heap dump文件所在目录可以在 ``be.conf`` 中通过``jeprofile_dir``变量进行配置,默认为``${DORIS_HOME}/log``
+
+```shell
+curl http://be_host:be_webport/jeheap/dump
+```
+
+##### 2. heap dump by JEMALLOC_CONF
+通过更改`start_be.sh` 中`JEMALLOC_CONF` 变量后重新启动BE 来进行heap dump
+
+1. 每1MB dump一次:
+
+   `JEMALLOC_CONF`变量中新增两个变量设置`prof:true,lg_prof_interval:20`  其中`prof:true`是打开profiling,`lg_prof_interval:20`中表示每1MB(2^20)生成一次dump 
+2. 每次达到新高时dump:
+   
+   `JEMALLOC_CONF`变量中新增两个变量设置`prof:true,prof_gdump:true` 其中`prof:true`是打开profiling,`prof_gdump:true` 代表内存使用达到新高时生成dump
+3. 程序退出时内存泄漏dump:
+   
+   `JEMALLOC_CONF`变量中新增三个变量设置`prof_leak:true,lg_prof_sample:0,prof_final:true`
+
+
+#### 3. jemalloc heap dump profiling
+
+3.1  生成纯文本分析结果
+   ```shell
+   jeprof lib/doris_be --base=heap_dump_file_1 heap_dump_file_2
+   ```
+   
+3.2 生成调用关系图片
+
+   安装绘图所需的依赖项
+   ```shell
+   yum install ghostscript graphviz
+   ```
+   通过在一短时间内多次运行上述命令可以生成多份dump 文件,可以选取第一份dump 文件作为baseline 进行diff对比分析
+   
+   ```shell
+   jeprof --dot lib/doris_be --base=heap_dump_file_1 heap_dump_file_2
+   ```
+   执行完上述命令,终端中会输出dot语法的图,将其贴到[在线dot绘图网站](http://www.webgraphviz.com/),生成内存分配图,然后进行分析,此种方式能够直接通过终端输出结果进行绘图,比较适用于传输文件不是很方便的服务器。
+   
+   也可以通过如下命令直接生成调用关系result.pdf文件传输到本地后进行查看
+   ```shell
+   jeprof --pdf lib/doris_be --base=heap_dump_file_1 heap_dump_file_2 > result.pdf
+   ```
+   
+上述jeprof相关命令中均去掉 `--base` 选项来只分析单个heap dump文件
+
+
 #### LSAN
 
 [LSAN](https://github.com/google/sanitizers/wiki/AddressSanitizerLeakSanitizer)是一个地址检查工具,GCC已经集成。在我们编译代码的时候开启相应的编译选项,就能够开启这个功能。当程序发生可以确定的内存泄露时,会将泄露堆栈打印。Doris BE已经集成了这个工具,只需要在编译的时候使用如下的命令进行编译就能够生成带有内存泄露检测版本的BE二进制


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