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 2020/08/13 15:41:54 UTC

[incubator-doris] branch master updated: [Compaction] Manually trigger compaction RESTapi interface (#4312)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d602886  [Compaction] Manually trigger compaction RESTapi interface (#4312)
d602886 is described below

commit d6028863f3e9d8f401f1dea34a119e48fd21c7fe
Author: ZhangYu0123 <67...@users.noreply.github.com>
AuthorDate: Thu Aug 13 23:41:46 2020 +0800

    [Compaction] Manually trigger compaction RESTapi interface (#4312)
    
    Add restapi to be which do compaction task by manual trigger. The detail design in #4311 .
---
 be/src/common/config.h                             |   3 +
 be/src/http/action/compaction_action.cpp           | 213 ++++++++++++++++++++-
 be/src/http/action/compaction_action.h             |  39 +++-
 be/src/olap/olap_server.cpp                        |  42 ++--
 be/src/service/http_service.cpp                    |   2 +
 docs/en/administrator-guide/config/be_config.md    |   8 +
 .../http-actions/compaction-action.md              |  94 ++++++++-
 docs/zh-CN/administrator-guide/config/be_config.md |   8 +
 .../http-actions/compaction-action.md              |  92 ++++++++-
 9 files changed, 468 insertions(+), 33 deletions(-)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index 1538638..beaae0f 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -256,6 +256,9 @@ namespace config {
     CONF_Bool(disable_storage_page_cache, "false");
 
     // be policy
+    // whether disable automatic compaction task
+    CONF_mBool(disable_auto_compaction, "false");
+
     // CONF_Int64(base_compaction_start_hour, "20");
     // CONF_Int64(base_compaction_end_hour, "7");
     CONF_mInt32(base_compaction_check_interval_seconds, "60");
diff --git a/be/src/http/action/compaction_action.cpp b/be/src/http/action/compaction_action.cpp
index efec3da..505db91 100644
--- a/be/src/http/action/compaction_action.cpp
+++ b/be/src/http/action/compaction_action.cpp
@@ -19,6 +19,7 @@
 
 #include <sstream>
 #include <string>
+#include <sys/syscall.h>
 
 #include "http/http_channel.h"
 #include "http/http_headers.h"
@@ -30,15 +31,19 @@
 #include "gutil/strings/substitute.h"
 #include "olap/olap_define.h"
 #include "olap/storage_engine.h"
-#include "olap/tablet.h"
+#include "olap/base_compaction.h"
+#include "olap/cumulative_compaction.h"
 #include "util/json_util.h"
 
 namespace doris {
 
 const static std::string HEADER_JSON = "application/json";
 
-// for viewing the compaction status
-Status CompactionAction::_handle_show_compaction(HttpRequest* req, std::string* json_result) {
+bool CompactionAction::_is_compaction_running = false;
+std::mutex CompactionAction::_compaction_running_mutex;
+
+Status CompactionAction::_check_param(HttpRequest* req, uint64_t* tablet_id, uint32_t* schema_hash) {
+    
     std::string req_tablet_id = req->param(TABLET_ID_KEY);
     std::string req_schema_hash = req->param(TABLET_SCHEMA_HASH_KEY);
     if (req_tablet_id == "" && req_schema_hash == "") {
@@ -46,17 +51,27 @@ Status CompactionAction::_handle_show_compaction(HttpRequest* req, std::string*
         return Status::NotSupported("The overall compaction status is not supported yet");
     }
 
-    uint64_t tablet_id = 0;
-    uint32_t schema_hash = 0;
     try {
-        tablet_id = std::stoull(req_tablet_id);
-        schema_hash = std::stoul(req_schema_hash);
+        *tablet_id = std::stoull(req_tablet_id);
+        *schema_hash = std::stoul(req_schema_hash);
     } catch (const std::exception& e) {
         LOG(WARNING) << "invalid argument.tablet_id:" << req_tablet_id
                      << ", schema_hash:" << req_schema_hash;
         return Status::InternalError(strings::Substitute("convert failed, $0", e.what()));
     }
 
+    return Status::OK();
+}
+
+// for viewing the compaction status
+Status CompactionAction::_handle_show_compaction(HttpRequest* req, std::string* json_result) {
+    
+    uint64_t tablet_id = 0;
+    uint32_t schema_hash = 0;
+    
+    Status status = _check_param(req, &tablet_id, &schema_hash);
+    RETURN_IF_ERROR(status);
+
     TabletSharedPtr tablet =
             StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id, schema_hash);
     if (tablet == nullptr) {
@@ -67,6 +82,171 @@ Status CompactionAction::_handle_show_compaction(HttpRequest* req, std::string*
     return Status::OK();
 }
 
+Status CompactionAction::_handle_run_compaction(HttpRequest *req, std::string* json_result) {
+
+    // 1. param check
+    uint64_t tablet_id = 0;
+    uint32_t schema_hash = 0;
+    
+    // check req_tablet_id and req_schema_hash is not empty
+    Status check_status = _check_param(req, &tablet_id, &schema_hash);
+    RETURN_IF_ERROR(check_status);
+
+    std::string compaction_type = req->param(PARAM_COMPACTION_TYPE);
+    // check compaction_type is not empty and equals base or cumulative
+    if (compaction_type == "" && !(compaction_type == PARAM_COMPACTION_BASE || compaction_type == PARAM_COMPACTION_CUMULATIVE)) {
+        return Status::NotSupported("The compaction type is not supported");
+    }
+
+    // 2. fetch the tablet by tablet_id and schema_hash
+    TabletSharedPtr tablet =
+            StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id, schema_hash);
+
+    if (tablet == nullptr) {
+        LOG(WARNING) << "invalid argument.tablet_id:" << tablet_id
+                     << ", schema_hash:" << schema_hash;
+        return Status::InternalError(
+                strings::Substitute("fail to get $0, $1", tablet_id, schema_hash));
+    }
+
+    // 3. execute compaction task
+    std::packaged_task<OLAPStatus()> task([this, tablet, compaction_type]() { 
+            return _execute_compaction_callback(tablet, compaction_type);
+    });
+    std::future<OLAPStatus> future_obj = task.get_future();
+
+    {
+        // 3.1 check is there compaction running
+        std::lock_guard<std::mutex> lock(_compaction_running_mutex);
+        if (_is_compaction_running) {
+            return Status::TooManyTasks("Manual compaction task is running");
+        } else {
+            // 3.2 execute the compaction task and set compaction task running 
+            _is_compaction_running = true;
+            std::thread(std::move(task)).detach();
+        }
+    }
+
+    // 4. wait for result for 2 seconds by async
+    std::future_status status = future_obj.wait_for(std::chrono::seconds(2));
+    if (status == std::future_status::ready) {
+        // fetch execute result
+        OLAPStatus olap_status = future_obj.get();
+        if (olap_status != OLAP_SUCCESS) {
+            return Status::InternalError(
+                    strings::Substitute("fail to execute compaction, error = $0", olap_status));
+        }
+    } else {
+        LOG(INFO) << "Manual compaction task is timeout for waiting " << (status == std::future_status::timeout);
+    }
+   
+    LOG(INFO) << "Manual compaction task is successfully triggered";
+    *json_result = "{\"status\": \"Success\", \"msg\": \"compaction task is successfully triggered.\"}";
+
+    return Status::OK();
+}
+
+Status CompactionAction::_handle_run_status_compaction(HttpRequest *req, std::string* json_result) {
+
+    uint64_t tablet_id = 0;
+    uint32_t schema_hash = 0;
+    
+    // check req_tablet_id and req_schema_hash is not empty
+    Status check_status = _check_param(req, &tablet_id, &schema_hash);
+    RETURN_IF_ERROR(check_status);
+
+    // fetch the tablet by tablet_id and schema_hash
+    TabletSharedPtr tablet =
+            StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id, schema_hash);
+
+    if (tablet == nullptr) {
+        LOG(WARNING) << "invalid argument.tablet_id:" << tablet_id
+                     << ", schema_hash:" << schema_hash;
+        return Status::InternalError(
+                strings::Substitute("fail to get $0, $1", tablet_id, schema_hash));
+    }
+
+    std::string json_template = R"({
+        "status" : "Success",
+        "run_status" : $0,
+        "msg" : "$1",
+        "tablet_id" : $2,
+        "schema_hash" : $3,
+        "compact_type" : "$4"
+})";
+
+    std::string msg = "this tablet_id is not running";
+    std::string compaction_type = "";
+    bool run_status = 0;
+
+    {
+        // use try lock to check this tablet is running cumulative compaction
+        MutexLock lock_cumulativie(tablet->get_cumulative_lock(), TRY_LOCK);
+        if (!lock_cumulativie.own_lock()) {
+            msg = "this tablet_id is running";
+            compaction_type = "cumulative";
+            run_status = 1;
+            *json_result = strings::Substitute(json_template, run_status, msg, tablet_id, schema_hash,
+                                            compaction_type);
+            return Status::OK();
+        }
+    }
+
+    {
+        // use try lock to check this tablet is running base compaction
+        MutexLock lock_base(tablet->get_base_lock(), TRY_LOCK);
+        if (!lock_base.own_lock()) {
+            msg = "this tablet_id is running";
+            compaction_type = "base";
+            run_status = 1;
+            *json_result = strings::Substitute(json_template, run_status, msg, tablet_id, schema_hash,
+                                            compaction_type);
+            return Status::OK();
+        }
+    }
+    // not running any compaction
+    *json_result = strings::Substitute(json_template, run_status, msg, tablet_id, schema_hash,
+                                           compaction_type);
+    return Status::OK();
+}
+
+OLAPStatus CompactionAction::_execute_compaction_callback(TabletSharedPtr tablet,
+                                                    const std::string& compaction_type) {
+    OLAPStatus status = OLAP_SUCCESS;
+    if (compaction_type == PARAM_COMPACTION_BASE) {
+        std::string tracker_label = "base compaction " + std::to_string(syscall(__NR_gettid));
+        BaseCompaction base_compaction(tablet, tracker_label, _compaction_mem_tracker);
+        OLAPStatus res = base_compaction.compact();
+        if (res != OLAP_SUCCESS) {
+            if (res != OLAP_ERR_BE_NO_SUITABLE_VERSION) {
+                DorisMetrics::instance()->base_compaction_request_failed.increment(1);
+                LOG(WARNING) << "failed to init base compaction. res=" << res
+                            << ", table=" << tablet->full_name();
+            }
+        }
+        status = res;
+    } else if (compaction_type == PARAM_COMPACTION_CUMULATIVE) {
+        std::string tracker_label = "cumulative compaction " + std::to_string(syscall(__NR_gettid));
+        CumulativeCompaction cumulative_compaction(tablet, tracker_label, _compaction_mem_tracker);
+
+        OLAPStatus res = cumulative_compaction.compact();
+        if (res != OLAP_SUCCESS) {
+            if (res != OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSIONS) {
+                DorisMetrics::instance()->cumulative_compaction_request_failed.increment(1);
+                LOG(WARNING) << "failed to do cumulative compaction. res=" << res
+                            << ", table=" << tablet->full_name();
+            }
+        }
+        status = res;
+    }
+    
+    LOG(INFO) << "Manual compaction task finish, status = " << status;
+    std::lock_guard<std::mutex> lock(_compaction_running_mutex);
+    _is_compaction_running = false;
+
+    return status;
+}
+
 void CompactionAction::handle(HttpRequest* req) {
     req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.c_str());
 
@@ -78,10 +258,23 @@ void CompactionAction::handle(HttpRequest* req) {
         } else {
             HttpChannel::send_reply(req, HttpStatus::OK, json_result);
         }
+    } else if (_type == CompactionActionType::RUN_COMPACTION) {
+        std::string json_result;
+        Status st = _handle_run_compaction(req, &json_result);
+        if (!st.ok()) {
+            HttpChannel::send_reply(req, HttpStatus::OK, to_json(st));
+        } else {
+            HttpChannel::send_reply(req, HttpStatus::OK, json_result);
+        }
     } else {
-        HttpChannel::send_reply(req, HttpStatus::OK,
-                                to_json(Status::NotSupported("Action not supported")));
+        std::string json_result;
+        Status st = _handle_run_status_compaction(req, &json_result);
+        if (!st.ok()) {
+            HttpChannel::send_reply(req, HttpStatus::OK, to_json(st));
+        } else {
+            HttpChannel::send_reply(req, HttpStatus::OK, json_result);
+        }
     }
-}
 
+}
 } // end namespace doris
