## Proposed changes

pick #41292 #41350 #41589 #41628 #41743 #41601 #41667 #41751

<!--Describe your changes.-->

---------

Co-authored-by: Pxl <pxl290@qq.com>
This commit is contained in:
Gabriel
2024-10-16 15:41:28 +08:00
committed by GitHub
parent e56216211e
commit 5bd33fc88c
27 changed files with 189 additions and 91 deletions

View File

@ -151,19 +151,19 @@ public:
}
Status merge(BloomFilterFuncBase* bloomfilter_func) {
DCHECK(bloomfilter_func != nullptr);
DCHECK(bloomfilter_func->_bloom_filter != nullptr);
// If `_inited` is false, there is no memory allocated in bloom filter and this is the first
// call for `merge` function. So we just reuse this bloom filter, and we don't need to
// allocate memory again.
if (!_inited) {
auto* other_func = static_cast<BloomFilterFuncBase*>(bloomfilter_func);
DCHECK(_bloom_filter == nullptr);
DCHECK(bloomfilter_func != nullptr);
_bloom_filter = bloomfilter_func->_bloom_filter;
_bloom_filter_alloced = other_func->_bloom_filter_alloced;
_inited = true;
return Status::OK();
}
DCHECK(bloomfilter_func != nullptr);
auto* other_func = static_cast<BloomFilterFuncBase*>(bloomfilter_func);
if (_bloom_filter_alloced != other_func->_bloom_filter_alloced) {
return Status::InternalError(

View File

@ -468,10 +468,10 @@ public:
const TExpr& probe_expr);
Status merge(const RuntimePredicateWrapper* wrapper) {
if (is_ignored() || wrapper->is_ignored()) {
_context->ignored = true;
if (wrapper->is_ignored()) {
return Status::OK();
}
_context->ignored = false;
bool can_not_merge_in_or_bloom =
_filter_type == RuntimeFilterType::IN_OR_BLOOM_FILTER &&
@ -489,7 +489,10 @@ public:
switch (_filter_type) {
case RuntimeFilterType::IN_FILTER: {
// try insert set
if (!_context->hybrid_set) {
_context->ignored = true;
return Status::OK();
}
_context->hybrid_set->insert(wrapper->_context->hybrid_set.get());
if (_max_in_num >= 0 && _context->hybrid_set->size() >= _max_in_num) {
_context->ignored = true;

View File

@ -98,12 +98,16 @@ public:
return Status::OK();
}
Status ignore_all_filters() {
for (auto filter : _runtime_filters) {
filter->set_ignored();
}
return Status::OK();
}
Status init_filters(RuntimeState* state, uint64_t local_hash_table_size) {
// process IN_OR_BLOOM_FILTER's real type
for (auto* filter : _runtime_filters) {
if (filter->get_ignored()) {
continue;
}
if (filter->type() == RuntimeFilterType::IN_OR_BLOOM_FILTER &&
get_real_size(filter, local_hash_table_size) > state->runtime_filter_max_in_num()) {
RETURN_IF_ERROR(filter->change_to_bloom_filter());
@ -140,7 +144,7 @@ public:
}
// publish runtime filter
Status publish(bool publish_local = false) {
Status publish(bool publish_local) {
for (auto& pair : _runtime_filters_map) {
for (auto& filter : pair.second) {
RETURN_IF_ERROR(filter->publish(publish_local));

View File

@ -189,13 +189,17 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) {
PUniqueId id;
id.set_hi(_state->query_id().hi);
id.set_lo(_state->query_id().lo);
_sink_buffer = std::make_unique<ExchangeSinkBuffer<ExchangeSinkLocalState>>(
id, p._dest_node_id, _sender_id, _state->be_number(), state, this);
register_channels(_sink_buffer.get());
_queue_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(),
"ExchangeSinkQueueDependency", true);
_sink_buffer->set_dependency(_queue_dependency, _finish_dependency);
if (!only_local_exchange) {
_sink_buffer = std::make_unique<ExchangeSinkBuffer<ExchangeSinkLocalState>>(
id, p._dest_node_id, _sender_id, _state->be_number(), state, this);
register_channels(_sink_buffer.get());
_queue_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(),
"ExchangeSinkQueueDependency", true);
_sink_buffer->set_dependency(_queue_dependency, _finish_dependency);
_finish_dependency->block();
}
if ((_part_type == TPartitionType::UNPARTITIONED || channels.size() == 1) &&
!only_local_exchange) {
_broadcast_dependency = Dependency::create_shared(
@ -304,7 +308,6 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) {
fmt::format("Crc32HashPartitioner({})", _partition_count));
}
_finish_dependency->block();
if (_part_type == TPartitionType::HASH_PARTITIONED ||
_part_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED ||
_part_type == TPartitionType::TABLE_SINK_HASH_PARTITIONED) {
@ -638,8 +641,9 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block
final_st = st;
}
}
local_state._sink_buffer->set_should_stop();
return final_st;
if (local_state._sink_buffer) {
local_state._sink_buffer->set_should_stop();
}
}
return final_st;
}
@ -725,11 +729,14 @@ Status ExchangeSinkOperatorX::channel_add_rows_with_idx(
std::string ExchangeSinkLocalState::debug_string(int indentation_level) const {
fmt::memory_buffer debug_string_buffer;
fmt::format_to(debug_string_buffer, "{}", Base::debug_string(indentation_level));
fmt::format_to(debug_string_buffer,
", Sink Buffer: (_should_stop = {}, _busy_channels = {}, _is_finishing = {}), "
"_reach_limit: {}",
_sink_buffer->_should_stop.load(), _sink_buffer->_busy_channels.load(),
_sink_buffer->_is_finishing.load(), _reach_limit.load());
if (_sink_buffer) {
fmt::format_to(
debug_string_buffer,
", Sink Buffer: (_should_stop = {}, _busy_channels = {}, _is_finishing = {}), "
"_reach_limit: {}",
_sink_buffer->_should_stop.load(), _sink_buffer->_busy_channels.load(),
_sink_buffer->_is_finishing.load(), _reach_limit.load());
}
return fmt::to_string(debug_string_buffer);
}

View File

@ -96,7 +96,9 @@ public:
std::vector<Dependency*> dependencies() const override {
std::vector<Dependency*> dep_vec;
dep_vec.push_back(_queue_dependency.get());
if (_queue_dependency) {
dep_vec.push_back(_queue_dependency.get());
}
if (_broadcast_dependency) {
dep_vec.push_back(_broadcast_dependency.get());
}

View File

@ -22,6 +22,7 @@
#include "exprs/bloom_filter_func.h"
#include "pipeline/exec/hashjoin_probe_operator.h"
#include "pipeline/exec/operator.h"
#include "pipeline/pipeline_x/pipeline_x_task.h"
#include "vec/data_types/data_type_nullable.h"
#include "vec/exec/join/vhash_join_node.h"
#include "vec/utils/template_helpers.hpp"
@ -122,6 +123,9 @@ Status HashJoinBuildSinkLocalState::open(RuntimeState* state) {
}
Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_status) {
if (_closed) {
return Status::OK();
}
auto p = _parent->cast<HashJoinBuildSinkOperatorX>();
Defer defer {[&]() {
if (_should_build_hash_table && p._shared_hashtable_controller) {
@ -130,25 +134,30 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu
}};
if (!_runtime_filter_slots || _runtime_filters.empty() || state->is_cancelled()) {
return Status::OK();
}
auto* block = _shared_state->build_block.get();
uint64_t hash_table_size = block ? block->rows() : 0;
{
SCOPED_TIMER(_runtime_filter_init_timer);
if (_should_build_hash_table) {
RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size));
}
RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state));
}
if (_should_build_hash_table && hash_table_size > 1) {
SCOPED_TIMER(_runtime_filter_compute_timer);
_runtime_filter_slots->insert(block);
return Base::close(state, exec_status);
}
if (state->get_task()->wake_up_by_downstream()) {
RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency));
RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters());
} else {
auto* block = _shared_state->build_block.get();
uint64_t hash_table_size = block ? block->rows() : 0;
{
SCOPED_TIMER(_runtime_filter_init_timer);
if (_should_build_hash_table) {
RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size));
}
RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state));
}
if (_should_build_hash_table && hash_table_size > 1) {
SCOPED_TIMER(_runtime_filter_compute_timer);
_runtime_filter_slots->insert(block);
}
}
SCOPED_TIMER(_publish_runtime_filter_timer);
RETURN_IF_ERROR(_runtime_filter_slots->publish(!_should_build_hash_table));
return Status::OK();
return Base::close(state, exec_status);
}
bool HashJoinBuildSinkLocalState::build_unique() const {
@ -519,6 +528,7 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
SCOPED_TIMER(local_state.exec_time_counter());
COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows());
local_state._eos = eos;
if (local_state._should_build_hash_table) {
// If eos or have already met a null value using short-circuit strategy, we do not need to pull
// data from probe side.
@ -582,6 +592,7 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
local_state.process_build_block(state, (*local_state._shared_state->build_block)));
if (_shared_hashtable_controller) {
_shared_hash_table_context->status = Status::OK();
_shared_hash_table_context->complete_build_stage = true;
// arena will be shared with other instances.
_shared_hash_table_context->arena = local_state._shared_state->arena;
_shared_hash_table_context->hash_table_variants =
@ -594,7 +605,8 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
local_state._runtime_filter_slots->copy_to_shared_context(_shared_hash_table_context);
_shared_hashtable_controller->signal(node_id());
}
} else if (!local_state._should_build_hash_table) {
} else if (!local_state._should_build_hash_table &&
_shared_hash_table_context->complete_build_stage) {
DCHECK(_shared_hashtable_controller != nullptr);
DCHECK(_shared_hash_table_context != nullptr);
// the instance which is not build hash table, it's should wait the signal of hash table build finished.

View File

@ -62,15 +62,15 @@ class MultiCastDataStreamSinkOperatorX final
using Base = DataSinkOperatorX<MultiCastDataStreamSinkLocalState>;
public:
MultiCastDataStreamSinkOperatorX(int sink_id, std::vector<int>& sources,
const int cast_sender_count, ObjectPool* pool,
MultiCastDataStreamSinkOperatorX(int sink_id, std::vector<int>& sources, ObjectPool* pool,
const TMultiCastDataStreamSink& sink,
const RowDescriptor& row_desc)
: Base(sink_id, -1, sources),
_pool(pool),
_row_desc(row_desc),
_cast_sender_count(cast_sender_count),
_sink(sink) {}
_cast_sender_count(sources.size()),
_sink(sink),
_num_dests(sources.size()) {}
~MultiCastDataStreamSinkOperatorX() override = default;
Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos) override {
@ -103,14 +103,19 @@ public:
}
const TMultiCastDataStreamSink& sink_node() { return _sink; }
bool count_down_destination() override {
DCHECK_GT(_num_dests, 0);
return _num_dests.fetch_sub(1) == 1;
}
private:
friend class MultiCastDataStreamSinkLocalState;
ObjectPool* _pool;
RowDescriptor _row_desc;
const int _cast_sender_count;
const size_t _cast_sender_count;
const TMultiCastDataStreamSink& _sink;
friend class MultiCastDataStreamSinkLocalState;
std::atomic<size_t> _num_dests;
};
} // namespace doris::pipeline

