## 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:
@ -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(
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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));
|
||||
|
||||
@ -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);
|
||||
}
|
||||
|
||||
|
||||
@ -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());
|
||||
}
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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
|
||||
|
||||
@ -258,7 +258,6 @@ public:
|
||||
|
||||
std::unique_ptr<RuntimeState> _runtime_state;
|
||||
|
||||
bool _eos = false;
|
||||
std::shared_ptr<Dependency> _finish_dependency;
|
||||
|
||||
// temp structures during spilling
|
||||
|
||||
@ -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;
|
||||
};
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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() {}
|
||||
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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>)
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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;
|
||||
};
|
||||
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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>;
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user