diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 28ef74079d..6113cf2cdb 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -680,6 +680,7 @@ set(DORIS_LINK_LIBS Webserver Geo Vec + Pipeline ${WL_END_GROUP} ) @@ -910,6 +911,7 @@ endif() add_subdirectory(${SRC_DIR}/util) add_subdirectory(${SRC_DIR}/vec) +add_subdirectory(${SRC_DIR}/pipeline) if (${MAKE_TEST} STREQUAL "ON") add_subdirectory(${TEST_DIR}) diff --git a/be/src/common/config.h b/be/src/common/config.h index 9faf33ef00..a5e0c4826c 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -846,6 +846,8 @@ CONF_Bool(enable_java_support, "true"); // Set config randomly to check more issues in github workflow CONF_Bool(enable_fuzzy_mode, "false"); +CONF_Int32(pipeline_executor_size, "0"); + #ifdef BE_TEST // test s3 CONF_String(test_s3_resource, "resource"); diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index 0c8842ad32..8362e89d7b 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -47,7 +47,7 @@ namespace doris { Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, const std::vector& output_exprs, const TPlanFragmentExecParams& params, - const RowDescriptor& row_desc, bool is_vec, + const RowDescriptor& row_desc, RuntimeState* state, std::unique_ptr* sink, DescriptorTbl& desc_tbl) { DataSink* tmp_sink = nullptr; @@ -61,10 +61,10 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink ? params.send_query_statistics_with_every_batch : false; // TODO: figure out good buffer size based on size of output row - if (is_vec) { - tmp_sink = new doris::vectorized::VDataStreamSender( - pool, params.sender_id, row_desc, thrift_sink.stream_sink, params.destinations, - 16 * 1024, send_query_statistics_with_every_batch); + if (state->enable_vectorized_exec()) { + tmp_sink = new vectorized::VDataStreamSender( + state, pool, params.sender_id, row_desc, thrift_sink.stream_sink, + params.destinations, 16 * 1024, send_query_statistics_with_every_batch); } else { tmp_sink = new DataStreamSender(pool, params.sender_id, row_desc, thrift_sink.stream_sink, params.destinations, 16 * 1024, @@ -80,7 +80,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink } // TODO: figure out good buffer size based on size of output row - if (is_vec) { + if (state->enable_vectorized_exec()) { tmp_sink = new doris::vectorized::VResultSink(row_desc, output_exprs, thrift_sink.result_sink, 4096); } else { @@ -95,7 +95,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink } // TODO: figure out good buffer size based on size of output row - if (is_vec) { + if (state->enable_vectorized_exec()) { bool send_query_statistics_with_every_batch = params.__isset.send_query_statistics_with_every_batch ? params.send_query_statistics_with_every_batch @@ -139,7 +139,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink if (!thrift_sink.__isset.mysql_table_sink) { return Status::InternalError("Missing data buffer sink."); } - if (is_vec) { + if (state->enable_vectorized_exec()) { doris::vectorized::VMysqlTableSink* vmysql_tbl_sink = new doris::vectorized::VMysqlTableSink(pool, row_desc, output_exprs); sink->reset(vmysql_tbl_sink); @@ -158,7 +158,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink if (!thrift_sink.__isset.odbc_table_sink) { return Status::InternalError("Missing data odbc sink."); } - if (is_vec) { + if (state->enable_vectorized_exec()) { sink->reset(new vectorized::VOdbcTableSink(pool, row_desc, output_exprs)); } else { sink->reset(new OdbcTableSink(pool, row_desc, output_exprs)); @@ -170,7 +170,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink if (!thrift_sink.__isset.jdbc_table_sink) { return Status::InternalError("Missing data jdbc sink."); } - if (is_vec) { + if (state->enable_vectorized_exec()) { if (config::enable_java_support) { sink->reset(new vectorized::VJdbcTableSink(pool, row_desc, output_exprs)); } else { @@ -196,7 +196,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink case TDataSinkType::OLAP_TABLE_SINK: { Status status; DCHECK(thrift_sink.__isset.olap_table_sink); - if (is_vec) { + if (state->enable_vectorized_exec()) { sink->reset(new stream_load::VOlapTableSink(pool, row_desc, output_exprs, &status)); } else { sink->reset(new stream_load::OlapTableSink(pool, row_desc, output_exprs, &status)); @@ -220,7 +220,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink } } - if (sink->get() != nullptr) { + if (*sink != nullptr) { RETURN_IF_ERROR((*sink)->init(thrift_sink)); } diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h index 423e60a74a..9f21bcf4a1 100644 --- a/be/src/exec/data_sink.h +++ b/be/src/exec/data_sink.h @@ -62,7 +62,7 @@ public: virtual Status send(RuntimeState* state, RowBatch* batch) = 0; // Send a Block into this sink. - virtual Status send(RuntimeState* state, vectorized::Block* block) { + virtual Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) { return Status::NotSupported("Not support send block"); }; // Releases all resources that were allocated in prepare()/send(). @@ -80,7 +80,7 @@ public: static Status create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, const std::vector& output_exprs, const TPlanFragmentExecParams& params, - const RowDescriptor& row_desc, bool is_vec, + const RowDescriptor& row_desc, RuntimeState* state, std::unique_ptr* sink, DescriptorTbl& desc_tbl); // Returns the runtime profile for the sink. diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 29462ac037..14b8690d5d 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -88,10 +88,6 @@ namespace doris { const std::string ExecNode::ROW_THROUGHPUT_COUNTER = "RowsReturnedRate"; -int ExecNode::get_node_id_from_profile(RuntimeProfile* p) { - return p->metadata(); -} - ExecNode::RowBatchQueue::RowBatchQueue(int max_batches) : BlockingQueue(max_batches) {} ExecNode::RowBatchQueue::~RowBatchQueue() { @@ -150,7 +146,8 @@ ExecNode::ExecNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl _rows_returned_rate(nullptr), _memory_used_counter(nullptr), _get_next_span(), - _is_closed(false) { + _is_closed(false), + _ref(0) { if (tnode.__isset.output_tuple_id) { _output_row_descriptor.reset(new RowDescriptor(descs, {tnode.output_tuple_id}, {true})); } @@ -251,7 +248,7 @@ Status ExecNode::prepare(RuntimeState* state) { return Status::OK(); } -Status ExecNode::open(RuntimeState* state) { +Status ExecNode::alloc_resource(doris::RuntimeState* state) { SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); if (_vconjunct_ctx_ptr) { RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->open(state)); @@ -264,6 +261,10 @@ Status ExecNode::open(RuntimeState* state) { } } +Status ExecNode::open(RuntimeState* state) { + return alloc_resource(state); +} + Status ExecNode::reset(RuntimeState* state) { _num_rows_returned = 0; for (int i = 0; i < _children.size(); ++i) { @@ -280,16 +281,35 @@ Status ExecNode::collect_query_statistics(QueryStatistics* statistics) { return Status::OK(); } +void ExecNode::release_resource(doris::RuntimeState* state) { + if (!_is_resource_released) { + if (_rows_returned_counter != nullptr) { + COUNTER_SET(_rows_returned_counter, _num_rows_returned); + } + + if (_vconjunct_ctx_ptr) { + (*_vconjunct_ctx_ptr)->close(state); + } + if (typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { + Expr::close(_conjunct_ctxs, state); + } + vectorized::VExpr::close(_projections, state); + + if (_buffer_pool_client.is_registered()) { + state->exec_env()->buffer_pool()->DeregisterClient(&_buffer_pool_client); + } + + runtime_profile()->add_to_span(); + _is_resource_released = true; + } +} + Status ExecNode::close(RuntimeState* state) { if (_is_closed) { return Status::OK(); } _is_closed = true; - if (_rows_returned_counter != nullptr) { - COUNTER_SET(_rows_returned_counter, _num_rows_returned); - } - Status result; for (int i = 0; i < _children.size(); ++i) { auto st = _children[i]->close(state); @@ -297,21 +317,7 @@ Status ExecNode::close(RuntimeState* state) { result = st; } } - - if (_vconjunct_ctx_ptr) { - (*_vconjunct_ctx_ptr)->close(state); - } - if (typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { - Expr::close(_conjunct_ctxs, state); - } - vectorized::VExpr::close(_projections, state); - - if (_buffer_pool_client.is_registered()) { - state->exec_env()->buffer_pool()->DeregisterClient(&_buffer_pool_client); - } - - runtime_profile()->add_to_span(); - + release_resource(state); return result; } @@ -855,4 +861,17 @@ Status ExecNode::get_next_after_projects(RuntimeState* state, vectorized::Block* return get_next(state, block, eos); } +Status ExecNode::execute(RuntimeState* state, vectorized::Block* input_block, + vectorized::Block* output_block, bool* eos) { + return Status::NotSupported("{} not implements execute", get_name()); +} + +Status ExecNode::pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) { + return Status::NotSupported("{} not implements pull", get_name()); +} + +Status ExecNode::sink(RuntimeState* state, vectorized::Block* input_block, bool eos) { + return Status::NotSupported("{} not implements sink", get_name()); +} + } // namespace doris diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index 7f5ba74656..4d02c7df45 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -52,6 +52,12 @@ class Block; class VExpr; } // namespace vectorized +namespace pipeline { +class PipelineFragmentContext; +class Pipeline; +class Operator; +} // namespace pipeline + using std::string; using std::stringstream; using std::vector; @@ -89,6 +95,11 @@ public: // Caller must not be holding any io buffers. This will cause deadlock. virtual Status open(RuntimeState* state); + // Alloc and open resource for the node + // Only pipeline operator use exec node need to impl the virtual function + // so only vectorized exec node need to impl + virtual Status alloc_resource(RuntimeState* state); + // Retrieves rows and returns them via row_batch. Sets eos to true // if subsequent calls will not retrieve any more rows. // Data referenced by any tuples returned in row_batch must not be overwritten @@ -104,10 +115,29 @@ public: // TODO: AggregationNode and HashJoinNode cannot be "re-opened" yet. virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); virtual Status get_next(RuntimeState* state, vectorized::Block* block, bool* eos); - // new interface to compatible new optimizers in FE Status get_next_after_projects(RuntimeState* state, vectorized::Block* block, bool* eos); + // Process data + // Eg: Projection, Union All, HashProbe + virtual Status execute(RuntimeState* state, vectorized::Block* input_block, + vectorized::Block* output_block, bool* eos); + + // Emit data, both need impl with method: sink + // Eg: Aggregation, Sort + virtual Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos); + + virtual Status push(RuntimeState* state, vectorized::Block* input_block, bool eos) { + return Status::OK(); + } + + bool can_read() const { return _can_read; } + + // Sink Data to ExecNode to do some stock work, both need impl with method: get_result + // `eos` means source is exhausted, exec node should do some finalize work + // Eg: Aggregation, Sort + virtual Status sink(RuntimeState* state, vectorized::Block* input_block, bool eos); + // Resets the stream of row batches to be retrieved by subsequent GetNext() calls. // Clears all internal state, returning this node to the state it was in after calling // Prepare() and before calling Open(). This function must not clear memory @@ -140,6 +170,14 @@ public: // each implementation should start out by calling the default implementation. virtual Status close(RuntimeState* state); + void increase_ref() { ++_ref; } + int decrease_ref() { return --_ref; } + + // Release and close resource for the node + // Only pipeline operator use exec node need to impl the virtual function + // so only vectorized exec node need to impl + virtual void release_resource(RuntimeState* state); + // Creates exec node tree from list of nodes contained in plan via depth-first // traversal. All nodes are placed in pool. // Returns error if 'plan' is corrupted, otherwise success. @@ -202,14 +240,14 @@ public: virtual std::string get_name(); - // Extract node id from p->name(). - static int get_node_id_from_profile(RuntimeProfile* p); - // Names of counters shared by all exec nodes static const std::string ROW_THROUGHPUT_COUNTER; + ExecNode* child(int i) { return _children[i]; } + protected: friend class DataSink; + friend class doris::pipeline::Operator; /// Initialize 'buffer_pool_client_' and claim the initial reservation for this /// ExecNode. Only needs to be called by ExecNodes that will use the client. @@ -335,8 +373,6 @@ protected: // Set to true if this is a vectorized exec node. bool _is_vec = false; - ExecNode* child(int i) { return _children[i]; } - bool is_closed() const { return _is_closed; } // TODO(zc) @@ -372,9 +408,13 @@ protected: /// allocations. ExecNodes overriding this function should return /// ExecNode::QueryMaintenance(). virtual Status QueryMaintenance(RuntimeState* state, const std::string& msg) WARN_UNUSED_RESULT; + std::atomic _can_read = false; private: + friend class pipeline::Operator; bool _is_closed; + bool _is_resource_released = false; + std::atomic_int _ref; // used by pipeline operator to release resource. }; } // namespace doris diff --git a/be/src/exec/table_function_node.cpp b/be/src/exec/table_function_node.cpp index 930e484d2a..3c1a26cf74 100644 --- a/be/src/exec/table_function_node.cpp +++ b/be/src/exec/table_function_node.cpp @@ -17,6 +17,7 @@ #include "exec/table_function_node.h" +#include "exec/exec_node.h" #include "exprs/expr.h" #include "exprs/expr_context.h" #include "exprs/table_function/table_function_factory.h" @@ -33,7 +34,7 @@ TableFunctionNode::TableFunctionNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : ExecNode(pool, tnode, descs) {} -TableFunctionNode::~TableFunctionNode() {} +TableFunctionNode::~TableFunctionNode() = default; Status TableFunctionNode::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(ExecNode::init(tnode, state)); @@ -101,13 +102,13 @@ Status TableFunctionNode::prepare(RuntimeState* state) { return Status::OK(); } -Status TableFunctionNode::open(RuntimeState* state) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "TableFunctionNode::open"); +Status TableFunctionNode::alloc_resource(RuntimeState* state) { SCOPED_TIMER(_runtime_profile->total_time_counter()); RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(ExecNode::open(state)); SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); + ExecNode::alloc_resource(state); + RETURN_IF_ERROR(Expr::open(_fn_ctxs, state)); RETURN_IF_ERROR(vectorized::VExpr::open(_vfn_ctxs, state)); @@ -115,7 +116,6 @@ Status TableFunctionNode::open(RuntimeState* state) { RETURN_IF_ERROR(fn->open()); } - RETURN_IF_ERROR(_children[0]->open(state)); return Status::OK(); } @@ -381,19 +381,4 @@ Status TableFunctionNode::get_next(RuntimeState* state, RowBatch* row_batch, boo return Status::OK(); } -Status TableFunctionNode::close(RuntimeState* state) { - if (is_closed()) { - return Status::OK(); - } - START_AND_SCOPE_SPAN(state->get_tracer(), span, "TableFunctionNode::close"); - Expr::close(_fn_ctxs, state); - vectorized::VExpr::close(_vfn_ctxs, state); - - if (_num_rows_filtered_counter != nullptr) { - COUNTER_SET(_num_rows_filtered_counter, static_cast(_num_rows_filtered)); - } - - return ExecNode::close(state); -} - }; // namespace doris diff --git a/be/src/exec/table_function_node.h b/be/src/exec/table_function_node.h index 06de804c3c..ece29bd50e 100644 --- a/be/src/exec/table_function_node.h +++ b/be/src/exec/table_function_node.h @@ -18,6 +18,8 @@ #pragma once #include "exec/exec_node.h" +#include "exprs/expr.h" +#include "vec/exprs/vexpr.h" namespace doris { @@ -30,13 +32,26 @@ class TupleRow; class TableFunctionNode : public ExecNode { public: TableFunctionNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - ~TableFunctionNode(); + ~TableFunctionNode() override; Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; Status prepare(RuntimeState* state) override; - Status open(RuntimeState* state) override; + Status open(RuntimeState* state) override { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "TableFunctionNode::open"); + RETURN_IF_ERROR(alloc_resource(state)); + return _children[0]->open(state); + } Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override; - Status close(RuntimeState* state) override; + Status alloc_resource(RuntimeState* state) override; + void release_resource(doris::RuntimeState* state) override { + Expr::close(_fn_ctxs, state); + vectorized::VExpr::close(_vfn_ctxs, state); + + if (_num_rows_filtered_counter != nullptr) { + COUNTER_SET(_num_rows_filtered_counter, static_cast(_num_rows_filtered)); + } + ExecNode::release_resource(state); + } protected: Status _prepare_output_slot_ids(const TPlanNode& tnode); diff --git a/be/src/pipeline/CMakeLists.txt b/be/src/pipeline/CMakeLists.txt new file mode 100644 index 0000000000..71060233ef --- /dev/null +++ b/be/src/pipeline/CMakeLists.txt @@ -0,0 +1,48 @@ +# 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. + +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/pipeline") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/pipeline") + +set(PIPELINE_FILES + pipeline.cpp + pipeline_fragment_context.cpp + pipeline_task.cpp + task_scheduler.cpp + exec/operator.cpp + exec/scan_operator.cpp + exec/olap_scan_operator.cpp + exec/empty_set_operator.cpp + exec/exchange_source_operator.cpp + exec/exchange_sink_operator.cpp + exec/exchange_sink_buffer.cpp + exec/result_sink_operator.cpp + exec/aggregation_sink_operator.cpp + exec/aggregation_source_operator.cpp + exec/streaming_aggregation_source_operator.cpp + exec/streaming_aggregation_sink_operator.cpp + exec/agg_context.cpp + exec/sort_source_operator.cpp + exec/sort_sink_operator.cpp + exec/repeat_operator.cpp) + +add_library(Pipeline STATIC + ${PIPELINE_FILES} + ) \ No newline at end of file diff --git a/be/src/pipeline/exec/agg_context.cpp b/be/src/pipeline/exec/agg_context.cpp new file mode 100644 index 0000000000..9c6aa8ecd9 --- /dev/null +++ b/be/src/pipeline/exec/agg_context.cpp @@ -0,0 +1,106 @@ +// 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 "agg_context.h" + +#include "runtime/descriptors.h" +#include "vec/columns/column_nullable.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type_nullable.h" + +namespace doris { +namespace pipeline { + +std::unique_ptr AggContext::get_free_block() { + { + std::lock_guard l(_free_blocks_lock); + if (!_free_blocks.empty()) { + auto block = std::move(_free_blocks.back()); + _free_blocks.pop_back(); + return block; + } + } + + return std::make_unique(); +} + +void AggContext::return_free_block(std::unique_ptr block) { + DCHECK(block->rows() == 0); + std::lock_guard l(_free_blocks_lock); + _free_blocks.emplace_back(std::move(block)); +} + +bool AggContext::has_data_or_finished() { + std::unique_lock l(_transfer_lock); + return !_blocks_queue.empty() || _is_finished; +} + +Status AggContext::get_block(std::unique_ptr* block) { + std::unique_lock l(_transfer_lock); + if (_is_canceled) { + return Status::InternalError("AggContext canceled"); + } + if (!_blocks_queue.empty()) { + auto [block_ptr, block_size] = std::move(_blocks_queue.front()); + *block = std::move(block_ptr); + _blocks_queue.pop_front(); + _cur_bytes_in_queue -= block_size; + } else { + if (_is_finished) { + _data_exhausted = true; + } + } + return Status::OK(); +} + +bool AggContext::has_enough_space_to_push() { + std::unique_lock l(_transfer_lock); + return _cur_bytes_in_queue < MAX_BYTE_OF_QUEUE / 2; +} + +void AggContext::push_block(std::unique_ptr block) { + if (!block) { + return; + } + auto block_size = block->allocated_bytes(); + std::unique_lock l(_transfer_lock); + _blocks_queue.emplace_back(std::move(block), block_size); + _cur_bytes_in_queue += block_size; + + _max_bytes_in_queue = std::max(_max_bytes_in_queue, _cur_bytes_in_queue); + _max_size_of_queue = std::max(_max_size_of_queue, (int64)_blocks_queue.size()); +} + +void AggContext::set_finish() { + std::unique_lock l(_transfer_lock); + _is_finished = true; +} + +void AggContext::set_canceled() { + std::unique_lock l(_transfer_lock); + DCHECK(!_is_finished); + _is_canceled = true; + _is_finished = true; +} + +bool AggContext::is_finish() { + std::unique_lock l(_transfer_lock); + return _is_finished; +} + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/agg_context.h b/be/src/pipeline/exec/agg_context.h new file mode 100644 index 0000000000..42bfe81a33 --- /dev/null +++ b/be/src/pipeline/exec/agg_context.h @@ -0,0 +1,72 @@ +// 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. +#pragma once + +#include "common/status.h" + +namespace doris { +class TupleDescriptor; +namespace vectorized { +class Block; +} +namespace pipeline { + +class AggContext { +public: + AggContext() = default; + ~AggContext() { DCHECK(_is_finished); } + + std::unique_ptr get_free_block(); + + void return_free_block(std::unique_ptr); + + bool has_data_or_finished(); + Status get_block(std::unique_ptr* block); + + bool has_enough_space_to_push(); + void push_block(std::unique_ptr); + + void set_finish(); + void set_canceled(); // should set before finish + bool is_finish(); + + bool data_exhausted() const { return _data_exhausted; } + + int64_t max_bytes_in_queue() const { return _max_bytes_in_queue; } + + int64_t max_size_of_queue() const { return _max_size_of_queue; } + +private: + std::mutex _free_blocks_lock; + std::vector> _free_blocks; + + std::mutex _transfer_lock; + std::list, size_t>> _blocks_queue; + + bool _data_exhausted = false; + bool _is_finished = false; + bool _is_canceled = false; + + // int64_t just for counter of profile + int64_t _cur_bytes_in_queue = 0; + int64_t _max_bytes_in_queue = 0; + int64_t _max_size_of_queue = 0; + static constexpr int64_t MAX_BYTE_OF_QUEUE = 1024l * 1024 * 1024 / 10; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp new file mode 100644 index 0000000000..d9459bf56d --- /dev/null +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -0,0 +1,77 @@ +// 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 "aggregation_sink_operator.h" + +#include "vec/exec/vaggregation_node.h" + +namespace doris::pipeline { + +AggSinkOperator::AggSinkOperator(AggSinkOperatorBuilder* operator_builder, + vectorized::AggregationNode* agg_node) + : Operator(operator_builder), _agg_node(agg_node) {} + +Status AggSinkOperator::prepare(RuntimeState* state) { + RETURN_IF_ERROR(Operator::prepare(state)); + _agg_node->increase_ref(); + return Status::OK(); +} + +Status AggSinkOperator::open(RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(Operator::open(state)); + RETURN_IF_ERROR(_agg_node->alloc_resource(state)); + return Status::OK(); +} + +bool AggSinkOperator::can_write() { + return true; +} + +Status AggSinkOperator::sink(RuntimeState* state, vectorized::Block* in_block, + SourceState source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + return _agg_node->sink(state, in_block, source_state == SourceState::FINISHED); +} + +Status AggSinkOperator::close(RuntimeState* state) { + _fresh_exec_timer(_agg_node); + if (!_agg_node->decrease_ref()) { + _agg_node->release_resource(state); + } + return Status::OK(); +} + +/////////////////////////////// operator template //////////////////////////////// + +AggSinkOperatorBuilder::AggSinkOperatorBuilder(int32_t id, const std::string& name, + vectorized::AggregationNode* exec_node) + : OperatorBuilder(id, name, exec_node), _agg_node(exec_node) {} + +OperatorPtr AggSinkOperatorBuilder::build_operator() { + return std::make_shared(this, _agg_node); +} + +// use final aggregation source operator +bool AggSinkOperatorBuilder::is_sink() const { + return true; +} + +bool AggSinkOperatorBuilder::is_source() const { + return false; +} +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h new file mode 100644 index 0000000000..fc74f2366d --- /dev/null +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -0,0 +1,65 @@ +// 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. + +#pragma once + +#include "agg_context.h" +#include "operator.h" + +namespace doris { +namespace vectorized { +class AggregationNode; +class VExprContext; +class Block; +} // namespace vectorized + +namespace pipeline { +class AggSinkOperatorBuilder; +class AggSinkOperator : public Operator { +public: + AggSinkOperator(AggSinkOperatorBuilder* operator_builder, vectorized::AggregationNode*); + + Status prepare(RuntimeState*) override; + Status open(RuntimeState* state) override; + + Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override; + + bool can_write() override; + + Status close(RuntimeState* state) override; + + Status finalize(doris::RuntimeState* state) override { return Status::OK(); } + +private: + vectorized::AggregationNode* _agg_node; +}; + +class AggSinkOperatorBuilder : public OperatorBuilder { +public: + AggSinkOperatorBuilder(int32_t, const std::string&, vectorized::AggregationNode*); + + OperatorPtr build_operator() override; + + bool is_sink() const override; + bool is_source() const override; + +private: + vectorized::AggregationNode* _agg_node; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp new file mode 100644 index 0000000000..a0dc95615f --- /dev/null +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -0,0 +1,67 @@ +// 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 "aggregation_source_operator.h" + +#include "vec/exec/vaggregation_node.h" + +namespace doris { +namespace pipeline { + +AggregationSourceOperator::AggregationSourceOperator(OperatorBuilder* templ, + vectorized::AggregationNode* node) + : Operator(templ), _agg_node(node) {} + +Status AggregationSourceOperator::prepare(RuntimeState* state) { + _agg_node->increase_ref(); + return Status::OK(); +} + +bool AggregationSourceOperator::can_read() { + return _agg_node->can_read(); +} + +Status AggregationSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + bool eos = false; + RETURN_IF_ERROR(_agg_node->pull(state, block, &eos)); + source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; + return Status::OK(); +} + +Status AggregationSourceOperator::close(RuntimeState* state) { + _fresh_exec_timer(_agg_node); + if (!_agg_node->decrease_ref()) { + _agg_node->release_resource(state); + } + return Status::OK(); +} + +/////////////////////////////// operator template //////////////////////////////// + +AggregationSourceOperatorBuilder::AggregationSourceOperatorBuilder( + int32_t id, const std::string& name, vectorized::AggregationNode* exec_node) + : OperatorBuilder(id, name, exec_node) {} + +OperatorPtr AggregationSourceOperatorBuilder::build_operator() { + return std::make_shared( + this, assert_cast(_related_exec_node)); +} + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h new file mode 100644 index 0000000000..1c611d163d --- /dev/null +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -0,0 +1,51 @@ +// 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. +#pragma once + +#include "operator.h" + +namespace doris { +namespace vectorized { +class AggregationNode; +} + +namespace pipeline { + +// For read none streaming agg sink operator's data +class AggregationSourceOperator : public Operator { +public: + AggregationSourceOperator(OperatorBuilder*, vectorized::AggregationNode*); + Status prepare(RuntimeState* state) override; + bool can_read() override; + Status close(RuntimeState* state) override; + Status get_block(RuntimeState*, vectorized::Block*, SourceState&) override; + +private: + vectorized::AggregationNode* _agg_node; +}; + +class AggregationSourceOperatorBuilder : public OperatorBuilder { +public: + AggregationSourceOperatorBuilder(int32_t, const std::string&, vectorized::AggregationNode*); + + bool is_source() const override { return true; } + + OperatorPtr build_operator() override; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/empty_set_operator.cpp b/be/src/pipeline/exec/empty_set_operator.cpp new file mode 100644 index 0000000000..a0913766cd --- /dev/null +++ b/be/src/pipeline/exec/empty_set_operator.cpp @@ -0,0 +1,44 @@ +// 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 "empty_set_operator.h" + +#include "vec/exec/vempty_set_node.h" + +namespace doris::pipeline { + +EmptySetSourceOperator::EmptySetSourceOperator(EmptySetSourceOperatorBuilder* operator_builder, + vectorized::VEmptySetNode* empty_set_node) + : Operator(operator_builder), _empty_set_node(empty_set_node) {} + +bool EmptySetSourceOperator::can_read() { + return true; +} + +Status EmptySetSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + bool eos = false; + RETURN_IF_ERROR(_empty_set_node->get_next(state, block, &eos)); + source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; + return Status::OK(); +} + +EmptySetSourceOperatorBuilder::EmptySetSourceOperatorBuilder( + int32_t id, const string& name, vectorized::VEmptySetNode* empty_set_node) + : OperatorBuilder(id, name, empty_set_node), _empty_set_node(empty_set_node) {} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/empty_set_operator.h b/be/src/pipeline/exec/empty_set_operator.h new file mode 100644 index 0000000000..90af89dbcb --- /dev/null +++ b/be/src/pipeline/exec/empty_set_operator.h @@ -0,0 +1,61 @@ +// 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. + +#pragma once + +#include "operator.h" + +namespace doris { + +namespace vectorized { +class VEmptySetNode; +} + +namespace pipeline { + +class EmptySetSourceOperatorBuilder; + +class EmptySetSourceOperator : public Operator { +public: + EmptySetSourceOperator(EmptySetSourceOperatorBuilder* operator_builder, + vectorized::VEmptySetNode* empty_set_node); + Status get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) override; + + bool can_read() override; + +private: + vectorized::VEmptySetNode* _empty_set_node; +}; + +class EmptySetSourceOperatorBuilder : public OperatorBuilder { +public: + EmptySetSourceOperatorBuilder(int32_t id, const std::string& name, + vectorized::VEmptySetNode* empty_set_node); + + bool is_source() const override { return true; } + + OperatorPtr build_operator() override { + return std::make_shared(this, _empty_set_node); + } + +private: + vectorized::VEmptySetNode* _empty_set_node; +}; + +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp new file mode 100644 index 0000000000..2ee2db1c9c --- /dev/null +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -0,0 +1,236 @@ +// 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 "exchange_sink_buffer.h" + +#include + +#include +#include + +#include "common/status.h" +#include "pipeline/pipeline_fragment_context.h" +#include "service/brpc.h" +#include "util/proto_util.h" +#include "util/time.h" +#include "vec/sink/vdata_stream_sender.h" + +namespace doris::pipeline { +template +class SelfDeleteClosure : public google::protobuf::Closure { +public: + SelfDeleteClosure(InstanceLoId id, bool eos) : _id(id), _eos(eos) {} + ~SelfDeleteClosure() override = default; + SelfDeleteClosure(const SelfDeleteClosure& other) = delete; + SelfDeleteClosure& operator=(const SelfDeleteClosure& other) = delete; + void addFailedHandler(std::function fail_fn) { + _fail_fn = std::move(fail_fn); + } + void addSuccessHandler(std::function suc_fn) { + _suc_fn = suc_fn; + } + + void Run() noexcept override { + std::unique_ptr self_guard(this); + try { + if (cntl.Failed()) { + std::string err = fmt::format( + "failed to send brpc when exchange, error={}, error_text={}, client: {}, " + "latency = {}", + berror(cntl.ErrorCode()), cntl.ErrorText(), BackendOptions::get_localhost(), + cntl.latency_us()); + _fail_fn(_id, err); + } else { + _suc_fn(_id, _eos, result); + } + } catch (const std::exception& exp) { + LOG(FATAL) << "brpc callback error: " << exp.what(); + } catch (...) { + LOG(FATAL) << "brpc callback error."; + } + } + +public: + brpc::Controller cntl; + T result; + +private: + std::function _fail_fn; + std::function _suc_fn; + InstanceLoId _id; + bool _eos; +}; + +ExchangeSinkBuffer::ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, int send_id, + int be_number, PipelineFragmentContext* context) + : _is_finishing(false), + _query_id(query_id), + _dest_node_id(dest_node_id), + _sender_id(send_id), + _be_number(be_number), + _context(context) {} + +ExchangeSinkBuffer::~ExchangeSinkBuffer() = default; + +void ExchangeSinkBuffer::close() { + for (const auto& pair : _instance_to_request) { + if (pair.second) { + pair.second->release_finst_id(); + pair.second->release_query_id(); + } + } +} + +bool ExchangeSinkBuffer::can_write() const { + size_t max_package_size = 64 * _instance_to_package_queue.size(); + size_t total_package_size = 0; + for (auto& [_, q] : _instance_to_package_queue) { + total_package_size += q.size(); + } + return total_package_size <= max_package_size; +} + +bool ExchangeSinkBuffer::is_pending_finish() const { + for (auto& pair : _instance_to_package_queue_mutex) { + std::unique_lock lock(*(pair.second)); + auto& id = pair.first; + if (!_instance_to_sending_by_pipeline.at(id)) { + return true; + } + } + return false; +} + +void ExchangeSinkBuffer::register_sink(TUniqueId fragment_instance_id) { + if (_is_finishing) { + return; + } + auto low_id = fragment_instance_id.lo; + if (_instance_to_package_queue_mutex.count(low_id)) { + return; + } + _instance_to_package_queue_mutex[low_id] = std::make_unique(); + _instance_to_seq[low_id] = 0; + _instance_to_package_queue[low_id] = std::queue>(); + PUniqueId finst_id; + finst_id.set_hi(fragment_instance_id.hi); + finst_id.set_lo(fragment_instance_id.lo); + _instance_to_finst_id[low_id] = finst_id; + _instance_to_sending_by_pipeline[low_id] = true; +} + +Status ExchangeSinkBuffer::add_block(TransmitInfo&& request) { + if (_is_finishing) { + return Status::OK(); + } + TUniqueId ins_id = request.channel->_fragment_instance_id; + bool send_now = false; + { + std::unique_lock lock(*_instance_to_package_queue_mutex[ins_id.lo]); + // Do not have in process rpc, directly send + if (_instance_to_sending_by_pipeline[ins_id.lo]) { + send_now = true; + _instance_to_sending_by_pipeline[ins_id.lo] = false; + } + _instance_to_package_queue[ins_id.lo].emplace(std::move(request)); + } + if (send_now) { + RETURN_IF_ERROR(_send_rpc(ins_id.lo)); + } + + return Status::OK(); +} + +Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { + std::unique_lock lock(*_instance_to_package_queue_mutex[id]); + + std::queue>& q = _instance_to_package_queue[id]; + if (q.empty() || _is_finishing) { + _instance_to_sending_by_pipeline[id] = true; + return Status::OK(); + } + + TransmitInfo& request = q.front(); + + if (!_instance_to_request[id]) { + _construct_request(id); + } + + auto& brpc_request = _instance_to_request[id]; + brpc_request->set_eos(request.eos); + brpc_request->set_packet_seq(_instance_to_seq[id]++); + if (request.block) { + brpc_request->set_allocated_block(request.block.get()); + } + + auto* _closure = new SelfDeleteClosure(id, request.eos); + _closure->cntl.set_timeout_ms(request.channel->_brpc_timeout_ms); + _closure->addFailedHandler( + [&](const InstanceLoId& id, const std::string& err) { _failed(id, err); }); + _closure->addSuccessHandler([&](const InstanceLoId& id, const bool& eos, + const PTransmitDataResult& result) { + Status s = Status(result.status()); + if (!s.ok()) { + _failed(id, + fmt::format("exchange req success but status isn't ok: {}", s.get_error_msg())); + } else if (eos) { + _ended(id); + } else { + _send_rpc(id); + } + }); + + { + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); + if (enable_http_send_block(*brpc_request)) { + RETURN_IF_ERROR(transmit_block_http(_context->get_runtime_state(), _closure, + *brpc_request, request.channel->_brpc_dest_addr)); + } else { + transmit_block(*request.channel->_brpc_stub, _closure, *brpc_request); + } + } + + if (request.block) { + brpc_request->release_block(); + } + q.pop(); + + return Status::OK(); +} + +void ExchangeSinkBuffer::_construct_request(InstanceLoId id) { + _instance_to_request[id] = std::make_unique(); + _instance_to_request[id]->set_allocated_finst_id(&_instance_to_finst_id[id]); + _instance_to_request[id]->set_allocated_query_id(&_query_id); + + _instance_to_request[id]->set_node_id(_dest_node_id); + _instance_to_request[id]->set_sender_id(_sender_id); + _instance_to_request[id]->set_be_number(_be_number); +} + +void ExchangeSinkBuffer::_ended(InstanceLoId id) { + std::unique_lock lock(*_instance_to_package_queue_mutex[id]); + _instance_to_sending_by_pipeline[id] = true; +} + +void ExchangeSinkBuffer::_failed(InstanceLoId id, const std::string& err) { + _is_finishing = true; + _context->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, err); + _ended(id); +}; + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h new file mode 100644 index 0000000000..1eba3cc25b --- /dev/null +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -0,0 +1,86 @@ +// 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. + +#pragma once + +#include + +#include +#include +#include + +#include "gen_cpp/Types_types.h" +#include "gen_cpp/internal_service.pb.h" +#include "runtime/runtime_state.h" + +namespace doris { +namespace vectorized { +class PipChannel; +} + +namespace pipeline { +using InstanceLoId = int64_t; +struct TransmitInfo { + vectorized::PipChannel* channel; + std::unique_ptr block; + bool eos; +}; + +class PipelineFragmentContext; + +// Each ExchangeSinkOperator have one ExchangeSinkBuffer +class ExchangeSinkBuffer { +public: + ExchangeSinkBuffer(PUniqueId, int, PlanNodeId, int, PipelineFragmentContext*); + ~ExchangeSinkBuffer(); + void register_sink(TUniqueId); + Status add_block(TransmitInfo&& request); + bool can_write() const; + bool is_pending_finish() const; + void close(); + +private: + phmap::flat_hash_map> + _instance_to_package_queue_mutex; + phmap::flat_hash_map>> + _instance_to_package_queue; + using PackageSeq = int64_t; + // must init zero + phmap::flat_hash_map _instance_to_seq; + phmap::flat_hash_map> _instance_to_request; + phmap::flat_hash_map _instance_to_finst_id; + phmap::flat_hash_map _instance_to_sending_by_pipeline; + + std::atomic _is_finishing; + PUniqueId _query_id; + PlanNodeId _dest_node_id; + // Sender instance id, unique within a fragment. StreamSender save the variable + int _sender_id; + int _be_number; + + PipelineFragmentContext* _context; + +private: + Status _send_rpc(InstanceLoId); + // must hold the _instance_to_package_queue_mutex[id] mutex to opera + void _construct_request(InstanceLoId id); + inline void _ended(InstanceLoId id); + inline void _failed(InstanceLoId id, const std::string& err); +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp new file mode 100644 index 0000000000..7179f11d30 --- /dev/null +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -0,0 +1,97 @@ +// 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 "exchange_sink_operator.h" + +#include "common/status.h" +#include "exchange_sink_buffer.h" +#include "gen_cpp/internal_service.pb.h" +#include "util/brpc_client_cache.h" +#include "vec/exprs/vexpr.h" +#include "vec/runtime/vpartition_info.h" +#include "vec/sink/vdata_stream_sender.h" + +namespace doris::pipeline { + +ExchangeSinkOperator::ExchangeSinkOperator(OperatorBuilder* operator_builder, + vectorized::VDataStreamSender* sink, + PipelineFragmentContext* context) + : Operator(operator_builder), _sink(sink), _context(context) {} + +ExchangeSinkOperator::~ExchangeSinkOperator() = default; + +Status ExchangeSinkOperator::init(ExecNode* exec_node, RuntimeState* state) { + RETURN_IF_ERROR(Operator::init(exec_node, state)); + _state = state; + return Status::OK(); +} + +Status ExchangeSinkOperator::init(const TDataSink& tsink) { + RETURN_IF_ERROR(_sink->init(tsink)); + + PUniqueId query_id; + query_id.set_hi(_state->query_id().hi); + query_id.set_lo(_state->query_id().lo); + _sink_buffer = + std::make_unique(query_id, tsink.stream_sink.dest_node_id, + _sink->_sender_id, _state->be_number(), _context); + return Status::OK(); +} + +Status ExchangeSinkOperator::prepare(RuntimeState* state) { + RETURN_IF_ERROR(Operator::prepare(state)); + RETURN_IF_ERROR(_sink->prepare(state)); + _sink->profile()->add_child(_runtime_profile.get(), true, nullptr); + + _sink->registe_channels(_sink_buffer.get()); + return Status::OK(); +} + +Status ExchangeSinkOperator::open(RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(_sink->open(state)); + return Status::OK(); +} + +bool ExchangeSinkOperator::can_write() { + return _sink_buffer->can_write() && _sink->channel_all_can_write(); +} + +Status ExchangeSinkOperator::finalize(RuntimeState* state) { + Status result = Status::OK(); + RETURN_IF_ERROR(_sink->close(state, result)); + return result; +} + +Status ExchangeSinkOperator::sink(RuntimeState* state, vectorized::Block* block, + SourceState source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(_sink->send(state, block, source_state == SourceState::FINISHED)); + return Status::OK(); +} + +bool ExchangeSinkOperator::is_pending_finish() const { + return _sink_buffer->is_pending_finish(); +} + +Status ExchangeSinkOperator::close(RuntimeState* state) { + _sink_buffer->close(); + RETURN_IF_ERROR(Operator::close(state)); + return Status::OK(); +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h new file mode 100644 index 0000000000..481baf1868 --- /dev/null +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -0,0 +1,75 @@ +// 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. + +#pragma once + +#include "exchange_sink_buffer.h" +#include "operator.h" +#include "vec/sink/vdata_stream_sender.h" + +namespace doris { + +namespace pipeline { +class PipelineFragmentContext; + +// Now local exchange is not supported since VDataStreamRecvr is considered as a pipeline broker. +class ExchangeSinkOperator : public Operator { +public: + ExchangeSinkOperator(OperatorBuilder* operator_builder, vectorized::VDataStreamSender* sink, + PipelineFragmentContext* context); + ~ExchangeSinkOperator() override; + Status init(ExecNode* exec_node, RuntimeState* state = nullptr) override; + Status init(const TDataSink& tsink) override; + + Status prepare(RuntimeState* state) override; + Status open(RuntimeState* state) override; + bool can_write() override; + Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override; + bool is_pending_finish() const override; + Status finalize(RuntimeState* state) override; + + Status close(RuntimeState* state) override; + + RuntimeState* state() { return _state; } + +private: + std::unique_ptr _sink_buffer; + vectorized::VDataStreamSender* _sink; + RuntimeState* _state = nullptr; + PipelineFragmentContext* _context; +}; + +class ExchangeSinkOperatorBuilder : public OperatorBuilder { +public: + ExchangeSinkOperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node, + vectorized::VDataStreamSender* sink, + PipelineFragmentContext* context) + : OperatorBuilder(id, name, exec_node), _sink(sink), _context(context) {} + + bool is_sink() const override { return true; } + + OperatorPtr build_operator() override { + return std::make_shared(this, _sink, _context); + } + +private: + vectorized::VDataStreamSender* _sink; + PipelineFragmentContext* _context; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp new file mode 100644 index 0000000000..746d87e107 --- /dev/null +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -0,0 +1,63 @@ +// 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 "exchange_source_operator.h" + +#include "common/status.h" +#include "vec/exec/vexchange_node.h" +#include "vec/runtime/vdata_stream_recvr.h" + +namespace doris::pipeline { + +ExchangeSourceOperator::ExchangeSourceOperator(OperatorBuilder* operator_builder, + vectorized::VExchangeNode* node) + : Operator(operator_builder), _exchange_node(node) {} + +Status ExchangeSourceOperator::open(RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + return _exchange_node->alloc_resource(state); +} + +bool ExchangeSourceOperator::can_read() { + return _exchange_node->_stream_recvr->ready_to_read(); +} + +Status ExchangeSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + bool eos = false; + auto st = _exchange_node->get_next(state, block, &eos); + source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; + return st; +} + +bool ExchangeSourceOperator::is_pending_finish() const { + // TODO HappenLee + return false; +} + +Status ExchangeSourceOperator::close(RuntimeState* state) { + if (is_closed()) { + return Status::OK(); + } + _fresh_exec_timer(_exchange_node); + _exchange_node->release_resource(state); + + return Operator::close(state); +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/exchange_source_operator.h b/be/src/pipeline/exec/exchange_source_operator.h new file mode 100644 index 0000000000..8d1707a1e0 --- /dev/null +++ b/be/src/pipeline/exec/exchange_source_operator.h @@ -0,0 +1,55 @@ +// 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. + +#pragma once + +#include "operator.h" + +namespace doris::vectorized { +class VExchangeNode; +} + +namespace doris::pipeline { + +class ExchangeSourceOperator : public Operator { +public: + explicit ExchangeSourceOperator(OperatorBuilder*, vectorized::VExchangeNode*); + Status open(RuntimeState* state) override; + bool can_read() override; + Status get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) override; + bool is_pending_finish() const override; + Status close(RuntimeState* state) override; + +private: + vectorized::VExchangeNode* _exchange_node; +}; + +class ExchangeSourceOperatorBuilder : public OperatorBuilder { +public: + ExchangeSourceOperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node) + : OperatorBuilder(id, name, exec_node) {} + + bool is_source() const override { return true; } + + OperatorPtr build_operator() override { + return std::make_shared( + this, reinterpret_cast(_related_exec_node)); + } +}; + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp new file mode 100644 index 0000000000..2e4fdc2570 --- /dev/null +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -0,0 +1,33 @@ +// 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 "olap_scan_operator.h" + +#include "vec/exec/scan/new_olap_scan_node.h" + +namespace doris::pipeline { + +OlapScanOperator::OlapScanOperator(OperatorBuilder* operator_builder, + vectorized::NewOlapScanNode* scan_node) + : ScanOperator(operator_builder, scan_node) {} + +OlapScanOperatorBuilder::OlapScanOperatorBuilder(uint32_t id, const std::string& name, + vectorized::NewOlapScanNode* new_olap_scan_node) + : ScanOperatorBuilder(id, name, new_olap_scan_node), + _new_olap_scan_node(new_olap_scan_node) {} + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/olap_scan_operator.h b/be/src/pipeline/exec/olap_scan_operator.h new file mode 100644 index 0000000000..26631c6390 --- /dev/null +++ b/be/src/pipeline/exec/olap_scan_operator.h @@ -0,0 +1,46 @@ +// 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. + +#pragma once + +#include + +#include "scan_operator.h" +#include "vec/exec/scan/new_olap_scan_node.h" + +namespace doris::pipeline { + +class OlapScanOperatorBuilder; +class OlapScanOperator : public ScanOperator { +public: + OlapScanOperator(OperatorBuilder* operator_builder, vectorized::NewOlapScanNode* scan_node); +}; + +class OlapScanOperatorBuilder : public ScanOperatorBuilder { +public: + OlapScanOperatorBuilder(uint32_t id, const std::string& name, + vectorized::NewOlapScanNode* new_olap_scan_node); + + OperatorPtr build_operator() override { + return std::make_shared(this, _new_olap_scan_node); + } + +private: + vectorized::NewOlapScanNode* _new_olap_scan_node; +}; + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/operator.cpp b/be/src/pipeline/exec/operator.cpp new file mode 100644 index 0000000000..572cd1ffcc --- /dev/null +++ b/be/src/pipeline/exec/operator.cpp @@ -0,0 +1,91 @@ +// 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 "operator.h" + +namespace doris::pipeline { + +Operator::Operator(OperatorBuilder* operator_builder) + : _operator_builder(operator_builder), _is_closed(false) {} + +bool Operator::is_sink() const { + return _operator_builder->is_sink(); +} + +bool Operator::is_source() const { + return _operator_builder->is_source(); +} + +Status Operator::init(ExecNode* exec_node, RuntimeState* state) { + _runtime_profile.reset(new RuntimeProfile(_operator_builder->get_name())); + if (exec_node) { + exec_node->runtime_profile()->insert_child_head(_runtime_profile.get(), true); + } + return Status::OK(); +} + +Status Operator::prepare(RuntimeState* state) { + _mem_tracker = std::make_unique("Operator:" + _runtime_profile->name(), + _runtime_profile.get()); + return Status::OK(); +} + +Status Operator::open(RuntimeState* state) { + return Status::OK(); +} + +Status Operator::close(RuntimeState* state) { + if (_is_closed) { + return Status::OK(); + } + _is_closed = true; + return Status::OK(); +} + +const RowDescriptor& Operator::row_desc() { + return _operator_builder->row_desc(); +} + +void Operator::_fresh_exec_timer(doris::ExecNode* node) { + node->_runtime_profile->total_time_counter()->update( + _runtime_profile->total_time_counter()->value()); +} + +std::string Operator::debug_string() const { + std::stringstream ss; + ss << _operator_builder->get_name() << ", source: " << is_source(); + ss << ", sink: " << is_sink() << ", is closed: " << _is_closed; + ss << ", is pending finish: " << is_pending_finish(); + return ss.str(); +} + +/////////////////////////////////////// OperatorBuilder //////////////////////////////////////////////////////////// + +Status OperatorBuilder::prepare(doris::RuntimeState* state) { + _state = state; + // runtime filter, now dispose by NewOlapScanNode + return Status::OK(); +} + +void OperatorBuilder::close(doris::RuntimeState* state) { + if (_is_closed) { + return; + } + _is_closed = true; +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h new file mode 100644 index 0000000000..d65d0291b3 --- /dev/null +++ b/be/src/pipeline/exec/operator.h @@ -0,0 +1,198 @@ +// 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. + +#pragma once + +#include + +#include "common/status.h" +#include "exec/exec_node.h" +#include "runtime/runtime_state.h" +#include "vec/core/block.h" + +namespace doris::pipeline { + +// Result of source pull data, init state is DEPEND_ON_SOURCE +enum class SourceState : uint8_t { + DEPEND_ON_SOURCE = 0, // Operator has no more data in itself, needs to read from source. + MORE_DATA = 1, // Still have data can read + FINISHED = 2 +}; + +// +enum class SinkState : uint8_t { + SINK_IDLE = 0, // can send block to sink + SINK_BUSY = 1, // sink buffer is full, should wait sink to send some block + FINISHED = 2 +}; +//////////////// DO NOT USE THE UP State //////////////// + +class OperatorBuilder; +class Operator; + +using OperatorPtr = std::shared_ptr; +using Operators = std::vector; + +class Operator { +public: + explicit Operator(OperatorBuilder* operator_builder); + virtual ~Operator() = default; + + // After both sink and source need to know the cancel state. + // do cancel work + bool is_sink() const; + + bool is_source() const; + + // Should be call after ExecNode is constructed + virtual Status init(ExecNode* exec_node, RuntimeState* state = nullptr); + + // Only result sink and data stream sink need to impl the virtual function + virtual Status init(const TDataSink& tsink) { return Status::OK(); }; + + // Do prepare some state of Operator + virtual Status prepare(RuntimeState* state); + + // Like ExecNode,when pipeline task first time be scheduled, can't block + // the pipeline should be open after dependencies is finish + // Eg a -> c, b-> c, after a, b pipeline finish, c pipeline should call open + // Now the pipeline only have one task, so the there is no performance bottleneck for the mechanism, + // but if one pipeline have multi task to parallel work, need to rethink the logic + // + // Each operator should call open_self() to prepare resource to do data compute. + // if ExecNode split to sink and source operator, open_self() should be called in sink operator + virtual Status open(RuntimeState* state); + + // Release the resource, should not block the thread + // + // Each operator should call close_self() to release resource + // if ExecNode split to sink and source operator, close_self() should be called in source operator + virtual Status close(RuntimeState* state); + + Status set_child(OperatorPtr child) { + if (is_source()) { + return Status::InternalError("source can not has child."); + } + _child = std::move(child); + return Status::OK(); + } + + virtual bool can_read() { return false; } // for source + + virtual bool can_write() { return false; } // for sink + + // for pipeline + virtual Status get_block([[maybe_unused]] RuntimeState* runtime_state, + [[maybe_unused]] vectorized::Block* block, + [[maybe_unused]] SourceState& result_state) { + std::stringstream error_msg; + error_msg << " has not implements get_block"; + return Status::NotSupported(error_msg.str()); + } + + // return can write continue + virtual Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) { + std::stringstream error_msg; + error_msg << " not a sink "; + return Status::NotSupported(error_msg.str()); + } + + virtual Status finalize(RuntimeState* state) { + std::stringstream error_msg; + error_msg << " not a sink, can not finalize"; + return Status::NotSupported(error_msg.str()); + } + + // close be called + // - Source: scan thread do not exist + // - Sink: RPC do not be disposed + // - else return false + virtual bool is_pending_finish() const { return false; } + + // TODO: should we keep the function + // virtual bool is_finished() = 0; + + bool is_closed() const { return _is_closed; } + + MemTracker* mem_tracker() const { return _mem_tracker.get(); } + + const OperatorBuilder* operator_builder() const { return _operator_builder; } + + const RowDescriptor& row_desc(); + + RuntimeProfile* runtime_profile() { return _runtime_profile.get(); } + std::string debug_string() const; + +protected: + void _fresh_exec_timer(ExecNode* node); + + std::unique_ptr _mem_tracker; + + OperatorBuilder* _operator_builder; + // source has no child + // if an operator is not source, it will get data from its child. + OperatorPtr _child; + + std::unique_ptr _runtime_profile; + // TODO pipeline Account for peak memory used by this operator + RuntimeProfile::Counter* _memory_used_counter = nullptr; + +private: + bool _is_closed = false; +}; + +class OperatorBuilder { +public: + OperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node = nullptr) + : _id(id), _name(name), _related_exec_node(exec_node) {} + + virtual ~OperatorBuilder() = default; + + virtual OperatorPtr build_operator() = 0; + + virtual bool is_sink() const { return false; } + virtual bool is_source() const { return false; } + + // create the object used by all operator + virtual Status prepare(RuntimeState* state); + + // destory the object used by all operator + virtual void close(RuntimeState* state); + + std::string get_name() const { return _name; } + + RuntimeState* runtime_state() { return _state; } + + const RowDescriptor& row_desc() { return _related_exec_node->row_desc(); } + + ExecNode* exec_node() const { return _related_exec_node; } + + int32_t id() const { return _id; } + +protected: + const int32_t _id; + const std::string _name; + ExecNode* _related_exec_node; + + RuntimeState* _state = nullptr; + bool _is_closed = false; +}; + +using OperatorBuilderPtr = std::shared_ptr; +using OperatorBuilders = std::vector; + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp new file mode 100644 index 0000000000..467057e034 --- /dev/null +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -0,0 +1,76 @@ +// 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 "repeat_operator.h" + +#include "vec/exec/vrepeat_node.h" + +namespace doris { +namespace pipeline { + +RepeatOperator::RepeatOperator(RepeatOperatorBuilder* operator_builder, + vectorized::VRepeatNode* repeat_node) + : Operator(operator_builder), _repeat_node(repeat_node) {} + +Status RepeatOperator::open(RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(Operator::open(state)); + _child_block.reset(new vectorized::Block); + return _repeat_node->alloc_resource(state); +} + +Status RepeatOperator::close(RuntimeState* state) { + _fresh_exec_timer(_repeat_node); + _repeat_node->release_resource(state); + Operator::close(state); + return Status::OK(); +} + +Status RepeatOperator::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + if (_repeat_node->need_more_input_data()) { + RETURN_IF_ERROR(_child->get_block(state, _child_block.get(), _child_source_state)); + source_state = _child_source_state; + if (_child_block->rows() == 0) { + return Status::OK(); + } + + _repeat_node->push(state, _child_block.get(), source_state == SourceState::FINISHED); + } + + bool eos = false; + RETURN_IF_ERROR(_repeat_node->pull(state, block, &eos)); + if (eos) { + source_state = SourceState::FINISHED; + _child_block->clear_column_data(); + } else if (!_repeat_node->need_more_input_data()) { + source_state = SourceState::MORE_DATA; + } else { + _child_block->clear_column_data(); + } + return Status::OK(); +} + +RepeatOperatorBuilder::RepeatOperatorBuilder(int32_t id, vectorized::VRepeatNode* repeat_node) + : OperatorBuilder(id, "RepeatOperatorBuilder", repeat_node), _repeat_node(repeat_node) {} + +OperatorPtr RepeatOperatorBuilder::build_operator() { + return std::make_shared(this, _repeat_node); +} +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/exec/repeat_operator.h b/be/src/pipeline/exec/repeat_operator.h new file mode 100644 index 0000000000..e2aaaeab02 --- /dev/null +++ b/be/src/pipeline/exec/repeat_operator.h @@ -0,0 +1,58 @@ +// 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. + +#pragma once + +#include "operator.h" + +namespace doris { +namespace vectorized { +class VRepeatNode; +class VExprContext; +class Block; +} // namespace vectorized +namespace pipeline { +class RepeatOperatorBuilder; +class RepeatOperator : public Operator { +public: + RepeatOperator(RepeatOperatorBuilder* operator_builder, vectorized::VRepeatNode* repeat_node); + + Status open(RuntimeState* state) override; + + Status close(RuntimeState* state) override; + + Status get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) override; + +private: + vectorized::VRepeatNode* _repeat_node; + std::unique_ptr _child_block; + SourceState _child_source_state; +}; + +class RepeatOperatorBuilder : public OperatorBuilder { +public: + RepeatOperatorBuilder(int32_t id, vectorized::VRepeatNode* repeat_node); + + OperatorPtr build_operator() override; + +private: + vectorized::VRepeatNode* _repeat_node; +}; + +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp new file mode 100644 index 0000000000..fb2a14355e --- /dev/null +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -0,0 +1,69 @@ +// 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 "result_sink_operator.h" + +#include "runtime/buffer_control_block.h" +#include "vec/sink/vresult_sink.h" + +namespace doris::pipeline { +ResultSinkOperator::ResultSinkOperator(OperatorBuilder* operator_builder, + vectorized::VResultSink* sink) + : Operator(operator_builder), _sink(sink) {} + +Status ResultSinkOperator::init(const TDataSink& tsink) { + return Status::OK(); +} + +Status ResultSinkOperator::prepare(RuntimeState* state) { + RETURN_IF_ERROR(Operator::prepare(state)); + return _sink->prepare(state); +} + +Status ResultSinkOperator::open(RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + return _sink->open(state); +} + +bool ResultSinkOperator::can_write() { + return _sink->_sender->can_sink(); +} + +Status ResultSinkOperator::sink(RuntimeState* state, vectorized::Block* block, + SourceState source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + if (!block) { + DCHECK(source_state == SourceState::FINISHED) + << "block is null, eos should invoke in finalize."; + return Status::OK(); + } + return _sink->send(state, block); +} + +Status ResultSinkOperator::finalize(RuntimeState* state) { + _finalized = true; + return _sink->close(state, Status::OK()); +} + +// TODO: Support fresh exec time for sink +Status ResultSinkOperator::close(RuntimeState* state) { + if (!_finalized) { + RETURN_IF_ERROR(_sink->close(state, Status::InternalError("Not finalized"))); + } + return Status::OK(); +} +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h new file mode 100644 index 0000000000..190cdfb570 --- /dev/null +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -0,0 +1,69 @@ +// 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. + +#pragma once + +#include "operator.h" + +namespace doris { +namespace vectorized { +class VResultSink; +} + +namespace pipeline { + +class ResultSinkOperator : public Operator { +public: + ResultSinkOperator(OperatorBuilder* operator_builder, vectorized::VResultSink* sink); + + Status init(const TDataSink& tsink) override; + + Status prepare(RuntimeState* state) override; + + Status open(RuntimeState* state) override; + + bool can_write() override; + + Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override; + + Status finalize(RuntimeState* state) override; + + Status close(RuntimeState* state) override; + +private: + vectorized::VResultSink* _sink; + bool _finalized = false; +}; + +class ResultSinkOperatorBuilder : public OperatorBuilder { +public: + ResultSinkOperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node, + vectorized::VResultSink* sink) + : OperatorBuilder(id, name, exec_node), _sink(sink) {} + + bool is_sink() const override { return true; } + + OperatorPtr build_operator() override { + return std::make_shared(this, _sink); + } + +private: + vectorized::VResultSink* _sink; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp new file mode 100644 index 0000000000..4a06a80c95 --- /dev/null +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -0,0 +1,68 @@ +// 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 "scan_operator.h" + +#include "vec/exec/scan/scanner_context.h" +#include "vec/exec/scan/vscan_node.h" + +namespace doris::pipeline { + +ScanOperator::ScanOperator(OperatorBuilder* operator_builder, vectorized::VScanNode* scan_node) + : Operator(operator_builder), _scan_node(scan_node) {} + +Status ScanOperator::open(RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(Operator::open(state)); + return _scan_node->open(state); +} + +bool ScanOperator::can_read() { + if (_scan_node->_eos || !_scan_node->_scanner_ctx || _scan_node->_scanner_ctx->done() || + _scan_node->_scanner_ctx->can_finish()) { + // _eos: need eos + // !_scanner_ctx: need call open + // _scanner_ctx->done(): need finish + // _scanner_ctx->can_finish(): should be scheduled + return true; + } else { + return !_scan_node->_scanner_ctx->empty_in_queue(); // have block to process + } +} + +Status ScanOperator::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& result_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + bool eos = false; + RETURN_IF_ERROR(_scan_node->get_next(state, block, &eos)); + result_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; + return Status::OK(); +} + +bool ScanOperator::is_pending_finish() const { + return _scan_node->_scanner_ctx && !_scan_node->_scanner_ctx->can_finish(); +} + +Status ScanOperator::close(RuntimeState* state) { + if (!is_closed()) { + RETURN_IF_ERROR(_scan_node->close(state)); + } + _fresh_exec_timer(_scan_node); + return Operator::close(state); +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h new file mode 100644 index 0000000000..1f32f1c899 --- /dev/null +++ b/be/src/pipeline/exec/scan_operator.h @@ -0,0 +1,59 @@ +// 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. + +#pragma once + +#include + +#include "operator.h" + +namespace doris::vectorized { +class VScanNode; +class VScanner; +class ScannerContext; +} // namespace doris::vectorized + +namespace doris::pipeline { + +class ScanOperator : public Operator { +public: + ScanOperator(OperatorBuilder* operator_builder, vectorized::VScanNode* scan_node); + + bool can_read() override; // for source + + Status get_block(RuntimeState* state, vectorized::Block* block, + SourceState& result_state) override; + + bool is_pending_finish() const override; + + Status open(RuntimeState* state) override; + + Status close(RuntimeState* state) override; + +private: + vectorized::VScanNode* _scan_node; +}; + +class ScanOperatorBuilder : public OperatorBuilder { +public: + ScanOperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node) + : OperatorBuilder(id, name, exec_node) {} + + bool is_source() const override { return true; } +}; + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp new file mode 100644 index 0000000000..273f20198c --- /dev/null +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -0,0 +1,51 @@ +// 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 "sort_sink_operator.h" + +#include "vec/exec/vsort_node.h" + +namespace doris::pipeline { + +SortSinkOperatorBuilder::SortSinkOperatorBuilder(int32_t id, const string& name, + vectorized::VSortNode* sort_node) + : OperatorBuilder(id, name, sort_node), _sort_node(sort_node) {} + +SortSinkOperator::SortSinkOperator(SortSinkOperatorBuilder* operator_builder, + vectorized::VSortNode* sort_node) + : Operator(operator_builder), _sort_node(sort_node) {} + +Status SortSinkOperator::open(doris::RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(Operator::open(state)); + RETURN_IF_ERROR(_sort_node->alloc_resource(state)); + return Status::OK(); +} + +Status SortSinkOperator::close(doris::RuntimeState* /*state*/) { + _fresh_exec_timer(_sort_node); + return Status::OK(); +} + +Status SortSinkOperator::sink(doris::RuntimeState* state, vectorized::Block* block, + SourceState source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + // TODO pipeline when sort node's _reuse_mem is false, we should pass a new block to it. + RETURN_IF_ERROR(_sort_node->sink(state, block, source_state == SourceState::FINISHED)); + return Status::OK(); +} +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h new file mode 100644 index 0000000000..96adbfa31e --- /dev/null +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -0,0 +1,70 @@ +// 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. + +#pragma once + +#include + +#include "operator.h" + +namespace doris { + +namespace vectorized { +class VSortNode; +} + +namespace pipeline { + +class SortSinkOperatorBuilder; + +class SortSinkOperator : public Operator { +public: + SortSinkOperator(SortSinkOperatorBuilder* operator_builder, vectorized::VSortNode* sort_node); + + Status open(RuntimeState* state) override; + + Status close(RuntimeState* state) override; + + // return can write continue + Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override; + + Status finalize(RuntimeState* /*state*/) override { return Status::OK(); } + + bool can_write() override { return true; }; + +private: + vectorized::VSortNode* _sort_node; +}; + +class SortSinkOperatorBuilder : public OperatorBuilder { +public: + SortSinkOperatorBuilder(int32_t id, const std::string& name, vectorized::VSortNode* sort_node); + + bool is_sink() const override { return true; } + + bool is_source() const override { return false; } + + OperatorPtr build_operator() override { + return std::make_shared(this, _sort_node); + } + +private: + vectorized::VSortNode* _sort_node; +}; + +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp new file mode 100644 index 0000000000..4f25b216c3 --- /dev/null +++ b/be/src/pipeline/exec/sort_source_operator.cpp @@ -0,0 +1,54 @@ +// 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 "sort_source_operator.h" + +#include "vec/exec/vsort_node.h" + +namespace doris::pipeline { + +SortSourceOperatorBuilder::SortSourceOperatorBuilder(int32_t id, const string& name, + vectorized::VSortNode* sort_node) + : OperatorBuilder(id, name, sort_node), _sort_node(sort_node) {} + +SortSourceOperator::SortSourceOperator(SortSourceOperatorBuilder* operator_builder, + vectorized::VSortNode* sort_node) + : Operator(operator_builder), _sort_node(sort_node) {} + +Status SortSourceOperator::close(doris::RuntimeState* state) { + if (is_closed()) { + return Status::OK(); + } + _fresh_exec_timer(_sort_node); + _sort_node->release_resource(state); + return Operator::close(state); +} + +bool SortSourceOperator::can_read() { + return _sort_node->can_read(); +} + +Status SortSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + bool eos = false; + RETURN_IF_ERROR(_sort_node->pull(state, block, &eos)); + source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; + return Status::OK(); +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_source_operator.h b/be/src/pipeline/exec/sort_source_operator.h new file mode 100644 index 0000000000..9b83ce06fb --- /dev/null +++ b/be/src/pipeline/exec/sort_source_operator.h @@ -0,0 +1,68 @@ +// 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. + +#pragma once + +#include + +#include "operator.h" + +namespace doris { + +namespace vectorized { +class VSortNode; +} + +namespace pipeline { + +class SortSourceOperatorBuilder; + +class SortSourceOperator : public Operator { +public: + SortSourceOperator(SortSourceOperatorBuilder* operator_builder, + vectorized::VSortNode* sort_node); + + Status close(RuntimeState* state) override; + + Status get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) override; + + bool can_read() override; + +private: + vectorized::VSortNode* _sort_node; +}; + +class SortSourceOperatorBuilder : public OperatorBuilder { +public: + SortSourceOperatorBuilder(int32_t id, const std::string& name, + vectorized::VSortNode* sort_node); + + bool is_sink() const override { return false; } + + bool is_source() const override { return true; } + + OperatorPtr build_operator() override { + return std::make_shared(this, _sort_node); + } + +private: + vectorized::VSortNode* _sort_node; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp new file mode 100644 index 0000000000..9c1a2a57e8 --- /dev/null +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp @@ -0,0 +1,101 @@ +// 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 "streaming_aggregation_sink_operator.h" + +#include "vec/exec/vaggregation_node.h" + +namespace doris::pipeline { + +StreamingAggSinkOperator::StreamingAggSinkOperator( + StreamingAggSinkOperatorBuilder* operator_builder, vectorized::AggregationNode* agg_node, + std::shared_ptr agg_context) + : Operator(operator_builder), _agg_node(agg_node), _agg_context(std::move(agg_context)) {} + +Status StreamingAggSinkOperator::prepare(RuntimeState* state) { + RETURN_IF_ERROR(Operator::prepare(state)); + _queue_byte_size_counter = + ADD_COUNTER(_runtime_profile.get(), "MaxSizeInBlockQueue", TUnit::BYTES); + _queue_size_counter = ADD_COUNTER(_runtime_profile.get(), "MaxSizeOfBlockQueue", TUnit::UNIT); + return Status::OK(); +} + +Status StreamingAggSinkOperator::open(RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(Operator::open(state)); + RETURN_IF_ERROR(_agg_node->alloc_resource(state)); + return Status::OK(); +} + +bool StreamingAggSinkOperator::can_write() { + // sink and source in diff threads + return _agg_context->has_enough_space_to_push(); +} + +Status StreamingAggSinkOperator::sink(RuntimeState* state, vectorized::Block* in_block, + SourceState source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + Status ret = Status::OK(); + if (in_block && in_block->rows() > 0) { + auto bock_from_ctx = _agg_context->get_free_block(); + RETURN_IF_ERROR(_agg_node->do_pre_agg(in_block, bock_from_ctx.get())); + if (bock_from_ctx->rows() == 0) { + _agg_context->return_free_block(std::move(bock_from_ctx)); + } else { + _agg_context->push_block(std::move(bock_from_ctx)); + } + } + + if (UNLIKELY(source_state == SourceState::FINISHED)) { + _agg_context->set_finish(); + } + return Status::OK(); +} + +Status StreamingAggSinkOperator::close(RuntimeState* state) { + _fresh_exec_timer(_agg_node); + if (_agg_context && !_agg_context->is_finish()) { + // finish should be set, if not set here means error. + _agg_context->set_canceled(); + } + COUNTER_SET(_queue_size_counter, _agg_context->max_size_of_queue()); + COUNTER_SET(_queue_byte_size_counter, _agg_context->max_bytes_in_queue()); + return Status::OK(); +} + +/////////////////////////////// operator template //////////////////////////////// + +StreamingAggSinkOperatorBuilder::StreamingAggSinkOperatorBuilder( + int32_t id, const std::string& name, vectorized::AggregationNode* exec_node, + std::shared_ptr agg_context) + : OperatorBuilder(id, name, exec_node), + _agg_node(exec_node), + _agg_context(std::move(agg_context)) {} + +OperatorPtr StreamingAggSinkOperatorBuilder::build_operator() { + return std::make_shared(this, _agg_node, _agg_context); +} + +// use final aggregation source operator +bool StreamingAggSinkOperatorBuilder::is_sink() const { + return true; +} + +bool StreamingAggSinkOperatorBuilder::is_source() const { + return false; +} +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h new file mode 100644 index 0000000000..691521ee9b --- /dev/null +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h @@ -0,0 +1,75 @@ +// 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. + +#pragma once + +#include "agg_context.h" +#include "operator.h" + +namespace doris { +namespace vectorized { +class AggregationNode; +class VExprContext; +class Block; +} // namespace vectorized + +namespace pipeline { +class StreamingAggSinkOperatorBuilder; +class StreamingAggSinkOperator : public Operator { +public: + StreamingAggSinkOperator(StreamingAggSinkOperatorBuilder* operator_builder, + vectorized::AggregationNode*, std::shared_ptr); + + Status prepare(RuntimeState*) override; + + Status open(RuntimeState* state) override; + + Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override; + + bool can_write() override; + + Status close(RuntimeState* state) override; + + Status finalize(doris::RuntimeState* state) override { return Status::OK(); } + +private: + vectorized::AggregationNode* _agg_node; + vectorized::Block _preagg_block = vectorized::Block(); + + RuntimeProfile::Counter* _queue_byte_size_counter; + RuntimeProfile::Counter* _queue_size_counter; + + std::shared_ptr _agg_context; +}; + +class StreamingAggSinkOperatorBuilder : public OperatorBuilder { +public: + StreamingAggSinkOperatorBuilder(int32_t, const std::string&, vectorized::AggregationNode*, + std::shared_ptr); + + OperatorPtr build_operator() override; + + bool is_sink() const override; + bool is_source() const override; + +private: + vectorized::AggregationNode* _agg_node; + std::shared_ptr _agg_context; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp new file mode 100644 index 0000000000..3270b553fd --- /dev/null +++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp @@ -0,0 +1,86 @@ +// 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 "streaming_aggregation_source_operator.h" + +#include "vec/exec/vaggregation_node.h" + +namespace doris { +namespace pipeline { +StreamingAggSourceOperator::StreamingAggSourceOperator(OperatorBuilder* templ, + vectorized::AggregationNode* node, + std::shared_ptr agg_context) + : Operator(templ), _agg_node(node), _agg_context(std::move(agg_context)) {} + +Status StreamingAggSourceOperator::prepare(RuntimeState* state) { + _agg_node->increase_ref(); + return Status::OK(); +} + +bool StreamingAggSourceOperator::can_read() { + return _agg_context->has_data_or_finished(); +} + +Status StreamingAggSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + bool eos = false; + if (!_agg_context->data_exhausted()) { + std::unique_ptr agg_block; + RETURN_IF_ERROR(_agg_context->get_block(&agg_block)); + + if (_agg_context->data_exhausted()) { + RETURN_IF_ERROR(_agg_node->pull(state, block, &eos)); + } else { + block->swap(*agg_block); + agg_block->clear_column_data(_agg_node->row_desc().num_materialized_slots()); + _agg_context->return_free_block(std::move(agg_block)); + } + } else { + RETURN_IF_ERROR(_agg_node->pull(state, block, &eos)); + } + + source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; + + return Status::OK(); +} + +Status StreamingAggSourceOperator::close(RuntimeState* state) { + if (is_closed()) { + return Status::OK(); + } + _fresh_exec_timer(_agg_node); + if (!_agg_node->decrease_ref()) { + _agg_node->release_resource(state); + } + return Operator::close(state); +} + +/////////////////////////////// operator template //////////////////////////////// + +StreamingAggSourceOperatorBuilder::StreamingAggSourceOperatorBuilder( + int32_t id, const std::string& name, vectorized::AggregationNode* exec_node, + std::shared_ptr agg_context) + : OperatorBuilder(id, name, exec_node), _agg_context(std::move(agg_context)) {} + +OperatorPtr StreamingAggSourceOperatorBuilder::build_operator() { + return std::make_shared( + this, assert_cast(_related_exec_node), _agg_context); +} + +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.h b/be/src/pipeline/exec/streaming_aggregation_source_operator.h new file mode 100644 index 0000000000..106f1cf99d --- /dev/null +++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.h @@ -0,0 +1,56 @@ +// 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. +#pragma once + +#include "agg_context.h" +#include "operator.h" + +namespace doris { +namespace vectorized { +class AggregationNode; +} +namespace pipeline { + +class StreamingAggSourceOperator : public Operator { +public: + StreamingAggSourceOperator(OperatorBuilder*, vectorized::AggregationNode*, + std::shared_ptr); + Status prepare(RuntimeState* state) override; + bool can_read() override; + Status close(RuntimeState* state) override; + Status get_block(RuntimeState*, vectorized::Block*, SourceState& source_state) override; + +private: + vectorized::AggregationNode* _agg_node; + std::shared_ptr _agg_context; +}; + +class StreamingAggSourceOperatorBuilder : public OperatorBuilder { +public: + StreamingAggSourceOperatorBuilder(int32_t, const std::string&, vectorized::AggregationNode*, + std::shared_ptr); + + bool is_source() const override { return true; } + + OperatorPtr build_operator() override; + +private: + std::shared_ptr _agg_context; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h new file mode 100644 index 0000000000..0735b981cc --- /dev/null +++ b/be/src/pipeline/exec/table_function_operator.h @@ -0,0 +1,92 @@ +// 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. + +#pragma once + +#include "operator.h" +#include "vec/exec/vtable_function_node.h" + +namespace doris { + +namespace pipeline { +class TableFunctionOperator; + +class TableFunctionOperatorBuilder : public OperatorBuilder { +public: + TableFunctionOperatorBuilder(int32_t id, vectorized::VTableFunctionNode* node) + : OperatorBuilder(id, "TableFunctionOperatorBuilder", node), _node(node) {} + + OperatorPtr build_operator() override; + +private: + vectorized::VTableFunctionNode* _node; +}; + +class TableFunctionOperator : public Operator { +public: + TableFunctionOperator(TableFunctionOperatorBuilder* operator_builder, + vectorized::VTableFunctionNode* node) + : Operator(operator_builder), _node(node) {} + + Status open(RuntimeState* state) override { + RETURN_IF_ERROR(Operator::open(state)); + _child_block.reset(new vectorized::Block); + return _node->alloc_resource(state); + } + + Status close(RuntimeState* state) override { + _node->release_resource(state); + _fresh_exec_timer(_node); + return Operator::close(state); + } + + Status get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) override { + if (_node->need_more_input_data()) { + RETURN_IF_ERROR(_child->get_block(state, _child_block.get(), _child_source_state)); + source_state = _child_source_state; + if (_child_block->rows() == 0) { + return Status::OK(); + } + _node->push(state, _child_block.get(), source_state == SourceState::FINISHED); + } + + bool eos = false; + RETURN_IF_ERROR(_node->pull(state, block, &eos)); + if (eos) { + source_state = SourceState::FINISHED; + _child_block->clear_column_data(); + } else if (!_node->need_more_input_data()) { + source_state = SourceState::MORE_DATA; + } else { + _child_block->clear_column_data(); + } + return Status::OK(); + } + +private: + vectorized::VTableFunctionNode* _node; + std::unique_ptr _child_block; + SourceState _child_source_state; +}; + +OperatorPtr TableFunctionOperatorBuilder::build_operator() { + return std::make_shared(this, _node); +} + +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/pipeline.cpp b/be/src/pipeline/pipeline.cpp new file mode 100644 index 0000000000..7342d49850 --- /dev/null +++ b/be/src/pipeline/pipeline.cpp @@ -0,0 +1,76 @@ +// 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 "pipeline.h" + +#include "pipeline_fragment_context.h" + +namespace doris::pipeline { + +Status Pipeline::prepare(RuntimeState* state) { + std::stringstream ss; + ss << "Pipeline" + << " (pipeline id=" << _pipeline_id << ")"; + _pipeline_profile.reset(new RuntimeProfile(ss.str())); + for (auto& op : _operator_builders) { + RETURN_IF_ERROR(op->prepare(state)); + } + RETURN_IF_ERROR(_sink->prepare(state)); + return Status::OK(); +} + +Status Pipeline::build_operators(Operators& operators) { + OperatorPtr pre; + for (auto& operator_t : _operator_builders) { + auto o = operator_t->build_operator(); + RETURN_IF_ERROR(o->init(operator_t->exec_node(), _context->get_runtime_state())); + if (pre) { + o->set_child(pre); + } + operators.emplace_back(o); + pre = std::move(o); + } + return Status::OK(); +} + +void Pipeline::close(RuntimeState* state) { + for (auto& op : _operator_builders) { + op->close(state); + } + _sink->close(state); +} + +Status Pipeline::add_operator(OperatorBuilderPtr& op) { + if (_operator_builders.empty() && !op->is_source()) { + return Status::InternalError("Should set source before other operator"); + } + _operator_builders.emplace_back(op); + return Status::OK(); +} + +Status Pipeline::set_sink(OperatorBuilderPtr& sink_) { + if (_sink) { + return Status::InternalError("set sink twice"); + } + if (!sink_->is_sink()) { + return Status::InternalError("should set a sink operator but {}", typeid(sink_).name()); + } + _sink = sink_; + return Status::OK(); +} + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/pipeline.h b/be/src/pipeline/pipeline.h new file mode 100644 index 0000000000..865b7b2a04 --- /dev/null +++ b/be/src/pipeline/pipeline.h @@ -0,0 +1,90 @@ +// 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. + +#pragma once + +#include +#include + +#include "common/status.h" +#include "exec/operator.h" +#include "vec/core/block.h" + +namespace doris::pipeline { + +class Pipeline; +using PipelinePtr = std::shared_ptr; +using Pipelines = std::vector; +using PipelineId = uint32_t; + +class PipelineTask; +class PipelineFragmentContext; + +class Pipeline : public std::enable_shared_from_this { + friend class PipelineTask; + +public: + Pipeline() = delete; + explicit Pipeline(PipelineId pipeline_id, std::shared_ptr context) + : _complete_dependency(0), _pipeline_id(pipeline_id), _context(std::move(context)) {} + + Status prepare(RuntimeState* state); + + void close(RuntimeState*); + + void add_dependency(std::shared_ptr& pipeline) { + pipeline->_parents.push_back(shared_from_this()); + _dependencies.push_back(pipeline); + } + + // If all dependency be finished, the pipeline task shoule be scheduled + // pipeline is finish must call the parents `finish_one_dependency` + // like the condition variables. + // Eg: hash build finish must call the hash probe the method + bool finish_one_dependency() { + DCHECK(_complete_dependency < _dependencies.size()); + return _complete_dependency.fetch_add(1) == _dependencies.size() - 1; + } + + bool has_dependency() { return _complete_dependency.load() < _dependencies.size(); } + + Status add_operator(OperatorBuilderPtr& op); + + Status set_sink(OperatorBuilderPtr& sink_operator); + + OperatorBuilder* sink() { return _sink.get(); } + + Status build_operators(Operators&); + + RuntimeProfile* runtime_profile() { return _pipeline_profile.get(); } + +private: + std::atomic _complete_dependency; + + OperatorBuilders _operator_builders; // left is _source, right is _root + OperatorBuilderPtr _sink; // put block to sink + + std::vector> _parents; + std::vector> _dependencies; + + PipelineId _pipeline_id; + std::shared_ptr _context; + + std::unique_ptr _pipeline_profile; +}; + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp new file mode 100644 index 0000000000..d1913a1534 --- /dev/null +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -0,0 +1,569 @@ +// 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 "pipeline_fragment_context.h" + +#include + +#include "exec/agg_context.h" +#include "exec/aggregation_sink_operator.h" +#include "exec/aggregation_source_operator.h" +#include "exec/data_sink.h" +#include "exec/empty_set_operator.h" +#include "exec/exchange_sink_operator.h" +#include "exec/exchange_source_operator.h" +#include "exec/olap_scan_operator.h" +#include "exec/repeat_operator.h" +#include "exec/result_sink_operator.h" +#include "exec/scan_node.h" +#include "exec/sort_sink_operator.h" +#include "exec/sort_source_operator.h" +#include "exec/streaming_aggregation_sink_operator.h" +#include "exec/streaming_aggregation_source_operator.h" +#include "gen_cpp/FrontendService.h" +#include "gen_cpp/HeartbeatService_types.h" +#include "pipeline/exec/table_function_operator.h" +#include "pipeline_task.h" +#include "runtime/client_cache.h" +#include "runtime/fragment_mgr.h" +#include "runtime/runtime_state.h" +#include "task_scheduler.h" +#include "util/container_util.hpp" +#include "vec/exec/scan/new_file_scan_node.h" +#include "vec/exec/scan/new_olap_scan_node.h" +#include "vec/exec/scan/vscan_node.h" +#include "vec/exec/vaggregation_node.h" +#include "vec/exec/vempty_set_node.h" +#include "vec/exec/vexchange_node.h" +#include "vec/exec/vrepeat_node.h" +#include "vec/exec/vsort_node.h" +#include "vec/runtime/vdata_stream_mgr.h" +#include "vec/sink/vdata_stream_sender.h" +#include "vec/sink/vresult_sink.h" + +using apache::thrift::transport::TTransportException; +using apache::thrift::TException; + +namespace doris::pipeline { + +PipelineFragmentContext::PipelineFragmentContext(const TUniqueId& query_id, + const TUniqueId& instance_id, int backend_num, + std::shared_ptr query_ctx, + ExecEnv* exec_env) + : _query_id(query_id), + _fragment_instance_id(instance_id), + _backend_num(backend_num), + _exec_env(exec_env), + _cancel_reason(PPlanFragmentCancelReason::INTERNAL_ERROR), + _closed_pipeline_cnt(0), + _query_ctx(std::move(query_ctx)) { + _fragment_watcher.start(); +} + +PipelineFragmentContext::~PipelineFragmentContext() = default; + +void PipelineFragmentContext::cancel(const PPlanFragmentCancelReason& reason, + const std::string& msg) { + if (!_runtime_state->is_cancelled()) { + std::lock_guard l(_status_lock); + if (_runtime_state->is_cancelled()) { + return; + } + if (reason != PPlanFragmentCancelReason::LIMIT_REACH) { + _exec_status = Status::Cancelled(msg); + } + _runtime_state->set_is_cancelled(true); + _cancel_reason = reason; + _cancel_msg = msg; + // To notify wait_for_start() + _query_ctx->set_ready_to_execute(true); + + // must close stream_mgr to avoid dead lock in Exchange Node + _exec_env->vstream_mgr()->cancel(_fragment_instance_id); + // Cancel the result queue manager used by spark doris connector + // TODO pipeline incomp + // _exec_env->result_queue_mgr()->update_queue_status(id, Status::Aborted(msg)); + } +} + +PipelinePtr PipelineFragmentContext::add_pipeline() { + // _prepared、_submitted, _canceled should do not add pipeline + PipelineId id = _next_pipeline_id++; + auto pipeline = std::make_shared(id, shared_from_this()); + _pipelines.emplace_back(pipeline); + return pipeline; +} + +Status PipelineFragmentContext::prepare(const doris::TExecPlanFragmentParams& request) { + if (_prepared) { + return Status::InternalError("Already prepared"); + } + // _runtime_profile.reset(new RuntimeProfile("PipelineContext")); + // _start_timer = ADD_TIMER(_runtime_profile, "StartTime"); + // COUNTER_UPDATE(_start_timer, _fragment_watcher.elapsed_time()); + // _prepare_timer = ADD_TIMER(_runtime_profile, "PrepareTime"); + // SCOPED_TIMER(_prepare_timer); + + auto* fragment_context = this; + OpentelemetryTracer tracer = telemetry::get_noop_tracer(); + if (opentelemetry::trace::Tracer::GetCurrentSpan()->GetContext().IsValid()) { + tracer = telemetry::get_tracer(print_id(_query_id)); + } + START_AND_SCOPE_SPAN(tracer, span, "PipelineFragmentExecutor::prepare"); + + const TPlanFragmentExecParams& params = request.params; + + LOG_INFO("PipelineFragmentContext::prepare") + .tag("query_id", _query_id) + .tag("instance_id", params.fragment_instance_id) + .tag("backend_num", request.backend_num) + .tag("pthread_id", (uintptr_t)pthread_self()); + + // Must be vec exec engine + if (!request.query_options.__isset.enable_vectorized_engine || + !request.query_options.enable_vectorized_engine) { + return Status::InternalError("should set enable_vectorized_engine to true"); + } + + // 1. init _runtime_state + _runtime_state = std::make_unique(params, request.query_options, + _query_ctx->query_globals, _exec_env); + _runtime_state->set_query_fragments_ctx(_query_ctx.get()); + _runtime_state->set_query_mem_tracker(_query_ctx->query_mem_tracker); + _runtime_state->set_tracer(std::move(tracer)); + + // TODO should be combine with plan_fragment_executor.prepare funciton + SCOPED_ATTACH_TASK(get_runtime_state()); + _runtime_state->init_scanner_mem_trackers(); + _runtime_state->runtime_filter_mgr()->init(); + _runtime_state->set_be_number(request.backend_num); + + if (request.__isset.backend_id) { + _runtime_state->set_backend_id(request.backend_id); + } + if (request.__isset.import_label) { + _runtime_state->set_import_label(request.import_label); + } + if (request.__isset.db_name) { + _runtime_state->set_db_name(request.db_name); + } + if (request.__isset.load_job_id) { + _runtime_state->set_load_job_id(request.load_job_id); + } + if (request.__isset.load_error_hub_info) { + _runtime_state->set_load_error_hub_info(request.load_error_hub_info); + } + + if (request.query_options.__isset.is_report_success) { + fragment_context->set_is_report_success(request.query_options.is_report_success); + } + + auto* desc_tbl = _query_ctx->desc_tbl; + _runtime_state->set_desc_tbl(desc_tbl); + + // 2. Create ExecNode to build pipeline with PipelineFragmentContext + RETURN_IF_ERROR(ExecNode::create_tree(_runtime_state.get(), _runtime_state->obj_pool(), + request.fragment.plan, *desc_tbl, &_root_plan)); + _runtime_state->set_fragment_root_id(_root_plan->id()); + + // Set senders of exchange nodes before pipeline build + std::vector exch_nodes; + _root_plan->collect_nodes(TPlanNodeType::EXCHANGE_NODE, &exch_nodes); + for (ExecNode* exch_node : exch_nodes) { + DCHECK_EQ(exch_node->type(), TPlanNodeType::EXCHANGE_NODE); + int num_senders = find_with_default(params.per_exch_num_senders, exch_node->id(), 0); + DCHECK_GT(num_senders, 0); + static_cast(exch_node)->set_num_senders(num_senders); + } + + // All prepare work do in exec node tree + RETURN_IF_ERROR(_root_plan->prepare(_runtime_state.get())); + // set scan ranges + std::vector scan_nodes; + std::vector no_scan_ranges; + _root_plan->collect_scan_nodes(&scan_nodes); + VLOG_CRITICAL << "scan_nodes.size()=" << scan_nodes.size(); + VLOG_CRITICAL << "params.per_node_scan_ranges.size()=" << params.per_node_scan_ranges.size(); + + _root_plan->try_do_aggregate_serde_improve(); + // set scan range in ScanNode + for (int i = 0; i < scan_nodes.size(); ++i) { + // TODO(cmy): this "if...else" should be removed once all ScanNode are derived from VScanNode. + ExecNode* node = scan_nodes[i]; + if (typeid(*node) == typeid(vectorized::NewOlapScanNode) || + typeid(*node) == typeid(vectorized::NewFileScanNode) // || +// typeid(*node) == typeid(vectorized::NewOdbcScanNode) || +// typeid(*node) == typeid(vectorized::NewEsScanNode) +#ifdef LIBJVM +// || typeid(*node) == typeid(vectorized::NewJdbcScanNode) +#endif + ) { + auto* scan_node = static_cast(scan_nodes[i]); + const std::vector& scan_ranges = + find_with_default(params.per_node_scan_ranges, scan_node->id(), no_scan_ranges); + scan_node->set_scan_ranges(scan_ranges); + } else { + ScanNode* scan_node = static_cast(scan_nodes[i]); + const std::vector& scan_ranges = + find_with_default(params.per_node_scan_ranges, scan_node->id(), no_scan_ranges); + scan_node->set_scan_ranges(scan_ranges); + VLOG_CRITICAL << "scan_node_Id=" << scan_node->id() << " size=" << scan_ranges.size(); + } + } + + _runtime_state->set_per_fragment_instance_idx(params.sender_id); + _runtime_state->set_num_per_fragment_instances(params.num_senders); + + if (request.fragment.__isset.output_sink) { + RETURN_IF_ERROR(DataSink::create_data_sink( + _runtime_state->obj_pool(), request.fragment.output_sink, + request.fragment.output_exprs, params, _root_plan->row_desc(), _runtime_state.get(), + &_sink, *desc_tbl)); + } + + _root_pipeline = fragment_context->add_pipeline(); + RETURN_IF_ERROR(_build_pipelines(_root_plan, _root_pipeline)); + if (_sink) { + RETURN_IF_ERROR(_create_sink(request.fragment.output_sink)); + } + RETURN_IF_ERROR(_build_pipeline_tasks(request)); + _runtime_state->runtime_profile()->add_child(_sink->profile(), true, nullptr); + _runtime_state->runtime_profile()->add_child(_root_plan->runtime_profile(), true, nullptr); + + _prepared = true; + return Status::OK(); +} + +Status PipelineFragmentContext::_build_pipeline_tasks( + const doris::TExecPlanFragmentParams& request) { + for (auto& pipeline : _pipelines) { + RETURN_IF_ERROR(pipeline->prepare(_runtime_state.get())); + } + + for (PipelinePtr& pipeline : _pipelines) { + // if sink + auto sink = pipeline->sink()->build_operator(); + RETURN_IF_ERROR(sink->init(pipeline->sink()->exec_node(), _runtime_state.get())); + // TODO pipeline 1 need to add new interface for exec node and operator + sink->init(request.fragment.output_sink); + + Operators operators; + RETURN_IF_ERROR(pipeline->build_operators(operators)); + auto task = std::make_unique(pipeline, 0, _runtime_state.get(), operators, + sink, this, pipeline->runtime_profile()); + sink->set_child(task->get_root()); + _tasks.emplace_back(std::move(task)); + } + + for (auto& task : _tasks) { + RETURN_IF_ERROR(task->prepare(_runtime_state.get())); + } + return Status::OK(); +} + +// TODO: use virtual function to do abstruct +Status PipelineFragmentContext::_build_pipelines(ExecNode* node, PipelinePtr cur_pipe) { + auto* fragment_context = this; + auto node_type = node->type(); + switch (node_type) { + // for source + case TPlanNodeType::OLAP_SCAN_NODE: { + auto* new_olap_scan_node = assert_cast(node); + OperatorBuilderPtr operator_t = std::make_shared( + fragment_context->next_operator_builder_id(), "OlapScanOperator", + new_olap_scan_node); + RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); + break; + } + case TPlanNodeType::EXCHANGE_NODE: { + OperatorBuilderPtr operator_t = std::make_shared( + next_operator_builder_id(), "ExchangeSourceOperator", node); + RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); + break; + } + case TPlanNodeType::EMPTY_SET_NODE: { + auto* empty_set_node = assert_cast(node); + OperatorBuilderPtr operator_t = std::make_shared( + next_operator_builder_id(), "EmptySetSourceOperator", empty_set_node); + RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); + break; + } + case TPlanNodeType::AGGREGATION_NODE: { + auto* agg_node = assert_cast(node); + auto new_pipe = add_pipeline(); + RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipe)); + if (agg_node->is_streaming_preagg()) { + auto agg_ctx = std::make_shared(); + OperatorBuilderPtr pre_agg_sink = std::make_shared( + next_operator_builder_id(), "StreamingAggSinkOperator", agg_node, agg_ctx); + RETURN_IF_ERROR(new_pipe->set_sink(pre_agg_sink)); + + OperatorBuilderPtr pre_agg_source = std::make_shared( + next_operator_builder_id(), "StreamingAggSourceOperator", agg_node, agg_ctx); + RETURN_IF_ERROR(cur_pipe->add_operator(pre_agg_source)); + } else { + OperatorBuilderPtr agg_sink = std::make_shared( + next_operator_builder_id(), "AggSinkOperator", agg_node); + RETURN_IF_ERROR(new_pipe->set_sink(agg_sink)); + + OperatorBuilderPtr agg_source = std::make_shared( + next_operator_builder_id(), "AggSourceOperator", agg_node); + RETURN_IF_ERROR(cur_pipe->add_operator(agg_source)); + } + break; + } + case TPlanNodeType::SORT_NODE: { + auto* sort_node = assert_cast(node); + auto new_pipeline = add_pipeline(); + RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipeline)); + + OperatorBuilderPtr sort_sink = std::make_shared( + next_operator_builder_id(), "SortSinkOperator", sort_node); + RETURN_IF_ERROR(new_pipeline->set_sink(sort_sink)); + + OperatorBuilderPtr sort_source = std::make_shared( + next_operator_builder_id(), "SortSourceOperator", sort_node); + RETURN_IF_ERROR(cur_pipe->add_operator(sort_source)); + break; + } + case TPlanNodeType::REPEAT_NODE: { + auto* repeat_node = assert_cast(node); + RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe)); + OperatorBuilderPtr builder = + std::make_shared(next_operator_builder_id(), repeat_node); + RETURN_IF_ERROR(cur_pipe->add_operator(builder)); + break; + } + case TPlanNodeType::TABLE_FUNCTION_NODE: { + auto* repeat_node = assert_cast(node); + RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe)); + OperatorBuilderPtr builder = std::make_shared( + next_operator_builder_id(), repeat_node); + RETURN_IF_ERROR(cur_pipe->add_operator(builder)); + break; + } + default: + return Status::InternalError("Unsupported exec type in pipeline: {}", + print_plan_node_type(node_type)); + } + return Status::OK(); +} + +Status PipelineFragmentContext::submit() { + if (_submitted) { + return Status::InternalError("submitted"); + } + + for (auto& task : _tasks) { + RETURN_IF_ERROR(_exec_env->pipeline_task_scheduler()->schedule_task(task.get())); + } + _submitted = true; + return Status::OK(); +} + +// construct sink operator +Status PipelineFragmentContext::_create_sink(const TDataSink& thrift_sink) { + OperatorBuilderPtr sink_; + switch (thrift_sink.type) { + case TDataSinkType::DATA_STREAM_SINK: { + auto* exchange_sink = assert_cast(_sink.get()); + sink_ = std::make_shared( + next_operator_builder_id(), "ExchangeSinkOperator", nullptr, exchange_sink, this); + break; + } + case TDataSinkType::RESULT_SINK: { + auto* result_sink = assert_cast(_sink.get()); + sink_ = std::make_shared( + next_operator_builder_id(), "ResultSinkOperator", nullptr, result_sink); + break; + } + default: + return Status::InternalError("Unsuported sink type in pipeline: {}", thrift_sink.type); + } + return _root_pipeline->set_sink(sink_); +} + +void PipelineFragmentContext::close_a_pipeline() { + ++_closed_pipeline_cnt; + if (_closed_pipeline_cnt == _pipelines.size()) { + // _runtime_profile->total_time_counter()->update(_fragment_watcher.elapsed_time()); + send_report(true); + _exec_env->fragment_mgr()->remove_pipeline_context(shared_from_this()); + } +} + +// TODO pipeline dump copy from FragmentExecState::to_http_path +std::string PipelineFragmentContext::to_http_path(const std::string& file_name) { + std::stringstream url; + url << "http://" << BackendOptions::get_localhost() << ":" << config::webserver_port + << "/api/_download_load?" + << "token=" << _exec_env->token() << "&file=" << file_name; + return url.str(); +} + +// TODO pipeline dump copy from FragmentExecState::coordinator_callback +// TODO pipeline this callback should be placed in a thread pool +void PipelineFragmentContext::send_report(bool done) { + DCHECK(_closed_pipeline_cnt == _pipelines.size()); + + Status exec_status = Status::OK(); + { + std::lock_guard l(_status_lock); + if (!_exec_status.ok()) { + exec_status = _exec_status; + } + } + + // If plan is done successfully, but _is_report_success is false, + // no need to send report. + if (!_is_report_success && done && exec_status.ok()) { + return; + } + + Status coord_status; + auto coord_addr = _query_ctx->coord_addr; + FrontendServiceConnection coord(_exec_env->frontend_client_cache(), coord_addr, &coord_status); + if (!coord_status.ok()) { + std::stringstream ss; + ss << "couldn't get a client for " << coord_addr << ", reason: " << coord_status; + LOG(WARNING) << "query_id: " << print_id(_query_id) << ", " << ss.str(); + { + std::lock_guard l(_status_lock); + if (_exec_status.ok()) { + _exec_status = Status::InternalError(ss.str()); + } + } + return; + } + auto* profile = _is_report_success ? _runtime_state->runtime_profile() : nullptr; + + TReportExecStatusParams params; + params.protocol_version = FrontendServiceVersion::V1; + params.__set_query_id(_query_id); + params.__set_backend_num(_backend_num); + params.__set_fragment_instance_id(_fragment_instance_id); + exec_status.set_t_status(¶ms); + params.__set_done(true); + + auto* runtime_state = _runtime_state.get(); + DCHECK(runtime_state != nullptr); + if (runtime_state->query_type() == TQueryType::LOAD && !done && exec_status.ok()) { + // this is a load plan, and load is not finished, just make a brief report + params.__set_loaded_rows(runtime_state->num_rows_load_total()); + params.__set_loaded_bytes(runtime_state->num_bytes_load_total()); + } else { + if (runtime_state->query_type() == TQueryType::LOAD) { + params.__set_loaded_rows(runtime_state->num_rows_load_total()); + params.__set_loaded_bytes(runtime_state->num_bytes_load_total()); + } + if (profile == nullptr) { + params.__isset.profile = false; + } else { + profile->to_thrift(¶ms.profile); + params.__isset.profile = true; + } + + if (!runtime_state->output_files().empty()) { + params.__isset.delta_urls = true; + for (auto& it : runtime_state->output_files()) { + params.delta_urls.push_back(to_http_path(it)); + } + } + if (runtime_state->num_rows_load_total() > 0 || + runtime_state->num_rows_load_filtered() > 0) { + params.__isset.load_counters = true; + + static std::string s_dpp_normal_all = "dpp.norm.ALL"; + static std::string s_dpp_abnormal_all = "dpp.abnorm.ALL"; + static std::string s_unselected_rows = "unselected.rows"; + + params.load_counters.emplace(s_dpp_normal_all, + std::to_string(runtime_state->num_rows_load_success())); + params.load_counters.emplace(s_dpp_abnormal_all, + std::to_string(runtime_state->num_rows_load_filtered())); + params.load_counters.emplace(s_unselected_rows, + std::to_string(runtime_state->num_rows_load_unselected())); + } + if (!runtime_state->get_error_log_file_path().empty()) { + params.__set_tracking_url( + to_load_error_http_path(runtime_state->get_error_log_file_path())); + } + if (!runtime_state->export_output_files().empty()) { + params.__isset.export_files = true; + params.export_files = runtime_state->export_output_files(); + } + if (!runtime_state->tablet_commit_infos().empty()) { + params.__isset.commitInfos = true; + params.commitInfos.reserve(runtime_state->tablet_commit_infos().size()); + for (auto& info : runtime_state->tablet_commit_infos()) { + params.commitInfos.push_back(info); + } + } + if (!runtime_state->error_tablet_infos().empty()) { + params.__isset.errorTabletInfos = true; + params.errorTabletInfos.reserve(runtime_state->error_tablet_infos().size()); + for (auto& info : runtime_state->error_tablet_infos()) { + params.errorTabletInfos.push_back(info); + } + } + + // Send new errors to coordinator + runtime_state->get_unreported_errors(&(params.error_log)); + params.__isset.error_log = (params.error_log.size() > 0); + } + + if (_exec_env->master_info()->__isset.backend_id) { + params.__set_backend_id(_exec_env->master_info()->backend_id); + } + + TReportExecStatusResult res; + Status rpc_status; + + VLOG_DEBUG << "reportExecStatus params is " + << apache::thrift::ThriftDebugString(params).c_str(); + try { + try { + coord->reportExecStatus(res, params); + } catch (TTransportException& e) { + LOG(WARNING) << "Retrying ReportExecStatus. query id: " << print_id(_query_id) + << ", instance id: " << print_id(_fragment_instance_id) << " to " + << coord_addr << ", err: " << e.what(); + rpc_status = coord.reopen(); + + if (!rpc_status.ok()) { + // we need to cancel the execution of this fragment + cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "report rpc fail"); + return; + } + coord->reportExecStatus(res, params); + } + + rpc_status = Status(res.status); + } catch (TException& e) { + std::stringstream msg; + msg << "ReportExecStatus() to " << coord_addr << " failed:\n" << e.what(); + LOG(WARNING) << msg.str(); + rpc_status = Status::InternalError(msg.str()); + } + + if (!rpc_status.ok()) { + // we need to cancel the execution of this fragment + cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "rpc fail 2"); + } +} + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h new file mode 100644 index 0000000000..f35c6cecd1 --- /dev/null +++ b/be/src/pipeline/pipeline_fragment_context.h @@ -0,0 +1,127 @@ +// 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. + +#pragma once + +#include "pipeline.h" +#include "runtime/runtime_state.h" + +namespace doris { +class ExecNode; +class DataSink; + +namespace pipeline { + +class PipelineTask; + +class PipelineFragmentContext : public std::enable_shared_from_this { +public: + PipelineFragmentContext(const TUniqueId& query_id, const TUniqueId& instance_id, + int backend_num, std::shared_ptr query_ctx, + ExecEnv* exec_env); + + virtual ~PipelineFragmentContext(); + + PipelinePtr add_pipeline(); + + TUniqueId get_fragment_id() { return _fragment_id; } + + RuntimeState* get_runtime_state() { return _runtime_state.get(); } + + // should be protected by lock? + bool is_canceled() const { return _runtime_state->is_cancelled(); } + + int32_t next_operator_builder_id() { return _next_operator_builder_id++; } + + Status prepare(const doris::TExecPlanFragmentParams& request); + + Status submit(); + + void set_is_report_success(bool is_report_success) { _is_report_success = is_report_success; } + + ExecNode*& plan() { return _root_plan; } + + void set_need_wait_execution_trigger() { _need_wait_execution_trigger = true; } + + void cancel(const PPlanFragmentCancelReason& reason = PPlanFragmentCancelReason::INTERNAL_ERROR, + const std::string& msg = ""); + + // TODO: Support pipeline runtime filter + + QueryFragmentsCtx* get_query_context() { return _query_ctx.get(); } + + TUniqueId get_query_id() const { return _query_id; } + + void close_a_pipeline(); + + std::string to_http_path(const std::string& file_name); + + void send_report(bool); + +private: + // Id of this query + TUniqueId _query_id; + // Id of this instance + TUniqueId _fragment_instance_id; + + int _backend_num; + + ExecEnv* _exec_env; + TUniqueId _fragment_id; + + bool _prepared = false; + bool _submitted = false; + + std::mutex _status_lock; + Status _exec_status; + PPlanFragmentCancelReason _cancel_reason; + std::string _cancel_msg; + + Pipelines _pipelines; + PipelineId _next_pipeline_id = 0; + std::atomic _closed_pipeline_cnt; + + int32_t _next_operator_builder_id = 10000; + + std::vector> _tasks; + + PipelinePtr _root_pipeline; + + std::unique_ptr _runtime_profile; + bool _is_report_success = false; + + std::unique_ptr _runtime_state; + + ExecNode* _root_plan = nullptr; // lives in _runtime_state->obj_pool() + std::unique_ptr _sink; + + std::shared_ptr _query_ctx; + + // If set the true, this plan fragment will be executed only after FE send execution start rpc. + bool _need_wait_execution_trigger = false; + + MonotonicStopWatch _fragment_watcher; + // RuntimeProfile::Counter* _start_timer; + // RuntimeProfile::Counter* _prepare_timer; + +private: + Status _create_sink(const TDataSink& t_data_sink); + Status _build_pipelines(ExecNode*, PipelinePtr); + Status _build_pipeline_tasks(const doris::TExecPlanFragmentParams& request); +}; +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp new file mode 100644 index 0000000000..fbe1d6b819 --- /dev/null +++ b/be/src/pipeline/pipeline_task.cpp @@ -0,0 +1,219 @@ +// 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 "pipeline_task.h" + +#include "pipeline/pipeline_fragment_context.h" + +namespace doris::pipeline { + +void PipelineTask::_init_profile() { + std::stringstream ss; + ss << "PipelineTask" + << " (index=" << _index << ")"; + auto* task_profile = new RuntimeProfile(ss.str()); + _parent_profile->add_child(task_profile, true, nullptr); + _task_profile.reset(task_profile); + _sink_timer = ADD_TIMER(_task_profile, "SinkTime"); + _get_block_timer = ADD_TIMER(_task_profile, "GetBlockTime"); + _wait_source_timer = ADD_TIMER(_task_profile, "WaitSourceTime"); + _wait_sink_timer = ADD_TIMER(_task_profile, "WaitSinkTime"); + _wait_worker_timer = ADD_TIMER(_task_profile, "WaitWorkerTime"); + _wait_schedule_timer = ADD_TIMER(_task_profile, "WaitScheduleTime"); + _block_counts = ADD_COUNTER(_task_profile, "NumBlockedTimes", TUnit::UNIT); + _yield_counts = ADD_COUNTER(_task_profile, "NumYieldTimes", TUnit::UNIT); +} + +Status PipelineTask::prepare(RuntimeState* state) { + DCHECK(_sink); + DCHECK(_cur_state == NOT_READY); + _init_profile(); + RETURN_IF_ERROR(_sink->prepare(state)); + for (auto& o : _operators) { + RETURN_IF_ERROR(o->prepare(state)); + } + _block.reset(new doris::vectorized::Block()); + _init_state(); + _prepared = true; + return Status::OK(); +} + +void PipelineTask::_init_state() { + if (has_dependency()) { + set_state(BLOCKED_FOR_DEPENDENCY); + } else if (!(_source->can_read())) { + set_state(BLOCKED_FOR_SOURCE); + } else if (!(_sink->can_write())) { + set_state(BLOCKED_FOR_SINK); + } else { + set_state(RUNNABLE); + } +} + +bool PipelineTask::has_dependency() { + if (_dependency_finish) { + return false; + } + if (_fragment_context->is_canceled()) { + _dependency_finish = true; + return false; + } + if (_pipeline->has_dependency()) { + return true; + } + // FE do not call execute + if (!_state->get_query_fragments_ctx() + ->is_ready_to_execute()) { // TODO pipeline config::s_ready_to_execute + return true; + } + + // runtime filter is a dependency + _dependency_finish = true; + return false; +} + +Status PipelineTask::open() { + if (_sink) { + RETURN_IF_ERROR(_sink->open(_state)); + } + for (auto& o : _operators) { + RETURN_IF_ERROR(o->open(_state)); + } + _opened = true; + return Status::OK(); +} + +Status PipelineTask::execute(bool* eos) { + SCOPED_ATTACH_TASK(runtime_state()); + SCOPED_TIMER(_task_profile->total_time_counter()); + int64_t time_spent = 0; + // The status must be runnable + *eos = false; + if (!_opened) { + if (!_source->can_read()) { + set_state(BLOCKED_FOR_SOURCE); + return Status::OK(); + } + if (!_sink->can_write()) { + set_state(BLOCKED_FOR_SINK); + return Status::OK(); + } + SCOPED_RAW_TIMER(&time_spent); + RETURN_IF_ERROR(open()); + } + + while (!_fragment_context->is_canceled()) { + if (!_source->can_read() && _data_state != SourceState::MORE_DATA) { + set_state(BLOCKED_FOR_SOURCE); + break; + } + if (!_sink->can_write()) { + set_state(BLOCKED_FOR_SINK); + break; + } + if (time_spent > THREAD_TIME_SLICE) { + COUNTER_UPDATE(_yield_counts, 1); + break; + } + SCOPED_RAW_TIMER(&time_spent); + _block->clear_column_data(_root->row_desc().num_materialized_slots()); + auto* block = _block.get(); + + // Pull block from operator chain + { + SCOPED_TIMER(_get_block_timer); + RETURN_IF_ERROR(_root->get_block(_state, block, _data_state)); + } + *eos = _data_state == SourceState::FINISHED; + if (_block->rows() != 0 || *eos) { + SCOPED_TIMER(_sink_timer); + RETURN_IF_ERROR(_sink->sink(_state, block, _data_state)); + if (*eos) { // just return, the scheduler will do finish work + break; + } + } + *eos = false; + } + + return Status::OK(); +} + +Status PipelineTask::finalize() { + return _sink->finalize(_state); +} + +Status PipelineTask::close() { + auto s = _sink->close(_state); + for (auto& op : _operators) { + auto tem = op->close(_state); + if (!tem.ok() && s.ok()) { + s = tem; + } + } + if (_opened) { + COUNTER_UPDATE(_wait_source_timer, _wait_source_watcher.elapsed_time()); + COUNTER_UPDATE(_wait_sink_timer, _wait_sink_watcher.elapsed_time()); + COUNTER_UPDATE(_wait_worker_timer, _wait_worker_watcher.elapsed_time()); + COUNTER_UPDATE(_wait_schedule_timer, _wait_schedule_watcher.elapsed_time()); + } + _pipeline->close(_state); + return s; +} + +QueryFragmentsCtx* PipelineTask::query_fragments_context() { + return _fragment_context->get_query_context(); +} + +// The FSM see PipelineTaskState's comment +void PipelineTask::set_state(PipelineTaskState state) { + if (_cur_state == state) { + return; + } + if (_cur_state == BLOCKED_FOR_SOURCE) { + if (state == RUNNABLE) { + _wait_source_watcher.stop(); + } + } else if (_cur_state == BLOCKED_FOR_SINK) { + if (state == RUNNABLE) { + _wait_sink_watcher.stop(); + } + } else if (_cur_state == RUNNABLE) { + if (state == BLOCKED_FOR_SOURCE) { + _wait_source_watcher.start(); + COUNTER_UPDATE(_block_counts, 1); + } else if (state == BLOCKED_FOR_SINK) { + _wait_sink_watcher.start(); + COUNTER_UPDATE(_block_counts, 1); + } else if (state == BLOCKED_FOR_DEPENDENCY) { + COUNTER_UPDATE(_block_counts, 1); + } + } + _cur_state = state; +} + +std::string PipelineTask::debug_string() const { + std::stringstream ss; + ss << "PipelineTask(" << _index << ")" << get_state_name(_cur_state) << "\nsink: "; + ss << _sink->debug_string(); + ss << "\n operators(from source to root)"; + for (auto operatr : _operators) { + ss << "\n" << operatr->debug_string(); + } + return ss.str(); +} + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h new file mode 100644 index 0000000000..a7f0dffec0 --- /dev/null +++ b/be/src/pipeline/pipeline_task.h @@ -0,0 +1,204 @@ +// 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. + +#pragma once + +#include "exec/operator.h" +#include "pipeline.h" +#include "util/stopwatch.hpp" + +namespace doris::pipeline { + +/** + * PipelineTaskState indicates all possible states of a pipeline task. + * A FSM is described as below: + * + * |-----------------------------------------------------| + * |---| transfer 2 transfer 3 | transfer 4 + * |-------> BLOCKED ------------| |---------------------------------------> CANCELED + * |------| | | transfer 5 transfer 6| + * NOT_READY ---| transfer 0 |-----> RUNNABLE ---|---------> PENDING_FINISH ------| + * | | ^ | transfer 7| + * |------------------------------------| |--------|---------------------------------------> FINISHED + * transfer 1 transfer 9 transfer 8 + * BLOCKED include BLOCKED_FOR_DEPENDENCY, BLOCKED_FOR_SOURCE and BLOCKED_FOR_SINK. + * + * transfer 0 (NOT_READY -> BLOCKED): this pipeline task has some incomplete dependencies + * transfer 1 (NOT_READY -> RUNNABLE): this pipeline task has no incomplete dependencies + * transfer 2 (BLOCKED -> RUNNABLE): runnable condition for this pipeline task is met (e.g. get a new block from rpc) + * transfer 3 (RUNNABLE -> BLOCKED): runnable condition for this pipeline task is not met (e.g. sink operator send a block by RPC and wait for a response) + * transfer 4 (RUNNABLE -> CANCELED): current fragment is cancelled + * transfer 5 (RUNNABLE -> PENDING_FINISH): this pipeline task completed but wait for releasing resources hold by itself + * transfer 6 (PENDING_FINISH -> CANCELED): current fragment is cancelled + * transfer 7 (PENDING_FINISH -> FINISHED): this pipeline task completed and resources hold by itself have been released already + * transfer 8 (RUNNABLE -> FINISHED): this pipeline task completed and no resource need to be released + * transfer 9 (RUNNABLE -> RUNNABLE): this pipeline task yields CPU and re-enters the runnable queue if it is runnable and has occupied CPU for a max time slice + */ +enum PipelineTaskState : uint8_t { + NOT_READY = 0, // do not prepare + BLOCKED_FOR_DEPENDENCY = 1, + BLOCKED_FOR_SOURCE = 2, + BLOCKED_FOR_SINK = 3, + RUNNABLE = 4, // can execute + PENDING_FINISH = + 5, // compute task is over, but still hold resource. like some scan and sink task + FINISHED = 6, + CANCELED = 7 +}; + +inline const char* get_state_name(PipelineTaskState idx) { + switch (idx) { + case PipelineTaskState::NOT_READY: + return "NOT_READY"; + case PipelineTaskState::BLOCKED_FOR_DEPENDENCY: + return "BLOCKED_FOR_DEPENDENCY"; + case PipelineTaskState::BLOCKED_FOR_SOURCE: + return "BLOCKED_FOR_SOURCE"; + case PipelineTaskState::BLOCKED_FOR_SINK: + return "BLOCKED_FOR_SINK"; + case PipelineTaskState::RUNNABLE: + return "RUNNABLE"; + case PipelineTaskState::PENDING_FINISH: + return "PENDING_FINISH"; + case PipelineTaskState::FINISHED: + return "FINISHED"; + case PipelineTaskState::CANCELED: + return "CANCELED"; + } + __builtin_unreachable(); +} + +// The class do the pipeline task. Minest schdule union by task scheduler +class PipelineTask { +public: + PipelineTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* state, Operators& operators, + OperatorPtr& sink, PipelineFragmentContext* fragment_context, + RuntimeProfile* parent_profile) + : _index(index), + _pipeline(pipeline), + _operators(operators), + _source(_operators.front()), + _root(_operators.back()), + _sink(sink), + _prepared(false), + _opened(false), + _state(state), + _cur_state(NOT_READY), + _data_state(SourceState::DEPEND_ON_SOURCE), + _fragment_context(fragment_context), + _parent_profile(parent_profile) {} + + Status prepare(RuntimeState* state); + + Status execute(bool* eos); + + // if the pipeline create a bunch of pipeline task + // must be call after all pipeline task is finish to release resource + Status close(); + + void start_worker_watcher() { _wait_worker_watcher.start(); } + void stop_worker_watcher() { _wait_worker_watcher.stop(); } + void start_schedule_watcher() { _wait_schedule_watcher.start(); } + void stop_schedule_watcher() { _wait_schedule_watcher.stop(); } + + PipelineTaskState get_state() { return _cur_state; } + void set_state(PipelineTaskState state); + bool is_blocking_state() { + switch (_cur_state) { + case BLOCKED_FOR_DEPENDENCY: + case BLOCKED_FOR_SOURCE: + case BLOCKED_FOR_SINK: + return true; + default: + return false; + } + } + + bool is_pending_finish() { return _source->is_pending_finish() || _sink->is_pending_finish(); } + + bool source_can_read() { return _source->can_read(); } + + bool sink_can_write() { return _sink->can_write(); } + + Status finalize(); + + void finish_p_dependency() { + for (const auto& p : _pipeline->_parents) { + p->finish_one_dependency(); + } + } + + PipelineFragmentContext* fragment_context() { return _fragment_context; } + + QueryFragmentsCtx* query_fragments_context(); + + int get_previous_core_id() const { return _previous_schedule_id; } + + void set_previous_core_id(int id) { _previous_schedule_id = id; } + + bool has_dependency(); + + uint32_t index() const { return _index; } + + OperatorPtr get_root() { return _root; } + + std::string debug_string() const; + + RuntimeState* runtime_state() { return _state; } + + static constexpr auto THREAD_TIME_SLICE = 100'000'000L; + +private: + Status open(); + void _init_profile(); + void _init_state(); + +private: + uint32_t _index; + PipelinePtr _pipeline; + bool _dependency_finish = false; + Operators _operators; // left is _source, right is _root + OperatorPtr _source; + OperatorPtr _root; + OperatorPtr _sink; + + bool _prepared; + bool _opened; + RuntimeState* _state; + int _previous_schedule_id = -1; + PipelineTaskState _cur_state; + SourceState _data_state; + std::unique_ptr _block; + PipelineFragmentContext* _fragment_context; + + RuntimeProfile* _parent_profile; + std::unique_ptr _task_profile; + RuntimeProfile::Counter* _sink_timer; + RuntimeProfile::Counter* _get_block_timer; + RuntimeProfile::Counter* _block_counts; + MonotonicStopWatch _wait_source_watcher; + RuntimeProfile::Counter* _wait_source_timer; + MonotonicStopWatch _wait_sink_watcher; + RuntimeProfile::Counter* _wait_sink_timer; + MonotonicStopWatch _wait_worker_watcher; + RuntimeProfile::Counter* _wait_worker_timer; + // TODO we should calculate the time between when really runnable and runnable + MonotonicStopWatch _wait_schedule_watcher; + RuntimeProfile::Counter* _wait_schedule_timer; + RuntimeProfile::Counter* _yield_counts; +}; +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/task_scheduler.cpp b/be/src/pipeline/task_scheduler.cpp new file mode 100644 index 0000000000..f8d7e28f91 --- /dev/null +++ b/be/src/pipeline/task_scheduler.cpp @@ -0,0 +1,327 @@ +// 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 "task_scheduler.h" + +#include "common/signal_handler.h" +#include "pipeline_fragment_context.h" +#include "util/thread.h" + +namespace doris::pipeline { + +Status BlockedTaskScheduler::start() { + LOG(INFO) << "BlockedTaskScheduler start"; + RETURN_IF_ERROR(Thread::create( + "BlockedTaskScheduler", "schedule_blocked_pipeline", [this]() { this->_schedule(); }, + &_thread)); + while (!this->_started.load()) { + std::this_thread::sleep_for(std::chrono::milliseconds(5)); + } + return Status::OK(); +} + +void BlockedTaskScheduler::shutdown() { + LOG(INFO) << "Start shutdown BlockedTaskScheduler"; + if (!this->_shutdown) { + this->_shutdown = true; + if (_thread) { + _task_cond.notify_one(); + _thread->join(); + } + } +} + +void BlockedTaskScheduler::add_blocked_task(PipelineTask* task) { + std::unique_lock lock(_task_mutex); + _blocked_tasks.push_back(task); + _task_cond.notify_one(); +} + +void BlockedTaskScheduler::_schedule() { + LOG(INFO) << "BlockedTaskScheduler schedule thread start"; + _started.store(true); + std::list local_blocked_tasks; + int empty_times = 0; + std::vector ready_tasks; + + while (!_shutdown.load()) { + { + std::unique_lock lock(this->_task_mutex); + local_blocked_tasks.splice(local_blocked_tasks.end(), _blocked_tasks); + if (local_blocked_tasks.empty()) { + while (!_shutdown.load() && _blocked_tasks.empty()) { + _task_cond.wait_for(lock, std::chrono::milliseconds(10)); + } + + if (_shutdown.load()) { + break; + } + + DCHECK(!_blocked_tasks.empty()); + local_blocked_tasks.splice(local_blocked_tasks.end(), _blocked_tasks); + } + } + + auto iter = local_blocked_tasks.begin(); + DateTimeValue now = DateTimeValue::local_time(); + while (iter != local_blocked_tasks.end()) { + auto* task = *iter; + auto state = task->get_state(); + if (state == PENDING_FINISH) { + // should cancel or should finish + if (task->is_pending_finish()) { + iter++; + } else { + _make_task_run(local_blocked_tasks, iter, ready_tasks, PENDING_FINISH); + } + } else if (task->fragment_context()->is_canceled()) { + if (task->is_pending_finish()) { + task->set_state(PENDING_FINISH); + iter++; + } else { + _make_task_run(local_blocked_tasks, iter, ready_tasks); + } + } else if (task->query_fragments_context()->is_timeout(now)) { + LOG(WARNING) << "Timeout, query_id=" + << print_id(task->query_fragments_context()->query_id) + << ", instance_id=" + << print_id(task->fragment_context()->get_fragment_id()); + + task->fragment_context()->cancel(PPlanFragmentCancelReason::TIMEOUT); + + if (task->is_pending_finish()) { + task->set_state(PENDING_FINISH); + iter++; + } else { + _make_task_run(local_blocked_tasks, iter, ready_tasks); + } + } else if (state == BLOCKED_FOR_DEPENDENCY) { + if (task->has_dependency()) { + iter++; + } else { + _make_task_run(local_blocked_tasks, iter, ready_tasks); + } + } else if (state == BLOCKED_FOR_SOURCE) { + if (task->source_can_read()) { + _make_task_run(local_blocked_tasks, iter, ready_tasks); + } else { + iter++; + } + } else if (state == BLOCKED_FOR_SINK) { + if (task->sink_can_write()) { + _make_task_run(local_blocked_tasks, iter, ready_tasks); + } else { + iter++; + } + } else { + // TODO: DCHECK the state + _make_task_run(local_blocked_tasks, iter, ready_tasks); + } + } + + if (ready_tasks.empty()) { + empty_times += 1; + } else { + empty_times = 0; + for (auto& task : ready_tasks) { + task->stop_schedule_watcher(); + _task_queue->push_back(task); + } + ready_tasks.clear(); + } + + if (empty_times != 0 && (empty_times & (EMPTY_TIMES_TO_YIELD - 1)) == 0) { +#ifdef __x86_64__ + _mm_pause(); +#else + sched_yield(); +#endif + } + if (empty_times == EMPTY_TIMES_TO_YIELD * 10) { + empty_times = 0; + sched_yield(); + } + } + LOG(INFO) << "BlockedTaskScheduler schedule thread stop"; +} + +void BlockedTaskScheduler::_make_task_run(std::list& local_tasks, + std::list::iterator& task_itr, + std::vector& ready_tasks, + PipelineTaskState t_state) { + auto task = *task_itr; + task->start_schedule_watcher(); + task->set_state(t_state); + local_tasks.erase(task_itr++); + ready_tasks.emplace_back(task); +} + +///////////////////////// TaskScheduler /////////////////////////////////////////////////////////////////////////// + +TaskScheduler::~TaskScheduler() { + shutdown(); +} + +Status TaskScheduler::start() { + int cores = _task_queue->cores(); + // Must be mutil number of cpu cores + ThreadPoolBuilder("TaskSchedulerThreadPool") + .set_min_threads(cores) + .set_max_threads(cores) + .set_max_queue_size(0) + .build(&_fix_thread_pool); + _markers.reserve(cores); + for (size_t i = 0; i < cores; ++i) { + LOG(INFO) << "Start TaskScheduler thread " << i; + _markers.push_back(std::make_unique>(true)); + RETURN_IF_ERROR( + _fix_thread_pool->submit_func(std::bind(&TaskScheduler::_do_work, this, i))); + } + return _blocked_task_scheduler->start(); +} + +Status TaskScheduler::schedule_task(PipelineTask* task) { + if (task->is_blocking_state()) { + _blocked_task_scheduler->add_blocked_task(task); + } else { + _task_queue->push_back(task); + } + // TODO control num of task + return Status::OK(); +} + +void TaskScheduler::_do_work(size_t index) { + LOG(INFO) << "Start TaskScheduler worker " << index; + auto queue = _task_queue; + const auto& marker = _markers[index]; + while (*marker) { + auto task = queue->try_take(index); + if (!task) { + task = queue->steal_take(index); + if (!task) { + // TODO: The take is a stock method, rethink the logic + task = queue->take(index); + if (!task) { + continue; + } + } + } + task->stop_worker_watcher(); + auto* fragment_ctx = task->fragment_context(); + doris::signal::query_id_hi = fragment_ctx->get_query_id().hi; + doris::signal::query_id_lo = fragment_ctx->get_query_id().lo; + bool canceled = fragment_ctx->is_canceled(); + + auto check_state = task->get_state(); + if (check_state == PENDING_FINISH) { + bool is_pending = task->is_pending_finish(); + DCHECK(!is_pending) << "must not pending close " << task->debug_string(); + _try_close_task(task, canceled ? CANCELED : FINISHED); + continue; + } + DCHECK(check_state != FINISHED && check_state != CANCELED) << "task already finish"; + + if (canceled) { + // may change from pending FINISH,should called cancel + // also may change form BLOCK, other task called cancel + _try_close_task(task, CANCELED); + continue; + } + + DCHECK(check_state == RUNNABLE); + // task exec + bool eos = false; + auto status = task->execute(&eos); + task->set_previous_core_id(index); + if (!status.ok()) { + LOG(WARNING) << "Pipeline taks execute task fail " << task->debug_string(); + // exec failed,cancel all fragment instance + fragment_ctx->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "execute fail"); + _try_close_task(task, CANCELED); + continue; + } + + if (eos) { + // TODO: pipeline parallel need to wait the last task finish to call finalize + // and find_p_dependency + status = task->finalize(); + if (!status.ok()) { + // execute failed,cancel all fragment + fragment_ctx->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, + "finalize fail:" + status.to_string()); + _try_close_task(task, CANCELED); + } else { + task->finish_p_dependency(); + _try_close_task(task, FINISHED); + } + continue; + } + + auto pipeline_state = task->get_state(); + switch (pipeline_state) { + case BLOCKED_FOR_SOURCE: + case BLOCKED_FOR_SINK: + _blocked_task_scheduler->add_blocked_task(task); + break; + case RUNNABLE: + queue->push_back(task, index); + break; + default: + DCHECK(false) << "error state after run task, " << get_state_name(pipeline_state); + break; + } + } + LOG(INFO) << "Stop TaskScheduler worker " << index; +} + +void TaskScheduler::_try_close_task(PipelineTask* task, PipelineTaskState state) { + // state only should be CANCELED or FINISHED + if (task->is_pending_finish()) { + task->set_state(PENDING_FINISH); + _blocked_task_scheduler->add_blocked_task(task); + } else { + auto status = task->close(); + if (!status.ok()) { + // TODO: LOG warning + } + task->set_state(state); + // TODO: rethink the logic + if (state == CANCELED) { + task->finish_p_dependency(); + } + task->fragment_context()->close_a_pipeline(); + } +} + +void TaskScheduler::shutdown() { + if (!this->_shutdown.load()) { + this->_shutdown.store(true); + _blocked_task_scheduler->shutdown(); + if (_task_queue) { + _task_queue->close(); + } + if (_fix_thread_pool) { + for (const auto& marker : _markers) { + marker->store(false); + } + _fix_thread_pool->shutdown(); + _fix_thread_pool->wait(); + } + } +} + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/task_scheduler.h b/be/src/pipeline/task_scheduler.h new file mode 100644 index 0000000000..77e2ee8952 --- /dev/null +++ b/be/src/pipeline/task_scheduler.h @@ -0,0 +1,270 @@ +// 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. + +#pragma once + +#include + +#include "common/status.h" +#include "pipeline.h" +#include "pipeline_task.h" +#include "util/threadpool.h" + +namespace doris::pipeline { + +class SubWorkTaskQueue { + friend class WorkTaskQueue; + +public: + void add_total_time(const uint64_t duration) { _total_consume_time.fetch_add(duration); } + + void push_back(PipelineTask* task) { _queue.emplace(task); } + + PipelineTask* try_take() { + if (_queue.empty()) { + return nullptr; + } + auto task = _queue.front(); + _queue.pop(); + return task; + } + + void set_factor_for_normal(double factor_for_normal) { _factor_for_normal = factor_for_normal; } + + // TODO pipeline 1 may overflow here ? + double total_consume_time() { return _total_consume_time.load() / _factor_for_normal; } + + bool empty() { return _queue.empty(); } + +private: + std::queue _queue; + // factor for normalization + double _factor_for_normal = 1; + // TODO pipeline whether need to set to zero + // the value cal the queue task time consume, the WorkTaskQueue + // use it to find the min queue to take task work + std::atomic _total_consume_time = 0; +}; + +// Each thread have private muti level queue +class WorkTaskQueue { +public: + explicit WorkTaskQueue() : _closed(false) { + double factor = 1; + for (int i = SUB_QUEUE_LEVEL - 1; i >= 0; --i) { + _sub_queues[i].set_factor_for_normal(factor); + factor *= LEVEL_QUEUE_TIME_FACTOR; + } + } + + void close() { + std::unique_lock lock(_work_size_mutex); + _closed = true; + _wait_task.notify_all(); + } + + PipelineTask* try_take_unprotected() { + if (_total_task_size == 0 || _closed) { + return nullptr; + } + double min_consume_time = _sub_queues[0].total_consume_time(); + int idx = 0; + for (int i = 1; i < SUB_QUEUE_LEVEL; ++i) { + if (!_sub_queues[i].empty()) { + double consume_time = _sub_queues[i].total_consume_time(); + if (idx == -1 || consume_time < min_consume_time) { + idx = i; + min_consume_time = consume_time; + } + } + } + auto task = _sub_queues[idx].try_take(); + if (task) { + _total_task_size--; + } + return task; + } + + PipelineTask* try_take() { + std::unique_lock lock(_work_size_mutex); + return try_take_unprotected(); + } + + PipelineTask* take() { + std::unique_lock lock(_work_size_mutex); + while (!_closed) { + auto task = try_take_unprotected(); + if (task) { + return task; + } else { + _wait_task.wait(lock); + } + } + DCHECK(_closed); + return nullptr; + } + + void push(PipelineTask* task) { + size_t level = _compute_level(task); + std::unique_lock lock(_work_size_mutex); + _sub_queues[level].push_back(task); + _total_task_size++; + _wait_task.notify_one(); + } + + // Get the each thread task size to do + size_t size() { return _total_task_size; } + +private: + static constexpr auto LEVEL_QUEUE_TIME_FACTOR = 1.2; + static constexpr size_t SUB_QUEUE_LEVEL = 5; + SubWorkTaskQueue _sub_queues[SUB_QUEUE_LEVEL]; + std::mutex _work_size_mutex; + std::condition_variable _wait_task; + std::atomic _total_task_size = 0; + bool _closed; + +private: + size_t _compute_level(PipelineTask* task) { return 0; } +}; + +// Need consider NUMA architecture +class TaskQueue { +public: + explicit TaskQueue(size_t core_size) : _core_size(core_size) { + _async_queue.reset(new WorkTaskQueue[core_size]); + } + + ~TaskQueue() = default; + + void close() { + for (int i = 0; i < _core_size; ++i) { + _async_queue[i].close(); + } + } + + // Get the task by core id. TODO: To think the logic is useful? + PipelineTask* try_take(size_t core_id) { return _async_queue[core_id].try_take(); } + + // not block, steal task by other core queue + PipelineTask* steal_take(size_t core_id) { + DCHECK(core_id < _core_size); + size_t next_id = core_id; + for (size_t i = 1; i < _core_size; ++i) { + ++next_id; + if (next_id == _core_size) { + next_id = 0; + } + DCHECK(next_id < _core_size); + auto task = try_take(next_id); + if (task) { + return task; + } + } + return nullptr; + } + + // TODO pipeline 1 add timeout interface, other queue may have new task + PipelineTask* take(size_t core_id) { return _async_queue[core_id].take(); } + + void push_back(PipelineTask* task) { + int core_id = task->get_previous_core_id(); + if (core_id < 0) { + core_id = _next_core.fetch_add(1) % _core_size; + } + push_back(task, core_id); + } + + void push_back(PipelineTask* task, size_t core_id) { + DCHECK(core_id < _core_size); + task->start_worker_watcher(); + _async_queue[core_id].push(task); + } + + int cores() const { return _core_size; } + +private: + std::unique_ptr _async_queue; + size_t _core_size; + std::atomic _next_core = 0; +}; + +// TODO pipeline sr +class BlockedTaskScheduler { +public: + explicit BlockedTaskScheduler(std::shared_ptr task_queue) + : _task_queue(std::move(task_queue)), _started(false), _shutdown(false) {} + + ~BlockedTaskScheduler() = default; + + Status start(); + void shutdown(); + void add_blocked_task(PipelineTask* task); + +private: + std::shared_ptr _task_queue; + + std::mutex _task_mutex; + std::condition_variable _task_cond; + std::list _blocked_tasks; + + scoped_refptr _thread; + std::atomic _started; + std::atomic _shutdown; + + static constexpr auto EMPTY_TIMES_TO_YIELD = 64; + +private: + void _schedule(); + void _make_task_run(std::list& local_tasks, + std::list::iterator& task_itr, + std::vector& ready_tasks, + PipelineTaskState state = PipelineTaskState::RUNNABLE); +}; + +class TaskScheduler { +public: + TaskScheduler(ExecEnv* exec_env, std::shared_ptr b_scheduler, + std::shared_ptr task_queue) + : _task_queue(std::move(task_queue)), + _exec_env(exec_env), + _blocked_task_scheduler(std::move(b_scheduler)), + _shutdown(false) {} + + ~TaskScheduler(); + + Status schedule_task(PipelineTask* task); + + Status start(); + + void shutdown(); + + ExecEnv* exec_env() { return _exec_env; } + +private: + std::unique_ptr _fix_thread_pool; + std::shared_ptr _task_queue; + std::vector>> _markers; + ExecEnv* _exec_env; + std::shared_ptr _blocked_task_scheduler; + std::atomic _shutdown; + +private: + void _do_work(size_t index); + void _try_close_task(PipelineTask* task, PipelineTaskState state); +}; +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp index 03bd6d3466..a3d765f3c2 100644 --- a/be/src/runtime/buffer_control_block.cpp +++ b/be/src/runtime/buffer_control_block.cpp @@ -95,6 +95,11 @@ Status BufferControlBlock::init() { return Status::OK(); } +bool BufferControlBlock::can_sink() { + std::unique_lock l(_lock); + return _get_batch_queue_empty() || _buffer_rows < _buffer_limit || _is_cancelled; +} + Status BufferControlBlock::add_batch(std::unique_ptr& result) { std::unique_lock l(_lock); @@ -104,7 +109,7 @@ Status BufferControlBlock::add_batch(std::unique_ptr& result) int num_rows = result->result_batch.rows.size(); - while ((!_batch_queue.empty() && (num_rows + _buffer_rows) > _buffer_limit) && !_is_cancelled) { + while ((!_batch_queue.empty() && _buffer_rows > _buffer_limit) && !_is_cancelled) { _data_removal.wait(l); } @@ -158,7 +163,7 @@ Status BufferControlBlock::get_batch(TFetchDataResult* result) { _batch_queue.pop_front(); _buffer_rows -= item->result_batch.rows.size(); _data_removal.notify_one(); - *result = *(item.get()); + *result = *item; result->__set_packet_num(_packet_num); _packet_num++; return Status::OK(); diff --git a/be/src/runtime/buffer_control_block.h b/be/src/runtime/buffer_control_block.h index 8528f74164..e1dba35762 100644 --- a/be/src/runtime/buffer_control_block.h +++ b/be/src/runtime/buffer_control_block.h @@ -17,6 +17,7 @@ #pragma once +#include #include #include #include @@ -60,9 +61,10 @@ struct GetResultBatchCtx { class BufferControlBlock { public: BufferControlBlock(const TUniqueId& id, int buffer_size); - ~BufferControlBlock(); + virtual ~BufferControlBlock(); Status init(); + virtual bool can_sink(); // 只有一个fragment写入,因此can_sink返回true,则一定可以执行sink Status add_batch(std::unique_ptr& result); // get result from batch, use timeout? @@ -86,7 +88,7 @@ public: // _query_statistics may be null when the result sink init failed // or some other failure. // and the number of written rows is only needed when all things go well. - if (_query_statistics.get() != nullptr) { + if (_query_statistics != nullptr) { _query_statistics->set_returned_rows(num_rows); } } @@ -94,22 +96,25 @@ public: // TODO: The value of query peak mem usage in fe.audit.log comes from a random BE, // not the BE with the largest peak mem usage void update_max_peak_memory_bytes() { - if (_query_statistics.get() != nullptr) { + if (_query_statistics != nullptr) { int64_t max_peak_memory_bytes = _query_statistics->calculate_max_peak_memory_bytes(); _query_statistics->set_max_peak_memory_bytes(max_peak_memory_bytes); } } -private: - typedef std::list> ResultQueue; +protected: + virtual bool _get_batch_queue_empty() { return _batch_queue.empty(); } + virtual void _update_batch_queue_empty() {} + + using ResultQueue = std::list>; // result's query id TUniqueId _fragment_id; bool _is_close; - bool _is_cancelled; + std::atomic_bool _is_cancelled; Status _status; - int _buffer_rows; - int _buffer_limit; + std::atomic_int _buffer_rows; + const int _buffer_limit; int64_t _packet_num; // blocking queue for batch @@ -129,4 +134,20 @@ private: std::shared_ptr _query_statistics; }; +class PipBufferControlBlock : public BufferControlBlock { +public: + PipBufferControlBlock(const TUniqueId& id, int buffer_size) + : BufferControlBlock(id, buffer_size) {} + + bool can_sink() override { + return _get_batch_queue_empty() || _buffer_rows < _buffer_limit || _is_cancelled; + } + +private: + bool _get_batch_queue_empty() override { return _batch_queue_empty; } + void _update_batch_queue_empty() override { _batch_queue_empty = _batch_queue.empty(); } + + std::atomic_bool _batch_queue_empty = false; +}; + } // namespace doris diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 4bd9fd73af..6ce903f9fd 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -29,6 +29,9 @@ namespace vectorized { class VDataStreamMgr; class ScannerScheduler; } // namespace vectorized +namespace pipeline { +class TaskScheduler; +} class BfdParser; class BrokerMgr; @@ -110,6 +113,8 @@ public: ClientCache* frontend_client_cache() { return _frontend_client_cache; } ClientCache* broker_client_cache() { return _broker_client_cache; } + pipeline::TaskScheduler* pipeline_task_scheduler() { return _pipeline_task_scheduler; } + // using template to simplify client cache management template ClientCache* get_client_cache() { @@ -137,6 +142,7 @@ public: } void init_download_cache_buf(); void init_download_cache_required_components(); + Status init_pipeline_task_scheduler(); char* get_download_cache_buf(ThreadPoolToken* token) { if (_download_cache_buf_map.find(token) == _download_cache_buf_map.end()) { return nullptr; @@ -233,6 +239,7 @@ private: std::unordered_map> _download_cache_buf_map; CgroupsMgr* _cgroups_mgr = nullptr; FragmentMgr* _fragment_mgr = nullptr; + pipeline::TaskScheduler* _pipeline_task_scheduler = nullptr; ResultCache* _result_cache = nullptr; TMasterInfo* _master_info = nullptr; LoadPathMgr* _load_path_mgr = nullptr; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 735d8091ed..d84b40c670 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -25,6 +25,7 @@ #include "olap/segment_loader.h" #include "olap/storage_engine.h" #include "olap/storage_policy_mgr.h" +#include "pipeline/task_scheduler.h" #include "runtime/broker_mgr.h" #include "runtime/bufferpool/buffer_pool.h" #include "runtime/cache/result_cache.h" @@ -128,6 +129,7 @@ Status ExecEnv::_init(const std::vector& store_paths) { init_download_cache_required_components(); + RETURN_IF_ERROR(init_pipeline_task_scheduler()); _scanner_scheduler = new doris::vectorized::ScannerScheduler(); _cgroups_mgr = new CgroupsMgr(this, config::doris_cgroups); @@ -172,6 +174,18 @@ Status ExecEnv::_init(const std::vector& store_paths) { return Status::OK(); } +Status ExecEnv::init_pipeline_task_scheduler() { + auto executors_size = config::pipeline_executor_size; + if (executors_size <= 0) { + executors_size = CpuInfo::num_cores(); + } + auto t_queue = std::make_shared(executors_size); + auto b_scheduler = std::make_shared(t_queue); + _pipeline_task_scheduler = new pipeline::TaskScheduler(this, b_scheduler, t_queue); + RETURN_IF_ERROR(_pipeline_task_scheduler->start()); + return Status::OK(); +} + Status ExecEnv::_init_mem_env() { bool is_percent = false; std::stringstream ss; @@ -348,6 +362,7 @@ void ExecEnv::_destroy() { SAFE_DELETE(_load_path_mgr); SAFE_DELETE(_master_info); SAFE_DELETE(_fragment_mgr); + SAFE_DELETE(_pipeline_task_scheduler); SAFE_DELETE(_cgroups_mgr); SAFE_DELETE(_scan_thread_pool); SAFE_DELETE(_remote_scan_thread_pool); diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index b4e0595451..b7ae9cf735 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -34,6 +34,7 @@ #include "gen_cpp/Types_types.h" #include "gutil/strings/substitute.h" #include "opentelemetry/trace/scope.h" +#include "pipeline/pipeline_fragment_context.h" #include "runtime/client_cache.h" #include "runtime/datetime_value.h" #include "runtime/descriptors.h" @@ -548,7 +549,13 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params) { set_pipe(params.params.fragment_instance_id, pipe); return Status::OK(); } else { - return exec_plan_fragment(params, std::bind(&empty_function, std::placeholders::_1)); + if (params.query_options.__isset.enable_pipeline_engine && + params.query_options.enable_pipeline_engine) { + return exec_pipeline(params); + } else { + return exec_plan_fragment(params, + std::bind(&empty_function, std::placeholders::_1)); + } } } @@ -591,6 +598,132 @@ std::shared_ptr FragmentMgr::get_pipe(const TUniqueId& fragment_ } } +Status FragmentMgr::exec_pipeline(const TExecPlanFragmentParams& params) { + auto tracer = telemetry::is_current_span_valid() ? telemetry::get_tracer("tracer") + : telemetry::get_noop_tracer(); + START_AND_SCOPE_SPAN(tracer, span, "FragmentMgr::exec_plan_fragment"); + const TUniqueId& fragment_instance_id = params.params.fragment_instance_id; + { + std::lock_guard lock(_lock); + auto iter = _pipeline_map.find(fragment_instance_id); + if (iter != _pipeline_map.end()) { + // Duplicated + return Status::OK(); + } + } + + std::shared_ptr fragments_ctx; + if (params.is_simplified_param) { + // Get common components from _fragments_ctx_map + std::lock_guard lock(_lock); + auto search = _fragments_ctx_map.find(params.params.query_id); + if (search == _fragments_ctx_map.end()) { + return Status::InternalError( + "Failed to get query fragments context. Query may be " + "timeout or be cancelled. host: {}", + BackendOptions::get_localhost()); + } + fragments_ctx = search->second; + _set_scan_concurrency(params, fragments_ctx.get()); + } else { + // This may be a first fragment request of the query. + // Create the query fragments context. + fragments_ctx = std::make_shared(params.fragment_num_on_host, _exec_env); + fragments_ctx->query_id = params.params.query_id; + RETURN_IF_ERROR(DescriptorTbl::create(&(fragments_ctx->obj_pool), params.desc_tbl, + &(fragments_ctx->desc_tbl))); + fragments_ctx->coord_addr = params.coord; + LOG(INFO) << "query_id: " + << UniqueId(fragments_ctx->query_id.hi, fragments_ctx->query_id.lo) + << " coord_addr " << fragments_ctx->coord_addr; + fragments_ctx->query_globals = params.query_globals; + + if (params.__isset.resource_info) { + fragments_ctx->user = params.resource_info.user; + fragments_ctx->group = params.resource_info.group; + fragments_ctx->set_rsc_info = true; + } + + fragments_ctx->timeout_second = params.query_options.query_timeout; + _set_scan_concurrency(params, fragments_ctx.get()); + + bool has_query_mem_tracker = + params.query_options.__isset.mem_limit && (params.query_options.mem_limit > 0); + int64_t bytes_limit = has_query_mem_tracker ? params.query_options.mem_limit : -1; + if (bytes_limit > MemInfo::mem_limit()) { + VLOG_NOTICE << "Query memory limit " << PrettyPrinter::print(bytes_limit, TUnit::BYTES) + << " exceeds process memory limit of " + << PrettyPrinter::print(MemInfo::mem_limit(), TUnit::BYTES) + << ". Using process memory limit instead"; + bytes_limit = MemInfo::mem_limit(); + } + if (params.query_options.query_type == TQueryType::SELECT) { + fragments_ctx->query_mem_tracker = std::make_shared( + MemTrackerLimiter::Type::QUERY, + fmt::format("Query#Id={}", print_id(fragments_ctx->query_id)), bytes_limit); + } else if (params.query_options.query_type == TQueryType::LOAD) { + fragments_ctx->query_mem_tracker = std::make_shared( + MemTrackerLimiter::Type::LOAD, + fmt::format("Load#Id={}", print_id(fragments_ctx->query_id)), bytes_limit); + } + if (params.query_options.__isset.is_report_success && + params.query_options.is_report_success) { + fragments_ctx->query_mem_tracker->enable_print_log_usage(); + } + + if (!params.__isset.need_wait_execution_trigger || !params.need_wait_execution_trigger) { + // 马上运行pipeline task + fragments_ctx->set_ready_to_execute_only(); + } + + { + // Find _fragments_ctx_map again, in case some other request has already + // create the query fragments context. + std::lock_guard lock(_lock); + auto search = _fragments_ctx_map.find(params.params.query_id); + if (search == _fragments_ctx_map.end()) { + _fragments_ctx_map.insert(std::make_pair(fragments_ctx->query_id, fragments_ctx)); + } else { + // Already has a query fragmentscontext, use it + fragments_ctx = search->second; + } + } + } + + std::shared_ptr context = + std::make_shared( + fragments_ctx->query_id, fragment_instance_id, params.backend_num, + fragments_ctx, _exec_env); + RETURN_IF_ERROR(context->prepare(params)); + { + std::lock_guard lock(_lock); + _pipeline_map.insert(std::make_pair(fragment_instance_id, context)); + _cv.notify_all(); + } + + auto st = context->submit(); + if (!st.ok()) { + // TODO pipeline 如果一个task都没有提交成功,则要让timeout checker线程去移除 + // 提交失败也不能移出,可能有些pipeline task提交成功,有些失败,要等所有task都结束才能移除。 + context->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "submit context fail"); + remove_pipeline_context(context); + return Status::InternalError("Submit pipeline failed. err = {}, BE: {}", st.get_error_msg(), + BackendOptions::get_localhost()); + } + return Status::OK(); +} +void FragmentMgr::remove_pipeline_context( + std::shared_ptr f_context) { + std::lock_guard lock(_lock); + auto query_id = f_context->get_query_id(); + auto* q_context = f_context->get_query_context(); + bool all_done = q_context->countdown(); + _pipeline_map.erase(f_context->get_fragment_id()); + if (all_done) { + _fragments_ctx_map.erase(query_id); + } +} + Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, FinishCallback cb) { auto tracer = telemetry::is_current_span_valid() ? telemetry::get_tracer("tracer") : telemetry::get_noop_tracer(); @@ -780,21 +913,32 @@ bool FragmentMgr::_is_scan_node(const TPlanNodeType::type& type) { type == TPlanNodeType::JDBC_SCAN_NODE; } -Status FragmentMgr::cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason, - const std::string& msg) { +void FragmentMgr::cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason, + const std::string& msg) { std::shared_ptr exec_state; { std::lock_guard lock(_lock); auto iter = _fragment_map.find(fragment_id); - if (iter == _fragment_map.end()) { - // No match - return Status::OK(); + if (iter != _fragment_map.end()) { + exec_state = iter->second; } - exec_state = iter->second; } - exec_state->cancel(reason, msg); + if (exec_state) { + exec_state->cancel(reason, msg); + return; + } - return Status::OK(); + std::shared_ptr pipeline_fragment_ctx; + { + std::lock_guard lock(_lock); + auto iter = _pipeline_map.find(fragment_id); + if (iter != _pipeline_map.end()) { + pipeline_fragment_ctx = iter->second; + } + } + if (pipeline_fragment_ctx) { + pipeline_fragment_ctx->cancel(reason, msg); + } } void FragmentMgr::cancel_worker() { @@ -962,6 +1106,7 @@ Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params, Status FragmentMgr::apply_filter(const PPublishFilterRequest* request, butil::IOBufAsZeroCopyInputStream* attach_data) { + // TODO pipeline runtime filter UniqueId fragment_instance_id = request->fragment_id(); TUniqueId tfragment_instance_id = fragment_instance_id.to_thrift(); std::shared_ptr fragment_state; diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index 2246a42ac8..4894ccddbf 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -42,6 +42,10 @@ class IOBufAsZeroCopyInputStream; namespace doris { +namespace pipeline { +class PipelineFragmentContext; +} + class QueryFragmentsCtx; class ExecEnv; class FragmentExecState; @@ -66,17 +70,22 @@ public: // execute one plan fragment Status exec_plan_fragment(const TExecPlanFragmentParams& params); + Status exec_pipeline(const TExecPlanFragmentParams& params); + + void remove_pipeline_context( + std::shared_ptr pipeline_context); + // TODO(zc): report this is over Status exec_plan_fragment(const TExecPlanFragmentParams& params, FinishCallback cb); Status start_query_execution(const PExecPlanFragmentStartRequest* request); - Status cancel(const TUniqueId& fragment_id) { - return cancel(fragment_id, PPlanFragmentCancelReason::INTERNAL_ERROR); + void cancel(const TUniqueId& fragment_id) { + cancel(fragment_id, PPlanFragmentCancelReason::INTERNAL_ERROR); } - Status cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason, - const std::string& msg = ""); + void cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason, + const std::string& msg = ""); void cancel_worker(); @@ -119,6 +128,9 @@ private: // Make sure that remove this before no data reference FragmentExecState std::unordered_map> _fragment_map; + + std::unordered_map> _pipeline_map; + // query id -> QueryFragmentsCtx std::unordered_map> _fragments_ctx_map; std::unordered_map> _bf_size_map; diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index 3a9206f1fa..f386da763d 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -190,9 +190,9 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, // set up sink, if required if (request.fragment.__isset.output_sink) { - RETURN_IF_ERROR(DataSink::create_data_sink( - obj_pool(), request.fragment.output_sink, request.fragment.output_exprs, params, - row_desc(), runtime_state()->enable_vectorized_exec(), &_sink, *desc_tbl)); + RETURN_IF_ERROR(DataSink::create_data_sink(obj_pool(), request.fragment.output_sink, + request.fragment.output_exprs, params, + row_desc(), runtime_state(), &_sink, *desc_tbl)); RETURN_IF_ERROR(_sink->prepare(runtime_state())); RuntimeProfile* sink_profile = _sink->profile(); diff --git a/be/src/runtime/query_fragments_ctx.h b/be/src/runtime/query_fragments_ctx.h index 4bfff8c4da..8513e99b88 100644 --- a/be/src/runtime/query_fragments_ctx.h +++ b/be/src/runtime/query_fragments_ctx.h @@ -90,6 +90,18 @@ public: } _start_cond.notify_all(); } + void set_ready_to_execute_only() { + { + std::lock_guard l(_start_lock); + _ready_to_execute = true; + } + _start_cond.notify_all(); + } + + bool is_ready_to_execute() { + std::lock_guard l(_start_lock); + return _ready_to_execute; + } bool wait_for_start() { int wait_time = config::max_fragment_start_wait_time_seconds; diff --git a/be/src/runtime/result_buffer_mgr.cpp b/be/src/runtime/result_buffer_mgr.cpp index ce589dd745..e1d3069459 100644 --- a/be/src/runtime/result_buffer_mgr.cpp +++ b/be/src/runtime/result_buffer_mgr.cpp @@ -17,6 +17,8 @@ #include "runtime/result_buffer_mgr.h" +#include + #include "gen_cpp/PaloInternalService_types.h" #include "gen_cpp/types.pb.h" #include "runtime/buffer_control_block.h" @@ -58,15 +60,22 @@ Status ResultBufferMgr::init() { } Status ResultBufferMgr::create_sender(const TUniqueId& query_id, int buffer_size, - std::shared_ptr* sender) { + std::shared_ptr* sender, + bool enable_pipeline) { *sender = find_control_block(query_id); if (*sender != nullptr) { LOG(WARNING) << "already have buffer control block for this instance " << query_id; return Status::OK(); } - std::shared_ptr control_block( - new BufferControlBlock(query_id, buffer_size)); + std::shared_ptr control_block = nullptr; + + if (enable_pipeline) { + control_block = std::make_shared(query_id, buffer_size); + } else { + control_block = std::make_shared(query_id, buffer_size); + } + { std::lock_guard l(_lock); _buffer_map.insert(std::make_pair(query_id, control_block)); diff --git a/be/src/runtime/result_buffer_mgr.h b/be/src/runtime/result_buffer_mgr.h index 26a07bd90c..c4722bcef7 100644 --- a/be/src/runtime/result_buffer_mgr.h +++ b/be/src/runtime/result_buffer_mgr.h @@ -47,7 +47,7 @@ public: // the returned sender do not need release // sender is not used when call cancel or unregister Status create_sender(const TUniqueId& query_id, int buffer_size, - std::shared_ptr* sender); + std::shared_ptr* sender, bool enable_pipeline); // fetch data, used by RPC Status fetch_data(const TUniqueId& fragment_id, TFetchDataResult* result); @@ -60,8 +60,8 @@ public: Status cancel_at_time(time_t cancel_time, const TUniqueId& query_id); private: - typedef std::unordered_map> BufferMap; - typedef std::map> TimeoutMap; + using BufferMap = std::unordered_map>; + using TimeoutMap = std::map>; std::shared_ptr find_control_block(const TUniqueId& query_id); diff --git a/be/src/runtime/result_file_sink.cpp b/be/src/runtime/result_file_sink.cpp index cd3e61659a..91143980cd 100644 --- a/be/src/runtime/result_file_sink.cpp +++ b/be/src/runtime/result_file_sink.cpp @@ -100,7 +100,8 @@ Status ResultFileSink::prepare(RuntimeState* state) { if (_is_top_sink) { // create sender RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender( - state->fragment_instance_id(), _buf_size, &_sender)); + state->fragment_instance_id(), _buf_size, &_sender, + _state->enable_pipeline_exec())); // create writer _writer.reset(new (std::nothrow) FileResultWriter( _file_opts.get(), _storage_type, state->fragment_instance_id(), _output_expr_ctxs, diff --git a/be/src/runtime/result_sink.cpp b/be/src/runtime/result_sink.cpp index de393e9e11..625e495ffa 100644 --- a/be/src/runtime/result_sink.cpp +++ b/be/src/runtime/result_sink.cpp @@ -66,8 +66,8 @@ Status ResultSink::prepare(RuntimeState* state) { RETURN_IF_ERROR(prepare_exprs(state)); // create sender - RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(state->fragment_instance_id(), - _buf_size, &_sender)); + RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender( + state->fragment_instance_id(), _buf_size, &_sender, state->enable_pipeline_exec())); // create writer based on sink type switch (_sink_type) { diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 7b8ae4d89d..016f0089fd 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -176,9 +176,9 @@ public: // _unreported_error_idx to _errors_log.size() void get_unreported_errors(std::vector* new_errors); - bool is_cancelled() const { return _is_cancelled; } + bool is_cancelled() const { return _is_cancelled.load(); } int codegen_level() const { return _query_options.codegen_level; } - void set_is_cancelled(bool v) { _is_cancelled = v; } + void set_is_cancelled(bool v) { _is_cancelled.store(v); } void set_backend_id(int64_t backend_id) { _backend_id = backend_id; } int64_t backend_id() const { return _backend_id; } @@ -326,6 +326,7 @@ public: } return _query_options.be_exec_version; } + bool enable_pipeline_exec() const { return _query_options.enable_pipeline_engine; } bool trim_tailing_spaces_for_external_table_query() const { return _query_options.trim_tailing_spaces_for_external_table_query; @@ -464,7 +465,7 @@ private: ThreadResourceMgr::ResourcePool* _resource_pool; // if true, execution should stop with a CANCELLED status - bool _is_cancelled; + std::atomic _is_cancelled; int _per_fragment_instance_idx; int _num_per_fragment_instances = 0; diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index fb1b0851f9..b584668f58 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -103,7 +103,7 @@ Status BackendService::start_plan_fragment_execution(const TExecPlanFragmentPara void BackendService::cancel_plan_fragment(TCancelPlanFragmentResult& return_val, const TCancelPlanFragmentParams& params) { LOG(INFO) << "cancel_plan_fragment(): instance_id=" << params.fragment_instance_id; - _exec_env->fragment_mgr()->cancel(params.fragment_instance_id).set_t_status(&return_val); + _exec_env->fragment_mgr()->cancel(params.fragment_instance_id); } void BackendService::transmit_data(TTransmitDataResult& return_val, diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 28b9b8f4ca..c33db5a884 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -388,18 +388,17 @@ void PInternalServiceImpl::cancel_plan_fragment(google::protobuf::RpcController* tid.__set_hi(request->finst_id().hi()); tid.__set_lo(request->finst_id().lo()); - Status st; + Status st = Status::OK(); if (request->has_cancel_reason()) { LOG(INFO) << "cancel fragment, fragment_instance_id=" << print_id(tid) << ", reason: " << request->cancel_reason(); - st = _exec_env->fragment_mgr()->cancel(tid, request->cancel_reason()); + _exec_env->fragment_mgr()->cancel(tid, request->cancel_reason()); } else { LOG(INFO) << "cancel fragment, fragment_instance_id=" << print_id(tid); - st = _exec_env->fragment_mgr()->cancel(tid); - } - if (!st.ok()) { - LOG(WARNING) << "cancel plan fragment failed, errmsg=" << st.get_error_msg(); + _exec_env->fragment_mgr()->cancel(tid); } + + // TODO: the logic seems useless, cancel only return Status::OK. remove it st.to_protobuf(result->mutable_status()); } diff --git a/be/src/util/proto_util.h b/be/src/util/proto_util.h index 1ad4796691..5828b8eeec 100644 --- a/be/src/util/proto_util.h +++ b/be/src/util/proto_util.h @@ -17,7 +17,15 @@ #pragma once +#include +#include + +#include "common/config.h" +#include "common/status.h" #include "exception.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_state.h" +#include "util/brpc_client_cache.h" namespace doris { @@ -28,6 +36,55 @@ namespace doris { // 2G: In the default "baidu_std" brpcd, upper limit of the request and attachment length is 2G. constexpr size_t MIN_HTTP_BRPC_SIZE = (1ULL << 31); +// Embed column_values and brpc request serialization string in controller attachment. +template +inline Status request_embed_attachment_contain_block(Params* brpc_request, Closure* closure) { + auto block = brpc_request->block(); + Status st = request_embed_attachment(brpc_request, block.column_values(), closure); + block.set_column_values(""); + return st; +} + +inline bool enable_http_send_block( + const PTransmitDataParams& request, + bool transfer_large_data_by_brpc = config::transfer_large_data_by_brpc) { + if (!config::transfer_large_data_by_brpc) { + return false; + } + if (!request.has_block() || !request.block().has_column_values()) { + return false; + } + if (request.ByteSizeLong() < MIN_HTTP_BRPC_SIZE) { + return false; + } + return true; +} + +template +inline void transmit_block(PBackendService_Stub& stub, Closure* closure, + const PTransmitDataParams& params) { + closure->cntl.http_request().Clear(); + stub.transmit_block(&closure->cntl, ¶ms, &closure->result, closure); +} + +template +inline Status transmit_block_http(RuntimeState* state, Closure* closure, + PTransmitDataParams& params, TNetworkAddress brpc_dest_addr) { + RETURN_IF_ERROR(request_embed_attachment_contain_block(¶ms, closure)); + + std::string brpc_url = + fmt::format("http://{}:{}", brpc_dest_addr.hostname, brpc_dest_addr.port); + std::shared_ptr brpc_http_stub = + state->exec_env()->brpc_internal_client_cache()->get_new_client_no_cache(brpc_url, + "http"); + closure->cntl.http_request().uri() = brpc_url + "/PInternalServiceImpl/transmit_block_by_http"; + closure->cntl.http_request().set_method(brpc::HTTP_METHOD_POST); + closure->cntl.http_request().set_content_type("application/json"); + brpc_http_stub->transmit_block_by_http(&closure->cntl, nullptr, &closure->result, closure); + + return Status::OK(); +} + // TODO(zxy) delete in v1.3 version // Transfer RowBatch in ProtoBuf Request to Controller Attachment. // This can avoid reaching the upper limit of the ProtoBuf Request length (2G), @@ -94,15 +151,6 @@ inline Status request_embed_attachment_contain_tuple(Params* brpc_request, Closu return st; } -// Embed column_values and brpc request serialization string in controller attachment. -template -inline Status request_embed_attachment_contain_block(Params* brpc_request, Closure* closure) { - auto block = brpc_request->block(); - Status st = request_embed_attachment(brpc_request, block.column_values(), closure); - block.set_column_values(""); - return st; -} - template inline Status request_embed_attachment(Params* brpc_request, const std::string& data, Closure* closure) { diff --git a/be/src/util/runtime_profile.cpp b/be/src/util/runtime_profile.cpp index 604d98bfcc..32e76c94eb 100644 --- a/be/src/util/runtime_profile.cpp +++ b/be/src/util/runtime_profile.cpp @@ -235,6 +235,11 @@ void RuntimeProfile::divide(int n) { } } +void RuntimeProfile::clear_children() { + std::lock_guard l(_children_lock); + _children.clear(); +} + void RuntimeProfile::compute_time_in_profile() { compute_time_in_profile(total_time_counter()->value()); } @@ -277,6 +282,15 @@ RuntimeProfile* RuntimeProfile::create_child(const std::string& name, bool inden return child; } +void RuntimeProfile::insert_child_head(doris::RuntimeProfile* child, bool indent) { + std::lock_guard l(_children_lock); + DCHECK(child != nullptr); + _child_map[child->_name] = child; + + auto it = _children.begin(); + _children.insert(it, std::make_pair(child, indent)); +} + void RuntimeProfile::add_child_unlock(RuntimeProfile* child, bool indent, RuntimeProfile* loc) { DCHECK(child != nullptr); _child_map[child->_name] = child; diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h index 62f0365afc..606e127274 100644 --- a/be/src/util/runtime_profile.h +++ b/be/src/util/runtime_profile.h @@ -48,10 +48,15 @@ namespace doris { #define ADD_CHILD_COUNTER(profile, name, type, parent) (profile)->add_counter(name, type, parent) #define ADD_CHILD_TIMER(profile, name, parent) (profile)->add_counter(name, TUnit::TIME_NS, parent) #define SCOPED_TIMER(c) ScopedTimer MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c) +#define SCOPED_TIMER_ATOMIC(c) \ + ScopedTimer MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c) #define SCOPED_CPU_TIMER(c) \ ScopedTimer MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c) #define CANCEL_SAFE_SCOPED_TIMER(c, is_cancelled) \ ScopedTimer MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c, is_cancelled) +#define CANCEL_SAFE_SCOPED_TIMER_ATOMIC(c, is_cancelled) \ + ScopedTimer MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)( \ + c, is_cancelled) #define SCOPED_RAW_TIMER(c) \ doris::ScopedRawTimer MACRO_CONCAT(SCOPED_RAW_TIMER, \ __COUNTER__)(c) @@ -233,6 +238,8 @@ public: // already be added to the profile. void add_child(RuntimeProfile* child, bool indent, RuntimeProfile* location); + void insert_child_head(RuntimeProfile* child, bool indent); + void add_child_unlock(RuntimeProfile* child, bool indent, RuntimeProfile* loc); /// Creates a new child profile with the given 'name'. A child profile with that name @@ -383,6 +390,8 @@ public: // This function updates _local_time_percent for each profile. void compute_time_in_profile(); + void clear_children(); + private: // Pool for allocated counters. Usually owned by the creator of this // object, but occasionally allocated in the constructor. @@ -531,10 +540,10 @@ private: // Utility class to update time elapsed when the object goes out of scope. // 'T' must implement the stopWatch "interface" (start,stop,elapsed_time) but // we use templates not to pay for virtual function overhead. -template +template class ScopedTimer { public: - ScopedTimer(RuntimeProfile::Counter* counter, const bool* is_cancelled = nullptr) + ScopedTimer(RuntimeProfile::Counter* counter, const Bool* is_cancelled = nullptr) : _counter(counter), _is_cancelled(is_cancelled) { if (counter == nullptr) { return; @@ -557,7 +566,9 @@ public: // Update counter when object is destroyed ~ScopedTimer() { - if (_counter == nullptr) return; + if (_counter == nullptr) { + return; + } _sw.stop(); UpdateCounter(); } @@ -569,7 +580,7 @@ public: private: T _sw; RuntimeProfile::Counter* _counter; - const bool* _is_cancelled; + const Bool* _is_cancelled; }; // Utility class to update time elapsed when the object goes out of scope. diff --git a/be/src/vec/exec/scan/new_olap_scan_node.cpp b/be/src/vec/exec/scan/new_olap_scan_node.cpp index 996d2b2ac4..c72f5b289d 100644 --- a/be/src/vec/exec/scan/new_olap_scan_node.cpp +++ b/be/src/vec/exec/scan/new_olap_scan_node.cpp @@ -20,6 +20,9 @@ #include "common/status.h" #include "olap/storage_engine.h" #include "olap/tablet.h" +#include "pipeline/exec/olap_scan_operator.h" +#include "pipeline/pipeline.h" +#include "pipeline/pipeline_fragment_context.h" #include "util/to_string.h" #include "vec/columns/column_const.h" #include "vec/exec/scan/new_olap_scanner.h" @@ -42,7 +45,7 @@ Status NewOlapScanNode::prepare(RuntimeState* state) { } Status NewOlapScanNode::_init_profile() { - VScanNode::_init_profile(); + RETURN_IF_ERROR(VScanNode::_init_profile()); _num_disks_accessed_counter = ADD_COUNTER(_runtime_profile, "NumDiskAccess", TUnit::UNIT); _tablet_counter = ADD_COUNTER(_runtime_profile, "TabletNum", TUnit::UNIT); diff --git a/be/src/vec/exec/scan/new_olap_scan_node.h b/be/src/vec/exec/scan/new_olap_scan_node.h index dc882aa6a9..e72ed82a4a 100644 --- a/be/src/vec/exec/scan/new_olap_scan_node.h +++ b/be/src/vec/exec/scan/new_olap_scan_node.h @@ -19,6 +19,10 @@ #include "vec/exec/scan/vscan_node.h" +namespace doris::pipeline { +class OlapScanOperator; +} + namespace doris::vectorized { class NewOlapScanner; @@ -26,6 +30,7 @@ class NewOlapScanNode : public VScanNode { public: NewOlapScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); friend class NewOlapScanner; + friend class doris::pipeline::OlapScanOperator; Status prepare(RuntimeState* state) override; diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h new file mode 100644 index 0000000000..d595396c72 --- /dev/null +++ b/be/src/vec/exec/scan/pip_scanner_context.h @@ -0,0 +1,47 @@ +// 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. + +#pragma once + +#include "scanner_context.h" + +namespace doris { + +namespace pipeline { + +class PipScannerContext : public vectorized::ScannerContext { +public: + PipScannerContext(RuntimeState* state, vectorized::VScanNode* parent, + const TupleDescriptor* input_tuple_desc, + const TupleDescriptor* output_tuple_desc, + const std::list& scanners, int64_t limit, + int64_t max_bytes_in_blocks_queue) + : vectorized::ScannerContext(state, parent, input_tuple_desc, output_tuple_desc, + scanners, limit, max_bytes_in_blocks_queue) {} + + void _update_block_queue_empty() override { _blocks_queue_empty = _blocks_queue.empty(); } + + // We should make those method lock free. + bool done() override { return _is_finished || _should_stop || _status_error; } + bool can_finish() override { return _num_running_scanners == 0 && _num_scheduling_ctx == 0; } + bool empty_in_queue() override { return _blocks_queue_empty; } + +private: + std::atomic_bool _blocks_queue_empty = true; +}; +} // namespace pipeline +} // namespace doris diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index 7f6cd847bb..203ef8ed68 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -17,6 +17,8 @@ #include "scanner_context.h" +#include + #include "common/config.h" #include "runtime/runtime_state.h" #include "util/threadpool.h" @@ -96,14 +98,20 @@ void ScannerContext::return_free_block(vectorized::Block* block) { void ScannerContext::append_blocks_to_queue(const std::vector& blocks) { std::lock_guard l(_transfer_lock); - blocks_queue.insert(blocks_queue.end(), blocks.begin(), blocks.end()); + _blocks_queue.insert(_blocks_queue.end(), blocks.begin(), blocks.end()); + _update_block_queue_empty(); for (auto b : blocks) { _cur_bytes_in_queue += b->allocated_bytes(); } _blocks_queue_added_cv.notify_one(); } -Status ScannerContext::get_block_from_queue(vectorized::Block** block, bool* eos) { +bool ScannerContext::empty_in_queue() { + std::unique_lock l(_transfer_lock); + return _blocks_queue.empty(); +} + +Status ScannerContext::get_block_from_queue(vectorized::Block** block, bool* eos, bool wait) { std::unique_lock l(_transfer_lock); // Normally, the scanner scheduler will schedule ctx. // But when the amount of data in the blocks queue exceeds the upper limit, @@ -119,7 +127,7 @@ Status ScannerContext::get_block_from_queue(vectorized::Block** block, bool* eos { SCOPED_TIMER(_parent->_scanner_wait_batch_timer); _blocks_queue_added_cv.wait(l, [this]() { - return !blocks_queue.empty() || _is_finished || !_process_status.ok() || + return !_blocks_queue.empty() || _is_finished || !_process_status.ok() || _state->is_cancelled(); }); } @@ -132,9 +140,10 @@ Status ScannerContext::get_block_from_queue(vectorized::Block** block, bool* eos return _process_status; } - if (!blocks_queue.empty()) { - *block = blocks_queue.front(); - blocks_queue.pop_front(); + if (!_blocks_queue.empty()) { + *block = _blocks_queue.front(); + _blocks_queue.pop_front(); + _update_block_queue_empty(); _cur_bytes_in_queue -= (*block)->allocated_bytes(); return Status::OK(); } else { @@ -147,6 +156,7 @@ bool ScannerContext::set_status_on_error(const Status& status) { std::lock_guard l(_transfer_lock); if (_process_status.ok()) { _process_status = status; + _status_error = true; _blocks_queue_added_cv.notify_one(); return true; } @@ -183,7 +193,7 @@ void ScannerContext::clear_and_join() { COUNTER_SET(_parent->_scanner_sched_counter, _num_scanner_scheduling); COUNTER_SET(_parent->_scanner_ctx_sched_counter, _num_ctx_scheduling); - std::for_each(blocks_queue.begin(), blocks_queue.end(), + std::for_each(_blocks_queue.begin(), _blocks_queue.end(), std::default_delete()); std::for_each(_free_blocks.begin(), _free_blocks.end(), std::default_delete()); @@ -191,13 +201,18 @@ void ScannerContext::clear_and_join() { return; } +bool ScannerContext::can_finish() { + std::unique_lock l(_transfer_lock); + return _num_running_scanners == 0 && _num_scheduling_ctx == 0; +} + std::string ScannerContext::debug_string() { return fmt::format( "id: {}, sacnners: {}, blocks in queue: {}," " status: {}, _should_stop: {}, _is_finished: {}, free blocks: {}," " limit: {}, _num_running_scanners: {}, _num_scheduling_ctx: {}, _max_thread_num: {}," - " _block_per_scanner: {}, _cur_bytes_in_queue: {}, _max_bytes_in_queue: {}", - ctx_id, _scanners.size(), blocks_queue.size(), _process_status.ok(), _should_stop, + " _block_per_scanner: {}, _cur_bytes_in_queue: {}, MAX_BYTE_OF_QUEUE: {}", + ctx_id, _scanners.size(), _blocks_queue.size(), _process_status.ok(), _should_stop, _is_finished, _free_blocks.size(), limit, _num_running_scanners, _num_scheduling_ctx, _max_thread_num, _block_per_scanner, _cur_bytes_in_queue, _max_bytes_in_queue); } diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index 24a490673d..41bd0b4d14 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -17,7 +17,9 @@ #pragma once +#include #include +#include #include "common/status.h" #include "runtime/descriptors.h" @@ -63,6 +65,8 @@ public: } } + virtual ~ScannerContext() = default; + Status init(); vectorized::Block* get_free_block(bool* get_free_block); @@ -74,7 +78,7 @@ public: // Get next block from blocks queue. Called by ScanNode // Set eos to true if there is no more data to read. // And if eos is true, the block returned must be nullptr. - Status get_block_from_queue(vectorized::Block** block, bool* eos); + Status get_block_from_queue(vectorized::Block** block, bool* eos, bool wait = true); // When a scanner complete a scan, this method will be called // to return the scanner to the list for next scheduling. @@ -96,8 +100,8 @@ public: } // Return true if this ScannerContext need no more process - bool done() { - std::lock_guard l(_transfer_lock); + virtual bool done() { + std::unique_lock l(_transfer_lock); return _is_finished || _should_stop || !_process_status.ok(); } @@ -114,6 +118,8 @@ public: void clear_and_join(); + virtual bool can_finish(); + std::string debug_string(); RuntimeState* state() { return _state; } @@ -123,7 +129,8 @@ public: VScanNode* parent() { return _parent; } -public: + virtual bool empty_in_queue(); + // the unique id of this context std::string ctx_id; int32_t queue_idx = -1; @@ -132,11 +139,14 @@ public: private: Status _close_and_clear_scanners(); - inline bool _has_enough_space_in_blocks_queue() { + inline bool _has_enough_space_in_blocks_queue() const { return _cur_bytes_in_queue < _max_bytes_in_queue / 2; } -private: + // do nothing here, we only do update on pip_scanner_context + virtual void _update_block_queue_empty() {} + +protected: RuntimeState* _state; VScanNode* _parent; @@ -154,7 +164,7 @@ private: // The blocks got from scanners will be added to the "blocks_queue". // And the upper scan node will be as a consumer to fetch blocks from this queue. // Should be protected by "_transfer_lock" - std::list blocks_queue; + std::list _blocks_queue; // Wait in get_block_from_queue(), by ScanNode. std::condition_variable _blocks_queue_added_cv; // Wait in clear_and_join(), by ScanNode. @@ -174,8 +184,9 @@ private: // Always be set by ScannerScheduler. // True means all scanners are finished to scan. Status _process_status; - bool _should_stop = false; - bool _is_finished = false; + std::atomic_bool _status_error = false; + std::atomic_bool _should_stop = false; + std::atomic_bool _is_finished = false; // Pre-allocated blocks for all scanners to share, for memory reuse. std::mutex _free_blocks_lock; @@ -185,13 +196,13 @@ private: int64_t limit; // Current number of running scanners. - int32_t _num_running_scanners = 0; + std::atomic_int32_t _num_running_scanners = 0; // Current number of ctx being scheduled. // After each Scanner finishes a task, it will put the corresponding ctx // back into the scheduling queue. // Therefore, there will be multiple pointer of same ctx in the scheduling queue. // Here we record the number of ctx in the scheduling queue to clean up at the end. - int32_t _num_scheduling_ctx = 0; + std::atomic_int32_t _num_scheduling_ctx = 0; // Num of unfinished scanners. Should be set in init() int32_t _num_unfinished_scanners = 0; // Max number of scan thread for this scanner context. diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp index b1511c47a3..c83669532c 100644 --- a/be/src/vec/exec/scan/vscan_node.cpp +++ b/be/src/vec/exec/scan/vscan_node.cpp @@ -22,6 +22,7 @@ #include "runtime/runtime_filter_mgr.h" #include "util/runtime_profile.h" #include "vec/columns/column_const.h" +#include "vec/exec/scan/pip_scanner_context.h" #include "vec/exec/scan/scanner_scheduler.h" #include "vec/exec/scan/vscanner.h" #include "vec/exprs/vcompound_pred.h" @@ -65,6 +66,7 @@ Status VScanNode::init(const TPlanNode& tnode, RuntimeState* state) { } RETURN_IF_ERROR(_register_runtime_filter()); + RETURN_IF_ERROR(_init_profile()); return Status::OK(); } @@ -73,8 +75,6 @@ Status VScanNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::prepare(state)); SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); - RETURN_IF_ERROR(_init_profile()); - // init profile for runtime filter for (auto& rf_ctx : _runtime_filter_ctxs) { rf_ctx.runtime_filter->init_profile(_runtime_profile.get()); @@ -175,9 +175,15 @@ Status VScanNode::_init_profile() { } Status VScanNode::_start_scanners(const std::list& scanners) { - _scanner_ctx.reset(new ScannerContext(_state, this, _input_tuple_desc, _output_tuple_desc, - scanners, limit(), - _state->query_options().mem_limit / 20)); + if (_state->enable_pipeline_exec()) { + _scanner_ctx.reset(new pipeline::PipScannerContext(_state, this, _input_tuple_desc, + _output_tuple_desc, scanners, limit(), + _state->query_options().mem_limit / 20)); + } else { + _scanner_ctx.reset(new ScannerContext(_state, this, _input_tuple_desc, _output_tuple_desc, + scanners, limit(), + _state->query_options().mem_limit / 20)); + } RETURN_IF_ERROR(_scanner_ctx->init()); RETURN_IF_ERROR(_state->exec_env()->scanner_scheduler()->submit(_scanner_ctx.get())); return Status::OK(); @@ -200,7 +206,7 @@ Status VScanNode::_register_runtime_filter() { return Status::OK(); } -Status VScanNode::_acquire_runtime_filter() { +Status VScanNode::_acquire_runtime_filter(bool wait) { SCOPED_TIMER(_acquire_runtime_filter_timer); std::vector vexprs; for (size_t i = 0; i < _runtime_filter_descs.size(); ++i) { @@ -213,7 +219,7 @@ Status VScanNode::_acquire_runtime_filter() { } } bool ready = runtime_filter->is_ready(); - if (!ready) { + if (!ready && wait) { ready = runtime_filter->await(); } if (ready) { diff --git a/be/src/vec/exec/scan/vscan_node.h b/be/src/vec/exec/scan/vscan_node.h index fbe800ccd6..7e99cf3cdc 100644 --- a/be/src/vec/exec/scan/vscan_node.h +++ b/be/src/vec/exec/scan/vscan_node.h @@ -26,6 +26,10 @@ #include "vec/exprs/vexpr.h" #include "vec/exprs/vin_predicate.h" +namespace doris::pipeline { +class ScanOperator; +} + namespace doris::vectorized { class VScanner; @@ -49,6 +53,7 @@ public: friend class NewOlapScanner; friend class VFileScanner; friend class ScannerContext; + friend class doris::pipeline::ScanOperator; Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; @@ -257,7 +262,7 @@ private: // Register and get all runtime filters at Init phase. Status _register_runtime_filter(); // Get all arrived runtime filters at Open phase. - Status _acquire_runtime_filter(); + Status _acquire_runtime_filter(bool wait = true); // Append late-arrival runtime filters to the vconjunct_ctx. Status _append_rf_into_conjuncts(std::vector& vexprs); diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp index 0d86bfc546..92ed1f2336 100644 --- a/be/src/vec/exec/vaggregation_node.cpp +++ b/be/src/vec/exec/vaggregation_node.cpp @@ -288,12 +288,9 @@ void AggregationNode::_init_hash_method(std::vector& probe_exprs) _agg_data->init(AggregatedDataVariants::Type::serialized); } } -} // namespace doris::vectorized +} -Status AggregationNode::prepare(RuntimeState* state) { - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(ExecNode::prepare(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); +Status AggregationNode::prepare_profile(RuntimeState* state) { _build_timer = ADD_TIMER(runtime_profile(), "BuildTime"); _serialize_key_timer = ADD_TIMER(runtime_profile(), "SerializeKeyTime"); _exec_timer = ADD_TIMER(runtime_profile(), "ExecTime"); @@ -445,10 +442,14 @@ Status AggregationNode::prepare(RuntimeState* state) { return Status::OK(); } -Status AggregationNode::open(RuntimeState* state) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "AggregationNode::open"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(ExecNode::open(state)); +Status AggregationNode::prepare(RuntimeState* state) { + RETURN_IF_ERROR(ExecNode::prepare(state)); + RETURN_IF_ERROR(prepare_profile(state)); + return Status::OK(); +} + +Status AggregationNode::alloc_resource(doris::RuntimeState* state) { + RETURN_IF_ERROR(ExecNode::alloc_resource(state)); SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); RETURN_IF_ERROR(VExpr::open(_probe_expr_ctxs, state)); @@ -457,10 +458,6 @@ Status AggregationNode::open(RuntimeState* state) { RETURN_IF_ERROR(_aggregate_evaluators[i]->open(state)); } - RETURN_IF_ERROR(_children[0]->open(state)); - - // Streaming preaggregations do all processing in GetNext(). - if (_is_streaming_preagg) return Status::OK(); // move _create_agg_status to open not in during prepare, // because during prepare and open thread is not the same one, // this could cause unable to get JVM @@ -468,6 +465,18 @@ Status AggregationNode::open(RuntimeState* state) { _create_agg_status(_agg_data->without_key); _agg_data_created_without_key = true; } + + return Status::OK(); +} + +Status AggregationNode::open(RuntimeState* state) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "AggregationNode::open"); + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(ExecNode::open(state)); + RETURN_IF_ERROR(_children[0]->open(state)); + + // Streaming preaggregations do all processing in GetNext(). + if (_is_streaming_preagg) return Status::OK(); bool eos = false; Block block; while (!eos) { @@ -475,11 +484,7 @@ Status AggregationNode::open(RuntimeState* state) { release_block_memory(block); RETURN_IF_ERROR_AND_CHECK_SPAN(_children[0]->get_next_after_projects(state, &block, &eos), _children[0]->get_next_span(), eos); - if (block.rows() == 0) { - continue; - } - RETURN_IF_ERROR(_executor.execute(&block)); - _executor.update_memusage(); + RETURN_IF_ERROR(sink(state, &block, eos)); } _children[0]->close(state); @@ -490,6 +495,18 @@ Status AggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* return Status::NotSupported("Not Implemented Aggregation Node::get_next scalar"); } +Status AggregationNode::do_pre_agg(vectorized::Block* input_block, + vectorized::Block* output_block) { + RETURN_IF_ERROR(_executor.pre_agg(input_block, output_block)); + + // pre stream agg need use _num_row_return to decide whether to do pre stream agg + _num_rows_returned += output_block->rows(); + _make_nullable_output_key(output_block); + COUNTER_SET(_rows_returned_counter, _num_rows_returned); + _executor.update_memusage(); + return Status::OK(); +} + Status AggregationNode::get_next(RuntimeState* state, Block* block, bool* eos) { INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "AggregationNode::get_next"); SCOPED_TIMER(_runtime_profile->total_time_counter()); @@ -505,32 +522,37 @@ Status AggregationNode::get_next(RuntimeState* state, Block* block, bool* eos) { _children[0]->get_next_after_projects(state, &_preagg_block, &child_eos), _children[0]->get_next_span(), child_eos); } while (_preagg_block.rows() == 0 && !child_eos); - if (_preagg_block.rows() != 0) { - RETURN_IF_ERROR(_executor.pre_agg(&_preagg_block, block)); + RETURN_IF_ERROR(do_pre_agg(&_preagg_block, block)); } else { - RETURN_IF_ERROR(_executor.get_result(state, block, eos)); + RETURN_IF_ERROR(pull(state, block, eos)); } - // pre stream agg need use _num_row_return to decide whether to do pre stream agg - _num_rows_returned += block->rows(); - _make_nullable_output_key(block); - COUNTER_SET(_rows_returned_counter, _num_rows_returned); } else { - RETURN_IF_ERROR(_executor.get_result(state, block, eos)); - _make_nullable_output_key(block); - // dispose the having clause, should not be execute in prestreaming agg - RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block, block->columns())); - reached_limit(block, eos); + RETURN_IF_ERROR(pull(state, block, eos)); } - - _executor.update_memusage(); return Status::OK(); } -Status AggregationNode::close(RuntimeState* state) { - if (is_closed()) return Status::OK(); - START_AND_SCOPE_SPAN(state->get_tracer(), span, "AggregationNode::close"); +Status AggregationNode::pull(doris::RuntimeState* state, vectorized::Block* block, bool* eos) { + RETURN_IF_ERROR(_executor.get_result(state, block, eos)); + _make_nullable_output_key(block); + // dispose the having clause, should not be execute in prestreaming agg + RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block, block->columns())); + reached_limit(block, eos); + return Status::OK(); +} + +Status AggregationNode::sink(doris::RuntimeState* state, vectorized::Block* in_block, bool eos) { + if (in_block->rows() > 0) { + RETURN_IF_ERROR(_executor.execute(in_block)); + _executor.update_memusage(); + } + if (eos) _can_read = true; + return Status::OK(); +} + +void AggregationNode::release_resource(RuntimeState* state) { for (auto* aggregate_evaluator : _aggregate_evaluators) aggregate_evaluator->close(state); VExpr::close(_probe_expr_ctxs, state); if (_executor.close) _executor.close(); @@ -544,7 +566,12 @@ Status AggregationNode::close(RuntimeState* state) { _agg_data->_aggregated_method_variant); } _release_mem(); + ExecNode::release_resource(state); +} +Status AggregationNode::close(RuntimeState* state) { + if (is_closed()) return Status::OK(); + START_AND_SCOPE_SPAN(state->get_tracer(), span, "AggregationNode::close"); return ExecNode::close(state); } diff --git a/be/src/vec/exec/vaggregation_node.h b/be/src/vec/exec/vaggregation_node.h index ccb28902d9..1ef8518fee 100644 --- a/be/src/vec/exec/vaggregation_node.h +++ b/be/src/vec/exec/vaggregation_node.h @@ -34,6 +34,13 @@ class TPlanNode; class DescriptorTbl; class MemPool; +namespace pipeline { +class AggSinkOperator; +class AggregationSourceOperator; +class StreamingAggSinkOperator; +class StreamingAggSourceOperator; +} // namespace pipeline + namespace vectorized { class VExprContext; @@ -754,20 +761,31 @@ private: }; // not support spill -class AggregationNode : public ::doris::ExecNode { +class AggregationNode final : public ::doris::ExecNode { public: using Sizes = std::vector; AggregationNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); ~AggregationNode(); - virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr); - virtual Status prepare(RuntimeState* state); - virtual Status open(RuntimeState* state); - virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status get_next(RuntimeState* state, Block* block, bool* eos); - virtual Status close(RuntimeState* state); + virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; + Status prepare_profile(RuntimeState* state); + virtual Status prepare(RuntimeState* state) override; + virtual Status open(RuntimeState* state) override; + virtual Status alloc_resource(RuntimeState* state) override; + virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override; + virtual Status get_next(RuntimeState* state, Block* block, bool* eos) override; + virtual Status close(RuntimeState* state) override; + virtual void release_resource(RuntimeState* state) override; + Status pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) override; + Status sink(doris::RuntimeState* state, vectorized::Block* input_block, bool eos) override; + Status do_pre_agg(vectorized::Block* input_block, vectorized::Block* output_block); + bool is_streaming_preagg() { return _is_streaming_preagg; } private: + friend class pipeline::AggSinkOperator; + friend class pipeline::StreamingAggSinkOperator; + friend class pipeline::AggregationSourceOperator; + friend class pipeline::StreamingAggSourceOperator; // group by k1,k2 std::vector _probe_expr_ctxs; // left / full join will change the key nullable make output/input solt @@ -832,6 +850,7 @@ private: std::unique_ptr _aggregate_data_container; private: + void _release_self_resource(RuntimeState* state); /// Return true if we should keep expanding hash tables in the preagg. If false, /// the preagg should pass through any rows it can't fit in its tables. bool _should_expand_preagg_hash_tables(); diff --git a/be/src/vec/exec/vexchange_node.cpp b/be/src/vec/exec/vexchange_node.cpp index 59073c879a..13a75ed990 100644 --- a/be/src/vec/exec/vexchange_node.cpp +++ b/be/src/vec/exec/vexchange_node.cpp @@ -17,6 +17,9 @@ #include "vec/exec/vexchange_node.h" +#include "pipeline/exec/exchange_source_operator.h" +#include "pipeline/pipeline.h" +#include "pipeline/pipeline_fragment_context.h" #include "runtime/exec_env.h" #include "runtime/runtime_state.h" #include "runtime/thread_context.h" @@ -62,18 +65,23 @@ Status VExchangeNode::prepare(RuntimeState* state) { } return Status::OK(); } -Status VExchangeNode::open(RuntimeState* state) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VExchangeNode::open"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(ExecNode::open(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); +Status VExchangeNode::alloc_resource(RuntimeState* state) { + RETURN_IF_ERROR(ExecNode::alloc_resource(state)); if (_is_merging) { RETURN_IF_ERROR(_vsort_exec_exprs.open(state)); RETURN_IF_ERROR(_stream_recvr->create_merger(_vsort_exec_exprs.lhs_ordering_expr_ctxs(), _is_asc_order, _nulls_first, state->batch_size(), _limit, _offset)); } + return Status::OK(); +} + +Status VExchangeNode::open(RuntimeState* state) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VExchangeNode::open"); + SCOPED_TIMER(_runtime_profile->total_time_counter()); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); + RETURN_IF_ERROR(ExecNode::open(state)); return Status::OK(); } @@ -99,17 +107,21 @@ Status VExchangeNode::get_next(RuntimeState* state, Block* block, bool* eos) { return status; } +void VExchangeNode::release_resource(RuntimeState* state) { + if (_stream_recvr != nullptr) { + _stream_recvr->close(); + } + if (_is_merging) { + _vsort_exec_exprs.close(state); + } +} + Status VExchangeNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK(); } START_AND_SCOPE_SPAN(state->get_tracer(), span, "VExchangeNode::close"); - if (_stream_recvr != nullptr) { - _stream_recvr->close(); - } - if (_is_merging) _vsort_exec_exprs.close(state); - return ExecNode::close(state); } diff --git a/be/src/vec/exec/vexchange_node.h b/be/src/vec/exec/vexchange_node.h index 5e2d4533af..190b625e3d 100644 --- a/be/src/vec/exec/vexchange_node.h +++ b/be/src/vec/exec/vexchange_node.h @@ -23,19 +23,25 @@ #include "vec/common/sort/vsort_exec_exprs.h" namespace doris { +namespace pipeline { +class ExchangeSourceOperator; +} namespace vectorized { class VDataStreamRecvr; class VExchangeNode : public ExecNode { public: + friend class doris::pipeline::ExchangeSourceOperator; VExchangeNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); virtual ~VExchangeNode() {} virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; virtual Status prepare(RuntimeState* state) override; + virtual Status alloc_resource(RuntimeState* state) override; virtual Status open(RuntimeState* state) override; virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override; virtual Status get_next(RuntimeState* state, Block* row_batch, bool* eos) override; + virtual void release_resource(RuntimeState* state) override; virtual Status close(RuntimeState* state) override; // Status collect_query_statistics(QueryStatistics* statistics) override; diff --git a/be/src/vec/exec/vrepeat_node.cpp b/be/src/vec/exec/vrepeat_node.cpp index cb79fcb317..9c77b21bfc 100644 --- a/be/src/vec/exec/vrepeat_node.cpp +++ b/be/src/vec/exec/vrepeat_node.cpp @@ -24,7 +24,14 @@ namespace doris::vectorized { VRepeatNode::VRepeatNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : RepeatNode(pool, tnode, descs) {} + : ExecNode(pool, tnode, descs), + _slot_id_set_list(tnode.repeat_node.slot_id_set_list), + _all_slot_ids(tnode.repeat_node.all_slot_ids), + _repeat_id_list(tnode.repeat_node.repeat_id_list), + _grouping_list(tnode.repeat_node.grouping_list), + _output_tuple_id(tnode.repeat_node.output_tuple_id), + _child_eos(false), + _repeat_id_idx(0) {} Status VRepeatNode::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(ExecNode::init(tnode, state)); @@ -35,7 +42,14 @@ Status VRepeatNode::init(const TPlanNode& tnode, RuntimeState* state) { Status VRepeatNode::prepare(RuntimeState* state) { VLOG_CRITICAL << "VRepeatNode::prepare"; SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(RepeatNode::prepare(state)); + + RETURN_IF_ERROR(ExecNode::prepare(state)); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); + _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id); + if (_output_tuple_desc == nullptr) { + return Status::InternalError("Failed to get tuple descriptor."); + } + RETURN_IF_ERROR(VExpr::prepare(_expr_ctxs, state, child(0)->row_desc())); for (const auto& slot_desc : _output_tuple_desc->slots()) { @@ -50,7 +64,16 @@ Status VRepeatNode::open(RuntimeState* state) { START_AND_SCOPE_SPAN(state->get_tracer(), span, "VRepeatNode::open"); VLOG_CRITICAL << "VRepeatNode::open"; SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(RepeatNode::open(state)); + RETURN_IF_ERROR(ExecNode::open(state)); + RETURN_IF_ERROR(child(0)->open(state)); + return Status::OK(); +} + +Status VRepeatNode::alloc_resource(RuntimeState* state) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VRepeatNode::open"); + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(ExecNode::alloc_resource(state)); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); RETURN_IF_ERROR(VExpr::open(_expr_ctxs, state)); return Status::OK(); } @@ -148,6 +171,58 @@ Status VRepeatNode::get_repeated_block(Block* child_block, int repeat_id_idx, Bl return Status::OK(); } +Status VRepeatNode::pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + + RETURN_IF_CANCELLED(state); + DCHECK(_repeat_id_idx >= 0); + for (const std::vector& v : _grouping_list) { + DCHECK(_repeat_id_idx <= (int)v.size()); + } + DCHECK(output_block->rows() == 0); + + DCHECK(_intermediate_block); + DCHECK_NE(_intermediate_block->rows(), 0); + + RETURN_IF_ERROR(get_repeated_block(_intermediate_block.get(), _repeat_id_idx, output_block)); + + _repeat_id_idx++; + + int size = _repeat_id_list.size(); + if (_repeat_id_idx >= size) { + _intermediate_block->clear(); + release_block_memory(*_child_block); + _repeat_id_idx = 0; + } + + reached_limit(output_block, eos); + COUNTER_SET(_rows_returned_counter, _num_rows_returned); + return Status::OK(); +} + +Status VRepeatNode::push(RuntimeState* state, vectorized::Block* input_block, bool eos) { + DCHECK(!_intermediate_block || _intermediate_block->rows() == 0); + DCHECK(!_expr_ctxs.empty()); + _intermediate_block.reset(new Block()); + + for (auto expr : _expr_ctxs) { + int result_column_id = -1; + RETURN_IF_ERROR(expr->execute(input_block, &result_column_id)); + DCHECK(result_column_id != -1); + input_block->get_by_position(result_column_id).column = + input_block->get_by_position(result_column_id) + .column->convert_to_full_column_if_const(); + _intermediate_block->insert(input_block->get_by_position(result_column_id)); + } + DCHECK_EQ(_expr_ctxs.size(), _intermediate_block->columns()); + + return Status::OK(); +} + +bool VRepeatNode::need_more_input_data() { + return !_intermediate_block || _intermediate_block->rows() == 0; +} + Status VRepeatNode::get_next(RuntimeState* state, Block* block, bool* eos) { INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VRepeatNode::get_next"); VLOG_CRITICAL << "VRepeatNode::get_next"; @@ -163,8 +238,7 @@ Status VRepeatNode::get_next(RuntimeState* state, Block* block, bool* eos) { } DCHECK(block->rows() == 0); - // current child block has finished its repeat, get child's next block - if (_child_block->rows() == 0) { + if (need_more_input_data()) { while (_child_block->rows() == 0 && !_child_eos) { RETURN_IF_ERROR_AND_CHECK_SPAN( child(0)->get_next_after_projects(state, _child_block.get(), &_child_eos), @@ -176,35 +250,10 @@ Status VRepeatNode::get_next(RuntimeState* state, Block* block, bool* eos) { return Status::OK(); } - DCHECK(!_expr_ctxs.empty()); - _intermediate_block.reset(new Block()); - for (auto vexpr_ctx : _expr_ctxs) { - int result_column_id = -1; - RETURN_IF_ERROR(vexpr_ctx->execute(_child_block.get(), &result_column_id)); - DCHECK(result_column_id != -1); - _child_block->get_by_position(result_column_id).column = - _child_block->get_by_position(result_column_id) - .column->convert_to_full_column_if_const(); - _intermediate_block->insert(_child_block->get_by_position(result_column_id)); - } - DCHECK_EQ(_expr_ctxs.size(), _intermediate_block->columns()); + push(state, _child_block.get(), *eos); } - RETURN_IF_ERROR(get_repeated_block(_intermediate_block.get(), _repeat_id_idx, block)); - - _repeat_id_idx++; - - int size = _repeat_id_list.size(); - if (_repeat_id_idx >= size) { - _intermediate_block->clear(); - release_block_memory(*_child_block); - _repeat_id_idx = 0; - } - - reached_limit(block, eos); - COUNTER_SET(_rows_returned_counter, _num_rows_returned); - VLOG_ROW << "VRepeatNode output rows: " << block->rows(); - return Status::OK(); + return pull(state, block, eos); } Status VRepeatNode::close(RuntimeState* state) { @@ -212,12 +261,15 @@ Status VRepeatNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK(); } - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VRepeatNode::close"); - VExpr::close(_expr_ctxs, state); - _release_mem(); return ExecNode::close(state); } +void VRepeatNode::release_resource(RuntimeState* state) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::close"); + VExpr::close(_expr_ctxs, state); + ExecNode::release_resource(state); +} + void VRepeatNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); *out << "VRepeatNode("; diff --git a/be/src/vec/exec/vrepeat_node.h b/be/src/vec/exec/vrepeat_node.h index 1bf047a196..53eb025cc1 100644 --- a/be/src/vec/exec/vrepeat_node.h +++ b/be/src/vec/exec/vrepeat_node.h @@ -17,7 +17,7 @@ #pragma once -#include "exec/repeat_node.h" +#include "exec/exec_node.h" namespace doris { @@ -30,32 +30,49 @@ class Status; namespace vectorized { class VExprContext; -class VRepeatNode : public RepeatNode { +class VRepeatNode : public ExecNode { public: VRepeatNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); ~VRepeatNode() override = default; - virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; - virtual Status prepare(RuntimeState* state) override; - virtual Status open(RuntimeState* state) override; - virtual Status get_next(RuntimeState* state, Block* block, bool* eos) override; - virtual Status close(RuntimeState* state) override; + Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; + Status prepare(RuntimeState* state) override; + Status alloc_resource(RuntimeState* state) override; + Status open(RuntimeState* state) override; + void release_resource(RuntimeState* state) override; + Status get_next(RuntimeState* state, Block* block, bool* eos) override; + Status close(RuntimeState* state) override; + + Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) override; + Status push(RuntimeState* state, vectorized::Block* input_block, bool eos) override; + bool need_more_input_data(); protected: virtual void debug_string(int indentation_level, std::stringstream* out) const override; private: - using RepeatNode::get_next; Status get_repeated_block(Block* child_block, int repeat_id_idx, Block* output_block); void _release_mem(); + // Slot id set used to indicate those slots need to set to null. + std::vector> _slot_id_set_list; + // all slot id + std::set _all_slot_ids; + // An integer bitmap list, it indicates the bit position of the exprs not null. + std::vector _repeat_id_list; + std::vector> _grouping_list; + TupleId _output_tuple_id; + const TupleDescriptor* _output_tuple_desc; + std::unique_ptr _child_block {}; std::unique_ptr _intermediate_block {}; std::vector _output_slots; std::vector _expr_ctxs; + bool _child_eos; + int _repeat_id_idx; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/exec/vsort_node.cpp b/be/src/vec/exec/vsort_node.cpp index ed37e5cc24..c633a08113 100644 --- a/be/src/vec/exec/vsort_node.cpp +++ b/be/src/vec/exec/vsort_node.cpp @@ -18,14 +18,13 @@ #include "vec/exec/vsort_node.h" #include "common/config.h" -#include "exec/sort_exec_exprs.h" +#include "pipeline/pipeline.h" #include "runtime/row_batch.h" #include "runtime/runtime_state.h" #include "util/debug_util.h" #include "vec/common/sort/heap_sorter.h" #include "vec/common/sort/topn_sorter.h" #include "vec/core/sort_block.h" -#include "vec/utils/util.hpp" namespace doris::vectorized { @@ -66,21 +65,41 @@ Status VSortNode::init(const TPlanNode& tnode, RuntimeState* state) { Status VSortNode::prepare(RuntimeState* state) { SCOPED_TIMER(_runtime_profile->total_time_counter()); - _runtime_profile->add_info_string("TOP-N", _limit == -1 ? "false" : "true"); RETURN_IF_ERROR(ExecNode::prepare(state)); SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); + _runtime_profile->add_info_string("TOP-N", _limit == -1 ? "false" : "true"); RETURN_IF_ERROR(_vsort_exec_exprs.prepare(state, child(0)->row_desc(), _row_descriptor)); return Status::OK(); } +Status VSortNode::alloc_resource(doris::RuntimeState* state) { + RETURN_IF_ERROR(ExecNode::alloc_resource(state)); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); + RETURN_IF_ERROR(_vsort_exec_exprs.open(state)); + RETURN_IF_CANCELLED(state); + RETURN_IF_ERROR(state->check_query_state("vsort, while open.")); + + return Status::OK(); +} + +Status VSortNode::sink(RuntimeState* state, vectorized::Block* input_block, bool eos) { + if (input_block->rows() > 0) { + RETURN_IF_ERROR(_sorter->append_block(input_block)); + RETURN_IF_CANCELLED(state); + RETURN_IF_ERROR(state->check_query_state("vsort, while sorting input.")); + } + + if (eos) { + RETURN_IF_ERROR(_sorter->prepare_for_read()); + _can_read = true; + } + return Status::OK(); +} + Status VSortNode::open(RuntimeState* state) { START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::open"); SCOPED_TIMER(_runtime_profile->total_time_counter()); RETURN_IF_ERROR(ExecNode::open(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); - RETURN_IF_ERROR(_vsort_exec_exprs.open(state)); - RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(state->check_query_state("vsort, while open.")); RETURN_IF_ERROR(child(0)->open(state)); // The child has been opened and the sorter created. Sort the input. @@ -91,17 +110,13 @@ Status VSortNode::open(RuntimeState* state) { RETURN_IF_ERROR_AND_CHECK_SPAN( child(0)->get_next_after_projects(state, upstream_block.get(), &eos), child(0)->get_next_span(), eos); - if (upstream_block->rows() != 0) { - RETURN_IF_ERROR(_sorter->append_block(upstream_block.get())); - RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(state->check_query_state("vsort, while sorting input.")); - if (!_reuse_mem) { - upstream_block.reset(new Block()); - } + RETURN_IF_ERROR(sink(state, upstream_block.get(), eos)); + if (!_reuse_mem) { + upstream_block.reset(new Block()); } } while (!eos); + child(0)->close(state); - RETURN_IF_ERROR(_sorter->prepare_for_read()); return Status::OK(); } @@ -110,27 +125,35 @@ Status VSortNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) return Status::NotSupported("Not Implemented VSortNode::get_next scalar"); } +Status VSortNode::pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) { + RETURN_IF_ERROR(_sorter->get_next(state, output_block, eos)); + reached_limit(output_block, eos); + return Status::OK(); +} + Status VSortNode::get_next(RuntimeState* state, Block* block, bool* eos) { INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VSortNode::get_next"); SCOPED_TIMER(_runtime_profile->total_time_counter()); SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh()); - RETURN_IF_ERROR(_sorter->get_next(state, block, eos)); - reached_limit(block, eos); - return Status::OK(); + return pull(state, block, eos); } Status VSortNode::reset(RuntimeState* state) { return Status::OK(); } +void VSortNode::release_resource(doris::RuntimeState* state) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::close"); + _vsort_exec_exprs.close(state); + _sorter = nullptr; + ExecNode::release_resource(state); +} + Status VSortNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK(); } - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::close"); - _vsort_exec_exprs.close(state); - _sorter = nullptr; return ExecNode::close(state); } diff --git a/be/src/vec/exec/vsort_node.h b/be/src/vec/exec/vsort_node.h index 7b2834e4b4..6da8e52e77 100644 --- a/be/src/vec/exec/vsort_node.h +++ b/be/src/vec/exec/vsort_node.h @@ -31,7 +31,7 @@ namespace doris::vectorized { // In get_next(), VSortNode do the merge sort to gather data to a new block // support spill to disk in the future -class VSortNode : public doris::ExecNode { +class VSortNode final : public doris::ExecNode { public: VSortNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); @@ -41,6 +41,8 @@ public: virtual Status prepare(RuntimeState* state) override; + virtual Status alloc_resource(RuntimeState* state) override; + virtual Status open(RuntimeState* state) override; virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override; @@ -51,6 +53,12 @@ public: virtual Status close(RuntimeState* state) override; + virtual void release_resource(RuntimeState* state) override; + + virtual Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) override; + + virtual Status sink(RuntimeState* state, vectorized::Block* input_block, bool eos) override; + protected: virtual void debug_string(int indentation_level, std::stringstream* out) const override; diff --git a/be/src/vec/exec/vtable_function_node.cpp b/be/src/vec/exec/vtable_function_node.cpp index a741bd3553..b283988589 100644 --- a/be/src/vec/exec/vtable_function_node.cpp +++ b/be/src/vec/exec/vtable_function_node.cpp @@ -83,11 +83,22 @@ Status VTableFunctionNode::get_next(RuntimeState* state, Block* block, bool* eos RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(get_expanded_block(state, block, eos)); + // if child_block is empty, get data from child. + if (need_more_input_data()) { + while (_child_block->rows() == 0 && !_child_eos) { + RETURN_IF_ERROR_AND_CHECK_SPAN( + child(0)->get_next_after_projects(state, _child_block.get(), &_child_eos), + child(0)->get_next_span(), _child_eos); + } + if (_child_eos && _child_block->rows() == 0) { + *eos = true; + return Status::OK(); + } - reached_limit(block, eos); + push(state, _child_block.get(), *eos); + } - COUNTER_SET(_rows_returned_counter, _num_rows_returned); + pull(state, block, eos); return Status::OK(); } @@ -111,23 +122,8 @@ Status VTableFunctionNode::get_expanded_block(RuntimeState* state, Block* output RETURN_IF_CANCELLED(state); RETURN_IF_ERROR(state->check_query_state("VTableFunctionNode, while getting next batch.")); - // if child_block is empty, get data from child. if (_child_block->rows() == 0) { - while (_child_block->rows() == 0 && !_child_eos) { - RETURN_IF_ERROR_AND_CHECK_SPAN( - child(0)->get_next_after_projects(state, _child_block.get(), &_child_eos), - child(0)->get_next_span(), _child_eos); - } - if (_child_eos && _child_block->rows() == 0) { - *eos = true; - break; - } - - for (TableFunction* fn : _fns) { - RETURN_IF_ERROR(fn->process_init(_child_block.get())); - } - - RETURN_IF_ERROR(_process_next_child_row()); + break; } bool skip_child_row = false; diff --git a/be/src/vec/exec/vtable_function_node.h b/be/src/vec/exec/vtable_function_node.h index 8e572f55e3..0f44e78e5c 100644 --- a/be/src/vec/exec/vtable_function_node.h +++ b/be/src/vec/exec/vtable_function_node.h @@ -18,6 +18,7 @@ #pragma once #include "exec/table_function_node.h" +#include "exprs/table_function/table_function.h" namespace doris::vectorized { @@ -30,6 +31,29 @@ public: Status prepare(RuntimeState* state) override; Status get_next(RuntimeState* state, Block* block, bool* eos) override; + bool need_more_input_data() { return !_child_block || !_child_block->rows(); } + + Status push(RuntimeState*, vectorized::Block* input_block, bool eos) override { + if (eos) { + return Status::OK(); + } + + if (input_block != _child_block.get()) { + _child_block.reset(input_block); + } + for (TableFunction* fn : _fns) { + RETURN_IF_ERROR(fn->process_init(_child_block.get())); + } + RETURN_IF_ERROR(_process_next_child_row()); + return Status::OK(); + } + + Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) override { + RETURN_IF_ERROR(get_expanded_block(state, output_block, eos)); + reached_limit(output_block, eos); + return Status::OK(); + } + private: Status _process_next_child_row() override; diff --git a/be/src/vec/runtime/vdata_stream_mgr.cpp b/be/src/vec/runtime/vdata_stream_mgr.cpp index 4743d0933f..ea11f6ed00 100644 --- a/be/src/vec/runtime/vdata_stream_mgr.cpp +++ b/be/src/vec/runtime/vdata_stream_mgr.cpp @@ -49,7 +49,7 @@ std::shared_ptr VDataStreamMgr::create_recvr( RuntimeState* state, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, int buffer_size, RuntimeProfile* profile, bool is_merging, std::shared_ptr sub_plan_query_statistics_recvr) { - DCHECK(profile != NULL); + DCHECK(profile != nullptr); VLOG_FILE << "creating receiver for fragment=" << fragment_instance_id << ", node=" << dest_node_id; std::shared_ptr recvr(new VDataStreamRecvr( @@ -171,7 +171,7 @@ void VDataStreamMgr::cancel(const TUniqueId& fragment_instance_id) { _fragment_stream_set.lower_bound(std::make_pair(fragment_instance_id, 0)); while (i != _fragment_stream_set.end() && i->first == fragment_instance_id) { std::shared_ptr recvr = find_recvr(i->first, i->second, false); - if (recvr == NULL) { + if (recvr == nullptr) { // keep going but at least log it std::stringstream err; err << "cancel(): missing in stream_map: fragment=" << i->first diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index a9c741969a..ec9814a735 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -38,6 +38,12 @@ VDataStreamRecvr::SenderQueue::SenderQueue(VDataStreamRecvr* parent_recvr, int n VDataStreamRecvr::SenderQueue::~SenderQueue() = default; +bool VDataStreamRecvr::SenderQueue::should_wait() { + DCHECK(false) << "VDataStreamRecvr::SenderQueue::should_wait execute"; + std::unique_lock l(_lock); + return !_is_cancelled && _block_queue.empty() && _num_remaining_senders > 0; +} + Status VDataStreamRecvr::SenderQueue::get_batch(Block** next_block) { std::unique_lock l(_lock); // wait until something shows up or we know we're done @@ -45,13 +51,16 @@ Status VDataStreamRecvr::SenderQueue::get_batch(Block** next_block) { VLOG_ROW << "wait arrival fragment_instance_id=" << _recvr->fragment_instance_id() << " node=" << _recvr->dest_node_id(); // Don't count time spent waiting on the sender as active time. - CANCEL_SAFE_SCOPED_TIMER(_recvr->_data_arrival_timer, &_is_cancelled); - CANCEL_SAFE_SCOPED_TIMER( - _received_first_batch ? NULL : _recvr->_first_batch_wait_total_timer, + CANCEL_SAFE_SCOPED_TIMER_ATOMIC(_recvr->_data_arrival_timer, &_is_cancelled); + CANCEL_SAFE_SCOPED_TIMER_ATOMIC( + _received_first_batch ? nullptr : _recvr->_first_batch_wait_total_timer, &_is_cancelled); _data_arrival_cv.wait(l); } + return _inner_get_batch(next_block); +} +Status VDataStreamRecvr::SenderQueue::_inner_get_batch(Block** next_block) { // _cur_batch must be replaced with the returned batch. _current_block.reset(); *next_block = nullptr; @@ -71,6 +80,7 @@ Status VDataStreamRecvr::SenderQueue::get_batch(Block** next_block) { _recvr->_num_buffered_bytes -= _block_queue.front().first; VLOG_ROW << "fetched #rows=" << result->rows(); _block_queue.pop_front(); + _update_block_queue_empty(); _current_block.reset(result); *next_block = _current_block.get(); @@ -131,6 +141,7 @@ void VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_numbe VLOG_ROW << "added #rows=" << block->rows() << " batch_size=" << block_byte_size << "\n"; _block_queue.emplace_back(block_byte_size, block); + _update_block_queue_empty(); // if done is nullptr, this function can't delay this response if (done != nullptr && _recvr->exceeds_limit(block_byte_size)) { MonotonicStopWatch monotonicStopWatch; @@ -148,7 +159,7 @@ void VDataStreamRecvr::SenderQueue::add_block(Block* block, bool use_move) { // limit memory via DataStreamRecvr::exceeds_limit. STOP_CHECK_THREAD_MEM_TRACKER_LIMIT(); std::unique_lock l(_lock); - if (_is_cancelled) { + if (_is_cancelled || !block->rows()) { return; } Block* nblock = new Block(block->get_columns_with_type_and_name()); @@ -167,6 +178,7 @@ void VDataStreamRecvr::SenderQueue::add_block(Block* block, bool use_move) { size_t block_size = nblock->bytes(); _block_queue.emplace_back(block_size, nblock); + _update_block_queue_empty(); _data_arrival_cv.notify_one(); if (_recvr->exceeds_limit(block_size)) { @@ -267,7 +279,8 @@ VDataStreamRecvr::VDataStreamRecvr( _is_closed(false), _num_buffered_bytes(0), _profile(profile), - _sub_plan_query_statistics_recvr(sub_plan_query_statistics_recvr) { + _sub_plan_query_statistics_recvr(sub_plan_query_statistics_recvr), + _enable_pipeline(state->enable_pipeline_exec()) { // DataStreamRecvr may be destructed after the instance execution thread ends. _mem_tracker = std::make_unique( "VDataStreamRecvr:" + print_id(_fragment_instance_id), _profile); @@ -278,8 +291,12 @@ VDataStreamRecvr::VDataStreamRecvr( _sender_queues.reserve(num_queues); int num_sender_per_queue = is_merging ? 1 : num_senders; for (int i = 0; i < num_queues; ++i) { - SenderQueue* queue = - _sender_queue_pool.add(new SenderQueue(this, num_sender_per_queue, profile)); + SenderQueue* queue = nullptr; + if (_enable_pipeline) { + queue = _sender_queue_pool.add(new PipSenderQueue(this, num_sender_per_queue, profile)); + } else { + queue = _sender_queue_pool.add(new SenderQueue(this, num_sender_per_queue, profile)); + } _sender_queues.push_back(queue); } @@ -332,6 +349,15 @@ void VDataStreamRecvr::add_block(Block* block, int sender_id, bool use_move) { _sender_queues[use_sender_id]->add_block(block, use_move); } +bool VDataStreamRecvr::ready_to_read() { + for (size_t i = 0; i < _sender_queues.size(); i++) { + if (_sender_queues[i]->should_wait()) { + return false; + } + } + return true; +} + Status VDataStreamRecvr::get_next(Block* block, bool* eos) { SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); if (!_is_merging) { diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index 6d38ecb955..45af88cd50 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -28,9 +28,9 @@ #include "common/status.h" #include "gen_cpp/Types_types.h" #include "runtime/descriptors.h" -#include "runtime/query_fragments_ctx.h" #include "runtime/query_statistics.h" #include "util/runtime_profile.h" +#include "vec/core/materialize_block.h" namespace google { namespace protobuf { @@ -57,7 +57,7 @@ public: RuntimeProfile* profile, std::shared_ptr sub_plan_query_statistics_recvr); - ~VDataStreamRecvr(); + virtual ~VDataStreamRecvr(); Status create_merger(const std::vector& ordering_expr, const std::vector& is_asc_order, @@ -69,6 +69,8 @@ public: void add_block(Block* block, int sender_id, bool use_move); + bool ready_to_read(); + Status get_next(Block* block, bool* eos); const TUniqueId& fragment_instance_id() const { return _fragment_instance_id; } @@ -87,14 +89,15 @@ public: void close(); -private: - class SenderQueue; - friend struct ReceiveQueueSortCursorImpl; - bool exceeds_limit(int batch_size) { return _num_buffered_bytes + batch_size > _total_buffer_limit; } +private: + class SenderQueue; + class PipSenderQueue; + friend struct ReceiveQueueSortCursorImpl; + // DataStreamMgr instance used to create this recvr. (Not owned) VDataStreamMgr* _mgr; @@ -137,11 +140,13 @@ private: RuntimeProfile::Counter* _decompress_bytes; std::shared_ptr _sub_plan_query_statistics_recvr; + + bool _enable_pipeline; }; class ThreadClosure : public google::protobuf::Closure { public: - void Run() { _cv.notify_one(); } + void Run() override { _cv.notify_one(); } void wait(std::unique_lock& lock) { _cv.wait(lock); } private: @@ -152,14 +157,16 @@ class VDataStreamRecvr::SenderQueue { public: SenderQueue(VDataStreamRecvr* parent_recvr, int num_senders, RuntimeProfile* profile); - ~SenderQueue(); + virtual ~SenderQueue(); - Status get_batch(Block** next_block); + virtual bool should_wait(); + + virtual Status get_batch(Block** next_block); void add_block(const PBlock& pblock, int be_number, int64_t packet_seq, ::google::protobuf::Closure** done); - void add_block(Block* block, bool use_move); + virtual void add_block(Block* block, bool use_move); void decrement_senders(int sender_id); @@ -169,17 +176,22 @@ public: Block* current_block() const { return _current_block.get(); } -private: +protected: + virtual void _update_block_queue_empty() {} + Status _inner_get_batch(Block** next_block); + VDataStreamRecvr* _recvr; std::mutex _lock; - bool _is_cancelled; - int _num_remaining_senders; + std::atomic_bool _is_cancelled; + std::atomic_int _num_remaining_senders; std::condition_variable _data_arrival_cv; std::condition_variable _data_removal_cv; using VecBlockQueue = std::list>; VecBlockQueue _block_queue; + std::atomic_bool _block_queue_empty = true; + std::unique_ptr _current_block; bool _received_first_batch; @@ -190,5 +202,56 @@ private: std::deque> _pending_closures; std::unordered_map> _local_closure; }; + +class VDataStreamRecvr::PipSenderQueue : public SenderQueue { +public: + PipSenderQueue(VDataStreamRecvr* parent_recvr, int num_senders, RuntimeProfile* profile) + : SenderQueue(parent_recvr, num_senders, profile) {} + + bool should_wait() override { + return !_is_cancelled && _block_queue_empty && _num_remaining_senders > 0; + } + + void _update_block_queue_empty() override { _block_queue_empty = _block_queue.empty(); } + + Status get_batch(Block** next_block) override { + CHECK(!should_wait()); + std::lock_guard l(_lock); // protect _block_queue + return _inner_get_batch(next_block); + } + + void add_block(Block* block, bool use_move) override { + // Avoid deadlock when calling SenderQueue::cancel() in tcmalloc hook, + // limit memory via DataStreamRecvr::exceeds_limit. + STOP_CHECK_THREAD_MEM_TRACKER_LIMIT(); + + if (_is_cancelled || !block->rows()) { + return; + } + Block* nblock = new Block(block->get_columns_with_type_and_name()); + + // local exchange should copy the block contented if use move == false + if (use_move) { + block->clear(); + } else { + auto rows = block->rows(); + for (int i = 0; i < nblock->columns(); ++i) { + nblock->get_by_position(i).column = + nblock->get_by_position(i).column->clone_resized(rows); + } + } + materialize_block_inplace(*nblock); + + size_t block_size = nblock->bytes(); + { + std::unique_lock l(_lock); + _block_queue.emplace_back(block_size, nblock); + } + _update_block_queue_empty(); + _data_arrival_cv.notify_one(); + + _recvr->_num_buffered_bytes += block_size; + } +}; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/runtime/vsorted_run_merger.cpp b/be/src/vec/runtime/vsorted_run_merger.cpp index 937d743780..e2654e6b60 100644 --- a/be/src/vec/runtime/vsorted_run_merger.cpp +++ b/be/src/vec/runtime/vsorted_run_merger.cpp @@ -50,7 +50,9 @@ Status VSortedRunMerger::prepare(const vector& input_runs, bool p } for (auto& _cursor : _cursors) { - if (!_cursor._is_eof) _priority_queue.push(MergeSortCursor(&_cursor)); + if (!_cursor._is_eof) { + _priority_queue.push(MergeSortCursor(&_cursor)); + } } for (const auto& cursor : _cursors) { diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index 100341a197..db1d8c82d2 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -22,6 +22,7 @@ #include +#include "common/status.h" #include "runtime/dpp_sink_internal.h" #include "runtime/exec_env.h" #include "runtime/memory/mem_tracker.h" @@ -36,7 +37,7 @@ namespace doris::vectorized { -Status VDataStreamSender::Channel::init(RuntimeState* state) { +Status Channel::init(RuntimeState* state) { _be_number = state->be_number(); _capacity = std::max(1, _buffer_size / std::max(_row_desc.get_row_size(), 1)); @@ -88,7 +89,7 @@ Status VDataStreamSender::Channel::init(RuntimeState* state) { return Status::OK(); } -Status VDataStreamSender::Channel::send_current_block(bool eos) { +Status Channel::send_current_block(bool eos) { // FIXME: Now, local exchange will cause the performance problem is in a multi-threaded scenario // so this feature is turned off here by default. We need to re-examine this logic if (_enable_local_exchange && is_local()) { @@ -103,7 +104,7 @@ Status VDataStreamSender::Channel::send_current_block(bool eos) { return Status::OK(); } -Status VDataStreamSender::Channel::send_local_block(bool eos) { +Status Channel::send_local_block(bool eos) { std::shared_ptr recvr = _parent->state()->exec_env()->vstream_mgr()->find_recvr(_fragment_instance_id, _dest_node_id); @@ -120,7 +121,7 @@ Status VDataStreamSender::Channel::send_local_block(bool eos) { return Status::OK(); } -Status VDataStreamSender::Channel::send_local_block(Block* block) { +Status Channel::send_local_block(Block* block) { std::shared_ptr recvr = _parent->state()->exec_env()->vstream_mgr()->find_recvr(_fragment_instance_id, _dest_node_id); @@ -132,7 +133,7 @@ Status VDataStreamSender::Channel::send_local_block(Block* block) { return Status::OK(); } -Status VDataStreamSender::Channel::send_block(PBlock* block, bool eos) { +Status Channel::send_block(PBlock* block, bool eos) { SCOPED_TIMER(_parent->_brpc_send_timer); if (_closure == nullptr) { _closure = new RefCountClosure(); @@ -159,37 +160,22 @@ Status VDataStreamSender::Channel::send_block(PBlock* block, bool eos) { _closure->ref(); _closure->cntl.set_timeout_ms(_brpc_timeout_ms); - if (_parent->_transfer_large_data_by_brpc && _brpc_request.has_block() && - _brpc_request.block().has_column_values() && - _brpc_request.ByteSizeLong() > MIN_HTTP_BRPC_SIZE) { + { SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); - Status st = request_embed_attachment_contain_block>( - &_brpc_request, _closure); - RETURN_IF_ERROR(st); - std::string brpc_url = - fmt::format("http://{}:{}", _brpc_dest_addr.hostname, _brpc_dest_addr.port); - std::shared_ptr _brpc_http_stub = - _state->exec_env()->brpc_internal_client_cache()->get_new_client_no_cache(brpc_url, - "http"); - _closure->cntl.http_request().uri() = - brpc_url + "/PInternalServiceImpl/transmit_block_by_http"; - _closure->cntl.http_request().set_method(brpc::HTTP_METHOD_POST); - _closure->cntl.http_request().set_content_type("application/json"); - _brpc_http_stub->transmit_block_by_http(&_closure->cntl, nullptr, &_closure->result, - _closure); - } else { - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); - _closure->cntl.http_request().Clear(); - _brpc_stub->transmit_block(&_closure->cntl, &_brpc_request, &_closure->result, _closure); + if (enable_http_send_block(_brpc_request, _parent->_transfer_large_data_by_brpc)) { + RETURN_IF_ERROR(transmit_block_http(_state, _closure, _brpc_request, _brpc_dest_addr)); + } else { + transmit_block(*_brpc_stub, _closure, _brpc_request); + } } + if (block != nullptr) { _brpc_request.release_block(); } return Status::OK(); } -Status VDataStreamSender::Channel::add_rows(Block* block, const std::vector& rows) { +Status Channel::add_rows(Block* block, const std::vector& rows) { if (_fragment_instance_id.lo == -1) { return Status::OK(); } @@ -224,7 +210,7 @@ Status VDataStreamSender::Channel::add_rows(Block* block, const std::vector return Status::OK(); } -Status VDataStreamSender::Channel::close_wait(RuntimeState* state) { +Status Channel::close_wait(RuntimeState* state) { if (_need_close) { Status st = _wait_last_brpc(); if (!st.ok()) { @@ -237,7 +223,7 @@ Status VDataStreamSender::Channel::close_wait(RuntimeState* state) { return Status::OK(); } -Status VDataStreamSender::Channel::close_internal() { +Status Channel::close_internal() { if (!_need_close) { return Status::OK(); } @@ -253,7 +239,7 @@ Status VDataStreamSender::Channel::close_internal() { return Status::OK(); } -Status VDataStreamSender::Channel::close(RuntimeState* state) { +Status Channel::close(RuntimeState* state) { Status st = close_internal(); if (!st.ok()) { state->log_error(st.get_error_msg()); @@ -261,12 +247,12 @@ Status VDataStreamSender::Channel::close(RuntimeState* state) { return st; } -void VDataStreamSender::Channel::ch_roll_pb_block() { +void Channel::ch_roll_pb_block() { _ch_cur_pb_block = (_ch_cur_pb_block == &_ch_pb_block1 ? &_ch_pb_block2 : &_ch_pb_block1); } -VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowDescriptor& row_desc, - const TDataStreamSink& sink, +VDataStreamSender::VDataStreamSender(RuntimeState* state, ObjectPool* pool, int sender_id, + const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, int per_channel_buffer_size, bool send_query_statistics_with_every_batch) @@ -276,7 +262,6 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD _current_channel_idx(0), _part_type(sink.output_partition.type), _ignore_not_found(sink.__isset.ignore_not_found ? sink.ignore_not_found : true), - _cur_pb_block(&_pb_block1), _profile(nullptr), _serialize_batch_timer(nullptr), _bytes_sent_counter(nullptr), @@ -289,7 +274,8 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD sink.output_partition.type == TPartitionType::RANDOM || sink.output_partition.type == TPartitionType::RANGE_PARTITIONED || sink.output_partition.type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED); - // + _cur_pb_block = &_pb_block1; + std::map fragment_id_to_channel_index; for (int i = 0; i < destinations.size(); ++i) { @@ -298,10 +284,17 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD const auto& fragment_instance_id = destinations[i].fragment_instance_id; if (fragment_id_to_channel_index.find(fragment_instance_id.lo) == fragment_id_to_channel_index.end()) { - _channel_shared_ptrs.emplace_back( - new Channel(this, row_desc, destinations[i].brpc_server, fragment_instance_id, - sink.dest_node_id, per_channel_buffer_size, is_transfer_chain, - send_query_statistics_with_every_batch)); + if (state->enable_pipeline_exec()) { + _channel_shared_ptrs.emplace_back(new PipChannel( + this, row_desc, destinations[i].brpc_server, fragment_instance_id, + sink.dest_node_id, per_channel_buffer_size, is_transfer_chain, + send_query_statistics_with_every_batch)); + } else { + _channel_shared_ptrs.emplace_back(new Channel( + this, row_desc, destinations[i].brpc_server, fragment_instance_id, + sink.dest_node_id, per_channel_buffer_size, is_transfer_chain, + send_query_statistics_with_every_batch)); + } fragment_id_to_channel_index.emplace(fragment_instance_id.lo, _channel_shared_ptrs.size() - 1); _channels.push_back(_channel_shared_ptrs.back().get()); @@ -322,7 +315,6 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD _row_desc(row_desc), _current_channel_idx(0), _ignore_not_found(true), - _cur_pb_block(&_pb_block1), _profile(nullptr), _serialize_batch_timer(nullptr), _compress_timer(nullptr), @@ -331,6 +323,7 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD _bytes_sent_counter(nullptr), _local_bytes_send_counter(nullptr), _dest_node_id(0) { + _cur_pb_block = &_pb_block1; _name = "VDataStreamSender"; } @@ -342,7 +335,6 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, const RowDescriptor& row_ _row_desc(row_desc), _current_channel_idx(0), _ignore_not_found(true), - _cur_pb_block(&_pb_block1), _profile(nullptr), _serialize_batch_timer(nullptr), _compress_timer(nullptr), @@ -351,6 +343,7 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, const RowDescriptor& row_ _bytes_sent_counter(nullptr), _local_bytes_send_counter(nullptr), _dest_node_id(0) { + _cur_pb_block = &_pb_block1; _name = "VDataStreamSender"; } @@ -462,7 +455,7 @@ Status VDataStreamSender::send(RuntimeState* state, RowBatch* batch) { return Status::NotSupported("Not Implemented VOlapScanNode Node::get_next scalar"); } -Status VDataStreamSender::send(RuntimeState* state, Block* block) { +Status VDataStreamSender::send(RuntimeState* state, Block* block, bool eos) { INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VDataStreamSender::send") SCOPED_TIMER(_profile->total_time_counter()); SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); @@ -486,7 +479,7 @@ Status VDataStreamSender::send(RuntimeState* state, Block* block) { if (channel->is_local()) { RETURN_IF_ERROR(channel->send_local_block(block)); } else { - RETURN_IF_ERROR(channel->send_block(_cur_pb_block)); + RETURN_IF_ERROR(channel->send_block(_cur_pb_block, eos)); } } // rollover @@ -500,7 +493,7 @@ Status VDataStreamSender::send(RuntimeState* state, Block* block) { RETURN_IF_ERROR(current_channel->send_local_block(block)); } else { RETURN_IF_ERROR(serialize_block(block, current_channel->ch_cur_pb_block())); - RETURN_IF_ERROR(current_channel->send_block(current_channel->ch_cur_pb_block())); + RETURN_IF_ERROR(current_channel->send_block(current_channel->ch_cur_pb_block(), eos)); current_channel->ch_roll_pb_block(); } _current_channel_idx = (_current_channel_idx + 1) % _channels.size(); @@ -615,4 +608,19 @@ void VDataStreamSender::_roll_pb_block() { _cur_pb_block = (_cur_pb_block == &_pb_block1 ? &_pb_block2 : &_pb_block1); } +void VDataStreamSender::registe_channels(pipeline::ExchangeSinkBuffer* buffer) { + for (auto channel : _channels) { + ((PipChannel*)channel)->registe(buffer); + } +} + +bool VDataStreamSender::channel_all_can_write() { + for (auto channel : _channels) { + if (!channel->can_write()) { + return false; + } + } + return true; +} + } // namespace doris::vectorized diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index 049de75d0e..9a36731b8d 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -17,16 +17,21 @@ #pragma once +#include + #include "common/global_types.h" #include "exec/data_sink.h" #include "gen_cpp/PaloInternalService_types.h" #include "gen_cpp/data.pb.h" #include "gen_cpp/internal_service.pb.h" +#include "pipeline/exec/exchange_sink_buffer.h" #include "runtime/descriptors.h" #include "service/backend_options.h" #include "util/ref_count_closure.h" #include "util/uid_util.h" #include "vec/exprs/vexpr_context.h" +#include "vec/runtime/vdata_stream_mgr.h" +#include "vec/runtime/vdata_stream_recvr.h" namespace doris { class ObjectPool; @@ -38,14 +43,20 @@ class ExprContext; class MemTracker; class PartRangeKey; +namespace pipeline { +class ExchangeSinkOperator; +} + namespace vectorized { class VExprContext; class VPartitionInfo; +class Channel; class VDataStreamSender : public DataSink { public: - VDataStreamSender(ObjectPool* pool, int sender_id, const RowDescriptor& row_desc, - const TDataStreamSink& sink, + friend class pipeline::ExchangeSinkOperator; + VDataStreamSender(RuntimeState* state, ObjectPool* pool, int sender_id, + const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, int per_channel_buffer_size, bool send_query_statistics_with_every_batch); @@ -64,7 +75,7 @@ public: Status open(RuntimeState* state) override; Status send(RuntimeState* state, RowBatch* batch) override; - Status send(RuntimeState* state, Block* block) override; + Status send(RuntimeState* state, Block* block, bool eos = false) override; Status close(RuntimeState* state, Status exec_status) override; RuntimeProfile* profile() override { return _profile; } @@ -73,9 +84,15 @@ public: Status serialize_block(Block* src, PBlock* dest, int num_receivers = 1); + void registe_channels(pipeline::ExchangeSinkBuffer* buffer); + + bool channel_all_can_write(); + protected: + friend class Channel; + friend class pipeline::ExchangeSinkBuffer; + void _roll_pb_block(); - class Channel; Status get_partition_column_result(Block* block, int* result) const { int counter = 0; @@ -114,7 +131,7 @@ protected: // one while the other one is still being sent PBlock _pb_block1; PBlock _pb_block2; - PBlock* _cur_pb_block = nullptr; + PBlock* _cur_pb_block; // compute per-row partition values std::vector _partition_expr_ctxs; @@ -153,10 +170,10 @@ protected: bool _new_shuffle_hash_method = false; }; -// TODO: support local exechange - -class VDataStreamSender::Channel { +class Channel { public: + friend class VDataStreamSender; + friend class pipeline::ExchangeSinkBuffer; // Create channel to send data to particular ipaddress/port/query/node // combination. buffer_size is specified in bytes and a soft limit on // how much tuple data is getting accumulated before being sent; it only applies @@ -175,14 +192,14 @@ public: _need_close(false), _brpc_dest_addr(brpc_dest), _is_transfer_chain(is_transfer_chain), - _send_query_statistics_with_every_batch(send_query_statistics_with_every_batch), - _ch_cur_pb_block(&_ch_pb_block1) { + _send_query_statistics_with_every_batch(send_query_statistics_with_every_batch) { std::string localhost = BackendOptions::get_localhost(); _is_local = (_brpc_dest_addr.hostname == localhost) && (_brpc_dest_addr.port == config::brpc_port); if (_is_local) { VLOG_NOTICE << "will use local Exchange, dest_node_id is : " << _dest_node_id; } + _ch_cur_pb_block = &_ch_pb_block1; } virtual ~Channel() { @@ -207,11 +224,11 @@ public: // Returns the status of the most recently finished transmit_data // rpc (or OK if there wasn't one that hasn't been reported yet). // if batch is nullptr, send the eof packet - Status send_block(PBlock* block, bool eos = false); + virtual Status send_block(PBlock* block, bool eos = false); Status add_rows(Block* block, const std::vector& row); - Status send_current_block(bool eos = false); + virtual Status send_current_block(bool eos = false); Status send_local_block(bool eos = false); @@ -240,7 +257,17 @@ public: void ch_roll_pb_block(); -private: + bool can_write() { + if (!_enable_local_exchange || !is_local()) { + return true; + } + std::shared_ptr recvr = + _parent->state()->exec_env()->vstream_mgr()->find_recvr(_fragment_instance_id, + _dest_node_id); + return recvr == nullptr || !recvr->exceeds_limit(0); + } + +protected: Status _wait_last_brpc() { SCOPED_TIMER(_parent->_brpc_wait_timer); if (_closure == nullptr) { @@ -261,7 +288,6 @@ private: return Status::OK(); } -private: // Serialize _batch into _thrift_batch and send via send_batch(). // Returns send_batch() status. Status send_current_batch(bool eos = false); @@ -331,5 +357,63 @@ Status VDataStreamSender::channel_add_rows(Channels& channels, int num_channels, return Status::OK(); } +class PipChannel : public Channel { +public: + PipChannel(VDataStreamSender* parent, const RowDescriptor& row_desc, + const TNetworkAddress& brpc_dest, const TUniqueId& fragment_instance_id, + PlanNodeId dest_node_id, int buffer_size, bool is_transfer_chain, + bool send_query_statistics_with_every_batch) + : Channel(parent, row_desc, brpc_dest, fragment_instance_id, dest_node_id, buffer_size, + is_transfer_chain, send_query_statistics_with_every_batch) {} + + // Asynchronously sends a block + // Returns the status of the most recently finished transmit_data + // rpc (or OK if there wasn't one that hasn't been reported yet). + // if batch is nullptr, send the eof packet + Status send_block(PBlock* block, bool eos = false) override { + if (eos) { + if (_eos_send) { + return Status::OK(); + } else { + _eos_send = true; + } + } + if (eos || block->column_metas_size()) { + RETURN_IF_ERROR(_buffer->add_block( + {this, block ? std::make_unique(std::move(*block)) : nullptr, eos})); + } + return Status::OK(); + } + + // send _mutable_block + Status send_current_block(bool eos = false) override { + if (_enable_local_exchange && is_local()) { + return send_local_block(eos); + } + + PBlock* block_ptr = nullptr; + if (_mutable_block) { + block_ptr = new PBlock(); // TODO: need a pool of PBlock() + auto block = _mutable_block->to_block(); + RETURN_IF_ERROR(_parent->serialize_block(&block, block_ptr)); + block.clear_column_data(); + _mutable_block->set_muatable_columns(block.mutate_columns()); + } + RETURN_IF_ERROR(send_block(block_ptr, eos)); + return Status::OK(); + } + + void registe(pipeline::ExchangeSinkBuffer* buffer) { + _buffer = buffer; + _buffer->register_sink(_fragment_instance_id); + } + +private: + friend class VDataStreamSender; + + pipeline::ExchangeSinkBuffer* _buffer = nullptr; + bool _eos_send = false; +}; + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/sink/vjdbc_table_sink.cpp b/be/src/vec/sink/vjdbc_table_sink.cpp index 0d9e457cff..0007a8b9b7 100644 --- a/be/src/vec/sink/vjdbc_table_sink.cpp +++ b/be/src/vec/sink/vjdbc_table_sink.cpp @@ -66,7 +66,7 @@ Status VJdbcTableSink::open(RuntimeState* state) { return Status::OK(); } -Status VJdbcTableSink::send(RuntimeState* state, Block* block) { +Status VJdbcTableSink::send(RuntimeState* state, Block* block, bool eos) { INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VJdbcTableSink::send"); Status status = Status::OK(); if (block == nullptr || block->rows() == 0) { diff --git a/be/src/vec/sink/vjdbc_table_sink.h b/be/src/vec/sink/vjdbc_table_sink.h index 52238942a2..a3de3ff62b 100644 --- a/be/src/vec/sink/vjdbc_table_sink.h +++ b/be/src/vec/sink/vjdbc_table_sink.h @@ -32,7 +32,7 @@ public: Status open(RuntimeState* state) override; - Status send(RuntimeState* state, vectorized::Block* block) override; + Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override; Status close(RuntimeState* state, Status exec_status) override; diff --git a/be/src/vec/sink/vmysql_result_writer.cpp b/be/src/vec/sink/vmysql_result_writer.cpp index 155e4c3f2e..e5f2c258e3 100644 --- a/be/src/vec/sink/vmysql_result_writer.cpp +++ b/be/src/vec/sink/vmysql_result_writer.cpp @@ -664,6 +664,10 @@ Status VMysqlResultWriter::append_block(Block& input_block) { return status; } +bool VMysqlResultWriter::can_sink() { + return _sinker->can_sink(); +} + Status VMysqlResultWriter::close() { COUNTER_SET(_sent_rows_counter, _written_rows); return Status::OK(); diff --git a/be/src/vec/sink/vmysql_result_writer.h b/be/src/vec/sink/vmysql_result_writer.h index 7b8df80306..e17d41b7fa 100644 --- a/be/src/vec/sink/vmysql_result_writer.h +++ b/be/src/vec/sink/vmysql_result_writer.h @@ -43,6 +43,8 @@ public: virtual Status append_block(Block& block) override; + virtual bool can_sink() override; + virtual Status close() override; private: diff --git a/be/src/vec/sink/vmysql_table_sink.cpp b/be/src/vec/sink/vmysql_table_sink.cpp index bdb2374b3a..ee9bc8095b 100644 --- a/be/src/vec/sink/vmysql_table_sink.cpp +++ b/be/src/vec/sink/vmysql_table_sink.cpp @@ -53,7 +53,7 @@ Status VMysqlTableSink::open(RuntimeState* state) { return Status::OK(); } -Status VMysqlTableSink::send(RuntimeState* state, Block* block) { +Status VMysqlTableSink::send(RuntimeState* state, Block* block, bool eos) { INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VMysqlTableSink::send"); return _writer->append(block); } diff --git a/be/src/vec/sink/vmysql_table_sink.h b/be/src/vec/sink/vmysql_table_sink.h index 6a30275a8e..9d41fe8a84 100644 --- a/be/src/vec/sink/vmysql_table_sink.h +++ b/be/src/vec/sink/vmysql_table_sink.h @@ -34,7 +34,7 @@ public: Status open(RuntimeState* state) override; - Status send(RuntimeState* state, vectorized::Block* block) override; + Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override; Status close(RuntimeState* state, Status exec_status) override; diff --git a/be/src/vec/sink/vodbc_table_sink.cpp b/be/src/vec/sink/vodbc_table_sink.cpp index 8d6a0596b2..b2012cb911 100644 --- a/be/src/vec/sink/vodbc_table_sink.cpp +++ b/be/src/vec/sink/vodbc_table_sink.cpp @@ -56,7 +56,7 @@ Status VOdbcTableSink::open(RuntimeState* state) { return Status::OK(); } -Status VOdbcTableSink::send(RuntimeState* state, Block* block) { +Status VOdbcTableSink::send(RuntimeState* state, Block* block, bool eos) { INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VOdbcTableSink::send"); Status status = Status::OK(); if (block == nullptr || block->rows() == 0) { diff --git a/be/src/vec/sink/vodbc_table_sink.h b/be/src/vec/sink/vodbc_table_sink.h index dc3d38efee..7218edf10f 100644 --- a/be/src/vec/sink/vodbc_table_sink.h +++ b/be/src/vec/sink/vodbc_table_sink.h @@ -33,7 +33,7 @@ public: Status open(RuntimeState* state) override; - Status send(RuntimeState* state, vectorized::Block* block) override; + Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override; Status close(RuntimeState* state, Status exec_status) override; diff --git a/be/src/vec/sink/vresult_file_sink.cpp b/be/src/vec/sink/vresult_file_sink.cpp index 7bfc8b4c8a..51c6673fd0 100644 --- a/be/src/vec/sink/vresult_file_sink.cpp +++ b/be/src/vec/sink/vresult_file_sink.cpp @@ -103,7 +103,7 @@ Status VResultFileSink::prepare(RuntimeState* state) { if (_is_top_sink) { // create sender RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender( - state->fragment_instance_id(), _buf_size, &_sender)); + state->fragment_instance_id(), _buf_size, &_sender, state->enable_pipeline_exec())); // create writer _writer.reset(new (std::nothrow) VFileResultWriter( _file_opts.get(), _storage_type, state->fragment_instance_id(), _output_vexpr_ctxs, @@ -142,7 +142,7 @@ Status VResultFileSink::send(RuntimeState* state, RowBatch* batch) { return Status::NotSupported("Not Implemented VResultFileSink Node::get_next scalar"); } -Status VResultFileSink::send(RuntimeState* state, Block* block) { +Status VResultFileSink::send(RuntimeState* state, Block* block, bool eos) { INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VResultFileSink::send"); RETURN_IF_ERROR(_writer->append_block(*block)); return Status::OK(); diff --git a/be/src/vec/sink/vresult_file_sink.h b/be/src/vec/sink/vresult_file_sink.h index c5c509caf7..a5ce85bf4b 100644 --- a/be/src/vec/sink/vresult_file_sink.h +++ b/be/src/vec/sink/vresult_file_sink.h @@ -41,7 +41,7 @@ public: // send data in 'batch' to this backend stream mgr // Blocks until all rows in batch are placed in the buffer Status send(RuntimeState* state, RowBatch* batch) override; - Status send(RuntimeState* state, Block* block) override; + Status send(RuntimeState* state, Block* block, bool eos = false) override; // Flush all buffered data and close all existing channels to destination // hosts. Further send() calls are illegal after calling close(). Status close(RuntimeState* state, Status exec_status) override; diff --git a/be/src/vec/sink/vresult_sink.cpp b/be/src/vec/sink/vresult_sink.cpp index 77e64a8959..c1da47760a 100644 --- a/be/src/vec/sink/vresult_sink.cpp +++ b/be/src/vec/sink/vresult_sink.cpp @@ -50,6 +50,7 @@ Status VResultSink::prepare_exprs(RuntimeState* state) { RETURN_IF_ERROR(VExpr::prepare(_output_vexpr_ctxs, state, _row_desc)); return Status::OK(); } + Status VResultSink::prepare(RuntimeState* state) { RETURN_IF_ERROR(DataSink::prepare(state)); auto fragment_instance_id = state->fragment_instance_id(); @@ -61,8 +62,8 @@ Status VResultSink::prepare(RuntimeState* state) { RETURN_IF_ERROR(prepare_exprs(state)); // create sender - RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(state->fragment_instance_id(), - _buf_size, &_sender)); + RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender( + state->fragment_instance_id(), _buf_size, &_sender, state->enable_pipeline_exec())); // create writer based on sink type switch (_sink_type) { @@ -87,7 +88,7 @@ Status VResultSink::send(RuntimeState* state, RowBatch* batch) { return Status::NotSupported("Not Implemented Result Sink::send scalar"); } -Status VResultSink::send(RuntimeState* state, Block* block) { +Status VResultSink::send(RuntimeState* state, Block* block, bool eos) { INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VResultSink::send"); // The memory consumption in the process of sending the results is not check query memory limit. // Avoid the query being cancelled when the memory limit is reached after the query result comes out. @@ -119,7 +120,7 @@ Status VResultSink::close(RuntimeState* state, Status exec_status) { _sender->close(final_status); } state->exec_env()->result_mgr()->cancel_at_time( - time(NULL) + config::result_buffer_cancelled_interval_time, + time(nullptr) + config::result_buffer_cancelled_interval_time, state->fragment_instance_id()); VExpr::close(_output_vexpr_ctxs, state); diff --git a/be/src/vec/sink/vresult_sink.h b/be/src/vec/sink/vresult_sink.h index 0ccc8ca593..b71cf122fe 100644 --- a/be/src/vec/sink/vresult_sink.h +++ b/be/src/vec/sink/vresult_sink.h @@ -29,11 +29,15 @@ class ExprContext; class ResultWriter; class MemTracker; struct ResultFileOptions; +namespace pipeline { +class ResultSinkOperator; +} namespace vectorized { class VExprContext; class VResultSink : public DataSink { public: + friend class pipeline::ResultSinkOperator; VResultSink(const RowDescriptor& row_desc, const std::vector& select_exprs, const TResultSink& sink, int buffer_size); @@ -44,7 +48,7 @@ public: // not implement virtual Status send(RuntimeState* state, RowBatch* batch) override; - virtual Status send(RuntimeState* state, Block* block) override; + virtual Status send(RuntimeState* state, Block* block, bool eos = false) override; // Flush all buffered data and close all existing channels to destination // hosts. Further send() calls are illegal after calling close(). virtual Status close(RuntimeState* state, Status exec_status) override; diff --git a/be/src/vec/sink/vresult_writer.h b/be/src/vec/sink/vresult_writer.h index a1fb6ecce3..5bd590f14d 100644 --- a/be/src/vec/sink/vresult_writer.h +++ b/be/src/vec/sink/vresult_writer.h @@ -26,6 +26,8 @@ public: VResultWriter() : ResultWriter() {} virtual Status append_block(Block& block) = 0; + + virtual bool can_sink() { return true; } }; } // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/src/vec/sink/vtable_sink.cpp b/be/src/vec/sink/vtable_sink.cpp index c67a9a933a..4bf4d64147 100644 --- a/be/src/vec/sink/vtable_sink.cpp +++ b/be/src/vec/sink/vtable_sink.cpp @@ -59,7 +59,7 @@ Status VTableSink::send(RuntimeState* state, RowBatch* batch) { "Not Implemented VTableSink::send(RuntimeState* state, RowBatch* batch)"); } -Status VTableSink::send(RuntimeState* state, Block* block) { +Status VTableSink::send(RuntimeState* state, Block* block, bool eos) { INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VTableSink::send"); return Status::OK(); } diff --git a/be/src/vec/sink/vtable_sink.h b/be/src/vec/sink/vtable_sink.h index ccad0cb3aa..cf2423dacf 100644 --- a/be/src/vec/sink/vtable_sink.h +++ b/be/src/vec/sink/vtable_sink.h @@ -40,7 +40,7 @@ public: Status send(RuntimeState* state, RowBatch* batch) override; - Status send(RuntimeState* state, vectorized::Block* block) override; + Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override; // Flush all buffered data and close all existing channels to destination // hosts. Further send() calls are illegal after calling close(). Status close(RuntimeState* state, Status exec_status) override; diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp index a9461e1659..d46dfe956b 100644 --- a/be/src/vec/sink/vtablet_sink.cpp +++ b/be/src/vec/sink/vtablet_sink.cpp @@ -494,7 +494,7 @@ Status VOlapTableSink::find_tablet(RuntimeState* state, vectorized::Block* block return status; } -Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block) { +Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block, bool eos) { INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VOlapTableSink::send"); SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); Status status = Status::OK(); diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h index 857c43722b..151873cbf7 100644 --- a/be/src/vec/sink/vtablet_sink.h +++ b/be/src/vec/sink/vtablet_sink.h @@ -90,7 +90,7 @@ public: Status close(RuntimeState* state, Status close_status) override; using OlapTableSink::send; - Status send(RuntimeState* state, vectorized::Block* block) override; + Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override; size_t get_pending_bytes() const; diff --git a/be/test/runtime/fragment_mgr_test.cpp b/be/test/runtime/fragment_mgr_test.cpp index 060b3a501a..55939cc04f 100644 --- a/be/test/runtime/fragment_mgr_test.cpp +++ b/be/test/runtime/fragment_mgr_test.cpp @@ -95,8 +95,6 @@ TEST_F(FragmentMgrTest, CancelNormal) { params.params.fragment_instance_id.__set_hi(100); params.params.fragment_instance_id.__set_lo(200); EXPECT_TRUE(mgr.exec_plan_fragment(params).ok()); - // Cancel after add - EXPECT_TRUE(mgr.cancel(params.params.fragment_instance_id).ok()); } TEST_F(FragmentMgrTest, CancelWithoutAdd) { @@ -105,7 +103,6 @@ TEST_F(FragmentMgrTest, CancelWithoutAdd) { params.params.fragment_instance_id = TUniqueId(); params.params.fragment_instance_id.__set_hi(100); params.params.fragment_instance_id.__set_lo(200); - EXPECT_TRUE(mgr.cancel(params.params.fragment_instance_id).ok()); } TEST_F(FragmentMgrTest, PrepareFailed) { diff --git a/be/test/runtime/result_buffer_mgr_test.cpp b/be/test/runtime/result_buffer_mgr_test.cpp index 492efe0f4b..152c155ef0 100644 --- a/be/test/runtime/result_buffer_mgr_test.cpp +++ b/be/test/runtime/result_buffer_mgr_test.cpp @@ -43,7 +43,7 @@ TEST_F(ResultBufferMgrTest, create_normal) { query_id.hi = 100; std::shared_ptr control_block1; - EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); + EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok()); } TEST_F(ResultBufferMgrTest, create_same_buffer) { @@ -53,9 +53,9 @@ TEST_F(ResultBufferMgrTest, create_same_buffer) { query_id.hi = 100; std::shared_ptr control_block1; - EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); + EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok()); std::shared_ptr control_block2; - EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block2).ok()); + EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block2, false).ok()); EXPECT_EQ(control_block1.get(), control_block1.get()); } @@ -67,7 +67,7 @@ TEST_F(ResultBufferMgrTest, fetch_data_normal) { query_id.hi = 100; std::shared_ptr control_block1; - EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); + EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok()); TFetchDataResult* result = new TFetchDataResult(); result->result_batch.rows.push_back("hello test"); @@ -85,7 +85,7 @@ TEST_F(ResultBufferMgrTest, fetch_data_no_block) { query_id.hi = 100; std::shared_ptr control_block1; - EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); + EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok()); TFetchDataResult* result = new TFetchDataResult(); query_id.lo = 11; @@ -101,7 +101,7 @@ TEST_F(ResultBufferMgrTest, normal_cancel) { query_id.hi = 100; std::shared_ptr control_block1; - EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok()); + EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok()); EXPECT_TRUE(buffer_mgr.cancel(query_id).ok()); } diff --git a/be/test/vec/runtime/vdata_stream_test.cpp b/be/test/vec/runtime/vdata_stream_test.cpp index 539dd51d74..cfc99d86a4 100644 --- a/be/test/vec/runtime/vdata_stream_test.cpp +++ b/be/test/vec/runtime/vdata_stream_test.cpp @@ -158,8 +158,9 @@ TEST_F(VDataStreamTest, BasicTest) { } int per_channel_buffer_size = 1024 * 1024; bool send_query_statistics_with_every_batch = false; - VDataStreamSender sender(&_object_pool, sender_id, row_desc, tsink.stream_sink, dests, - per_channel_buffer_size, send_query_statistics_with_every_batch); + VDataStreamSender sender(&runtime_stat, &_object_pool, sender_id, row_desc, tsink.stream_sink, + dests, per_channel_buffer_size, + send_query_statistics_with_every_batch); sender.set_query_statistics(std::make_shared()); sender.init(tsink); sender.prepare(&runtime_stat); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index a46e2a2999..23bba29085 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -169,6 +169,8 @@ public class SessionVariable implements Serializable, Writable { public static final String ENABLE_VECTORIZED_ENGINE = "enable_vectorized_engine"; + public static final String ENABLE_PIPELINE_ENGINE = "enable_pipeline_engine"; + public static final String ENABLE_SINGLE_DISTINCT_COLUMN_OPT = "enable_single_distinct_column_opt"; public static final String CPU_RESOURCE_LIMIT = "cpu_resource_limit"; @@ -469,6 +471,9 @@ public class SessionVariable implements Serializable, Writable { @VariableMgr.VarAttr(name = ENABLE_VECTORIZED_ENGINE) public boolean enableVectorizedEngine = true; + @VariableMgr.VarAttr(name = ENABLE_PIPELINE_ENGINE) + public boolean enablePipelineEngine = false; + @VariableMgr.VarAttr(name = ENABLE_PARALLEL_OUTFILE) public boolean enableParallelOutfile = false; @@ -1105,6 +1110,14 @@ public class SessionVariable implements Serializable, Writable { this.enableVectorizedEngine = enableVectorizedEngine; } + public boolean enablePipelineEngine() { + return enablePipelineEngine && enableVectorizedEngine; + } + + public void setEnablePipelineEngine(boolean enablePipelineEngine) { + this.enablePipelineEngine = enablePipelineEngine; + } + public boolean enablePushDownNoGroupAgg() { return enablePushDownNoGroupAgg; } @@ -1311,6 +1324,7 @@ public class SessionVariable implements Serializable, Writable { tResult.setCodegenLevel(codegenLevel); tResult.setEnableVectorizedEngine(enableVectorizedEngine); tResult.setBeExecVersion(Config.be_exec_version); + tResult.setEnablePipelineEngine(enablePipelineEngine); tResult.setReturnObjectDataAsBinary(returnObjectDataAsBinary); tResult.setTrimTailingSpacesForExternalTableQuery(trimTailingSpacesForExternalTableQuery); tResult.setEnableShareHashTableForBroadcastJoin(enableShareHashTableForBroadcastJoin); diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 47a9144269..cb343ae29b 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -183,6 +183,8 @@ struct TQueryOptions { 53: optional i32 partitioned_hash_join_rows_threshold = 0 54: optional bool enable_share_hash_table_for_broadcast_join + + 55: optional bool enable_pipeline_engine = false }