View File

@ -258,7 +258,6 @@ public:
std::unique_ptr<RuntimeState> _runtime_state;
bool _eos = false;
std::shared_ptr<Dependency> _finish_dependency;
// temp structures during spilling

View File

@ -55,7 +55,6 @@ private:
RuntimeProfile::Counter* _spill_merge_sort_timer = nullptr;
bool _eos = false;
vectorized::SpillStreamSPtr _spilling_stream;
std::shared_ptr<Dependency> _finish_dependency;
};

View File

@ -22,6 +22,7 @@
#include <utility>
#include "pipeline/exec/operator.h"
#include "pipeline/pipeline_task.h"
namespace doris::pipeline {
@ -99,4 +100,14 @@ Status Pipeline::set_sink(DataSinkOperatorXPtr& sink) {
return Status::OK();
}
} // namespace doris::pipeline
void Pipeline::make_all_runnable() {
if (_sink_x->count_down_destination()) {
for (auto* task : _tasks) {
if (task) {
task->clear_blocking_state(true);
}
}
}
}
} // namespace doris::pipeline

View File

@ -50,6 +50,7 @@ public:
std::weak_ptr<PipelineFragmentContext> context)
: _pipeline_id(pipeline_id), _context(std::move(context)), _num_tasks(num_tasks) {
_init_profile();
_tasks.resize(_num_tasks, nullptr);
}
void add_dependency(std::shared_ptr<Pipeline>& pipeline) {
@ -155,14 +156,24 @@ public:
void set_children(std::shared_ptr<Pipeline> child) { _children.push_back(child); }
void set_children(std::vector<std::shared_ptr<Pipeline>> children) { _children = children; }
void incr_created_tasks() { _num_tasks_created++; }
void incr_created_tasks(int i, PipelineTask* task) {
_num_tasks_created++;
_num_tasks_running++;
DCHECK_LT(i, _tasks.size());
_tasks[i] = task;
}
void make_all_runnable();
void set_num_tasks(int num_tasks) {
_num_tasks = num_tasks;
_tasks.resize(_num_tasks, nullptr);
for (auto& op : operatorXs) {
op->set_parallel_tasks(_num_tasks);
}
}
int num_tasks() const { return _num_tasks; }
bool close_task() { return _num_tasks_running.fetch_sub(1) == 1; }
std::string debug_string() {
fmt::memory_buffer debug_string_buffer;
@ -243,6 +254,10 @@ private:
int _num_tasks = 1;
// How many tasks are already created?
std::atomic<int> _num_tasks_created = 0;
// How many tasks are already created and not finished?
std::atomic<int> _num_tasks_running = 0;
// Tasks in this pipeline.
std::vector<PipelineTask*> _tasks;
};
} // namespace doris::pipeline

