Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions be/src/runtime/fragment_mgr.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -670,6 +670,7 @@ void FragmentMgr::remove_pipeline_context(std::pair<TUniqueId, int> key) {
}

void FragmentMgr::remove_query_context(const TUniqueId& key) {
_query_ctx_map_delay_delete.erase(key);
#ifndef BE_TEST
_query_ctx_map.erase(key);
#endif
Expand Down Expand Up @@ -768,6 +769,9 @@ Status FragmentMgr::_get_or_create_query_ctx(const TPipelineFragmentParams& para

query_ctx->runtime_filter_mgr()->set_runtime_filter_params(
info.runtime_filter_params);
if (!handler->empty()) {
_query_ctx_map_delay_delete.insert(query_id, query_ctx);
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should also deal with the timeout logic if FE not send cancel message to BE. like the code at line 974

if (info.__isset.topn_filter_descs) {
query_ctx->init_runtime_predicates(info.topn_filter_descs);
Expand Down
3 changes: 3 additions & 0 deletions be/src/runtime/fragment_mgr.h
Original file line number Diff line number Diff line change
Expand Up @@ -209,6 +209,9 @@ class FragmentMgr : public RestMonitorIface {

// query id -> QueryContext
ConcurrentContextMap<TUniqueId, std::weak_ptr<QueryContext>, QueryContext> _query_ctx_map;
// keep query ctx do not delete immediately to make rf coordinator merge filter work well after query eos
ConcurrentContextMap<TUniqueId, std::shared_ptr<QueryContext>, QueryContext>
_query_ctx_map_delay_delete;

CountDownLatch _stop_background_threads_latch;
std::shared_ptr<Thread> _cancel_thread;
Expand Down
8 changes: 4 additions & 4 deletions be/src/runtime/query_context.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -231,13 +231,13 @@ QueryContext::~QueryContext() {
_runtime_predicates.clear();
file_scan_range_params_map.clear();
obj_pool.clear();
if (_merge_controller_handler) {
_merge_controller_handler->release_undone_filters(this);
}
_merge_controller_handler.reset();

DorisMetrics::instance()->query_ctx_cnt->increment(-1);
ExecEnv::GetInstance()->fragment_mgr()->remove_query_context(this->_query_id);
// fragment_mgr is nullptr in unittest
if (ExecEnv::GetInstance()->fragment_mgr()) {
ExecEnv::GetInstance()->fragment_mgr()->remove_query_context(this->_query_id);
}
// the only one msg shows query's end. any other msg should append to it if need.
LOG_INFO("Query {} deconstructed, mem_tracker: {}", print_id(this->_query_id), mem_tracker_msg);
}
Expand Down
24 changes: 0 additions & 24 deletions be/src/runtime_filter/runtime_filter_mgr.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -433,30 +433,6 @@ Status RuntimeFilterMergeControllerEntity::_send_rf_to_target(GlobalMergeContext
return st;
}

void RuntimeFilterMergeControllerEntity::release_undone_filters(QueryContext* query_ctx) {
std::unique_lock<std::shared_mutex> guard(_filter_map_mutex);
for (auto& [filter_id, ctx] : _filter_map) {
if (!ctx.done && !ctx.targetv2_info.empty()) {
{
std::lock_guard<std::mutex> l(ctx.mtx);
ctx.merger->set_wrapper_state_and_ready_to_apply(
RuntimeFilterWrapper::State::DISABLED,
"rf coordinator's query context released before runtime filter is ready to "
"apply");
}
auto st = _send_rf_to_target(ctx, std::weak_ptr<QueryContext> {}, 0,
UniqueId(query_ctx->query_id()).to_proto(),
query_ctx->execution_timeout());
if (!st.ok()) {
LOG(WARNING)
<< "Failed to send runtime filter to target before query done. filter_id:"
<< filter_id << " " << ctx.merger->debug_string() << " reason:" << st;
}
}
}
_filter_map.clear();
}

std::string RuntimeFilterMergeControllerEntity::debug_string() {
std::string result = "RuntimeFilterMergeControllerEntity Info:\n";
std::shared_lock<std::shared_mutex> guard(_filter_map_mutex);
Expand Down
5 changes: 4 additions & 1 deletion be/src/runtime_filter/runtime_filter_mgr.h
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,10 @@ class RuntimeFilterMergeControllerEntity {

std::string debug_string();

void release_undone_filters(QueryContext* query_ctx);
bool empty() {
std::shared_lock<std::shared_mutex> read_lock(_filter_map_mutex);
return _filter_map.empty();
}

private:
Status _init_with_desc(std::shared_ptr<QueryContext> query_ctx,
Expand Down
Loading