diff --git a/be/src/http/action/compaction_action.h b/be/src/http/action/compaction_action.h
index 7458c5d..1a287bd 100644
--- a/be/src/http/action/compaction_action.h
+++ b/be/src/http/action/compaction_action.h
@@ -19,20 +19,31 @@
 
 #include "http/http_handler.h"
 #include "common/status.h"
+#include "olap/storage_engine.h"
+#include "olap/base_compaction.h"
+#include "olap/tablet.h"
 
 namespace doris {
 
 
 enum CompactionActionType {
     SHOW_INFO = 1,
-    RUN_COMPACTION = 2
+    RUN_COMPACTION = 2,
+    RUN_COMPACTION_STATUS = 3,
 };
 
-// This action is used for viewing the compaction status.
-// See compaction-action.md for details.
+const std::string PARAM_COMPACTION_TYPE = "compact_type";
+const std::string PARAM_COMPACTION_BASE = "base";
+const std::string PARAM_COMPACTION_CUMULATIVE = "cumulative";
+
+/// This action is used for viewing the compaction status.
+/// See compaction-action.md for details.
 class CompactionAction : public HttpHandler {
 public:
-    CompactionAction(CompactionActionType type) : _type(type) {}
+    CompactionAction(CompactionActionType type)
+            : _type(type),
+              _compaction_mem_tracker(
+                      MemTracker::CreateTracker(-1, "manual compaction mem tracker(unlimited)")) {}
 
     virtual ~CompactionAction() {}
 
@@ -41,8 +52,28 @@ public:
 private:
     Status _handle_show_compaction(HttpRequest *req, std::string* json_result);
 
+    /// execute compaction request to run compaction task
+    /// param compact_type in req to distinguish the task type, base or cumulative
+    Status _handle_run_compaction(HttpRequest *req, std::string* json_result);
+
+    /// thread callback function for the tablet to do compaction
+    OLAPStatus _execute_compaction_callback(TabletSharedPtr tablet, const std::string& compaction_type);
+
+    /// fetch compaction running status
+    Status _handle_run_status_compaction(HttpRequest* req, std::string* json_result);
+
+    /// check param and fetch tablet_id and schema_hash from req
+    Status _check_param(HttpRequest* req, uint64_t* tablet_id, uint32_t* schema_hash);
+
 private:
     CompactionActionType _type;
+
+    /// running check mutex
+    static std::mutex _compaction_running_mutex;
+    /// whether there is manual compaction running
+    static bool _is_compaction_running;
+    /// memory tracker
+    std::shared_ptr<MemTracker> _compaction_mem_tracker;
 };
 
 } // end namespace doris
diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp
index e5b5056..760a751 100644
--- a/be/src/olap/olap_server.cpp
+++ b/be/src/olap/olap_server.cpp
@@ -83,6 +83,7 @@ Status StorageEngine::start_bg_threads() {
     }
     int32_t data_dir_num = data_dirs.size();
 
+    
     // base and cumulative compaction threads
     int32_t base_compaction_num_threads_per_disk = std::max<int32_t>(1, config::base_compaction_num_threads_per_disk);
     int32_t cumulative_compaction_num_threads_per_disk = std::max<int32_t>(1, config::cumulative_compaction_num_threads_per_disk);
@@ -91,8 +92,9 @@ Status StorageEngine::start_bg_threads() {
     // calc the max concurrency of compaction tasks
     int32_t max_compaction_concurrency = config::max_compaction_concurrency;
     if (max_compaction_concurrency < 0
-        || max_compaction_concurrency > base_compaction_num_threads + cumulative_compaction_num_threads) {
-        max_compaction_concurrency = base_compaction_num_threads + cumulative_compaction_num_threads;
+        || max_compaction_concurrency > base_compaction_num_threads + cumulative_compaction_num_threads + 1) {
+        // reserve 1 thread for manual execution
+        max_compaction_concurrency = base_compaction_num_threads + cumulative_compaction_num_threads + 1;
     }
     Compaction::init(max_compaction_concurrency);
 
@@ -192,20 +194,24 @@ void* StorageEngine::_base_compaction_thread_callback(void* arg, DataDir* data_d
     //string last_base_compaction_fs;
     //TTabletId last_base_compaction_tablet_id = -1;
     while (!_stop_bg_worker) {
-        // must be here, because this thread is start on start and
-        // cgroup is not initialized at this time
-        // add tid to cgroup
-        CgroupsMgr::apply_system_cgroup();
-        if (!data_dir->reach_capacity_limit(0)) {
-            _perform_base_compaction(data_dir);
+
+        if (!config::disable_auto_compaction) {
+            // must be here, because this thread is start on start and
+            // cgroup is not initialized at this time
+            // add tid to cgroup
+            CgroupsMgr::apply_system_cgroup();
+            if (!data_dir->reach_capacity_limit(0)) {
+                _perform_base_compaction(data_dir);
+            }
         }
 
         int32_t interval = config::base_compaction_check_interval_seconds;
         if (interval <= 0) {
             OLAP_LOG_WARNING("base compaction check interval config is illegal: [%d], "
-                             "force set to 1", interval);
+                            "force set to 1", interval);
             interval = 1;
         }
+        
         SLEEP_IN_BG_WORKER(interval);
     }
 
@@ -284,18 +290,20 @@ void* StorageEngine::_cumulative_compaction_thread_callback(void* arg, DataDir*
     LOG(INFO) << "try to start cumulative compaction process!";
 
     while (!_stop_bg_worker) {
-        // must be here, because this thread is start on start and
-        // cgroup is not initialized at this time
-        // add tid to cgroup
-        CgroupsMgr::apply_system_cgroup();
-        if (!data_dir->reach_capacity_limit(0)) {
-            _perform_cumulative_compaction(data_dir);
-        }
 
+        if (!config::disable_auto_compaction) {
+            // must be here, because this thread is start on start and
+            // cgroup is not initialized at this time
+            // add tid to cgroup
+            CgroupsMgr::apply_system_cgroup();
+            if (!data_dir->reach_capacity_limit(0)) {
+                _perform_cumulative_compaction(data_dir);
+            }
+        }
         int32_t interval = config::cumulative_compaction_check_interval_seconds;
         if (interval <= 0) {
             LOG(WARNING) << "cumulative compaction check interval config is illegal:" << interval
-                         << "will be forced set to one";
+                        << "will be forced set to one";
             interval = 1;
         }
         SLEEP_IN_BG_WORKER(interval);
diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp
index 0b236c1..191d40a 100644
--- a/be/src/service/http_service.cpp
+++ b/be/src/service/http_service.cpp
@@ -124,6 +124,8 @@ Status HttpService::start() {
     _ev_http_server->register_handler(HttpMethod::GET, "/api/compaction/show", show_compaction_action);
     CompactionAction* run_compaction_action = new CompactionAction(CompactionActionType::RUN_COMPACTION);
     _ev_http_server->register_handler(HttpMethod::POST, "/api/compaction/run", run_compaction_action);
+    CompactionAction* run_status_compaction_action = new CompactionAction(CompactionActionType::RUN_COMPACTION_STATUS);
+    _ev_http_server->register_handler(HttpMethod::GET, "/api/compaction/run_status", run_status_compaction_action);
 
     UpdateConfigAction* update_config_action = new UpdateConfigAction();
     _ev_http_server->register_handler(HttpMethod::POST, "/api/update_config", update_config_action);
diff --git a/docs/en/administrator-guide/config/be_config.md b/docs/en/administrator-guide/config/be_config.md
index bf6c0bf..f9f30e3 100644
--- a/docs/en/administrator-guide/config/be_config.md
+++ b/docs/en/administrator-guide/config/be_config.md
@@ -132,6 +132,14 @@ User can set this configuration to a larger value to get better QPS performance.
 
 ### `create_tablet_worker_count`
 
+### `disable_auto_compaction`
+
+* Type: bool
+* Description: Whether disable automatic compaction task
+* Default value: false
+
+Generally it needs to be turned off. When you want to manually operate the compaction task in the debugging or test environment, you can turn on the configuration.
+
 ### `cumulative_compaction_budgeted_bytes`
 
 ### `cumulative_compaction_check_interval_seconds`
diff --git a/docs/en/administrator-guide/http-actions/compaction-action.md b/docs/en/administrator-guide/http-actions/compaction-action.md
index 9627f8f..a524ca2 100644
--- a/docs/en/administrator-guide/http-actions/compaction-action.md
+++ b/docs/en/administrator-guide/http-actions/compaction-action.md
@@ -95,4 +95,96 @@ curl -X GET http://192.168.10.24:8040/api/compaction/show?tablet_id=10015\&schem
 
 ## Manually trigger Compaction
 
-(TODO)
+```
+curl -X POST http://be_host:webserver_port/api/compaction/run?tablet_id=xxxx\&schema_hash=yyyy\&compact_type=cumulative
+```
+
+The only one manual compaction task that can be performed at a moment, and the value range of compact_type is base or cumulative
+
+If the tablet does not exist, an error in JSON format is returned:
+
+```
+{
+    "status": "Fail",
+    "msg": "Tablet not found"
+}
+```
+
+If the compaction execution task fails to be triggered, an error in JSON format is returned:
+
+```
+{
+    "status": "Fail",
+    "msg": "fail to execute compaction, error = -2000"
+}
+```
+
+If the compaction execution task successes to be triggered, an error in JSON format is returned:
+
+```
+{
+    "status": "Success",
+    "msg": "compaction task is successfully triggered."
+}
+```
+
+Explanation of results:
+
+* status: Trigger task status, when it is successfully triggered, it is Success; when for some reason (for example, the appropriate version is not obtained), it returns Fail.
+* msg: Give specific success or failure information.
+
+### Examples
+
+```
+curl -X POST http://192.168.10.24:8040/api/compaction/run?tablet_id=10015\&schema_hash=1294206575\&compact_type=cumulative
+```
+
+## Manual Compaction execution status
+
+```
+curl -X GET http://be_host:webserver_port/api/compaction/run_status?tablet_id=xxxx\&schema_hash=yyyy
+```
+If the tablet does not exist, an error in JSON format is returned:
+
+```
+{
+    "status": "Fail",
+    "msg": "Tablet not found"
+}
+```
+
+If the tablet exists and the tablet is not running, JSON format is returned:
+
+```
+{
+    "status" : "Success",
+    "run_status" : false,
+    "msg" : "this tablet_id is not running",
+    "tablet_id" : 11308,
+    "schema_hash" : 700967178,
+    "compact_type" : ""
+}
+```
+
+If the tablet exists and the tablet is running, JSON format is returned:
+
+```
+{
+    "status" : "Success",
+    "run_status" : true,
+    "msg" : "this tablet_id is running",
+    "tablet_id" : 11308,
+    "schema_hash" : 700967178,
+    "compact_type" : "cumulative"
+}
+```
+
+Explanation of results:
+
+* run_status: Get the current manual compaction task execution status.
+
+### Examples
+
+```
+curl -X GET http://192.168.10.24:8040/api/compaction/run_status?tablet_id=10015\&schema_hash=1294206575
+
diff --git a/docs/zh-CN/administrator-guide/config/be_config.md b/docs/zh-CN/administrator-guide/config/be_config.md
index 79712ed..98269cf 100644
--- a/docs/zh-CN/administrator-guide/config/be_config.md
+++ b/docs/zh-CN/administrator-guide/config/be_config.md
@@ -130,6 +130,14 @@ Metrics: {"filtered_rows":0,"input_row_num":3346807,"input_rowsets_count":42,"in
 
 ### `create_tablet_worker_count`
 
+### `disable_auto_compaction`
+
+* 类型:bool
+* 描述:关闭自动执行compaction任务
+* 默认值:false
+
+一般需要为关闭状态,当调试或测试环境中想要手动操作compaction任务时,可以对该配置进行开启
+
 ### `cumulative_compaction_budgeted_bytes`
 
 ### `cumulative_compaction_check_interval_seconds`
diff --git a/docs/zh-CN/administrator-guide/http-actions/compaction-action.md b/docs/zh-CN/administrator-guide/http-actions/compaction-action.md
index c4a04e7..5386ca2 100644
--- a/docs/zh-CN/administrator-guide/http-actions/compaction-action.md
+++ b/docs/zh-CN/administrator-guide/http-actions/compaction-action.md
@@ -95,5 +95,95 @@ curl -X GET http://192.168.10.24:8040/api/compaction/show?tablet_id=10015\&schem
 
 ## 手动触发 Compaction
 
-(TODO)
+```
+curl -X POST http://be_host:webserver_port/api/compaction/run?tablet_id=xxxx\&schema_hash=yyyy\&compact_type=cumulative
+```
+
+当前仅能执行一个手动compaction任务,其中compact_type取值为base或cumulative
+
+若 tablet 不存在,返回 JSON 格式的错误:
+
+```
+{
+    "status": "Fail",
+    "msg": "Tablet not found"
+}
+```
+
+若 compaction 执行任务触发失败时,返回 JSON 格式的错误:
+
+```
+{
+    "status": "Fail",
+    "msg": "fail to execute compaction, error = -2000"
+}
+```
+
+若 compaction 执行触发成功时,则返回 JSON 格式的结果:
+
+```
+{
+    "status": "Success",
+    "msg": "compaction task is successfully triggered."
+}
+```
+
+结果说明:
 
+* status:触发任务状态,当成功触发时为Success;当因某些原因(比如,没有获取到合适的版本)时,返回Fail。
+* msg:给出具体的成功或失败的信息。
+### 示例
+
+```
+curl -X POST http://192.168.10.24:8040/api/compaction/run?tablet_id=10015\&schema_hash=1294206575\&compact_type=cumulative
+```
+
+## 手动 Compaction 执行状态
+
+```
+curl -X GET http://be_host:webserver_port/api/compaction/run_status?tablet_id=xxxx\&schema_hash=yyyy
+```
+
+若 tablet 不存在,返回 JSON 格式:
+
+```
+{
+    "status": "Fail",
+    "msg": "Tablet not found"
+}
+```
+
+若 tablet 存在并且 tablet 不在正在执行 compaction,返回 JSON 格式:
+
+```
+{
+    "status" : "Success",
+    "run_status" : false,
+    "msg" : "this tablet_id is not running",
+    "tablet_id" : 11308,
+    "schema_hash" : 700967178,
+    "compact_type" : ""
+}
+```
+
+若 tablet 存在并且 tablet 正在执行 compaction,返回 JSON 格式:
+
+```
+{
+    "status" : "Success",
+    "run_status" : true,
+    "msg" : "this tablet_id is running",
+    "tablet_id" : 11308,
+    "schema_hash" : 700967178,
+    "compact_type" : "cumulative"
+}
+```
+
+结果说明:
+
+* run_status:获取当前手动 compaction 任务执行状态
+
+### 示例
+
+```
+curl -X GET http://192.168.10.24:8040/api/compaction/run_status?tablet_id=10015\&schema_hash=1294206575


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