View File

@ -813,7 +813,7 @@ void PipelineFragmentContext::close_if_prepare_failed(Status /*st*/) {
DCHECK(!task->is_pending_finish());
WARN_IF_ERROR(task->close(Status::OK()),
fmt::format("Query {} closed since prepare failed", print_id(_query_id)));
close_a_pipeline();
close_a_pipeline(task->pipeline_id());
}
}
@ -960,7 +960,7 @@ void PipelineFragmentContext::_close_fragment_instance() {
std::dynamic_pointer_cast<PipelineFragmentContext>(shared_from_this()));
}
void PipelineFragmentContext::close_a_pipeline() {
void PipelineFragmentContext::close_a_pipeline(PipelineId pipeline_id) {
std::lock_guard<std::mutex> l(_task_mutex);
g_pipeline_tasks_count << -1;
++_closed_tasks;

View File

@ -110,7 +110,7 @@ public:
[[nodiscard]] int get_fragment_id() const { return _fragment_id; }
void close_a_pipeline();
virtual void close_a_pipeline(PipelineId pipeline_id);
virtual void clear_finished_tasks() {}

View File

@ -291,6 +291,10 @@ public:
std::string task_name() const { return fmt::format("task{}({})", _index, _pipeline->_name); }
PipelineId pipeline_id() const { return _pipeline->id(); }
virtual void clear_blocking_state(bool wake_up_by_downstream = false) {}
protected:
void _finish_p_dependency() {
for (const auto& p : _pipeline->_parents) {

View File

@ -52,28 +52,16 @@ Status LocalExchangeSinkLocalState::open(RuntimeState* state) {
return Status::OK();
}
Status LocalExchangeSinkLocalState::close(RuntimeState* state, Status exec_status) {
if (_closed) {
return Status::OK();
}
RETURN_IF_ERROR(Base::close(state, exec_status));
if (exec_status.ok()) {
DCHECK(_release_count) << "Do not finish correctly! " << debug_string(0);
}
return Status::OK();
}
std::string LocalExchangeSinkLocalState::debug_string(int indentation_level) const {
fmt::memory_buffer debug_string_buffer;
fmt::format_to(debug_string_buffer,
"{}, _use_global_shuffle: {}, _channel_id: {}, _num_partitions: {}, "
"_num_senders: {}, _num_sources: {}, "
"_running_sink_operators: {}, _running_source_operators: {}, _release_count: {}",
"_running_sink_operators: {}, _running_source_operators: {}",
Base::debug_string(indentation_level),
_parent->cast<LocalExchangeSinkOperatorX>()._use_global_shuffle, _channel_id,
_exchanger->_num_partitions, _exchanger->_num_senders, _exchanger->_num_sources,
_exchanger->_running_sink_operators, _exchanger->_running_source_operators,
_release_count);
_exchanger->_running_sink_operators, _exchanger->_running_source_operators);
return fmt::to_string(debug_string_buffer);
}
@ -121,13 +109,11 @@ Status LocalExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
// If all exchange sources ended due to limit reached, current task should also finish
if (local_state._exchanger->_running_source_operators == 0) {
local_state._release_count = true;
local_state._shared_state->sub_running_sink_operators();
return Status::EndOfFile("receiver eof");
}
if (eos) {
local_state._shared_state->sub_running_sink_operators();
local_state._release_count = true;
}
return Status::OK();

View File

@ -38,7 +38,6 @@ public:
Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
Status open(RuntimeState* state) override;
Status close(RuntimeState* state, Status exec_status) override;
std::string debug_string(int indentation_level) const override;
private:
@ -62,7 +61,6 @@ private:
// Used by random passthrough exchanger
int _channel_id = 0;
bool _release_count = false;
};
// A single 32-bit division on a recent x64 processor has a throughput of one instruction every six cycles with a latency of 26 cycles.

View File

@ -534,6 +534,7 @@ protected:
// Set to true after close() has been called. subclasses should check and set this in
// close().
bool _closed = false;
std::atomic<bool> _eos = false;
//NOTICE: now add a faker profile, because sometimes the profile record is useless
//so we want remove some counters and timers, eg: in join node, if it's broadcast_join
//and shared hash table, some counter/timer about build hash table is useless,
@ -683,6 +684,8 @@ public:
virtual bool should_dry_run(RuntimeState* state) { return false; }
[[nodiscard]] virtual bool count_down_destination() { return true; }
protected:
template <typename Writer, typename Parent>
requires(std::is_base_of_v<vectorized::AsyncResultWriter, Writer>)

View File

@ -127,6 +127,8 @@ PipelineXFragmentContext::~PipelineXFragmentContext() {
} else {
_call_back(nullptr, &st);
}
_dag.clear();
_pip_id_to_pipeline.clear();
_runtime_state.reset();
_runtime_filter_states.clear();
_runtime_filter_mgr_map.clear();
@ -465,8 +467,7 @@ Status PipelineXFragmentContext::_create_data_sink(ObjectPool* pool, const TData
}
_sink.reset(new MultiCastDataStreamSinkOperatorX(
sink_id, sources, thrift_sink.multi_cast_stream_sink.sinks.size(), pool,
thrift_sink.multi_cast_stream_sink, row_desc));
sink_id, sources, pool, thrift_sink.multi_cast_stream_sink, row_desc));
for (int i = 0; i < sender_size; ++i) {
auto new_pipeline = add_pipeline();
RowDescriptor* _row_desc = nullptr;
@ -525,6 +526,7 @@ Status PipelineXFragmentContext::_build_pipeline_x_tasks(
_task_runtime_states.resize(_pipelines.size());
for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) {
_task_runtime_states[pip_idx].resize(_pipelines[pip_idx]->num_tasks());
_pip_id_to_pipeline[_pipelines[pip_idx]->id()] = _pipelines[pip_idx].get();
}
auto& pipeline_id_to_profile = _runtime_state->pipeline_id_to_profile();
DCHECK(pipeline_id_to_profile.empty());
@ -638,6 +640,8 @@ Status PipelineXFragmentContext::_build_pipeline_x_tasks(
task_runtime_state.get(), ctx,
pipeline_id_to_profile[pip_idx].get(),
get_local_exchange_state(pipeline), i);
pipeline->incr_created_tasks(i, task.get());
task_runtime_state->set_task(task.get());
pipeline_id_to_task.insert({pipeline->id(), task.get()});
_tasks[i].emplace_back(std::move(task));
}
@ -737,7 +741,6 @@ Status PipelineXFragmentContext::_build_pipeline_x_tasks(
}
}
_pipeline_parent_map.clear();
_dag.clear();
_op_id_to_le_state.clear();
return Status::OK();
@ -1498,7 +1501,7 @@ void PipelineXFragmentContext::close_if_prepare_failed(Status st) {
for (auto& t : task) {
DCHECK(!t->is_pending_finish());
WARN_IF_ERROR(t->close(Status::OK()), "close_if_prepare_failed failed: ");
close_a_pipeline();
close_a_pipeline(t->pipeline_id());
}
}
_query_ctx->cancel(st.to_string(), st, _fragment_id);
@ -1587,4 +1590,17 @@ std::string PipelineXFragmentContext::debug_string() {
return fmt::to_string(debug_string_buffer);
}
void PipelineXFragmentContext::close_a_pipeline(PipelineId pipeline_id) {
// If all tasks of this pipeline has been closed, upstream tasks is never needed, and we just make those runnable here
DCHECK(_pip_id_to_pipeline.contains(pipeline_id));
if (_pip_id_to_pipeline[pipeline_id]->close_task()) {
if (_dag.contains(pipeline_id)) {
for (auto dep : _dag[pipeline_id]) {
_pip_id_to_pipeline[dep]->make_all_runnable();
}
}
}
PipelineFragmentContext::close_a_pipeline(pipeline_id);
}
} // namespace doris::pipeline

