This is an automated email from the ASF dual-hosted git repository.
dataroaring pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push:
new 95d42aefb1f [fix](merge-cloud) add injection test http api and fix
cloud compaction http api (#33268)
95d42aefb1f is described below
commit 95d42aefb1f75c9cc5fc5f02d4bfebfc475ea3a0
Author: Luwei <[email protected]>
AuthorDate: Fri Apr 5 18:52:12 2024 +0800
[fix](merge-cloud) add injection test http api and fix cloud compaction
http api (#33268)
---
be/CMakeLists.txt | 4 +
be/src/cloud/cloud_compaction_action.cpp | 171 ++++++-------------------
be/src/cloud/cloud_compaction_action.h | 3 -
be/src/cloud/cloud_storage_engine.h | 15 +++
be/src/cloud/cloud_tablet.cpp | 3 +-
be/src/cloud/injection_point_action.cpp | 213 +++++++++++++++++++++++++++++++
be/src/cloud/injection_point_action.h | 31 +++++
be/src/common/config.cpp | 5 +
be/src/common/config.h | 3 +
be/src/service/http_service.cpp | 6 +
build.sh | 6 +
11 files changed, 320 insertions(+), 140 deletions(-)
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index d9b9d957a54..28b3fee115a 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -329,6 +329,10 @@ if (WITH_MYSQL)
add_definitions(-DDORIS_WITH_MYSQL)
endif()
+if (ENABLE_INJECTION_POINT)
+ set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -DENABLE_INJECTION_POINT")
+endif()
+
# Enable memory tracker, which allows BE to limit the memory of tasks such as
query, load,
# and compaction,and observe the memory of BE through
be_ip:http_port/MemTracker.
# Adding the option `USE_MEM_TRACKER=OFF sh build.sh` when compiling can turn
off the memory tracker,
diff --git a/be/src/cloud/cloud_compaction_action.cpp
b/be/src/cloud/cloud_compaction_action.cpp
index fbc05e608b8..4d836fa0736 100644
--- a/be/src/cloud/cloud_compaction_action.cpp
+++ b/be/src/cloud/cloud_compaction_action.cpp
@@ -121,58 +121,29 @@ Status
CloudCompactionAction::_handle_run_compaction(HttpRequest* req, std::stri
RETURN_NOT_OK_STATUS_WITH_WARN(_check_param(req, &tablet_id, &table_id),
"check param failed");
// check compaction_type equals 'base' or 'cumulative'
- std::string compaction_type = req->param(PARAM_COMPACTION_TYPE);
+ auto& compaction_type = req->param(PARAM_COMPACTION_TYPE);
if (compaction_type != PARAM_COMPACTION_BASE &&
compaction_type != PARAM_COMPACTION_CUMULATIVE &&
compaction_type != PARAM_COMPACTION_FULL) {
return Status::NotSupported("The compaction type '{}' is not
supported", compaction_type);
}
- if (tablet_id == 0 && table_id != 0) {
- /*
- std::vector<TabletSharedPtr> tablet_vec =
_engine.tablet_manager()->get_all_tablet(
- [table_id](Tablet* tablet) -> bool { return
tablet->get_table_id() == table_id; });
- */
- auto tablet_vec = _engine.tablet_mgr().get_weak_tablets();
- for (const auto& weak_tablet : tablet_vec) {
- if (auto tablet = weak_tablet.lock()) {
- if (tablet->table_id() != table_id) {
- continue;
- }
- RETURN_IF_ERROR(
- _engine.submit_compaction_task(tablet,
CompactionType::FULL_COMPACTION));
- }
- }
- } else {
- // 2. fetch the tablet by tablet_id
- CloudTabletSPtr tablet =
DORIS_TRY(_engine.tablet_mgr().get_tablet(tablet_id));
- if (tablet == nullptr) {
- return Status::NotFound("Tablet not found. tablet_id={}",
tablet_id);
- }
+ CloudTabletSPtr tablet =
DORIS_TRY(_engine.tablet_mgr().get_tablet(tablet_id));
+ if (tablet == nullptr) {
+ return Status::NotFound("Tablet not found. tablet_id={}", tablet_id);
+ }
- // 3. execute compaction task
- std::packaged_task<Status()> task([this, tablet, compaction_type]() {
- return _execute_compaction_callback(tablet, compaction_type);
- });
- std::future<Status> future_obj = task.get_future();
- std::thread(std::move(task)).detach();
+ // 3. submit compaction task
+ RETURN_IF_ERROR(_engine.submit_compaction_task(
+ tablet, compaction_type == PARAM_COMPACTION_BASE ?
CompactionType::BASE_COMPACTION
+ : compaction_type == PARAM_COMPACTION_CUMULATIVE
+ ? CompactionType::CUMULATIVE_COMPACTION
+ : CompactionType::FULL_COMPACTION));
- // 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
- Status olap_status = future_obj.get();
- if (!olap_status.ok()) {
- return 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 =
- R"({"status": "Success", "msg": "compaction task is successfully
triggered. Table id: )" +
+ "{\"status\": \"Success\", \"msg\": \"compaction task is
successfully triggered. Table "
+ "id: " +
std::to_string(table_id) + ". Tablet id: " +
std::to_string(tablet_id) + "\"}";
return Status::OK();
}
@@ -190,14 +161,6 @@ Status
CloudCompactionAction::_handle_run_status_compaction(HttpRequest* req,
RETURN_IF_ERROR(_engine.get_compaction_status_json(json_result));
return Status::OK();
} else {
- // fetch the tablet by tablet_id
- //TabletSharedPtr tablet =
_engine.tablet_manager()->get_tablet(tablet_id);
- CloudTabletSPtr tablet =
DORIS_TRY(_engine.tablet_mgr().get_tablet(tablet_id));
- if (tablet == nullptr) {
- LOG(WARNING) << "invalid argument.tablet_id:" << tablet_id;
- return Status::InternalError("fail to get {}", tablet_id);
- }
-
std::string json_template = R"({
"status" : "Success",
"run_status" : $0,
@@ -210,32 +173,31 @@ Status
CloudCompactionAction::_handle_run_status_compaction(HttpRequest* req,
std::string compaction_type;
bool run_status = false;
- {
- // use try lock to check this tablet is running cumulative
compaction
- std::unique_lock<std::mutex>
lock_cumulative(tablet->get_cumulative_compaction_lock(),
- std::try_to_lock);
- if (!lock_cumulative.owns_lock()) {
- msg = "compaction task for this tablet is running";
- compaction_type = "cumulative";
- run_status = true;
- *json_result = strings::Substitute(json_template, run_status,
msg, tablet_id,
- compaction_type);
- return Status::OK();
- }
+ if (_engine.has_cumu_compaction(tablet_id)) {
+ msg = "compaction task for this tablet is running";
+ compaction_type = "cumulative";
+ run_status = true;
+ *json_result =
+ strings::Substitute(json_template, run_status, msg,
tablet_id, compaction_type);
+ return Status::OK();
}
- {
- // use try lock to check this tablet is running base compaction
- std::unique_lock<std::mutex>
lock_base(tablet->get_base_compaction_lock(),
- std::try_to_lock);
- if (!lock_base.owns_lock()) {
- msg = "compaction task for this tablet is running";
- compaction_type = "base";
- run_status = true;
- *json_result = strings::Substitute(json_template, run_status,
msg, tablet_id,
- compaction_type);
- return Status::OK();
- }
+ if (_engine.has_base_compaction(tablet_id)) {
+ msg = "compaction task for this tablet is running";
+ compaction_type = "base";
+ run_status = true;
+ *json_result =
+ strings::Substitute(json_template, run_status, msg,
tablet_id, compaction_type);
+ return Status::OK();
+ }
+
+ if (_engine.has_full_compaction(tablet_id)) {
+ msg = "compaction task for this tablet is running";
+ compaction_type = "full";
+ run_status = true;
+ *json_result =
+ strings::Substitute(json_template, run_status, msg,
tablet_id, compaction_type);
+ return Status::OK();
}
// not running any compaction
*json_result =
@@ -244,67 +206,6 @@ Status
CloudCompactionAction::_handle_run_status_compaction(HttpRequest* req,
}
}
-Status CloudCompactionAction::_execute_compaction_callback(CloudTabletSPtr
tablet,
- const std::string&
compaction_type) {
- MonotonicStopWatch timer;
- timer.start();
-
- std::shared_ptr<CumulativeCompactionPolicy> cumulative_compaction_policy =
-
CumulativeCompactionPolicyFactory::create_cumulative_compaction_policy(
- tablet->tablet_meta()->compaction_policy());
- /*
- if (tablet->get_cumulative_compaction_policy() == nullptr) {
- tablet->set_cumulative_compaction_policy(cumulative_compaction_policy);
- }
- */
- Status res = Status::OK();
- auto do_compact = [](Compaction& compaction) {
- RETURN_IF_ERROR(compaction.prepare_compact());
- return compaction.execute_compact();
- };
- if (compaction_type == PARAM_COMPACTION_BASE) {
- CloudBaseCompaction base_compaction(_engine, tablet);
- res = do_compact(base_compaction);
- if (!res) {
- if (!res.is<BE_NO_SUITABLE_VERSION>()) {
-
DorisMetrics::instance()->base_compaction_request_failed->increment(1);
- }
- }
- } else if (compaction_type == PARAM_COMPACTION_CUMULATIVE) {
- CloudCumulativeCompaction cumulative_compaction(_engine, tablet);
- res = do_compact(cumulative_compaction);
- if (!res) {
- if (res.is<CUMULATIVE_NO_SUITABLE_VERSION>()) {
- // Ignore this error code.
- VLOG_NOTICE << "failed to init cumulative compaction due to no
suitable version,"
- << "tablet=" << tablet->tablet_id();
- } else {
-
DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1);
- LOG(WARNING) << "failed to do cumulative compaction. res=" <<
res
- << ", table=" << tablet->tablet_id();
- }
- }
- } else if (compaction_type == PARAM_COMPACTION_FULL) {
- CloudFullCompaction full_compaction(_engine, tablet);
- res = do_compact(full_compaction);
- if (!res) {
- if (res.is<FULL_NO_SUITABLE_VERSION>()) {
- // Ignore this error code.
- VLOG_NOTICE << "failed to init full compaction due to no
suitable version,"
- << "tablet=" << tablet->tablet_id();
- } else {
- LOG(WARNING) << "failed to do full compaction. res=" << res
- << ", table=" << tablet->tablet_id();
- }
- }
- }
-
- timer.stop();
- LOG(INFO) << "Manual compaction task finish, status=" << res
- << ", compaction_use_time=" << timer.elapsed_time() / 1000000 <<
"ms";
- return res;
-}
-
void CloudCompactionAction::handle(HttpRequest* req) {
req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.c_str());
diff --git a/be/src/cloud/cloud_compaction_action.h
b/be/src/cloud/cloud_compaction_action.h
index dbee3a61c6e..648d00d9203 100644
--- a/be/src/cloud/cloud_compaction_action.h
+++ b/be/src/cloud/cloud_compaction_action.h
@@ -65,9 +65,6 @@ private:
/// 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
- Status _execute_compaction_callback(CloudTabletSPtr tablet, const
std::string& compaction_type);
-
/// fetch compaction running status
Status _handle_run_status_compaction(HttpRequest* req, std::string*
json_result);
diff --git a/be/src/cloud/cloud_storage_engine.h
b/be/src/cloud/cloud_storage_engine.h
index 9756a5637e4..b8c4da9cf64 100644
--- a/be/src/cloud/cloud_storage_engine.h
+++ b/be/src/cloud/cloud_storage_engine.h
@@ -95,6 +95,21 @@ public:
Status get_compaction_status_json(std::string* result);
+ bool has_base_compaction(int64_t tablet_id) const {
+ std::lock_guard lock(_compaction_mtx);
+ return _submitted_base_compactions.count(tablet_id);
+ }
+
+ bool has_cumu_compaction(int64_t tablet_id) const {
+ std::lock_guard lock(_compaction_mtx);
+ return _submitted_cumu_compactions.count(tablet_id);
+ }
+
+ bool has_full_compaction(int64_t tablet_id) const {
+ std::lock_guard lock(_compaction_mtx);
+ return _submitted_full_compactions.count(tablet_id);
+ }
+
private:
void _refresh_storage_vault_info_thread_callback();
void _vacuum_stale_rowsets_thread_callback();
diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp
index 9da67a827b5..53f92721a1a 100644
--- a/be/src/cloud/cloud_tablet.cpp
+++ b/be/src/cloud/cloud_tablet.cpp
@@ -505,8 +505,7 @@ std::vector<RowsetSharedPtr>
CloudTablet::pick_candidate_rowsets_to_base_compact
{
std::shared_lock rlock(_meta_lock);
for (const auto& [version, rs] : _rs_version_map) {
- // Do compaction on local rowsets only.
- if (version.first < _cumulative_point && rs->is_local()) {
+ if (version.first != 0 && version.first < _cumulative_point) {
candidate_rowsets.push_back(rs);
}
}
diff --git a/be/src/cloud/injection_point_action.cpp
b/be/src/cloud/injection_point_action.cpp
new file mode 100644
index 00000000000..adcc6cad708
--- /dev/null
+++ b/be/src/cloud/injection_point_action.cpp
@@ -0,0 +1,213 @@
+// 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 "cloud/injection_point_action.h"
+
+#include <glog/logging.h>
+
+#include <chrono>
+#include <mutex>
+
+#include "common/status.h"
+#include "common/sync_point.h"
+#include "http/http_channel.h"
+#include "http/http_request.h"
+#include "http/http_status.h"
+#include "olap/rowset/rowset.h"
+#include "util/stack_util.h"
+
+namespace doris {
+namespace {
+
+// TODO(cyx): Provide an object pool
+// `suite_map` won't be modified after `register_suites`
+std::map<std::string, std::function<void()>> suite_map;
+std::once_flag register_suites_once;
+
+// only call once
+void register_suites() {
+ suite_map.emplace("test_compaction", [] {
+ auto sp = SyncPoint::get_instance();
+ sp->set_call_back("new_cumulative_point", [](auto&& args) {
+ auto output_rowset = try_any_cast<Rowset*>(args[0]);
+ auto last_cumulative_point = try_any_cast<int64_t>(args[1]);
+ auto pair = try_any_cast<std::pair<int64_t, bool>*>(args.back());
+ pair->first = output_rowset->start_version() ==
last_cumulative_point
+ ? output_rowset->end_version() + 1
+ : last_cumulative_point;
+ pair->second = true;
+ });
+ });
+}
+
+void set_sleep(const std::string& point, HttpRequest* req) {
+ int duration = 0;
+ auto& duration_str = req->param("duration");
+ if (!duration_str.empty()) {
+ try {
+ duration = std::stoi(duration_str);
+ } catch (const std::exception&) {
+ HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST,
+ "invalid duration: " + duration_str);
+ return;
+ }
+ }
+ auto sp = SyncPoint::get_instance();
+ sp->set_call_back(point, [duration](auto&& args) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(duration));
+ });
+ HttpChannel::send_reply(req, HttpStatus::OK, "OK");
+}
+
+void set_return(const std::string& point, HttpRequest* req) {
+ auto sp = SyncPoint::get_instance();
+ sp->set_call_back(point, [](auto&& args) {
+ try {
+ auto pred = try_any_cast<bool*>(args.back());
+ *pred = true;
+ } catch (const std::bad_any_cast&) {
+ LOG_EVERY_N(ERROR, 10) << "failed to process `return` callback\n"
<< get_stack_trace();
+ }
+ });
+ HttpChannel::send_reply(req, HttpStatus::OK, "OK");
+}
+
+void set_return_ok(const std::string& point, HttpRequest* req) {
+ auto sp = SyncPoint::get_instance();
+ sp->set_call_back(point, [](auto&& args) {
+ try {
+ auto* pair = try_any_cast_ret<Status>(args);
+ pair->first = Status::OK();
+ pair->second = true;
+ } catch (const std::bad_any_cast&) {
+ LOG_EVERY_N(ERROR, 10) << "failed to process `return_ok`
callback\n"
+ << get_stack_trace();
+ }
+ });
+ HttpChannel::send_reply(req, HttpStatus::OK, "OK");
+}
+
+void set_return_error(const std::string& point, HttpRequest* req) {
+ const std::string CODE_PARAM = "code";
+ int code = ErrorCode::INTERNAL_ERROR;
+ auto& code_str = req->param(CODE_PARAM);
+ if (!code_str.empty()) {
+ try {
+ code = std::stoi(code_str);
+ } catch (const std::exception& e) {
+ HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST,
+ fmt::format("convert topn failed, {}",
e.what()));
+ return;
+ }
+ }
+
+ auto sp = SyncPoint::get_instance();
+ sp->set_call_back(point, [code](auto&& args) {
+ try {
+ auto* pair = try_any_cast_ret<Status>(args);
+ pair->first = Status::Error<false>(code, "injected error");
+ pair->second = true;
+ } catch (const std::bad_any_cast&) {
+ LOG_EVERY_N(ERROR, 10) << "failed to process `return_error`
callback\n"
+ << get_stack_trace();
+ }
+ });
+ HttpChannel::send_reply(req, HttpStatus::OK, "OK");
+}
+
+void handle_set(HttpRequest* req) {
+ auto& point = req->param("name");
+ if (point.empty()) {
+ HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, "empty point
name");
+ return;
+ }
+ auto& behavior = req->param("behavior");
+ if (behavior.empty()) {
+ HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, "empty
behavior");
+ return;
+ }
+ if (behavior == "sleep") {
+ set_sleep(point, req);
+ return;
+ } else if (behavior == "return") {
+ set_return(point, req);
+ return;
+ } else if (behavior == "return_ok") {
+ set_return_ok(point, req);
+ return;
+ } else if (behavior == "return_error") {
+ set_return_error(point, req);
+ return;
+ }
+ HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, "unknown behavior: "
+ behavior);
+}
+
+void handle_clear(HttpRequest* req) {
+ auto& point = req->param("name");
+ if (point.empty()) {
+ HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, "empty point
name");
+ return;
+ }
+ auto sp = SyncPoint::get_instance();
+ if (point == "all") {
+ sp->clear_all_call_backs();
+ HttpChannel::send_reply(req, HttpStatus::OK, "OK");
+ return;
+ }
+ sp->clear_call_back(point);
+ HttpChannel::send_reply(req, HttpStatus::OK, "OK");
+}
+
+void handle_suite(HttpRequest* req) {
+ auto& suite = req->param("name");
+ if (suite.empty()) {
+ HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, "empty suite
name");
+ return;
+ }
+
+ std::call_once(register_suites_once, register_suites);
+ if (auto it = suite_map.find(suite); it != suite_map.end()) {
+ it->second(); // set injection callbacks
+ HttpChannel::send_reply(req, HttpStatus::OK, "OK");
+ return;
+ }
+ HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, "unknown
suite: " + suite);
+}
+
+} // namespace
+
+InjectionPointAction::InjectionPointAction() {
+ SyncPoint::get_instance()->enable_processing();
+}
+
+void InjectionPointAction::handle(HttpRequest* req) {
+ LOG(INFO) << req->debug_string();
+ auto& op = req->param("op");
+ if (op == "set") {
+ handle_set(req);
+ return;
+ } else if (op == "clear") {
+ handle_clear(req);
+ return;
+ } else if (op == "apply_suite") {
+ handle_suite(req);
+ return;
+ }
+ HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, "unknown op: " + op);
+}
+
+} // namespace doris
diff --git a/be/src/cloud/injection_point_action.h
b/be/src/cloud/injection_point_action.h
new file mode 100644
index 00000000000..67a1cd2ace5
--- /dev/null
+++ b/be/src/cloud/injection_point_action.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.
+
+#include "http/http_handler.h"
+
+namespace doris {
+
+class InjectionPointAction : public HttpHandler {
+public:
+ InjectionPointAction();
+
+ ~InjectionPointAction() override = default;
+
+ void handle(HttpRequest* req) override;
+};
+
+} // namespace doris
diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp
index 0bd7a767abb..f29ec307914 100644
--- a/be/src/common/config.cpp
+++ b/be/src/common/config.cpp
@@ -1197,6 +1197,11 @@ DEFINE_mInt64(hive_sink_max_file_size, "1073741824"); //
1GB
DEFINE_mInt32(thrift_client_open_num_tries, "1");
+DEFINE_Bool(enable_index_compaction, "false");
+
+// enable injection point in regression-test
+DEFINE_mBool(enable_injection_point, "false");
+
// clang-format off
#ifdef BE_TEST
// test s3
diff --git a/be/src/common/config.h b/be/src/common/config.h
index 00b1f1a43b3..52c4988545a 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -1281,6 +1281,9 @@ DECLARE_mInt64(hive_sink_max_file_size); // 1GB
// Retry the Open num_retries time waiting 100 milliseconds between retries.
DECLARE_mInt32(thrift_client_open_num_tries);
+// enable injection point in regression-test
+DECLARE_mBool(enable_injection_point);
+
#ifdef BE_TEST
// test s3
DECLARE_String(test_s3_resource);
diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp
index 041bb9fd509..2ebbf5e5b69 100644
--- a/be/src/service/http_service.cpp
+++ b/be/src/service/http_service.cpp
@@ -25,6 +25,7 @@
#include "cloud/cloud_compaction_action.h"
#include "cloud/config.h"
+#include "cloud/injection_point_action.h"
#include "common/config.h"
#include "common/status.h"
#include "http/action/adjust_log_level.h"
@@ -361,6 +362,11 @@ void
HttpService::register_cloud_handler(CloudStorageEngine& engine) {
TPrivilegeHier::GLOBAL,
TPrivilegeType::ADMIN));
_ev_http_server->register_handler(HttpMethod::GET,
"/api/compaction/run_status",
run_status_compaction_action);
+#ifdef ENABLE_INJECTION_POINT
+ InjectionPointAction* injection_point_action = _pool.add(new
InjectionPointAction);
+ _ev_http_server->register_handler(HttpMethod::GET,
"/api/injection_point/{op}/{name}",
+ injection_point_action);
+#endif
ClearFileCacheAction* clear_file_cache_action = _pool.add(new
ClearFileCacheAction());
_ev_http_server->register_handler(HttpMethod::POST,
"/api/clear_file_cache",
clear_file_cache_action);
diff --git a/build.sh b/build.sh
index 8b5514e0bee..dbe6fb38b67 100755
--- a/build.sh
+++ b/build.sh
@@ -413,6 +413,10 @@ if [[ -z "${DISABLE_JAVA_CHECK_STYLE}" ]]; then
DISABLE_JAVA_CHECK_STYLE='OFF'
fi
+if [[ -z "${ENABLE_INJECTION_POINT}" ]]; then
+ ENABLE_INJECTION_POINT='OFF'
+fi
+
if [[ -z "${RECORD_COMPILER_SWITCHES}" ]]; then
RECORD_COMPILER_SWITCHES='OFF'
fi
@@ -459,6 +463,7 @@ echo "Get params:
USE_JEMALLOC -- ${USE_JEMALLOC}
USE_BTHREAD_SCANNER -- ${USE_BTHREAD_SCANNER}
ENABLE_STACKTRACE -- ${ENABLE_STACKTRACE}
+ ENABLE_INJECTION_POINT -- ${ENABLE_INJECTION_POINT}
DENABLE_CLANG_COVERAGE -- ${DENABLE_CLANG_COVERAGE}
DISPLAY_BUILD_TIME -- ${DISPLAY_BUILD_TIME}
ENABLE_PCH -- ${ENABLE_PCH}
@@ -543,6 +548,7 @@ if [[ "${BUILD_BE}" -eq 1 ]]; then
-DCMAKE_MAKE_PROGRAM="${MAKE_PROGRAM}" \
-DCMAKE_EXPORT_COMPILE_COMMANDS=ON \
-DCMAKE_BUILD_TYPE="${CMAKE_BUILD_TYPE}" \
+ -DENABLE_INJECTION_POINT="${ENABLE_INJECTION_POINT}" \
-DMAKE_TEST=OFF \
-DBUILD_FS_BENCHMARK="${BUILD_FS_BENCHMARK}" \
${CMAKE_USE_CCACHE:+${CMAKE_USE_CCACHE}} \
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]