[Enhancement] BE wait for at least one heartbeat from frontend during graceful exit (#59428)
Signed-off-by: Kevin Cai <kevin.cai@celerdata.com>
This commit is contained in:
parent
900dbea505
commit
2638c2aee5
|
|
@ -151,6 +151,10 @@ void HeartbeatServer::heartbeat(THeartbeatResult& heartbeat_result, const TMaste
|
|||
}
|
||||
heartbeat_result.backend_info.__set_reboot_time(reboot_time);
|
||||
}
|
||||
if (process_exit_in_progress()) {
|
||||
// Just assume this response can reach the frontend side.
|
||||
set_frontend_aware_of_exit();
|
||||
}
|
||||
}
|
||||
|
||||
std::string HeartbeatServer::print_master_info(const TMasterInfo& master_info) const {
|
||||
|
|
|
|||
|
|
@ -1168,6 +1168,12 @@ CONF_mInt16(pulsar_client_log_level, "2");
|
|||
// max loop count when be waiting its fragments to finish. It has no effect if the var is configured with value <= 0.
|
||||
CONF_mInt64(loop_count_wait_fragments_finish, "2");
|
||||
|
||||
// Determines whether to await at least one frontend heartbeat response indicating SHUTDOWN status before completing graceful exit.
|
||||
//
|
||||
// When enabled, the graceful shutdown process remains active until a SHUTDOWN confirmation is responded via heartbeat RPC,
|
||||
// ensuring the frontend has sufficient time to detect the termination state between two regular heartbeat intervals.
|
||||
CONF_mBool(graceful_exit_wait_for_frontend_heartbeat, "false");
|
||||
|
||||
// the maximum number of connections in the connection pool for a single jdbc url
|
||||
CONF_Int16(jdbc_connection_pool_size, "8");
|
||||
// the minimum number of idle connections that connection pool tries to maintain.
|
||||
|
|
|
|||
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
namespace starrocks {
|
||||
|
||||
// TODO: use bitmask to merge all the atomic variables into one atomic variable.
|
||||
|
||||
// NOTE: when BE receiving SIGTERM, this flag will be set to true. Then BE will reject
|
||||
// all ExecPlanFragments call by returning a fail status(brpc::EINTERNAL).
|
||||
// After all existing fragments executed, BE will exit.
|
||||
|
|
@ -29,6 +31,11 @@ std::atomic<bool> k_starrocks_exit;
|
|||
// but also waiting for all threads to exit gracefully.
|
||||
std::atomic<bool> k_starrocks_quick_exit;
|
||||
|
||||
// NOTE: when backend is going to shutdown, if FE's heartbeat request received, the status code
|
||||
// in the response will be set to SHUTDOWN, then FE leader will know the node is in shutdown immediately.
|
||||
// This flag records whether there is a such response with status SHUTDOWN sent back to FE.
|
||||
std::atomic<bool> k_starrocks_fe_heartbeat_aware_shutdown = false;
|
||||
|
||||
bool set_process_exit() {
|
||||
bool expected = false;
|
||||
return k_starrocks_exit.compare_exchange_strong(expected, true);
|
||||
|
|
@ -47,4 +54,16 @@ bool process_quick_exit_in_progress() {
|
|||
return k_starrocks_quick_exit.load(std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
void set_frontend_aware_of_exit() {
|
||||
k_starrocks_fe_heartbeat_aware_shutdown.store(true);
|
||||
}
|
||||
|
||||
void clear_frontend_aware_of_exit() {
|
||||
k_starrocks_fe_heartbeat_aware_shutdown.store(false);
|
||||
}
|
||||
|
||||
bool is_frontend_aware_of_exit() {
|
||||
return k_starrocks_fe_heartbeat_aware_shutdown.load(std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -42,4 +42,16 @@ bool process_exit_in_progress();
|
|||
// - false: process is not in quick exit
|
||||
bool process_quick_exit_in_progress();
|
||||
|
||||
// set the flag of FE leader awareness of the shutdown
|
||||
void set_frontend_aware_of_exit();
|
||||
|
||||
// whether the FE leader is aware of the shutdown
|
||||
// returns:
|
||||
// - true: at least one response is marked as shutdown to FE's heartbeat request
|
||||
// - false: no response is marked as shutdown to FE's heartbeat request
|
||||
bool is_frontend_aware_of_exit();
|
||||
|
||||
// clear the flag of frontend awareness of the shutdown.
|
||||
void clear_frontend_aware_of_exit();
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -43,6 +43,7 @@
|
|||
#include "common/config.h"
|
||||
#include "common/configbase.h"
|
||||
#include "common/logging.h"
|
||||
#include "common/process_exit.h"
|
||||
#include "exec/pipeline/driver_limiter.h"
|
||||
#include "exec/pipeline/pipeline_driver_executor.h"
|
||||
#include "exec/pipeline/query_context.h"
|
||||
|
|
@ -795,8 +796,14 @@ void ExecEnv::_wait_for_fragments_finish() {
|
|||
size_t running_fragments = _get_running_fragments_count();
|
||||
size_t loop_secs = 0;
|
||||
|
||||
while (running_fragments > 0 && loop_secs < max_loop_secs) {
|
||||
LOG(INFO) << running_fragments << " fragment(s) are still running...";
|
||||
// TODO: decouple the heartbeat with the graceful exit
|
||||
// only wait for frontend's heartbeat when the node is ever received heartbeats from the frontend
|
||||
bool need_wait_frontend_hb = config::graceful_exit_wait_for_frontend_heartbeat && get_backend_id().has_value();
|
||||
|
||||
while ((running_fragments > 0 || (need_wait_frontend_hb && !is_frontend_aware_of_exit())) &&
|
||||
loop_secs < max_loop_secs) {
|
||||
LOG(INFO) << "Frontend is aware of exit: " << is_frontend_aware_of_exit() << ", " << running_fragments
|
||||
<< " fragment(s) are still running...";
|
||||
sleep(1);
|
||||
running_fragments = _get_running_fragments_count();
|
||||
loop_secs++;
|
||||
|
|
|
|||
|
|
@ -14,6 +14,7 @@
|
|||
|
||||
#include "agent/heartbeat_server.h"
|
||||
|
||||
#include "common/process_exit.h"
|
||||
#include "gen_cpp/Types_types.h"
|
||||
#include "gtest/gtest.h"
|
||||
#include "service/backend_options.h"
|
||||
|
|
@ -110,4 +111,32 @@ TEST(HeartbeatServerTest, test_unmatched_node_type_heartbeat) {
|
|||
EXPECT_TRUE(res.status().message().find("actually BE"));
|
||||
}
|
||||
|
||||
TEST(HeartbeatServerTest, test_frontend_aware_of_exit) {
|
||||
clear_frontend_aware_of_exit();
|
||||
ASSERT_FALSE(is_frontend_aware_of_exit());
|
||||
|
||||
HeartbeatServer server;
|
||||
{
|
||||
THeartbeatResult result;
|
||||
TMasterInfo info;
|
||||
server.heartbeat(result, info);
|
||||
EXPECT_EQ(TStatusCode::OK, result.status.status_code);
|
||||
// service is running, no exit at all
|
||||
ASSERT_FALSE(is_frontend_aware_of_exit());
|
||||
}
|
||||
|
||||
k_starrocks_exit = true;
|
||||
{
|
||||
THeartbeatResult result;
|
||||
TMasterInfo info;
|
||||
server.heartbeat(result, info);
|
||||
EXPECT_EQ(TStatusCode::SHUTDOWN, result.status.status_code);
|
||||
Status status(result.status);
|
||||
EXPECT_TRUE(status.is_shutdown());
|
||||
// service is in shutdown, the shutdown response is replied to the frontend
|
||||
ASSERT_TRUE(is_frontend_aware_of_exit());
|
||||
}
|
||||
k_starrocks_exit = false;
|
||||
}
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -3796,6 +3796,15 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
- Description: The number of loops to be waited when the BE/CN process exits. Each loop is a fixed interval of 10 seconds. You can set it to `0` to disable the loop wait. From v3.4 onwards, this item is changed to mutable and its default value is changed from `0` to `2`.
|
||||
- Introduced in: v2.5
|
||||
|
||||
##### graceful_exit_wait_for_frontend_heartbeat
|
||||
|
||||
- Default: false
|
||||
- Type: Boolean
|
||||
- Unit: -
|
||||
- Is mutable: Yes
|
||||
- Description: Determines whether to await at least one frontend heartbeat response indicating SHUTDOWN status before completing graceful exit. When enabled, the graceful shutdown process remains active until a SHUTDOWN confirmation is responded via heartbeat RPC, ensuring the frontend has sufficient time to detect the termination state between two regular heartbeat intervals.
|
||||
- Introduced in: v3.4.5
|
||||
|
||||
### Data Lake
|
||||
|
||||
##### jdbc_connection_pool_size
|
||||
|
|
|
|||
|
|
@ -3755,6 +3755,15 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
- 描述:BE/CN 退出时需要等待正在执行的查询完成的轮次,一轮次固定 10 秒。设置为 `0` 表示禁用轮询等待,立即退出。自 v3.4 起,该参数变为动态参数,且默认值由 `0` 变为 `2`。
|
||||
- 引入版本:v2.5
|
||||
|
||||
##### graceful_exit_wait_for_frontend_heartbeat
|
||||
|
||||
- 默认值:false
|
||||
- 类型: Boolean
|
||||
- 单位:-
|
||||
- 是否动态:是
|
||||
- 描述: 确定是否在完成优雅退出前等待至少一个指示SHUTDOWN状态的FE心跳响应。启用后,优雅关闭进程将持续运行直至通过心跳RPC返回给FE SHUTDOWN状态变化,确保FE在两次常规心跳探测间隔期间有足够时间感知终止状态。
|
||||
- 引入版本:v3.4.5
|
||||
|
||||
### 数据湖
|
||||
|
||||
##### jdbc_connection_pool_size
|
||||
|
|
|
|||
Loading…
Reference in New Issue