View File

@ -116,6 +116,8 @@ public:
std::string debug_string() override;
void close_a_pipeline(PipelineId pipeline_id) override;
private:
void _close_fragment_instance() override;
Status _build_pipeline_x_tasks(const doris::TPipelineFragmentParams& request,
@ -222,6 +224,7 @@ private:
std::map<int, std::pair<std::shared_ptr<LocalExchangeSharedState>, std::shared_ptr<Dependency>>>
_op_id_to_le_state;
std::map<PipelineId, Pipeline*> _pip_id_to_pipeline;
// UniqueId -> runtime mgr
std::map<UniqueId, std::unique_ptr<RuntimeFilterMgr>> _runtime_filter_mgr_map;

View File

@ -66,7 +66,6 @@ PipelineXTask::PipelineXTask(
if (shared_state) {
_sink_shared_state = shared_state;
}
pipeline->incr_created_tasks();
}
Status PipelineXTask::prepare(const TPipelineInstanceParams& local_params, const TDataSink& tsink,
@ -248,7 +247,7 @@ Status PipelineXTask::execute(bool* eos) {
cpu_qs->add_cpu_nanos(delta_cpu_time);
}
}};
*eos = _sink->is_finished(_state);
*eos = _sink->is_finished(_state) || _wake_up_by_downstream || is_final_state(_cur_state);
if (*eos) {
return Status::OK();
}
@ -256,10 +255,18 @@ Status PipelineXTask::execute(bool* eos) {
set_state(PipelineTaskState::BLOCKED_FOR_DEPENDENCY);
return Status::OK();
}
if (_wake_up_by_downstream) {
*eos = true;
return Status::OK();
}
if (_runtime_filter_blocked_dependency() != nullptr) {
set_state(PipelineTaskState::BLOCKED_FOR_RF);
return Status::OK();
}
if (_wake_up_by_downstream) {
*eos = true;
return Status::OK();
}
// The status must be runnable
if (!_opened) {
{
@ -270,10 +277,18 @@ Status PipelineXTask::execute(bool* eos) {
set_state(PipelineTaskState::BLOCKED_FOR_SOURCE);
return Status::OK();
}
if (_wake_up_by_downstream) {
*eos = true;
return Status::OK();
}
if (!sink_can_write()) {
set_state(PipelineTaskState::BLOCKED_FOR_SINK);
return Status::OK();
}
if (_wake_up_by_downstream) {
*eos = true;
return Status::OK();
}
}
Status status = Status::OK();
@ -283,10 +298,18 @@ Status PipelineXTask::execute(bool* eos) {
set_state(PipelineTaskState::BLOCKED_FOR_SOURCE);
break;
}
if (_wake_up_by_downstream) {
*eos = true;
return Status::OK();
}
if (!sink_can_write()) {
set_state(PipelineTaskState::BLOCKED_FOR_SINK);
break;
}
if (_wake_up_by_downstream) {
*eos = true;
return Status::OK();
}
/// When a task is cancelled,
/// its blocking state will be cleared and it will transition to a ready state (though it is not truly ready).
@ -445,12 +468,13 @@ std::string PipelineXTask::debug_string() {
// If at the same time FE cancel this pipeline task and logging debug_string before _blocked_dep is cleared,
// it will think _blocked_dep is not nullptr and call _blocked_dep->debug_string().
auto* cur_blocked_dep = _blocked_dep;
fmt::format_to(debug_string_buffer,
"PipelineTask[this = {}, state = {}, dry run = {}, elapse time "
"= {}s], block dependency = {}, is running = {}\noperators: ",
(void*)this, get_state_name(_cur_state), _dry_run, elapsed,
cur_blocked_dep && !_finished ? cur_blocked_dep->debug_string() : "NULL",
is_running());
fmt::format_to(
debug_string_buffer,
"PipelineTask[this = {}, state = {}, dry run = {}, elapse time "
"= {}s], _wake_up_by_downstream = {}, block dependency = {}, is running = "
"{}\noperators: ",
(void*)this, get_state_name(_cur_state), _dry_run, elapsed, _wake_up_by_downstream,
cur_blocked_dep && !_finished ? cur_blocked_dep->debug_string() : "NULL", is_running());
for (size_t i = 0; i < _operators.size(); i++) {
fmt::format_to(debug_string_buffer, "\n{}",
_opened && !_finished ? _operators[i]->debug_string(_state, i)

View File

@ -138,7 +138,8 @@ public:
int task_id() const { return _index; };
void clear_blocking_state() {
void clear_blocking_state(bool wake_up_by_downstream = false) override {
_wake_up_by_downstream = _wake_up_by_downstream || wake_up_by_downstream;
_state->get_query_ctx()->get_execution_dependency()->set_always_ready();
// We use a lock to assure all dependencies are not deconstructed here.
std::unique_lock<std::mutex> lc(_dependency_lock);
@ -176,6 +177,8 @@ public:
static bool should_revoke_memory(RuntimeState* state, int64_t revocable_mem_bytes);
bool wake_up_by_downstream() const { return _wake_up_by_downstream; }
private:
friend class RuntimeFilterDependency;
Dependency* _write_blocked_dependency() {
@ -252,6 +255,7 @@ private:
std::atomic<bool> _finished {false};
std::mutex _dependency_lock;
std::atomic<bool> _wake_up_by_downstream = false;
};
} // namespace doris::pipeline

View File

@ -258,7 +258,7 @@ void _close_task(PipelineTask* task, PipelineTaskState state, Status exec_status
task->set_close_pipeline_time();
task->finalize();
task->set_running(false);
task->fragment_context()->close_a_pipeline();
task->fragment_context()->close_a_pipeline(task->pipeline_id());
}
void TaskScheduler::_do_work(size_t index) {

View File

@ -145,6 +145,7 @@ Status RuntimeFilterMgr::register_local_merge_producer_filter(
RETURN_IF_ERROR(IRuntimeFilter::create(_state, &_pool, &desc, &options,
RuntimeFilterRole::PRODUCER, -1, &merge_filter,
build_bf_exactly, true));
merge_filter->set_ignored();
iter->second.filters.emplace_back(merge_filter);
}
iter->second.merge_time++;
@ -166,7 +167,6 @@ Status RuntimeFilterMgr::get_local_merge_producer_filters(
}
*local_merge_filters = &iter->second;
DCHECK(!iter->second.filters.empty());
DCHECK_GT(iter->second.merge_time, 0);
return Status::OK();
}
@ -253,6 +253,7 @@ Status RuntimeFilterMergeControllerEntity::_init_with_desc(
auto filter_id = runtime_filter_desc->filter_id;
RETURN_IF_ERROR(cnt_val->filter->init_with_desc(&cnt_val->runtime_filter_desc, query_options,
-1, false));
cnt_val->filter->set_ignored();
_filter_map.emplace(filter_id, cnt_val);
return Status::OK();
}
@ -272,6 +273,7 @@ Status RuntimeFilterMergeControllerEntity::_init_with_desc(
new IRuntimeFilter(_state, &_state->get_query_ctx()->obj_pool, runtime_filter_desc));
auto filter_id = runtime_filter_desc->filter_id;
RETURN_IF_ERROR(cnt_val->filter->init_with_desc(&cnt_val->runtime_filter_desc, query_options));
cnt_val->filter->set_ignored();
std::unique_lock<std::shared_mutex> guard(_filter_map_mutex);
_filter_map.emplace(filter_id, cnt_val);

View File

@ -568,6 +568,10 @@ public:
: 0;
}
void set_task(pipeline::PipelineXTask* task) { _task = task; }
pipeline::PipelineXTask* get_task() const { return _task; }
inline bool enable_delete_sub_pred_v2() const {
return _query_options.__isset.enable_delete_sub_predicate_v2 &&
_query_options.enable_delete_sub_predicate_v2;
@ -784,6 +788,7 @@ private:
// prohibit copies
RuntimeState(const RuntimeState&);
pipeline::PipelineXTask* _task;
vectorized::ColumnInt64* _partial_update_auto_inc_column;
};

View File

@ -91,7 +91,7 @@ Status process_runtime_filter_build(RuntimeState* state, Block* block, Parent* p
}
{
SCOPED_TIMER(parent->_publish_runtime_filter_timer);
RETURN_IF_ERROR(parent->_runtime_filter_slots->publish());
RETURN_IF_ERROR(parent->_runtime_filter_slots->publish(false));
}
return Status::OK();

View File

@ -66,6 +66,7 @@ struct SharedHashTableContext {
std::map<int, RuntimeFilterContextSPtr> runtime_filters;
std::atomic<bool> signaled = false;
bool short_circuit_for_null_in_probe_side = false;
std::atomic<bool> complete_build_stage = false;
};
using SharedHashTableContextPtr = std::shared_ptr<SharedHashTableContext>;

View File

@ -103,11 +103,6 @@ IRuntimeFilter* create_runtime_filter(TRuntimeFilterType::type type, TQueryOptio
EXPECT_TRUE(status.ok()) << status.to_string();
if (auto bf = runtime_filter->get_bloomfilter()) {
status = bf->init_with_fixed_length();
EXPECT_TRUE(status.ok()) << status.to_string();
}
return status.ok() ? runtime_filter : nullptr;
}