From ff7b9dd9baa90dc0af7b4f7adb82226cb3367389 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Mon, 14 Jan 2019 15:57:36 +0800 Subject: [PATCH 01/18] Add cpu and io indicates to audit log --- be/src/exec/analytic_eval_node.cpp | 6 + be/src/exec/analytic_eval_node.h | 1 + be/src/exec/data_sink.cpp | 9 +- be/src/exec/data_sink.h | 6 + be/src/exec/exchange_node.cpp | 17 +- be/src/exec/exchange_node.h | 7 +- be/src/exec/exec_node.cpp | 8 + be/src/exec/exec_node.h | 6 + be/src/exec/hash_join_node.cpp | 19 +- be/src/exec/hash_join_node.h | 5 +- .../exec/new_partitioned_aggregation_node.cc | 6 + .../exec/new_partitioned_aggregation_node.h | 1 + be/src/exec/olap_scan_node.cpp | 6 + be/src/exec/olap_scan_node.h | 1 + be/src/exec/sort_node.cpp | 7 + be/src/exec/sort_node.h | 1 + be/src/exec/union_node.cpp | 6 + be/src/exec/union_node.h | 1 + be/src/runtime/buffer_control_block.cpp | 10 +- be/src/runtime/buffer_control_block.h | 10 +- be/src/runtime/data_stream_mgr.cpp | 13 +- be/src/runtime/data_stream_mgr.h | 6 +- be/src/runtime/data_stream_recvr.cc | 6 +- be/src/runtime/data_stream_recvr.h | 10 +- be/src/runtime/data_stream_sender.cpp | 27 ++- be/src/runtime/data_stream_sender.h | 2 +- be/src/runtime/plan_fragment_executor.cpp | 7 + be/src/runtime/plan_fragment_executor.h | 4 + be/src/runtime/query_statistic.h | 217 ++++++++++++++++++ be/src/runtime/result_sink.cpp | 4 + be/src/runtime/result_sink.h | 2 + be/src/runtime/runtime_state.cpp | 2 +- be/src/runtime/runtime_state.h | 1 + be/src/service/backend_service.cpp | 16 +- be/src/service/internal_service.cpp | 12 + .../proc/CurrentQueryFragmentProcNode.java | 4 +- .../common/proc/CurrentQueryInfoProvider.java | 19 +- .../proc/CurrentQueryStatisticsProcDir.java | 4 +- .../apache/doris/planner/PlanFragment.java | 14 +- .../org/apache/doris/planner/Planner.java | 38 +++ .../org/apache/doris/qe/ConnectProcessor.java | 10 +- .../java/org/apache/doris/qe/Coordinator.java | 18 +- .../org/apache/doris/qe/ResultReceiver.java | 14 +- .../java/org/apache/doris/qe/RowBatch.java | 55 +++++ .../org/apache/doris/qe/StmtExecutor.java | 59 ++++- .../apache/doris/rpc/PFetchDataResult.java | 2 + .../org/apache/doris/rpc/PQueryStatistic.java | 29 +++ .../apache/doris/qe/ConnectProcessorTest.java | 4 +- gensrc/proto/data.proto | 5 + gensrc/proto/internal_service.proto | 2 + gensrc/thrift/PaloInternalService.thrift | 1 + 51 files changed, 672 insertions(+), 68 deletions(-) create mode 100644 be/src/runtime/query_statistic.h create mode 100644 fe/src/main/java/org/apache/doris/qe/RowBatch.java create mode 100644 fe/src/main/java/org/apache/doris/rpc/PQueryStatistic.java diff --git a/be/src/exec/analytic_eval_node.cpp b/be/src/exec/analytic_eval_node.cpp index 99aa5e8b51a71d..2a80d2707c3a86 100644 --- a/be/src/exec/analytic_eval_node.cpp +++ b/be/src/exec/analytic_eval_node.cpp @@ -839,6 +839,12 @@ Status AnalyticEvalNode::get_next(RuntimeState* state, RowBatch* row_batch, bool return Status::OK; } +Status AnalyticEvalNode::collect_query_statistic(QueryStatistic* statistic) { + RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); + statistic->add_cpu_by_row(_process_rows_counter->value()); + return Status::OK; +} + Status AnalyticEvalNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK; diff --git a/be/src/exec/analytic_eval_node.h b/be/src/exec/analytic_eval_node.h index 6b92c37ea57e08..14ebb525e149bc 100644 --- a/be/src/exec/analytic_eval_node.h +++ b/be/src/exec/analytic_eval_node.h @@ -68,6 +68,7 @@ class AnalyticEvalNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); + virtual Status collect_query_statistic(QueryStatistic* statistic); virtual Status close(RuntimeState* state); protected: diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index 8f480df1e091b0..51109e9a1e0fcd 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -49,11 +49,16 @@ Status DataSink::create_data_sink( if (!thrift_sink.__isset.stream_sink) { return Status("Missing data stream sink."); } - + bool send_query_statistic_with_every_batch = false; + if (params.__isset.send_query_statistic_with_every_batch) { + send_query_statistic_with_every_batch = + params.send_query_statistic_with_every_batch; + } // TODO: figure out good buffer size based on size of output row tmp_sink = new DataStreamSender( pool, params.sender_id, row_desc, - thrift_sink.stream_sink, params.destinations, 16 * 1024); + thrift_sink.stream_sink, params.destinations, 16 * 1024, + send_query_statistic_with_every_batch); // RETURN_IF_ERROR(sender->prepare(state->obj_pool(), thrift_sink.stream_sink)); sink->reset(tmp_sink); break; diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h index 2064bf62e374c2..2fa73f2b357abb 100644 --- a/be/src/exec/data_sink.h +++ b/be/src/exec/data_sink.h @@ -25,6 +25,7 @@ #include "gen_cpp/DataSinks_types.h" #include "gen_cpp/Exprs_types.h" #include "runtime/mem_tracker.h" +#include "runtime/query_statistic.h" namespace doris { @@ -78,11 +79,16 @@ class DataSink { // Returns the runtime profile for the sink. virtual RuntimeProfile* profile() = 0; + virtual void set_query_statistic(boost::shared_ptr statistic) { + _query_statistic = statistic; + } protected: // Set to true after close() has been called. subclasses should check and set this in // close(). bool _closed; std::unique_ptr _expr_mem_tracker; + + boost::shared_ptr _query_statistic; }; } // namespace doris diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp index 4c7828c30966a8..9664b4de01ec1a 100644 --- a/be/src/exec/exchange_node.cpp +++ b/be/src/exec/exchange_node.cpp @@ -43,7 +43,8 @@ ExchangeNode::ExchangeNode( _next_row_idx(0), _is_merging(tnode.exchange_node.__isset.sort_info), _offset(tnode.exchange_node.__isset.offset ? tnode.exchange_node.offset : 0), - _num_rows_skipped(0) { + _num_rows_skipped(0), + _merge_rows_counter(nullptr) { DCHECK_GE(_offset, 0); DCHECK(_is_merging || (_offset == 0)); } @@ -63,16 +64,16 @@ Status ExchangeNode::init(const TPlanNode& tnode, RuntimeState* state) { Status ExchangeNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::prepare(state)); _convert_row_batch_timer = ADD_TIMER(runtime_profile(), "ConvertRowBatchTime"); - + _merge_rows_counter = ADD_COUNTER(runtime_profile(), "MergeRows", TUnit::UNIT); + _sub_plan_statistic.reset(new QueryStatistic()); // TODO: figure out appropriate buffer size DCHECK_GT(_num_senders, 0); _stream_recvr = state->exec_env()->stream_mgr()->create_recvr( state, _input_row_desc, state->fragment_instance_id(), _id, _num_senders, config::exchg_node_buffer_size_bytes, - state->runtime_profile(), _is_merging); + state->runtime_profile(), _is_merging, _sub_plan_statistic.get()); if (_is_merging) { - _merge_rows_counter = ADD_COUNTER(runtime_profile(), "MergeRows", TUnit::UNIT); RETURN_IF_ERROR(_sort_exec_exprs.prepare( state, _row_descriptor, _row_descriptor, expr_mem_tracker())); // AddExprCtxsToFree(_sort_exec_exprs); @@ -95,6 +96,14 @@ Status ExchangeNode::open(RuntimeState* state) { return Status::OK; } +Status ExchangeNode::collect_query_statistic(QueryStatistic* statistic) { + RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); + statistic->add_cpu_by_row(_merge_rows_counter->value()); + QueryStatistic* sub_plan_statistic = _sub_plan_statistic.get(); + statistic->add(*sub_plan_statistic); + return Status::OK; +} + Status ExchangeNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK; diff --git a/be/src/exec/exchange_node.h b/be/src/exec/exchange_node.h index 6450d39f913026..4fb8914a4d506e 100644 --- a/be/src/exec/exchange_node.h +++ b/be/src/exec/exchange_node.h @@ -21,11 +21,11 @@ #include #include "exec/exec_node.h" #include "exec/sort_exec_exprs.h" +#include "runtime/data_stream_recvr.h" namespace doris { class RowBatch; -class DataStreamRecvr; class RuntimeProfile; // Receiver node for data streams. The data stream receiver is created in Prepare() @@ -49,6 +49,7 @@ class ExchangeNode : public ExecNode { // Blocks until the first batch is available for consumption via GetNext(). virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); + virtual Status collect_query_statistic(QueryStatistic* statistic); virtual Status close(RuntimeState* state); // the number of senders needs to be set after the c'tor, because it's not @@ -61,6 +62,7 @@ class ExchangeNode : public ExecNode { virtual void debug_string(int indentation_level, std::stringstream* out) const; private: + // Implements GetNext() for the case where _is_merging is true. Delegates the GetNext() // call to the underlying DataStreamRecvr. Status get_next_merging(RuntimeState* state, RowBatch* output_batch, bool* eos); @@ -109,6 +111,9 @@ class ExchangeNode : public ExecNode { int64_t _num_rows_skipped; RuntimeProfile::Counter* _merge_rows_counter; + + // Query statistic from sub plan. + boost::scoped_ptr _sub_plan_statistic; }; }; diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 37b6b3a2dc7491..4aca701268fd1a 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -212,6 +212,14 @@ Status ExecNode::reset(RuntimeState* state) { return Status::OK; } +Status ExecNode::collect_query_statistic(QueryStatistic* statistic) { + DCHECK(statistic != nullptr); + for (auto child_node : _children) { + child_node->collect_query_statistic(statistic); + } + return Status::OK; +} + Status ExecNode::close(RuntimeState* state) { if (_is_closed) { return Status::OK; diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index bf2df357d66b29..fa8924e2b1bdc4 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -29,6 +29,7 @@ #include "util/runtime_profile.h" #include "util/blocking_queue.hpp" #include "runtime/bufferpool/buffer_pool.h" +#include "runtime/query_statistic.h" namespace llvm { class Function; @@ -114,6 +115,11 @@ class ExecNode { // so should be fast. virtual Status reset(RuntimeState* state); + // collect_query_statistic() should be called before close() and after get_next(), it + // responsible for collecting statistics returned with query result, it can't be called + // when prepare() return error. + virtual Status collect_query_statistic(QueryStatistic* statistic); + // close() will get called for every exec node, regardless of what else is called and // the status of these calls (i.e. prepare() may never have been called, or // prepare()/open()/get_next() returned with an error). diff --git a/be/src/exec/hash_join_node.cpp b/be/src/exec/hash_join_node.cpp index 378f8f5a0a727e..cedf1ed7c02172 100644 --- a/be/src/exec/hash_join_node.cpp +++ b/be/src/exec/hash_join_node.cpp @@ -92,11 +92,11 @@ Status HashJoinNode::prepare(RuntimeState* state) { ADD_TIMER(runtime_profile(), "PushDownComputeTime"); _probe_timer = ADD_TIMER(runtime_profile(), "ProbeTime"); - _build_row_counter = + _build_rows_counter = ADD_COUNTER(runtime_profile(), "BuildRows", TUnit::UNIT); _build_buckets_counter = ADD_COUNTER(runtime_profile(), "BuildBuckets", TUnit::UNIT); - _probe_row_counter = + _probe_rows_counter = ADD_COUNTER(runtime_profile(), "ProbeRows", TUnit::UNIT); _hash_tbl_load_factor_counter = ADD_COUNTER(runtime_profile(), "LoadFactor", TUnit::DOUBLE_VALUE); @@ -177,6 +177,13 @@ Status HashJoinNode::prepare(RuntimeState* state) { return Status::OK; } +Status HashJoinNode::collect_query_statistic(QueryStatistic* statistic) { + RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); + statistic->add_cpu_by_row(_probe_rows_counter->value()); + statistic->add_cpu_by_row(_build_rows_counter->value()); + return Status::OK; +} + Status HashJoinNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK; @@ -243,7 +250,7 @@ Status HashJoinNode::construct_hash_table(RuntimeState* state) { VLOG_ROW << _hash_tbl->debug_string(true, &child(1)->row_desc()); - COUNTER_SET(_build_row_counter, _hash_tbl->size()); + COUNTER_SET(_build_rows_counter, _hash_tbl->size()); COUNTER_SET(_build_buckets_counter, _hash_tbl->num_buckets()); COUNTER_SET(_hash_tbl_load_factor_counter, _hash_tbl->load_factor()); build_batch.reset(); @@ -380,7 +387,7 @@ Status HashJoinNode::open(RuntimeState* state) { // seed probe batch and _current_probe_row, etc. while (true) { RETURN_IF_ERROR(child(0)->get_next(state, _probe_batch.get(), &_probe_eos)); - COUNTER_UPDATE(_probe_row_counter, _probe_batch->num_rows()); + COUNTER_UPDATE(_probe_rows_counter, _probe_batch->num_rows()); _probe_batch_pos = 0; if (_probe_batch->num_rows() == 0) { @@ -571,7 +578,7 @@ Status HashJoinNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* eo continue; } else { - COUNTER_UPDATE(_probe_row_counter, _probe_batch->num_rows()); + COUNTER_UPDATE(_probe_rows_counter, _probe_batch->num_rows()); break; } } @@ -695,7 +702,7 @@ Status HashJoinNode::left_join_get_next(RuntimeState* state, probe_timer.stop(); RETURN_IF_ERROR(child(0)->get_next(state, _probe_batch.get(), &_probe_eos)); probe_timer.start(); - COUNTER_UPDATE(_probe_row_counter, _probe_batch->num_rows()); + COUNTER_UPDATE(_probe_rows_counter, _probe_batch->num_rows()); } } } diff --git a/be/src/exec/hash_join_node.h b/be/src/exec/hash_join_node.h index 8a64fb3ab142b8..607f6bdec4e986 100644 --- a/be/src/exec/hash_join_node.h +++ b/be/src/exec/hash_join_node.h @@ -56,6 +56,7 @@ class HashJoinNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); + virtual Status collect_query_statistic(QueryStatistic* statistic); virtual Status close(RuntimeState* state); static const char* _s_llvm_class_name; @@ -134,8 +135,8 @@ class HashJoinNode : public ExecNode { RuntimeProfile::Counter* _push_down_timer; // time to build hash table RuntimeProfile::Counter* _push_compute_timer; RuntimeProfile::Counter* _probe_timer; // time to probe - RuntimeProfile::Counter* _build_row_counter; // num build rows - RuntimeProfile::Counter* _probe_row_counter; // num probe rows + RuntimeProfile::Counter* _build_rows_counter; // num build rows + RuntimeProfile::Counter* _probe_rows_counter; // num probe rows RuntimeProfile::Counter* _build_buckets_counter; // num buckets in hash table RuntimeProfile::Counter* _hash_tbl_load_factor_counter; diff --git a/be/src/exec/new_partitioned_aggregation_node.cc b/be/src/exec/new_partitioned_aggregation_node.cc index 72ba8876e8b3ff..fab455f0b6f2d7 100644 --- a/be/src/exec/new_partitioned_aggregation_node.cc +++ b/be/src/exec/new_partitioned_aggregation_node.cc @@ -671,6 +671,12 @@ Status NewPartitionedAggregationNode::reset(RuntimeState* state) { return ExecNode::reset(state); } +Status NewPartitionedAggregationNode::collect_query_statistic(QueryStatistic* statistic) { + RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); + statistic->add_cpu_by_row(_build_rows_counter->value()); + return Status::OK; +} + Status NewPartitionedAggregationNode::close(RuntimeState* state) { if (is_closed()) return Status::OK; diff --git a/be/src/exec/new_partitioned_aggregation_node.h b/be/src/exec/new_partitioned_aggregation_node.h index 227cbed44baed8..1dbf8228b1b32c 100644 --- a/be/src/exec/new_partitioned_aggregation_node.h +++ b/be/src/exec/new_partitioned_aggregation_node.h @@ -137,6 +137,7 @@ class NewPartitionedAggregationNode : public ExecNode { virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); virtual Status reset(RuntimeState* state); + virtual Status collect_query_statistic(QueryStatistic* statistic); virtual Status close(RuntimeState* state); static const char* LLVM_CLASS_NAME; diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp index b0a456d99b934e..88b881b8055a10 100644 --- a/be/src/exec/olap_scan_node.cpp +++ b/be/src/exec/olap_scan_node.cpp @@ -306,6 +306,12 @@ Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo return _status; } +Status OlapScanNode::collect_query_statistic(QueryStatistic* statistic) { + RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); + statistic->add_io_by_byte(_read_compressed_counter->value()); + return Status::OK; +} + Status OlapScanNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK; diff --git a/be/src/exec/olap_scan_node.h b/be/src/exec/olap_scan_node.h index 0db834d0a013aa..045d3c496a37b1 100644 --- a/be/src/exec/olap_scan_node.h +++ b/be/src/exec/olap_scan_node.h @@ -56,6 +56,7 @@ class OlapScanNode : public ScanNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); + virtual Status collect_query_statistic(QueryStatistic* statistic); virtual Status close(RuntimeState* state); virtual Status set_scan_ranges(const std::vector& scan_ranges); diff --git a/be/src/exec/sort_node.cpp b/be/src/exec/sort_node.cpp index 3b816e9edcfb59..254da1f7577577 100644 --- a/be/src/exec/sort_node.cpp +++ b/be/src/exec/sort_node.cpp @@ -117,6 +117,13 @@ Status SortNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) { return Status::OK; } + +Status SortNode::collect_query_statistic(QueryStatistic* statistic) { + RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); + statistic->add_cpu_by_row(_sort_rows_counter->value()); + return Status::OK; +} + Status SortNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK; diff --git a/be/src/exec/sort_node.h b/be/src/exec/sort_node.h index 68dcfd8edbe430..2559bef8b58d47 100644 --- a/be/src/exec/sort_node.h +++ b/be/src/exec/sort_node.h @@ -42,6 +42,7 @@ class SortNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); + virtual Status collect_query_statistic(QueryStatistic* statistic); virtual Status close(RuntimeState* state); protected: diff --git a/be/src/exec/union_node.cpp b/be/src/exec/union_node.cpp index a1494cc1ef8017..2094acaf946fcd 100644 --- a/be/src/exec/union_node.cpp +++ b/be/src/exec/union_node.cpp @@ -337,6 +337,12 @@ Status UnionNode::reset(RuntimeState* state) { } #endif +Status UnionNode::collect_query_statistic(QueryStatistic* statistic) { + RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); + statistic->add_cpu_by_row(_materialize_rows_counter->value()); + return Status::OK; +} + Status UnionNode::close(RuntimeState* state) { if (is_closed()) return Status::OK; _child_batch.reset(); diff --git a/be/src/exec/union_node.h b/be/src/exec/union_node.h index 7daf2a623ae645..2ca14f90c7e1fc 100644 --- a/be/src/exec/union_node.h +++ b/be/src/exec/union_node.h @@ -48,6 +48,7 @@ class UnionNode : public ExecNode { virtual void codegen(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); + virtual Status collect_query_statistic(QueryStatistic* statistic); // virtual Status reset(RuntimeState* state); virtual Status close(RuntimeState* state); diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp index f4486dea08720a..1e2c271c776147 100644 --- a/be/src/runtime/buffer_control_block.cpp +++ b/be/src/runtime/buffer_control_block.cpp @@ -31,9 +31,13 @@ void GetResultBatchCtx::on_failure(const Status& status) { delete this; } -void GetResultBatchCtx::on_close(int64_t packet_seq) { +void GetResultBatchCtx::on_close(int64_t packet_seq, + QueryStatistic* statistic) { Status status; status.to_protobuf(result->mutable_status()); + if (statistic != nullptr) { + statistic->serialize(result->mutable_query_statistic()); + } result->set_packet_seq(packet_seq); result->set_eos(true); done->Run(); @@ -183,7 +187,7 @@ void BufferControlBlock::get_batch(GetResultBatchCtx* ctx) { return; } if (_is_close) { - ctx->on_close(_packet_num); + ctx->on_close(_packet_num, _query_statistic.get()); return; } // no ready data, push ctx to waiting list @@ -200,7 +204,7 @@ Status BufferControlBlock::close(Status exec_status) { if (!_waiting_rpc.empty()) { if (_status.ok()) { for (auto& ctx : _waiting_rpc) { - ctx->on_close(_packet_num); + ctx->on_close(_packet_num, _query_statistic.get()); } } else { for (auto& ctx : _waiting_rpc) { diff --git a/be/src/runtime/buffer_control_block.h b/be/src/runtime/buffer_control_block.h index 701ef5f70a1e9e..8a053494556e8a 100644 --- a/be/src/runtime/buffer_control_block.h +++ b/be/src/runtime/buffer_control_block.h @@ -24,6 +24,7 @@ #include #include "common/status.h" #include "gen_cpp/Types_types.h" +#include "runtime/query_statistic.h" namespace google { namespace protobuf { @@ -52,7 +53,7 @@ struct GetResultBatchCtx { } void on_failure(const Status& status); - void on_close(int64_t packet_seq); + void on_close(int64_t packet_seq, QueryStatistic* statistic = nullptr); void on_data(TFetchDataResult* t_result, int64_t packet_seq, bool eos = false); }; @@ -80,6 +81,9 @@ class BufferControlBlock { return _fragment_id; } + void set_query_statistic(boost::shared_ptr statistic) { + _query_statistic = statistic; + } private: typedef std::list ResultQueue; @@ -100,8 +104,10 @@ class BufferControlBlock { boost::condition_variable _data_arriaval; // signal removal of data by stream consumer boost::condition_variable _data_removal; - + std::deque _waiting_rpc; + + boost::shared_ptr _query_statistic; }; } diff --git a/be/src/runtime/data_stream_mgr.cpp b/be/src/runtime/data_stream_mgr.cpp index 0c75fbb04a64af..b983ee5f8b240d 100644 --- a/be/src/runtime/data_stream_mgr.cpp +++ b/be/src/runtime/data_stream_mgr.cpp @@ -51,14 +51,14 @@ inline uint32_t DataStreamMgr::get_hash_value( shared_ptr DataStreamMgr::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) { + bool is_merging, QueryStatistic* query_statistic) { DCHECK(profile != NULL); VLOG_FILE << "creating receiver for fragment=" << fragment_instance_id << ", node=" << dest_node_id; shared_ptr recvr( new DataStreamRecvr(this, state->instance_mem_tracker(), row_desc, fragment_instance_id, dest_node_id, num_senders, is_merging, buffer_size, - profile)); + profile, query_statistic)); uint32_t hash_value = get_hash_value(fragment_instance_id, dest_node_id); lock_guard l(_lock); _fragment_stream_set.insert(std::make_pair(fragment_instance_id, dest_node_id)); @@ -118,6 +118,15 @@ Status DataStreamMgr::add_data( return Status::OK; } +Status DataStreamMgr::update_query_statistic(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, + int sender_id, const PQueryStatistic& query_statistic) { + shared_ptr recvr = find_recvr(fragment_instance_id, dest_node_id); + if (recvr != NULL) { + recvr->update_sub_plan_statistic(query_statistic, sender_id); + } + return Status::OK; +} + Status DataStreamMgr::close_sender(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int sender_id, diff --git a/be/src/runtime/data_stream_mgr.h b/be/src/runtime/data_stream_mgr.h index a4880b65e14458..6f63e9ede3c984 100644 --- a/be/src/runtime/data_stream_mgr.h +++ b/be/src/runtime/data_stream_mgr.h @@ -30,6 +30,7 @@ #include "common/object_pool.h" #include "runtime/descriptors.h" // for PlanNodeId #include "runtime/mem_tracker.h" +#include "runtime/query_statistic.h" #include "util/runtime_profile.h" #include "gen_cpp/Types_types.h" // for TUniqueId @@ -76,7 +77,7 @@ class DataStreamMgr { 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); + bool is_merging, QueryStatistic* query_statistic); // Adds a row batch to the recvr identified by fragment_instance_id/dest_node_id // if the recvr has not been cancelled. sender_id identifies the sender instance @@ -98,6 +99,9 @@ class DataStreamMgr { Status close_sender(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int sender_id, int be_number); + Status update_query_statistic(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, + int sender_id, const PQueryStatistic& query_statistic); + // Closes all receivers registered for fragment_instance_id immediately. void cancel(const TUniqueId& fragment_instance_id); diff --git a/be/src/runtime/data_stream_recvr.cc b/be/src/runtime/data_stream_recvr.cc index f007e6f67b361f..2b599707b466d4 100644 --- a/be/src/runtime/data_stream_recvr.cc +++ b/be/src/runtime/data_stream_recvr.cc @@ -242,6 +242,7 @@ void DataStreamRecvr::SenderQueue::add_batch( // it in this thread. batch = new RowBatch(_recvr->row_desc(), pb_batch, _recvr->mem_tracker()); } + VLOG_ROW << "added #rows=" << batch->num_rows() << " batch_size=" << batch_size << "\n"; _batch_queue.emplace_back(batch_size, batch); @@ -350,7 +351,7 @@ DataStreamRecvr::DataStreamRecvr( DataStreamMgr* stream_mgr, MemTracker* parent_tracker, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, bool is_merging, int total_buffer_limit, - RuntimeProfile* profile) : + RuntimeProfile* profile, QueryStatistic* query_statistic) : _mgr(stream_mgr), _fragment_instance_id(fragment_instance_id), _dest_node_id(dest_node_id), @@ -358,7 +359,8 @@ DataStreamRecvr::DataStreamRecvr( _row_desc(row_desc), _is_merging(is_merging), _num_buffered_bytes(0), - _profile(profile) { + _profile(profile), + _sub_plan_query_statistic(query_statistic) { _mem_tracker.reset(new MemTracker(-1, "DataStreamRecvr", parent_tracker)); // Create one queue per sender if is_merging is true. diff --git a/be/src/runtime/data_stream_recvr.h b/be/src/runtime/data_stream_recvr.h index 22bca1cecddd19..140e3071f41274 100644 --- a/be/src/runtime/data_stream_recvr.h +++ b/be/src/runtime/data_stream_recvr.h @@ -25,6 +25,7 @@ #include "common/status.h" #include "gen_cpp/Types_types.h" // for TUniqueId #include "runtime/descriptors.h" +#include "runtime/query_statistic.h" #include "util/tuple_row_compare.h" namespace google { @@ -99,6 +100,10 @@ class DataStreamRecvr { const RowDescriptor& row_desc() const { return _row_desc; } MemTracker* mem_tracker() const { return _mem_tracker.get(); } + void update_sub_plan_statistic(const PQueryStatistic& statistic, int sender_id) { + _sub_plan_query_statistic->deserialize(statistic, sender_id); + } + private: friend class DataStreamMgr; class SenderQueue; @@ -106,7 +111,7 @@ class DataStreamRecvr { DataStreamRecvr(DataStreamMgr* stream_mgr, MemTracker* parent_tracker, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, bool is_merging, int total_buffer_limit, - RuntimeProfile* profile); + RuntimeProfile* profile, QueryStatistic* query_statistic); // If receive queue is full, done is enqueue pending, and return with *done is nullptr void add_batch(const PRowBatch& batch, int sender_id, @@ -194,6 +199,9 @@ class DataStreamRecvr { // Wall time senders spend waiting for the recv buffer to have capacity. RuntimeProfile::Counter* _buffer_full_wall_timer; + // Query statistic returned with batch + QueryStatistic* _sub_plan_query_statistic; + // Total time spent waiting for data to arrive in the recv buffer // RuntimeProfile::Counter* _data_arrival_timer; }; diff --git a/be/src/runtime/data_stream_sender.cpp b/be/src/runtime/data_stream_sender.cpp index d0c60960fc166d..557beb5faadfac 100644 --- a/be/src/runtime/data_stream_sender.cpp +++ b/be/src/runtime/data_stream_sender.cpp @@ -71,7 +71,10 @@ class DataStreamSender::Channel { Channel(DataStreamSender* parent, const RowDescriptor& row_desc, const TNetworkAddress& brpc_dest, const TUniqueId& fragment_instance_id, - PlanNodeId dest_node_id, int buffer_size) : + PlanNodeId dest_node_id, + int buffer_size, + bool is_transfer_chain, + bool send_query_statistic_with_every_batch) : _parent(parent), _buffer_size(buffer_size), _row_desc(row_desc), @@ -80,7 +83,9 @@ class DataStreamSender::Channel { _num_data_bytes_sent(0), _packet_seq(0), _need_close(false), - _brpc_dest_addr(brpc_dest) { + _brpc_dest_addr(brpc_dest), + _is_transfer_chain(is_transfer_chain), + _send_query_statistic_with_every_batch(send_query_statistic_with_every_batch) { } virtual ~Channel() { @@ -163,6 +168,9 @@ class DataStreamSender::Channel { palo::PInternalService_Stub* _brpc_stub = nullptr; RefCountClosure* _closure = nullptr; int32_t _brpc_timeout_ms = 500; + // whether the dest can be treated as consumption transfer chain. + bool _is_transfer_chain; + bool _send_query_statistic_with_every_batch; }; Status DataStreamSender::Channel::init(RuntimeState* state) { @@ -203,6 +211,10 @@ Status DataStreamSender::Channel::send_batch(PRowBatch* batch, bool eos) { } VLOG_ROW << "Channel::send_batch() instance_id=" << _fragment_instance_id << " dest_node=" << _dest_node_id; + if (_is_transfer_chain && (_send_query_statistic_with_every_batch || eos)) { + auto statistic = _brpc_request.mutable_query_statistic(); + _parent->_query_statistic->serialize(statistic); + } _brpc_request.set_eos(eos); if (batch != nullptr) { @@ -285,7 +297,8 @@ DataStreamSender::DataStreamSender( ObjectPool* pool, int sender_id, const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, - int per_channel_buffer_size) : + int per_channel_buffer_size, + bool send_query_statistic_with_every_batch) : _sender_id(sender_id), _pool(pool), _row_desc(row_desc), @@ -305,11 +318,17 @@ DataStreamSender::DataStreamSender( || sink.output_partition.type == TPartitionType::RANGE_PARTITIONED); // TODO: use something like google3's linked_ptr here (scoped_ptr isn't copyable) for (int i = 0; i < destinations.size(); ++i) { + bool is_transfer_chain = false; + // Select first dest as transfer chain. + if (i == 0) { + is_transfer_chain = true; + } _channel_shared_ptrs.emplace_back( new Channel(this, row_desc, destinations[i].brpc_server, destinations[i].fragment_instance_id, - sink.dest_node_id, per_channel_buffer_size)); + sink.dest_node_id, per_channel_buffer_size, + is_transfer_chain, send_query_statistic_with_every_batch)); _channels.push_back(_channel_shared_ptrs[i].get()); } } diff --git a/be/src/runtime/data_stream_sender.h b/be/src/runtime/data_stream_sender.h index 30f26fbe889927..22174541b7f6b2 100644 --- a/be/src/runtime/data_stream_sender.h +++ b/be/src/runtime/data_stream_sender.h @@ -59,7 +59,7 @@ class DataStreamSender : public DataSink { DataStreamSender(ObjectPool* pool, int sender_id, const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, - int per_channel_buffer_size); + int per_channel_buffer_size, bool send_query_statistic_with_every_batch); virtual ~DataStreamSender(); virtual Status init(const TDataSink& thrift_sink); diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index 1e769ee6dfb0a8..0019c165be1960 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -226,6 +226,9 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) { // _row_batch->tuple_data_pool()->set_limits(*_runtime_state->mem_trackers()); VLOG(3) << "plan_root=\n" << _plan->debug_string(); _prepared = true; + + _query_statistic.reset(new QueryStatistic()); + _sink->set_query_statistic(_query_statistic); return Status::OK; } @@ -317,7 +320,10 @@ Status PlanFragmentExecutor::open_internal() { } SCOPED_TIMER(profile()->total_time_counter()); + // Collect this plan and sub plan statistics, and send to parent plan. + _plan->collect_query_statistic(_query_statistic.get()); RETURN_IF_ERROR(_sink->send(runtime_state(), batch)); + _query_statistic->clear(); } // Close the sink *before* stopping the report thread. Close may @@ -333,6 +339,7 @@ Status PlanFragmentExecutor::open_internal() { // audit the sinks to check that this is ok, or change that behaviour. { SCOPED_TIMER(profile()->total_time_counter()); + _plan->collect_query_statistic(_query_statistic.get()); Status status = _sink->close(runtime_state(), _status); RETURN_IF_ERROR(status); } diff --git a/be/src/runtime/plan_fragment_executor.h b/be/src/runtime/plan_fragment_executor.h index 299191b598136b..b3e06c4dbf42a9 100644 --- a/be/src/runtime/plan_fragment_executor.h +++ b/be/src/runtime/plan_fragment_executor.h @@ -204,6 +204,10 @@ class PlanFragmentExecutor { // of the execution. RuntimeProfile::Counter* _average_thread_tokens; + // This plan and it's sub plan query statisic. Because plan may have been finished + // when fe fetch data and query statistic, this will be shared with BufferControlBlock. + boost::shared_ptr _query_statistic; + ObjectPool* obj_pool() { return _runtime_state->obj_pool(); } diff --git a/be/src/runtime/query_statistic.h b/be/src/runtime/query_statistic.h new file mode 100644 index 00000000000000..52c5fb1054e257 --- /dev/null +++ b/be/src/runtime/query_statistic.h @@ -0,0 +1,217 @@ +// 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. + +#ifndef DORIS_BE_EXEC_QUERY_STATISTIC_H +#define DORIS_BE_EXEC_QUERY_STATISTIC_H + +#include "common/atomic.h" +#include "gen_cpp/data.pb.h" +#include "util/spinlock.h" + +namespace doris { + +// This is responsible for collecting query statistic, usually it consist of +// two parts, one is current fragment or plan's statistic, the other is sub fragment +// or plan's statistic.Now cpu is measured by number of rows, io is measured by byte. +class QueryStatistic { +public: + + class Statistic { + public: + + Statistic() : cpu_by_row(0), io_by_byte(0) { + } + + void add(const Statistic& other) { + cpu_by_row += other.cpu_by_row; + io_by_byte += other.io_by_byte; + } + + void add(const PQueryStatistic& other) { + cpu_by_row += other.cpu_by_row(); + io_by_byte += other.io_by_byte(); + } + + void reset() { + cpu_by_row = 0; + io_by_byte = 0; + } + + void set_io_by_byte(int64_t io_by_byte) { + this->io_by_byte = io_by_byte; + } + + void add_io_by_byte(int64_t io_by_byte) { + this->io_by_byte += io_by_byte; + } + + long get_io_by_byte() { + return io_by_byte; + } + + void set_cpu_by_row(int64_t cpu_by_row) { + this->cpu_by_row = cpu_by_row; + } + + void add_cpu_by_row(int64_t cpu_by_row) { + this->cpu_by_row += cpu_by_row; + } + + long get_cpu_by_row() { + return cpu_by_row; + } + + void serialize(PQueryStatistic* statistic) { + DCHECK(statistic != nullptr); + statistic->set_cpu_by_row(cpu_by_row); + statistic->set_io_by_byte(io_by_byte); + } + + void deserialize(const PQueryStatistic& statistic) { + cpu_by_row = statistic.cpu_by_row(); + io_by_byte = statistic.io_by_byte(); + } + + private: + + long cpu_by_row; + long io_by_byte; + }; + + QueryStatistic() { + } + + void add(const QueryStatistic& other) { + boost::lock_guard l(_lock); + auto other_iter = other._statistics.begin(); + while (other_iter != other._statistics.end()) { + auto iter = _statistics.find(other_iter->first); + Statistic* statistic = nullptr; + if (iter == _statistics.end()) { + statistic = new Statistic(); + _statistics[other_iter->first] = statistic; + } else { + statistic = iter->second; + } + Statistic* other_statistic = other_iter->second; + statistic->add(*other_statistic); + other_iter++; + } + } + + void add_cpu_by_row(long cpu_by_row) { + boost::lock_guard l(_lock); + auto statistic = find(DEFAULT_SENDER_ID); + statistic->add_cpu_by_row(cpu_by_row); + } + + void add_io_by_byte(long io_by_byte) { + boost::lock_guard l(_lock); + auto statistic = find(DEFAULT_SENDER_ID); + statistic->add_io_by_byte(io_by_byte); + } + + void deserialize(const PQueryStatistic& other, int sender_id) { + boost::lock_guard l(_lock); + auto statistic = find(sender_id); + statistic->deserialize(other); + } + + void serialize(PQueryStatistic* statistic) { + boost::lock_guard l(_lock); + DCHECK(statistic != nullptr); + Statistic total_statistic = get_total_statistic(); + total_statistic.serialize(statistic); + } + + long get_cpu_by_row() { + boost::lock_guard l(_lock); + Statistic statistic = get_total_statistic(); + return statistic.get_cpu_by_row(); + } + + long get_io_by_byte() { + boost::lock_guard l(_lock); + Statistic statistic = get_total_statistic(); + return statistic.get_io_by_byte(); + } + + long get_cpu_by_row(int sender_id) { + boost::lock_guard l(_lock); + auto statistic = find(sender_id); + return statistic->get_cpu_by_row(); + } + + long get_io_by_byte(int sender_id) { + boost::lock_guard l(_lock); + auto statistic = find(sender_id); + return statistic->get_io_by_byte(); + } + + void clear() { + boost::lock_guard l(_lock); + auto iter = _statistics.begin(); + while (iter != _statistics.end()) { + iter->second->reset(); + iter++; + } + } + + ~QueryStatistic() { + boost::lock_guard l(_lock); + auto iter = _statistics.begin(); + while (iter != _statistics.end()) { + delete iter->second; + iter++; + } + } + +private: + + Statistic* find(int sender_id) { + auto iter = _statistics.find(sender_id); + Statistic* statistic = nullptr; + if (iter == _statistics.end()) { + statistic = new Statistic(); + _statistics[sender_id] = statistic; + } else { + statistic = iter->second; + } + return statistic; + } + + Statistic get_total_statistic() { + Statistic total_statistic; + auto iter = _statistics.begin(); + while (iter != _statistics.end()) { + total_statistic.add(*(iter->second)); + iter++; + } + return total_statistic; + } + + // Map lock. + SpinLock _lock; + // Sender id to statistic. + std::map _statistics; + // To index current plan. + static const int DEFAULT_SENDER_ID = -1; +}; + +} + +#endif diff --git a/be/src/runtime/result_sink.cpp b/be/src/runtime/result_sink.cpp index b5eb2c978bf6fd..9f05f50200fb89 100644 --- a/be/src/runtime/result_sink.cpp +++ b/be/src/runtime/result_sink.cpp @@ -92,5 +92,9 @@ Status ResultSink::close(RuntimeState* state, Status exec_status) { return Status::OK; } +void ResultSink::set_query_statistic(boost::shared_ptr statistic) { + _sender->set_query_statistic(statistic); +} + } /* vim: set ts=4 sw=4 sts=4 tw=100 : */ diff --git a/be/src/runtime/result_sink.h b/be/src/runtime/result_sink.h index 5b831fcee1c1da..f5c1d7cc83d200 100644 --- a/be/src/runtime/result_sink.h +++ b/be/src/runtime/result_sink.h @@ -56,6 +56,8 @@ class ResultSink : public DataSink { return _profile; } + virtual void set_query_statistic(boost::shared_ptr statistic); + private: Status prepare_exprs(RuntimeState* state); diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 1f6fde88eb6a93..97981cc6196147 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -32,6 +32,7 @@ #include "runtime/descriptors.h" #include "runtime/exec_env.h" #include "runtime/initial_reservations.h" +#include "runtime/query_statistic.h" #include "runtime/runtime_state.h" #include "runtime/load_path_mgr.h" #include "util/cpu_info.h" @@ -188,7 +189,6 @@ Status RuntimeState::init( } _db_name = "insert_stmt"; _import_label = print_id(fragment_instance_id); - return Status::OK; } diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index b58ef954932035..aa4026fa47a976 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -60,6 +60,7 @@ class LoadErrorHub; class ReservationTracker; class InitialReservations; class RowDescriptor; +class QueryStatistic; // A collection of items that are part of the global state of a // query and shared across all execution nodes of that query. diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index 056495a2816e6d..496fa89131806c 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -130,14 +130,14 @@ void BackendService::transmit_data(TTransmitDataResult& return_val, } if (params.eos) { - Status status = _exec_env->stream_mgr()->close_sender( - params.dest_fragment_instance_id, - params.dest_node_id, - params.sender_id, - params.be_number); - VLOG_ROW << "params.eos: " << (params.eos ? "true" : "false") - << " close_sender status: " << status.get_error_msg(); - status.set_t_status(&return_val); + // Status status = _exec_env->stream_mgr()->close_sender( + // params.dest_fragment_instance_id, + // params.dest_node_id, + // params.sender_id, + // params.be_number); + //VLOG_ROW << "params.eos: " << (params.eos ? "true" : "false") + // << " close_sender status: " << status.get_error_msg(); + //status.set_t_status(&return_val); } } diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 2f4b3f347a09d6..1bf219f6b27bcb 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -54,6 +54,18 @@ void PInternalServiceImpl::transmit_data(google::protobuf::RpcController* cnt request->be_number(), request->packet_seq(), eos ? nullptr : &done); } + + if (request->has_query_statistic()) { + TUniqueId finst_id; + finst_id.__set_hi(request->finst_id().hi()); + finst_id.__set_lo(request->finst_id().lo()); + _exec_env->stream_mgr()->update_query_statistic( + finst_id, + request->node_id(), + request->sender_id(), + request->query_statistic()); + } + if (eos) { TUniqueId finst_id; finst_id.__set_hi(request->finst_id().hi()); diff --git a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryFragmentProcNode.java b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryFragmentProcNode.java index 21ffef73fa9b63..4d995696fa50da 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryFragmentProcNode.java +++ b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryFragmentProcNode.java @@ -79,8 +79,8 @@ private ProcResult requestFragmentExecInfos() throws AnalysisException { rowData.add(instanceConsumption.getFragmentId()); rowData.add(instanceConsumption.getInstanceId().toString()); rowData.add(instanceConsumption.getAddress().toString()); - rowData.add(String.valueOf(instanceConsumption.getTotalIoConsumption())); - rowData.add(String.valueOf(instanceConsumption.getTotalCpuConsumption())); + rowData.add(instanceConsumption.getFormattingIo()); + rowData.add(instanceConsumption.getFormattingCpu()); sortedRowDatas.add(rowData); } diff --git a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java index f18d9d1399c0f7..9827e5ee3ef779 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java +++ b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import java.util.Collection; +import java.util.Formatter; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -334,7 +335,7 @@ private String parsePossibleExecNodeName(String str) { } } - public long getTotalCpuConsumption() { + private long getTotalCpuConsumption() { long cpu = 0; for (ConsumptionCalculator consumption : calculators) { cpu += consumption.getCpu(); @@ -342,13 +343,27 @@ public long getTotalCpuConsumption() { return cpu; } - public long getTotalIoConsumption() { + private long getTotalIoConsumption() { long io = 0; for (ConsumptionCalculator consumption : calculators) { io += consumption.getIo(); } return io; } + + public String getFormattingCpu() { + final StringBuilder builder = new StringBuilder(); + builder.append(getTotalCpuConsumption()).append(" Rows"); + return builder.toString(); + } + + public String getFormattingIo() { + final Pair pair = DebugUtil.getByteUint(getTotalIoConsumption()); + final Formatter fmt = new Formatter(); + final StringBuilder builder = new StringBuilder(); + builder.append(fmt.format("%.2f", pair.first)).append(" ").append(pair.second); + return builder.toString(); + } } public static class InstanceConsumption extends Consumption { diff --git a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryStatisticsProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryStatisticsProcDir.java index a59c3dbe9e07bf..79808423e30417 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryStatisticsProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryStatisticsProcDir.java @@ -76,8 +76,8 @@ public ProcResult fetchResult() throws AnalysisException { values.add(item.getDb()); values.add(item.getUser()); final CurrentQueryInfoProvider.Consumption consumption = consumptions.get(item.getQueryId()); - values.add(String.valueOf(consumption.getTotalIoConsumption())); - values.add(String.valueOf(consumption.getTotalCpuConsumption())); + values.add(consumption.getFormattingIo()); + values.add(consumption.getFormattingCpu()); values.add(item.getQueryExecTime()); sortedRowData.add(values); } diff --git a/fe/src/main/java/org/apache/doris/planner/PlanFragment.java b/fe/src/main/java/org/apache/doris/planner/PlanFragment.java index 5c9220332867b2..93d487b395fbed 100644 --- a/fe/src/main/java/org/apache/doris/planner/PlanFragment.java +++ b/fe/src/main/java/org/apache/doris/planner/PlanFragment.java @@ -96,6 +96,11 @@ public class PlanFragment extends TreeNode { // if the output is UNPARTITIONED, it is being broadcast private DataPartition outputPartition; + // Whether query statistic is sent with every batch. In order to get the query + // statistic correctly when query contains limit, it is necessary to send query + // statistic in every batch, or only in close. + private boolean transferQueryStatisticWithEveryBatch; + // TODO: SubstitutionMap outputSmap; // substitution map to remap exprs onto the output of this fragment, to be applied // at destination fragment @@ -108,6 +113,7 @@ public PlanFragment(PlanFragmentId id, PlanNode root, DataPartition partition) { this.planRoot = root; this.dataPartition = partition; this.outputPartition = DataPartition.UNPARTITIONED; + this.transferQueryStatisticWithEveryBatch = false; setFragmentInPlanTree(planRoot); } @@ -180,7 +186,6 @@ public TPlanFragment toThrift() { // TODO chenhao , calculated by cost result.setMin_reservation_bytes(0); result.setInitial_reservation_total_claims(0); - return result; } @@ -270,4 +275,11 @@ public PlanFragmentId getFragmentId() { return fragmentId; } + public void setTransferQueryStatisticWithEveryBatch(boolean value) { + transferQueryStatisticWithEveryBatch = value; + } + + public boolean isTransferQueryStatisticWithEveryBatch() { + return transferQueryStatisticWithEveryBatch; + } } diff --git a/fe/src/main/java/org/apache/doris/planner/Planner.java b/fe/src/main/java/org/apache/doris/planner/Planner.java index 96d3c5dedaa0d8..e2a501f758b70f 100644 --- a/fe/src/main/java/org/apache/doris/planner/Planner.java +++ b/fe/src/main/java/org/apache/doris/planner/Planner.java @@ -176,7 +176,11 @@ public void createPlanFragments(StatementBase statment, Analyzer analyzer, TQuer fragments = distributedPlanner.createPlanFragments(singleNodePlan); } + // Optimize the transfer of query statistic when query does't contain limit. PlanFragment rootFragment = fragments.get(fragments.size() - 1); + QueryStatisticTransferOptimizer queryStatisticTransferOptimizer = new QueryStatisticTransferOptimizer(rootFragment); + queryStatisticTransferOptimizer.optimizeTransferQueryStatistic(); + if (statment instanceof InsertStmt) { InsertStmt insertStmt = (InsertStmt) statment; rootFragment = distributedPlanner.createInsertFragment(rootFragment, insertStmt, fragments); @@ -230,4 +234,38 @@ private PlanNode addUnassignedConjuncts(Analyzer analyzer, PlanNode root) Preconditions.checkState(selectNode.hasValidStats()); return selectNode; } + + private static class QueryStatisticTransferOptimizer { + private final PlanFragment root; + + public QueryStatisticTransferOptimizer(PlanFragment root) { + Preconditions.checkNotNull(root); + this.root = root; + } + + public void optimizeTransferQueryStatistic() { + optimizeTransferQueryStatistic(root, null); + } + + private void optimizeTransferQueryStatistic(PlanFragment fragment, PlanFragment parent) { + if (parent != null && hasLimit(parent.getPlanRoot())) { + fragment.setTransferQueryStatisticWithEveryBatch(true); + } + for (PlanFragment child : fragment.getChildren()) { + optimizeTransferQueryStatistic(child, fragment); + } + } + + private boolean hasLimit(PlanNode planNode) { + if (planNode.hasLimit()) { + return true; + } + for (PlanNode child : planNode.getChildren()) { + if (hasLimit(child)) { + return true; + }; + } + return false; + } + } } diff --git a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 560d538bb2589e..0f82109c50d2d2 100644 --- a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -37,6 +37,7 @@ import org.apache.doris.thrift.TMasterOpRequest; import org.apache.doris.thrift.TMasterOpResult; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import org.apache.logging.log4j.LogManager; @@ -92,12 +93,16 @@ private void handlePing() { ctx.getState().setOk(); } - private void auditAfterExec(String origStmt, StatementBase parsedStmt) { + private void auditAfterExec(String origStmt, StatementBase parsedStmt, + StmtExecutor.QueryStatistic queryStatistic) { // slow query long elapseMs = System.currentTimeMillis() - ctx.getStartTime(); // query state log ctx.getAuditBuilder().put("state", ctx.getState()); ctx.getAuditBuilder().put("time", elapseMs); + Preconditions.checkNotNull(queryStatistic); + ctx.getAuditBuilder().put("cpu", queryStatistic.getFormattingCpu()); + ctx.getAuditBuilder().put("io", queryStatistic.getFormattingIo()); ctx.getAuditBuilder().put("returnRows", ctx.getReturnRows()); ctx.getAuditBuilder().put("stmt_id", ctx.getStmtId()); @@ -177,7 +182,8 @@ private void handleQuery() { // audit after exec // replace '\n' to '\\n' to make string in one line - auditAfterExec(stmt.replace("\n", " \\n"), executor.getParsedStmt()); + auditAfterExec(stmt.replace("\n", " \\n"), executor.getParsedStmt(), + executor.getQueryStatisticForAuditLog()); } // Get the column definitions of a table diff --git a/fe/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/src/main/java/org/apache/doris/qe/Coordinator.java index 252a9984e6e1cf..11317ac63b07cd 100644 --- a/fe/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/src/main/java/org/apache/doris/qe/Coordinator.java @@ -574,12 +574,12 @@ void updateStatus(Status status) { } } - TResultBatch getNext() throws Exception { + public RowBatch getNext() throws Exception { if (receiver == null) { throw new UserException("There is no receiver."); } - TResultBatch resultBatch; + RowBatch resultBatch; Status status = new Status(); resultBatch = receiver.getNext(status); @@ -611,7 +611,7 @@ TResultBatch getNext() throws Exception { } } - if (resultBatch == null) { + if (resultBatch.isEos()) { this.returnedAllResults = true; // if this query is a block query do not cancel. @@ -622,7 +622,7 @@ TResultBatch getNext() throws Exception { cancelInternal(); } } else { - numReceivedRows += resultBatch.getRowsSize(); + numReceivedRows += resultBatch.getBatch().getRowsSize(); } return resultBatch; @@ -1038,7 +1038,7 @@ private Set getScanHosts(PlanNodeId id, FragmentExecParams frag return result; } - public void createScanInstance(PlanNodeId leftMostScanId, FragmentExecParams fragmentExecParams) + private void createScanInstance(PlanNodeId leftMostScanId, FragmentExecParams fragmentExecParams) throws UserException { int maxNumInstance = queryOptions.mt_dop; if (maxNumInstance == 0) { @@ -1150,7 +1150,7 @@ private void validate() { } // create collocated instance according to inputFragments - public void createCollocatedInstance(FragmentExecParams fragmentExecParams) { + private void createCollocatedInstance(FragmentExecParams fragmentExecParams) { Preconditions.checkState(fragmentExecParams.inputFragments.size() >= 1); final FragmentExecParams inputFragmentParams = fragmentExecParamsMap.get(fragmentExecParams. inputFragments.get(0)); @@ -1169,7 +1169,7 @@ private TUniqueId getNextInstanceId() { } - public void createUnionInstance(FragmentExecParams fragmentExecParams) { + private void createUnionInstance(FragmentExecParams fragmentExecParams) { final PlanFragment fragment = fragmentExecParams.fragment; // Add hosts of scan nodes List scanNodeIds = findScanNodes(fragment.getPlanRoot()); @@ -1563,7 +1563,6 @@ List toThrift(int backendNum) { params.setResource_info(tResourceInfo); params.params.setQuery_id(queryId); params.params.setFragment_instance_id(instanceExecParam.instanceId); - Map> scanRanges = instanceExecParam.perNodeScanRanges; if (scanRanges == null) { scanRanges = Maps.newHashMap(); @@ -1579,7 +1578,8 @@ List toThrift(int backendNum) { params.setBackend_num(backendNum++); params.setQuery_globals(queryGlobals); params.setQuery_options(queryOptions); - + params.params.setSend_query_statistic_with_every_batch( + fragment.isTransferQueryStatisticWithEveryBatch()); if (queryOptions.getQuery_type() == TQueryType.LOAD) { LoadErrorHub.Param param = Catalog.getCurrentCatalog().getLoadInstance().getLoadErrorHubInfo(); if (param != null) { diff --git a/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java b/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java index acb87f135d8fa6..993f47171e3729 100644 --- a/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java +++ b/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java @@ -58,11 +58,11 @@ public ResultReceiver(TUniqueId tid, Long backendId, TNetworkAddress address, in this.timeoutTs = System.currentTimeMillis() + timeoutMs; } - public TResultBatch getNext(Status status) throws TException { + public RowBatch getNext(Status status) throws TException { if (isDone) { return null; } - + final RowBatch rowBatch = new RowBatch(); try { while (!isDone && !isCancel) { PFetchDataRequest request = new PFetchDataRequest(finstId); @@ -90,6 +90,10 @@ public TResultBatch getNext(Status status) throws TException { if (code != TStatusCode.OK) { status.setPstatus(pResult.status); return null; + } + + if (pResult.queryStatistic != null) { + rowBatch.setQueryStatistic(pResult.queryStatistic); } if (packetIdx != pResult.packetSeq) { @@ -106,7 +110,9 @@ public TResultBatch getNext(Status status) throws TException { TResultBatch resultBatch = new TResultBatch(); TDeserializer deserializer = new TDeserializer(); deserializer.deserialize(resultBatch, serialResult); - return resultBatch; + rowBatch.setBatch(resultBatch); + rowBatch.setEos(pResult.eos); + return rowBatch; } } } catch (RpcException e) { @@ -134,7 +140,7 @@ public TResultBatch getNext(Status status) throws TException { if (isCancel) { status.setStatus(Status.CANCELLED); } - return null; + return rowBatch; } public void cancel() { diff --git a/fe/src/main/java/org/apache/doris/qe/RowBatch.java b/fe/src/main/java/org/apache/doris/qe/RowBatch.java new file mode 100644 index 00000000000000..deeba8bef4920e --- /dev/null +++ b/fe/src/main/java/org/apache/doris/qe/RowBatch.java @@ -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. + +package org.apache.doris.qe; + +import org.apache.doris.rpc.PQueryStatistic; +import org.apache.doris.thrift.TResultBatch; + +public final class RowBatch { + private TResultBatch batch; + private PQueryStatistic queryStatistic; + private boolean eos; + + public RowBatch() { + eos = true; + } + + public TResultBatch getBatch() { + return batch; + } + + public void setBatch(TResultBatch batch) { + this.batch = batch; + } + + public PQueryStatistic getQueryStatistic() { + return queryStatistic; + } + + public void setQueryStatistic(PQueryStatistic queryStatistic) { + this.queryStatistic = queryStatistic; + } + + public boolean isEos() { + return eos; + } + + public void setEos(boolean eos) { + this.eos = eos; + } +} diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 53b6c023091503..5884a0e7109b41 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -17,6 +17,7 @@ package org.apache.doris.qe; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -52,6 +53,7 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.NotImplementedException; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.ProfileManager; @@ -63,6 +65,7 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.planner.Planner; import org.apache.doris.rewrite.ExprRewriter; +import org.apache.doris.rpc.PQueryStatistic; import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TQueryOptions; @@ -75,6 +78,7 @@ import java.io.IOException; import java.io.StringReader; import java.nio.ByteBuffer; +import java.util.Formatter; import java.util.List; import java.util.Map; import java.util.UUID; @@ -101,6 +105,7 @@ public class StmtExecutor { private Planner planner; private boolean isProxy; private ShowResultSet proxyResultSet = null; + private QueryStatistic statisticForAuditLog; public StmtExecutor(ConnectContext context, String stmt, boolean isProxy) { this.context = context; @@ -537,26 +542,34 @@ private void handleQueryStmt() throws Exception { // so We need to send fields after first batch arrived // send result - TResultBatch batch; + RowBatch batch; MysqlChannel channel = context.getMysqlChannel(); boolean isSendFields = false; - while ((batch = coord.getNext()) != null) { + + while ((batch = coord.getNext()) != null && !batch.isEos()) { if (!isSendFields) { sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); } isSendFields = true; - - for (ByteBuffer row : batch.getRows()) { + for (ByteBuffer row : batch.getBatch().getRows()) { channel.sendOnePacket(row); } - context.updateReturnRows(batch.getRows().size()); + context.updateReturnRows(batch.getBatch().getRows().size()); } + setConsumptionForAuditLog(batch); if (!isSendFields) { sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); } context.getState().setEof(); } + private void setConsumptionForAuditLog(RowBatch batch) { + if (batch != null) { + final PQueryStatistic queryStatistic = batch.getQueryStatistic(); + statisticForAuditLog = new QueryStatistic(queryStatistic.cpu, queryStatistic.io); + } + } + // Process a select statement. private void handleInsertStmt() throws Exception { // Every time set no send flag and clean all data in buffer @@ -774,4 +787,40 @@ private void handleExportStmt() throws Exception { ExportStmt exportStmt = (ExportStmt) parsedStmt; context.getCatalog().getExportMgr().addExportJob(exportStmt); } + + public QueryStatistic getQueryStatisticForAuditLog() { + if (statisticForAuditLog == null) { + statisticForAuditLog = new QueryStatistic(); + } + return statisticForAuditLog; + } + + public static class QueryStatistic { + private final long cpuByRow; + private final long ioByByte; + + public QueryStatistic() { + this.cpuByRow = 0; + this.ioByByte = 0; + } + + public QueryStatistic(long cpuByRow, long ioByByte) { + this.cpuByRow = cpuByRow; + this.ioByByte = ioByByte; + } + + public String getFormattingCpu() { + final StringBuilder builder = new StringBuilder(); + builder.append(cpuByRow).append(" Rows"); + return builder.toString(); + } + + public String getFormattingIo() { + final Pair pair = DebugUtil.getByteUint(ioByByte); + final Formatter fmt = new Formatter(); + final StringBuilder builder = new StringBuilder(); + builder.append(fmt.format("%.2f", pair.first)).append(" ").append(pair.second); + return builder.toString(); + } + } } diff --git a/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java b/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java index cd4af477e49196..f8f46aad8f5f49 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java +++ b/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java @@ -28,4 +28,6 @@ public class PFetchDataResult { public long packetSeq; @Protobuf(order = 3, required = false) public boolean eos; + @Protobuf(order = 4, required = false) + public PQueryStatistic queryStatistic; } diff --git a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistic.java b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistic.java new file mode 100644 index 00000000000000..b95a45e2ff4971 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistic.java @@ -0,0 +1,29 @@ +// 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. + +package org.apache.doris.rpc; + +import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; +import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; + +@ProtobufClass +public class PQueryStatistic { + @Protobuf(order = 1, required = false) + public long cpu; + @Protobuf(order = 2, required = false) + public long io; +} diff --git a/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java b/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java index e3e4a573c60f17..43eda0c3ceff1f 100644 --- a/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java @@ -231,6 +231,7 @@ public void testQuery() throws Exception { // Mock statement executor StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); + EasyMock.expect(qe.getQueryStatisticForAuditLog()).andReturn(new StmtExecutor.QueryStatistic()); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(qe); PowerMock.expectNew( @@ -254,11 +255,11 @@ public void testQueryFail() throws Exception { StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); EasyMock.expectLastCall().andThrow(new IOException("Fail")).anyTimes(); + EasyMock.expect(qe.getQueryStatisticForAuditLog()).andReturn(new StmtExecutor.QueryStatistic()); EasyMock.replay(qe); PowerMock.expectNew(StmtExecutor.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(String.class)) .andReturn(qe).anyTimes(); PowerMock.replay(StmtExecutor.class); - processor.processOnce(); Assert.assertEquals(MysqlCommand.COM_QUERY, myContext.getCommand()); } @@ -272,6 +273,7 @@ public void testQueryFail2() throws Exception { // Mock statement executor StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); + EasyMock.expect(qe.getQueryStatisticForAuditLog()).andReturn(new StmtExecutor.QueryStatistic()); EasyMock.expectLastCall().andThrow(new NullPointerException("Fail")).anyTimes(); EasyMock.replay(qe); PowerMock.expectNew(StmtExecutor.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(String.class)) diff --git a/gensrc/proto/data.proto b/gensrc/proto/data.proto index bec67edbc3413c..49fd79b7d042d9 100644 --- a/gensrc/proto/data.proto +++ b/gensrc/proto/data.proto @@ -19,6 +19,11 @@ syntax="proto2"; package doris; +message PQueryStatistic { + optional int64 cpu_by_row = 1; + optional int64 io_by_byte = 2; +} + message PRowBatch { required int32 num_rows = 1; repeated int32 row_tuples = 2; diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 71782934b2c690..255962232055bd 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -41,6 +41,7 @@ message PTransmitDataParams { optional PRowBatch row_batch = 6; // different per packet required int64 packet_seq = 7; + optional PQueryStatistic query_statistic = 8; }; message PTransmitDataResult { @@ -129,6 +130,7 @@ message PFetchDataResult { // valid when status is ok optional int64 packet_seq = 2; optional bool eos = 3; + optional PQueryStatistic query_statistic = 4; }; message PTriggerProfileReportRequest { diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 4bc4d5ee13ef22..1599b7afedabee 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -175,6 +175,7 @@ struct TPlanFragmentExecParams { // Id of this fragment in its role as a sender. 9: optional i32 sender_id 10: optional i32 num_senders + 11: optional bool send_query_statistic_with_every_batch } // Global query parameters assigned by the coordinator. From 5f9a6d3a657e561182a367d6815e2de921058d6a Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Tue, 15 Jan 2019 14:17:57 +0800 Subject: [PATCH 02/18] Modify variables name --- be/src/exec/analytic_eval_node.cpp | 6 +- be/src/exec/analytic_eval_node.h | 2 +- be/src/exec/data_sink.cpp | 10 +- be/src/exec/data_sink.h | 8 +- be/src/exec/exchange_node.cpp | 14 +- be/src/exec/exchange_node.h | 6 +- be/src/exec/exec_node.cpp | 6 +- be/src/exec/exec_node.h | 8 +- be/src/exec/hash_join_node.cpp | 8 +- be/src/exec/hash_join_node.h | 2 +- .../exec/new_partitioned_aggregation_node.cc | 10 +- .../exec/new_partitioned_aggregation_node.h | 2 +- be/src/exec/olap_scan_node.cpp | 6 +- be/src/exec/olap_scan_node.h | 2 +- be/src/exec/sort_node.cpp | 6 +- be/src/exec/sort_node.h | 2 +- be/src/exec/union_node.cpp | 6 +- be/src/exec/union_node.h | 2 +- be/src/runtime/buffer_control_block.cpp | 10 +- be/src/runtime/buffer_control_block.h | 10 +- be/src/runtime/data_stream_mgr.cpp | 10 +- be/src/runtime/data_stream_mgr.h | 8 +- be/src/runtime/data_stream_recvr.cc | 4 +- be/src/runtime/data_stream_recvr.h | 15 +- be/src/runtime/data_stream_sender.cpp | 21 +- be/src/runtime/data_stream_sender.h | 2 +- be/src/runtime/plan_fragment_executor.cpp | 28 ++- be/src/runtime/plan_fragment_executor.h | 9 +- be/src/runtime/query_statistic.h | 217 ------------------ be/src/runtime/query_statistics.h | 217 ++++++++++++++++++ be/src/runtime/result_sink.cpp | 4 +- be/src/runtime/result_sink.h | 2 +- be/src/runtime/runtime_state.cpp | 1 - be/src/runtime/runtime_state.h | 1 - be/src/service/internal_service.cpp | 6 +- .../proc/CurrentQueryFragmentProcNode.java | 6 +- .../common/proc/CurrentQueryInfoProvider.java | 4 +- .../proc/CurrentQueryStatisticsProcDir.java | 6 +- .../org/apache/doris/planner/Planner.java | 31 ++- .../org/apache/doris/qe/ConnectProcessor.java | 10 +- .../java/org/apache/doris/qe/Coordinator.java | 2 +- .../org/apache/doris/qe/ResultReceiver.java | 4 +- .../java/org/apache/doris/qe/RowBatch.java | 12 +- .../org/apache/doris/qe/StmtExecutor.java | 42 ++-- .../apache/doris/rpc/PFetchDataResult.java | 2 +- ...ryStatistic.java => PQueryStatistics.java} | 6 +- gensrc/proto/data.proto | 6 +- gensrc/proto/internal_service.proto | 4 +- gensrc/thrift/PaloInternalService.thrift | 2 +- 49 files changed, 419 insertions(+), 389 deletions(-) delete mode 100644 be/src/runtime/query_statistic.h create mode 100644 be/src/runtime/query_statistics.h rename fe/src/main/java/org/apache/doris/rpc/{PQueryStatistic.java => PQueryStatistics.java} (92%) diff --git a/be/src/exec/analytic_eval_node.cpp b/be/src/exec/analytic_eval_node.cpp index 2a80d2707c3a86..9fae59bac4031c 100644 --- a/be/src/exec/analytic_eval_node.cpp +++ b/be/src/exec/analytic_eval_node.cpp @@ -839,9 +839,9 @@ Status AnalyticEvalNode::get_next(RuntimeState* state, RowBatch* row_batch, bool return Status::OK; } -Status AnalyticEvalNode::collect_query_statistic(QueryStatistic* statistic) { - RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); - statistic->add_cpu_by_row(_process_rows_counter->value()); +Status AnalyticEvalNode::collect_query_statistics(QueryStatistics* statistics) { + RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); + statistics->add_process_rows(_process_rows_counter->value()); return Status::OK; } diff --git a/be/src/exec/analytic_eval_node.h b/be/src/exec/analytic_eval_node.h index 14ebb525e149bc..1c47988c84497b 100644 --- a/be/src/exec/analytic_eval_node.h +++ b/be/src/exec/analytic_eval_node.h @@ -68,7 +68,7 @@ class AnalyticEvalNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistic(QueryStatistic* statistic); + virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); protected: diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index 51109e9a1e0fcd..1ef29f70ccb509 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -49,16 +49,16 @@ Status DataSink::create_data_sink( if (!thrift_sink.__isset.stream_sink) { return Status("Missing data stream sink."); } - bool send_query_statistic_with_every_batch = false; - if (params.__isset.send_query_statistic_with_every_batch) { - send_query_statistic_with_every_batch = - params.send_query_statistic_with_every_batch; + bool send_query_statistics_with_every_batch = false; + if (params.__isset.send_query_statistics_with_every_batch) { + send_query_statistics_with_every_batch = + params.send_query_statistics_with_every_batch; } // TODO: figure out good buffer size based on size of output row tmp_sink = new DataStreamSender( pool, params.sender_id, row_desc, thrift_sink.stream_sink, params.destinations, 16 * 1024, - send_query_statistic_with_every_batch); + send_query_statistics_with_every_batch); // RETURN_IF_ERROR(sender->prepare(state->obj_pool(), thrift_sink.stream_sink)); sink->reset(tmp_sink); break; diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h index 2fa73f2b357abb..3227e875bdf3e1 100644 --- a/be/src/exec/data_sink.h +++ b/be/src/exec/data_sink.h @@ -25,7 +25,7 @@ #include "gen_cpp/DataSinks_types.h" #include "gen_cpp/Exprs_types.h" #include "runtime/mem_tracker.h" -#include "runtime/query_statistic.h" +#include "runtime/query_statistics.h" namespace doris { @@ -79,8 +79,8 @@ class DataSink { // Returns the runtime profile for the sink. virtual RuntimeProfile* profile() = 0; - virtual void set_query_statistic(boost::shared_ptr statistic) { - _query_statistic = statistic; + virtual void set_query_statistics(std::shared_ptr statistics) { + _query_statistics = statistics; } protected: // Set to true after close() has been called. subclasses should check and set this in @@ -88,7 +88,7 @@ class DataSink { bool _closed; std::unique_ptr _expr_mem_tracker; - boost::shared_ptr _query_statistic; + std::shared_ptr _query_statistics; }; } // namespace doris diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp index 9664b4de01ec1a..0f48b4d9a4a664 100644 --- a/be/src/exec/exchange_node.cpp +++ b/be/src/exec/exchange_node.cpp @@ -65,14 +65,14 @@ Status ExchangeNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::prepare(state)); _convert_row_batch_timer = ADD_TIMER(runtime_profile(), "ConvertRowBatchTime"); _merge_rows_counter = ADD_COUNTER(runtime_profile(), "MergeRows", TUnit::UNIT); - _sub_plan_statistic.reset(new QueryStatistic()); + _sub_plan_statistics.reset(new QueryStatistics()); // TODO: figure out appropriate buffer size DCHECK_GT(_num_senders, 0); _stream_recvr = state->exec_env()->stream_mgr()->create_recvr( state, _input_row_desc, state->fragment_instance_id(), _id, _num_senders, config::exchg_node_buffer_size_bytes, - state->runtime_profile(), _is_merging, _sub_plan_statistic.get()); + state->runtime_profile(), _is_merging, _sub_plan_statistics.get()); if (_is_merging) { RETURN_IF_ERROR(_sort_exec_exprs.prepare( state, _row_descriptor, _row_descriptor, expr_mem_tracker())); @@ -96,11 +96,11 @@ Status ExchangeNode::open(RuntimeState* state) { return Status::OK; } -Status ExchangeNode::collect_query_statistic(QueryStatistic* statistic) { - RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); - statistic->add_cpu_by_row(_merge_rows_counter->value()); - QueryStatistic* sub_plan_statistic = _sub_plan_statistic.get(); - statistic->add(*sub_plan_statistic); +Status ExchangeNode::collect_query_statistics(QueryStatistics* statistics) { + RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); + statistics->add_process_rows(_merge_rows_counter->value()); + QueryStatistics* sub_plan_statistics = _sub_plan_statistics.get(); + statistics->add(*sub_plan_statistics); return Status::OK; } diff --git a/be/src/exec/exchange_node.h b/be/src/exec/exchange_node.h index 4fb8914a4d506e..c31951f2f1e769 100644 --- a/be/src/exec/exchange_node.h +++ b/be/src/exec/exchange_node.h @@ -49,7 +49,7 @@ class ExchangeNode : public ExecNode { // Blocks until the first batch is available for consumption via GetNext(). virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistic(QueryStatistic* statistic); + virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); // the number of senders needs to be set after the c'tor, because it's not @@ -112,8 +112,8 @@ class ExchangeNode : public ExecNode { RuntimeProfile::Counter* _merge_rows_counter; - // Query statistic from sub plan. - boost::scoped_ptr _sub_plan_statistic; + // Query statistics from sub plan. + boost::scoped_ptr _sub_plan_statistics; }; }; diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 4aca701268fd1a..65b04c9f1763d3 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -212,10 +212,10 @@ Status ExecNode::reset(RuntimeState* state) { return Status::OK; } -Status ExecNode::collect_query_statistic(QueryStatistic* statistic) { - DCHECK(statistic != nullptr); +Status ExecNode::collect_query_statistics(QueryStatistics* statistics) { + DCHECK(statistics != nullptr); for (auto child_node : _children) { - child_node->collect_query_statistic(statistic); + child_node->collect_query_statistics(statistics); } return Status::OK; } diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index fa8924e2b1bdc4..a164efd53d80a1 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -29,7 +29,7 @@ #include "util/runtime_profile.h" #include "util/blocking_queue.hpp" #include "runtime/bufferpool/buffer_pool.h" -#include "runtime/query_statistic.h" +#include "runtime/query_statistics.h" namespace llvm { class Function; @@ -115,10 +115,10 @@ class ExecNode { // so should be fast. virtual Status reset(RuntimeState* state); - // collect_query_statistic() should be called before close() and after get_next(), it - // responsible for collecting statistics returned with query result, it can't be called + // collect_query_statistics() should be called before close() and after get_next(), it + // responsible for collecting statisticss returned with query result, it can't be called // when prepare() return error. - virtual Status collect_query_statistic(QueryStatistic* statistic); + virtual Status collect_query_statistics(QueryStatistics* statistics); // close() will get called for every exec node, regardless of what else is called and // the status of these calls (i.e. prepare() may never have been called, or diff --git a/be/src/exec/hash_join_node.cpp b/be/src/exec/hash_join_node.cpp index cedf1ed7c02172..a6a5e7c17ca7cb 100644 --- a/be/src/exec/hash_join_node.cpp +++ b/be/src/exec/hash_join_node.cpp @@ -177,10 +177,10 @@ Status HashJoinNode::prepare(RuntimeState* state) { return Status::OK; } -Status HashJoinNode::collect_query_statistic(QueryStatistic* statistic) { - RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); - statistic->add_cpu_by_row(_probe_rows_counter->value()); - statistic->add_cpu_by_row(_build_rows_counter->value()); +Status HashJoinNode::collect_query_statistics(QueryStatistics* statistics) { + RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); + statistics->add_process_rows(_probe_rows_counter->value()); + statistics->add_process_rows(_build_rows_counter->value()); return Status::OK; } diff --git a/be/src/exec/hash_join_node.h b/be/src/exec/hash_join_node.h index 607f6bdec4e986..a45d120db4c6b6 100644 --- a/be/src/exec/hash_join_node.h +++ b/be/src/exec/hash_join_node.h @@ -56,7 +56,7 @@ class HashJoinNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistic(QueryStatistic* statistic); + virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); static const char* _s_llvm_class_name; diff --git a/be/src/exec/new_partitioned_aggregation_node.cc b/be/src/exec/new_partitioned_aggregation_node.cc index fab455f0b6f2d7..0dd66165f7857f 100644 --- a/be/src/exec/new_partitioned_aggregation_node.cc +++ b/be/src/exec/new_partitioned_aggregation_node.cc @@ -593,7 +593,7 @@ bool NewPartitionedAggregationNode::ShouldExpandPreaggHashTables() const { ht_rows += ht->size(); } - // Need some rows in tables to have valid statistics. + // Need some rows in tables to have valid statisticss. if (ht_rows == 0) return true; // Find the appropriate reduction factor in our table for the current hash table sizes. @@ -621,7 +621,7 @@ bool NewPartitionedAggregationNode::ShouldExpandPreaggHashTables() const { // set, N is the number of input rows, excluding passed-through rows, and n is the // number of rows inserted or merged into the hash tables. This is a very rough // approximation but is good enough to be useful. - // TODO: consider collecting more statistics to better estimate reduction. + // TODO: consider collecting more statisticss to better estimate reduction. // double estimated_reduction = aggregated_input_rows >= expected_input_rows // ? current_reduction // : 1 + (expected_input_rows / aggregated_input_rows) * (current_reduction - 1); @@ -671,9 +671,9 @@ Status NewPartitionedAggregationNode::reset(RuntimeState* state) { return ExecNode::reset(state); } -Status NewPartitionedAggregationNode::collect_query_statistic(QueryStatistic* statistic) { - RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); - statistic->add_cpu_by_row(_build_rows_counter->value()); +Status NewPartitionedAggregationNode::collect_query_statistics(QueryStatistics* statistics) { + RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); + statistics->add_process_rows(_build_rows_counter->value()); return Status::OK; } diff --git a/be/src/exec/new_partitioned_aggregation_node.h b/be/src/exec/new_partitioned_aggregation_node.h index 1dbf8228b1b32c..f613ef19a6c514 100644 --- a/be/src/exec/new_partitioned_aggregation_node.h +++ b/be/src/exec/new_partitioned_aggregation_node.h @@ -137,7 +137,7 @@ class NewPartitionedAggregationNode : public ExecNode { virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); virtual Status reset(RuntimeState* state); - virtual Status collect_query_statistic(QueryStatistic* statistic); + virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); static const char* LLVM_CLASS_NAME; diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp index 88b881b8055a10..3d98c3a7fa78ac 100644 --- a/be/src/exec/olap_scan_node.cpp +++ b/be/src/exec/olap_scan_node.cpp @@ -306,9 +306,9 @@ Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo return _status; } -Status OlapScanNode::collect_query_statistic(QueryStatistic* statistic) { - RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); - statistic->add_io_by_byte(_read_compressed_counter->value()); +Status OlapScanNode::collect_query_statistics(QueryStatistics* statistics) { + RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); + statistics->add_scan_bytes(_read_compressed_counter->value()); return Status::OK; } diff --git a/be/src/exec/olap_scan_node.h b/be/src/exec/olap_scan_node.h index 045d3c496a37b1..1a08b176b9ab38 100644 --- a/be/src/exec/olap_scan_node.h +++ b/be/src/exec/olap_scan_node.h @@ -56,7 +56,7 @@ class OlapScanNode : public ScanNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistic(QueryStatistic* statistic); + virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); virtual Status set_scan_ranges(const std::vector& scan_ranges); diff --git a/be/src/exec/sort_node.cpp b/be/src/exec/sort_node.cpp index 254da1f7577577..d6a0036d3e8868 100644 --- a/be/src/exec/sort_node.cpp +++ b/be/src/exec/sort_node.cpp @@ -118,9 +118,9 @@ Status SortNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) { } -Status SortNode::collect_query_statistic(QueryStatistic* statistic) { - RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); - statistic->add_cpu_by_row(_sort_rows_counter->value()); +Status SortNode::collect_query_statistics(QueryStatistics* statistics) { + RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); + statistics->add_process_rows(_sort_rows_counter->value()); return Status::OK; } diff --git a/be/src/exec/sort_node.h b/be/src/exec/sort_node.h index 2559bef8b58d47..ee52549fbd0d59 100644 --- a/be/src/exec/sort_node.h +++ b/be/src/exec/sort_node.h @@ -42,7 +42,7 @@ class SortNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistic(QueryStatistic* statistic); + virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); protected: diff --git a/be/src/exec/union_node.cpp b/be/src/exec/union_node.cpp index 2094acaf946fcd..6e0df5fae28126 100644 --- a/be/src/exec/union_node.cpp +++ b/be/src/exec/union_node.cpp @@ -337,9 +337,9 @@ Status UnionNode::reset(RuntimeState* state) { } #endif -Status UnionNode::collect_query_statistic(QueryStatistic* statistic) { - RETURN_IF_ERROR(ExecNode::collect_query_statistic(statistic)); - statistic->add_cpu_by_row(_materialize_rows_counter->value()); +Status UnionNode::collect_query_statistics(QueryStatistics* statistics) { + RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); + statistics->add_process_rows(_materialize_rows_counter->value()); return Status::OK; } diff --git a/be/src/exec/union_node.h b/be/src/exec/union_node.h index 2ca14f90c7e1fc..a67aa4ed56e60a 100644 --- a/be/src/exec/union_node.h +++ b/be/src/exec/union_node.h @@ -48,7 +48,7 @@ class UnionNode : public ExecNode { virtual void codegen(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistic(QueryStatistic* statistic); + virtual Status collect_query_statistics(QueryStatistics* statistics); // virtual Status reset(RuntimeState* state); virtual Status close(RuntimeState* state); diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp index 1e2c271c776147..24237f240d29c8 100644 --- a/be/src/runtime/buffer_control_block.cpp +++ b/be/src/runtime/buffer_control_block.cpp @@ -32,11 +32,11 @@ void GetResultBatchCtx::on_failure(const Status& status) { } void GetResultBatchCtx::on_close(int64_t packet_seq, - QueryStatistic* statistic) { + QueryStatistics* statistics) { Status status; status.to_protobuf(result->mutable_status()); - if (statistic != nullptr) { - statistic->serialize(result->mutable_query_statistic()); + if (statistics != nullptr) { + statistics->serialize(result->mutable_query_statistics()); } result->set_packet_seq(packet_seq); result->set_eos(true); @@ -187,7 +187,7 @@ void BufferControlBlock::get_batch(GetResultBatchCtx* ctx) { return; } if (_is_close) { - ctx->on_close(_packet_num, _query_statistic.get()); + ctx->on_close(_packet_num, _query_statistics.get()); return; } // no ready data, push ctx to waiting list @@ -204,7 +204,7 @@ Status BufferControlBlock::close(Status exec_status) { if (!_waiting_rpc.empty()) { if (_status.ok()) { for (auto& ctx : _waiting_rpc) { - ctx->on_close(_packet_num, _query_statistic.get()); + ctx->on_close(_packet_num, _query_statistics.get()); } } else { for (auto& ctx : _waiting_rpc) { diff --git a/be/src/runtime/buffer_control_block.h b/be/src/runtime/buffer_control_block.h index 8a053494556e8a..34a9ea4c899d74 100644 --- a/be/src/runtime/buffer_control_block.h +++ b/be/src/runtime/buffer_control_block.h @@ -24,7 +24,7 @@ #include #include "common/status.h" #include "gen_cpp/Types_types.h" -#include "runtime/query_statistic.h" +#include "runtime/query_statistics.h" namespace google { namespace protobuf { @@ -53,7 +53,7 @@ struct GetResultBatchCtx { } void on_failure(const Status& status); - void on_close(int64_t packet_seq, QueryStatistic* statistic = nullptr); + void on_close(int64_t packet_seq, QueryStatistics* statistics = nullptr); void on_data(TFetchDataResult* t_result, int64_t packet_seq, bool eos = false); }; @@ -81,8 +81,8 @@ class BufferControlBlock { return _fragment_id; } - void set_query_statistic(boost::shared_ptr statistic) { - _query_statistic = statistic; + void set_query_statistics(std::shared_ptr statistics) { + _query_statistics = statistics; } private: typedef std::list ResultQueue; @@ -107,7 +107,7 @@ class BufferControlBlock { std::deque _waiting_rpc; - boost::shared_ptr _query_statistic; + std::shared_ptr _query_statistics; }; } diff --git a/be/src/runtime/data_stream_mgr.cpp b/be/src/runtime/data_stream_mgr.cpp index b983ee5f8b240d..c8781c6d75f1f0 100644 --- a/be/src/runtime/data_stream_mgr.cpp +++ b/be/src/runtime/data_stream_mgr.cpp @@ -51,14 +51,14 @@ inline uint32_t DataStreamMgr::get_hash_value( shared_ptr DataStreamMgr::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, QueryStatistic* query_statistic) { + bool is_merging, QueryStatistics* query_statistics) { DCHECK(profile != NULL); VLOG_FILE << "creating receiver for fragment=" << fragment_instance_id << ", node=" << dest_node_id; shared_ptr recvr( new DataStreamRecvr(this, state->instance_mem_tracker(), row_desc, fragment_instance_id, dest_node_id, num_senders, is_merging, buffer_size, - profile, query_statistic)); + profile, query_statistics)); uint32_t hash_value = get_hash_value(fragment_instance_id, dest_node_id); lock_guard l(_lock); _fragment_stream_set.insert(std::make_pair(fragment_instance_id, dest_node_id)); @@ -118,11 +118,11 @@ Status DataStreamMgr::add_data( return Status::OK; } -Status DataStreamMgr::update_query_statistic(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, - int sender_id, const PQueryStatistic& query_statistic) { +Status DataStreamMgr::update_query_statistics(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, + int sender_id, const PQueryStatistics& query_statistics) { shared_ptr recvr = find_recvr(fragment_instance_id, dest_node_id); if (recvr != NULL) { - recvr->update_sub_plan_statistic(query_statistic, sender_id); + recvr->update_sub_plan_statistics(query_statistics, sender_id); } return Status::OK; } diff --git a/be/src/runtime/data_stream_mgr.h b/be/src/runtime/data_stream_mgr.h index 6f63e9ede3c984..2c26053ce02d52 100644 --- a/be/src/runtime/data_stream_mgr.h +++ b/be/src/runtime/data_stream_mgr.h @@ -30,7 +30,7 @@ #include "common/object_pool.h" #include "runtime/descriptors.h" // for PlanNodeId #include "runtime/mem_tracker.h" -#include "runtime/query_statistic.h" +#include "runtime/query_statistics.h" #include "util/runtime_profile.h" #include "gen_cpp/Types_types.h" // for TUniqueId @@ -77,7 +77,7 @@ class DataStreamMgr { 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, QueryStatistic* query_statistic); + bool is_merging, QueryStatistics* query_statistics); // Adds a row batch to the recvr identified by fragment_instance_id/dest_node_id // if the recvr has not been cancelled. sender_id identifies the sender instance @@ -99,8 +99,8 @@ class DataStreamMgr { Status close_sender(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int sender_id, int be_number); - Status update_query_statistic(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, - int sender_id, const PQueryStatistic& query_statistic); + Status update_query_statistics(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, + int sender_id, const PQueryStatistics& query_statistics); // Closes all receivers registered for fragment_instance_id immediately. void cancel(const TUniqueId& fragment_instance_id); diff --git a/be/src/runtime/data_stream_recvr.cc b/be/src/runtime/data_stream_recvr.cc index 2b599707b466d4..4bf10a1c98f9e7 100644 --- a/be/src/runtime/data_stream_recvr.cc +++ b/be/src/runtime/data_stream_recvr.cc @@ -351,7 +351,7 @@ DataStreamRecvr::DataStreamRecvr( DataStreamMgr* stream_mgr, MemTracker* parent_tracker, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, bool is_merging, int total_buffer_limit, - RuntimeProfile* profile, QueryStatistic* query_statistic) : + RuntimeProfile* profile, QueryStatistics* query_statistics) : _mgr(stream_mgr), _fragment_instance_id(fragment_instance_id), _dest_node_id(dest_node_id), @@ -360,7 +360,7 @@ DataStreamRecvr::DataStreamRecvr( _is_merging(is_merging), _num_buffered_bytes(0), _profile(profile), - _sub_plan_query_statistic(query_statistic) { + _sub_plan_query_statistics(query_statistics) { _mem_tracker.reset(new MemTracker(-1, "DataStreamRecvr", parent_tracker)); // Create one queue per sender if is_merging is true. diff --git a/be/src/runtime/data_stream_recvr.h b/be/src/runtime/data_stream_recvr.h index 140e3071f41274..2d1b807398663e 100644 --- a/be/src/runtime/data_stream_recvr.h +++ b/be/src/runtime/data_stream_recvr.h @@ -25,7 +25,7 @@ #include "common/status.h" #include "gen_cpp/Types_types.h" // for TUniqueId #include "runtime/descriptors.h" -#include "runtime/query_statistic.h" +#include "runtime/query_statistics.h" #include "util/tuple_row_compare.h" namespace google { @@ -100,10 +100,13 @@ class DataStreamRecvr { const RowDescriptor& row_desc() const { return _row_desc; } MemTracker* mem_tracker() const { return _mem_tracker.get(); } - void update_sub_plan_statistic(const PQueryStatistic& statistic, int sender_id) { - _sub_plan_query_statistic->deserialize(statistic, sender_id); + void update_sub_plan_statistics(const PQueryStatistics& statistics, int sender_id) { + _sub_plan_query_statistics->deserialize(statistics, sender_id); } + QueryStatistics* get() { + return _sub_plan_query_statistics; + } private: friend class DataStreamMgr; class SenderQueue; @@ -111,7 +114,7 @@ class DataStreamRecvr { DataStreamRecvr(DataStreamMgr* stream_mgr, MemTracker* parent_tracker, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, bool is_merging, int total_buffer_limit, - RuntimeProfile* profile, QueryStatistic* query_statistic); + RuntimeProfile* profile, QueryStatistics* query_statistics); // If receive queue is full, done is enqueue pending, and return with *done is nullptr void add_batch(const PRowBatch& batch, int sender_id, @@ -199,8 +202,8 @@ class DataStreamRecvr { // Wall time senders spend waiting for the recv buffer to have capacity. RuntimeProfile::Counter* _buffer_full_wall_timer; - // Query statistic returned with batch - QueryStatistic* _sub_plan_query_statistic; + // Query statistics returned with batch + QueryStatistics* _sub_plan_query_statistics; // Total time spent waiting for data to arrive in the recv buffer // RuntimeProfile::Counter* _data_arrival_timer; diff --git a/be/src/runtime/data_stream_sender.cpp b/be/src/runtime/data_stream_sender.cpp index 557beb5faadfac..db5c475853e679 100644 --- a/be/src/runtime/data_stream_sender.cpp +++ b/be/src/runtime/data_stream_sender.cpp @@ -74,7 +74,7 @@ class DataStreamSender::Channel { PlanNodeId dest_node_id, int buffer_size, bool is_transfer_chain, - bool send_query_statistic_with_every_batch) : + bool send_query_statistics_with_every_batch) : _parent(parent), _buffer_size(buffer_size), _row_desc(row_desc), @@ -85,7 +85,7 @@ class DataStreamSender::Channel { _need_close(false), _brpc_dest_addr(brpc_dest), _is_transfer_chain(is_transfer_chain), - _send_query_statistic_with_every_batch(send_query_statistic_with_every_batch) { + _send_query_statistics_with_every_batch(send_query_statistics_with_every_batch) { } virtual ~Channel() { @@ -170,7 +170,7 @@ class DataStreamSender::Channel { int32_t _brpc_timeout_ms = 500; // whether the dest can be treated as consumption transfer chain. bool _is_transfer_chain; - bool _send_query_statistic_with_every_batch; + bool _send_query_statistics_with_every_batch; }; Status DataStreamSender::Channel::init(RuntimeState* state) { @@ -211,9 +211,9 @@ Status DataStreamSender::Channel::send_batch(PRowBatch* batch, bool eos) { } VLOG_ROW << "Channel::send_batch() instance_id=" << _fragment_instance_id << " dest_node=" << _dest_node_id; - if (_is_transfer_chain && (_send_query_statistic_with_every_batch || eos)) { - auto statistic = _brpc_request.mutable_query_statistic(); - _parent->_query_statistic->serialize(statistic); + if (_is_transfer_chain && (_send_query_statistics_with_every_batch || eos)) { + auto statistic = _brpc_request.mutable_query_statistics(); + _parent->_query_statistics->serialize(statistic); } _brpc_request.set_eos(eos); @@ -298,7 +298,7 @@ DataStreamSender::DataStreamSender( const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, int per_channel_buffer_size, - bool send_query_statistic_with_every_batch) : + bool send_query_statistics_with_every_batch) : _sender_id(sender_id), _pool(pool), _row_desc(row_desc), @@ -318,17 +318,14 @@ DataStreamSender::DataStreamSender( || sink.output_partition.type == TPartitionType::RANGE_PARTITIONED); // TODO: use something like google3's linked_ptr here (scoped_ptr isn't copyable) for (int i = 0; i < destinations.size(); ++i) { - bool is_transfer_chain = false; // Select first dest as transfer chain. - if (i == 0) { - is_transfer_chain = true; - } + bool is_transfer_chain = (i == 0); _channel_shared_ptrs.emplace_back( new Channel(this, row_desc, destinations[i].brpc_server, destinations[i].fragment_instance_id, sink.dest_node_id, per_channel_buffer_size, - is_transfer_chain, send_query_statistic_with_every_batch)); + is_transfer_chain, send_query_statistics_with_every_batch)); _channels.push_back(_channel_shared_ptrs[i].get()); } } diff --git a/be/src/runtime/data_stream_sender.h b/be/src/runtime/data_stream_sender.h index 22174541b7f6b2..a4787491d6e4a0 100644 --- a/be/src/runtime/data_stream_sender.h +++ b/be/src/runtime/data_stream_sender.h @@ -59,7 +59,7 @@ class DataStreamSender : public DataSink { DataStreamSender(ObjectPool* pool, int sender_id, const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, - int per_channel_buffer_size, bool send_query_statistic_with_every_batch); + int per_channel_buffer_size, bool send_query_statistics_with_every_batch); virtual ~DataStreamSender(); virtual Status init(const TDataSink& thrift_sink); diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index 0019c165be1960..bc8a329d8af431 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -54,7 +54,8 @@ PlanFragmentExecutor::PlanFragmentExecutor( _prepared(false), _closed(false), _has_thread_token(false), - _is_report_success(true) { + _is_report_success(true), + _collect_query_statistics_every_batch(false) { } PlanFragmentExecutor::~PlanFragmentExecutor() { @@ -196,6 +197,11 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) { if (sink_profile != NULL) { profile()->add_child(sink_profile, true, NULL); } + + if (params.__isset.send_query_statistics_with_every_batch) { + _collect_query_statistics_every_batch = + params.send_query_statistics_with_every_batch; + } } else { _sink.reset(NULL); } @@ -227,8 +233,8 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) { VLOG(3) << "plan_root=\n" << _plan->debug_string(); _prepared = true; - _query_statistic.reset(new QueryStatistic()); - _sink->set_query_statistic(_query_statistic); + _query_statistics.reset(new QueryStatistics()); + _sink->set_query_statistics(_query_statistics); return Status::OK; } @@ -318,12 +324,13 @@ Status PlanFragmentExecutor::open_internal() { VLOG_ROW << row->to_string(row_desc()); } } - + SCOPED_TIMER(profile()->total_time_counter()); - // Collect this plan and sub plan statistics, and send to parent plan. - _plan->collect_query_statistic(_query_statistic.get()); + // Collect this plan and sub plan statisticss, and send to parent plan. + if (_collect_query_statistics_every_batch) { + collect_query_statistics(); + } RETURN_IF_ERROR(_sink->send(runtime_state(), batch)); - _query_statistic->clear(); } // Close the sink *before* stopping the report thread. Close may @@ -339,7 +346,7 @@ Status PlanFragmentExecutor::open_internal() { // audit the sinks to check that this is ok, or change that behaviour. { SCOPED_TIMER(profile()->total_time_counter()); - _plan->collect_query_statistic(_query_statistic.get()); + collect_query_statistics() Status status = _sink->close(runtime_state(), _status); RETURN_IF_ERROR(status); } @@ -356,6 +363,11 @@ Status PlanFragmentExecutor::open_internal() { return Status::OK; } +void PlanFragmentExecutor::collect_query_statistics() { + _query_statistics->clear(); + _plan->collect_query_statistics(_query_statistics.get()); +} + void PlanFragmentExecutor::report_profile() { VLOG_FILE << "report_profile(): instance_id=" << _runtime_state->fragment_instance_id(); diff --git a/be/src/runtime/plan_fragment_executor.h b/be/src/runtime/plan_fragment_executor.h index b3e06c4dbf42a9..d298ae7e73a3fd 100644 --- a/be/src/runtime/plan_fragment_executor.h +++ b/be/src/runtime/plan_fragment_executor.h @@ -24,6 +24,7 @@ #include "common/status.h" #include "common/object_pool.h" +#include "runtime/query_statistics.h" #include "runtime/runtime_state.h" namespace doris { @@ -205,8 +206,9 @@ class PlanFragmentExecutor { RuntimeProfile::Counter* _average_thread_tokens; // This plan and it's sub plan query statisic. Because plan may have been finished - // when fe fetch data and query statistic, this will be shared with BufferControlBlock. - boost::shared_ptr _query_statistic; + // when fe fetch data and query statistics, this will be shared with BufferControlBlock. + std::shared_ptr _query_statistics; + bool _collect_query_statistics_every_batch; ObjectPool* obj_pool() { return _runtime_state->obj_pool(); @@ -260,6 +262,9 @@ class PlanFragmentExecutor { const DescriptorTbl& desc_tbl() { return _runtime_state->desc_tbl(); } + + void collect_query_statistics(); + }; } diff --git a/be/src/runtime/query_statistic.h b/be/src/runtime/query_statistic.h deleted file mode 100644 index 52c5fb1054e257..00000000000000 --- a/be/src/runtime/query_statistic.h +++ /dev/null @@ -1,217 +0,0 @@ -// 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. - -#ifndef DORIS_BE_EXEC_QUERY_STATISTIC_H -#define DORIS_BE_EXEC_QUERY_STATISTIC_H - -#include "common/atomic.h" -#include "gen_cpp/data.pb.h" -#include "util/spinlock.h" - -namespace doris { - -// This is responsible for collecting query statistic, usually it consist of -// two parts, one is current fragment or plan's statistic, the other is sub fragment -// or plan's statistic.Now cpu is measured by number of rows, io is measured by byte. -class QueryStatistic { -public: - - class Statistic { - public: - - Statistic() : cpu_by_row(0), io_by_byte(0) { - } - - void add(const Statistic& other) { - cpu_by_row += other.cpu_by_row; - io_by_byte += other.io_by_byte; - } - - void add(const PQueryStatistic& other) { - cpu_by_row += other.cpu_by_row(); - io_by_byte += other.io_by_byte(); - } - - void reset() { - cpu_by_row = 0; - io_by_byte = 0; - } - - void set_io_by_byte(int64_t io_by_byte) { - this->io_by_byte = io_by_byte; - } - - void add_io_by_byte(int64_t io_by_byte) { - this->io_by_byte += io_by_byte; - } - - long get_io_by_byte() { - return io_by_byte; - } - - void set_cpu_by_row(int64_t cpu_by_row) { - this->cpu_by_row = cpu_by_row; - } - - void add_cpu_by_row(int64_t cpu_by_row) { - this->cpu_by_row += cpu_by_row; - } - - long get_cpu_by_row() { - return cpu_by_row; - } - - void serialize(PQueryStatistic* statistic) { - DCHECK(statistic != nullptr); - statistic->set_cpu_by_row(cpu_by_row); - statistic->set_io_by_byte(io_by_byte); - } - - void deserialize(const PQueryStatistic& statistic) { - cpu_by_row = statistic.cpu_by_row(); - io_by_byte = statistic.io_by_byte(); - } - - private: - - long cpu_by_row; - long io_by_byte; - }; - - QueryStatistic() { - } - - void add(const QueryStatistic& other) { - boost::lock_guard l(_lock); - auto other_iter = other._statistics.begin(); - while (other_iter != other._statistics.end()) { - auto iter = _statistics.find(other_iter->first); - Statistic* statistic = nullptr; - if (iter == _statistics.end()) { - statistic = new Statistic(); - _statistics[other_iter->first] = statistic; - } else { - statistic = iter->second; - } - Statistic* other_statistic = other_iter->second; - statistic->add(*other_statistic); - other_iter++; - } - } - - void add_cpu_by_row(long cpu_by_row) { - boost::lock_guard l(_lock); - auto statistic = find(DEFAULT_SENDER_ID); - statistic->add_cpu_by_row(cpu_by_row); - } - - void add_io_by_byte(long io_by_byte) { - boost::lock_guard l(_lock); - auto statistic = find(DEFAULT_SENDER_ID); - statistic->add_io_by_byte(io_by_byte); - } - - void deserialize(const PQueryStatistic& other, int sender_id) { - boost::lock_guard l(_lock); - auto statistic = find(sender_id); - statistic->deserialize(other); - } - - void serialize(PQueryStatistic* statistic) { - boost::lock_guard l(_lock); - DCHECK(statistic != nullptr); - Statistic total_statistic = get_total_statistic(); - total_statistic.serialize(statistic); - } - - long get_cpu_by_row() { - boost::lock_guard l(_lock); - Statistic statistic = get_total_statistic(); - return statistic.get_cpu_by_row(); - } - - long get_io_by_byte() { - boost::lock_guard l(_lock); - Statistic statistic = get_total_statistic(); - return statistic.get_io_by_byte(); - } - - long get_cpu_by_row(int sender_id) { - boost::lock_guard l(_lock); - auto statistic = find(sender_id); - return statistic->get_cpu_by_row(); - } - - long get_io_by_byte(int sender_id) { - boost::lock_guard l(_lock); - auto statistic = find(sender_id); - return statistic->get_io_by_byte(); - } - - void clear() { - boost::lock_guard l(_lock); - auto iter = _statistics.begin(); - while (iter != _statistics.end()) { - iter->second->reset(); - iter++; - } - } - - ~QueryStatistic() { - boost::lock_guard l(_lock); - auto iter = _statistics.begin(); - while (iter != _statistics.end()) { - delete iter->second; - iter++; - } - } - -private: - - Statistic* find(int sender_id) { - auto iter = _statistics.find(sender_id); - Statistic* statistic = nullptr; - if (iter == _statistics.end()) { - statistic = new Statistic(); - _statistics[sender_id] = statistic; - } else { - statistic = iter->second; - } - return statistic; - } - - Statistic get_total_statistic() { - Statistic total_statistic; - auto iter = _statistics.begin(); - while (iter != _statistics.end()) { - total_statistic.add(*(iter->second)); - iter++; - } - return total_statistic; - } - - // Map lock. - SpinLock _lock; - // Sender id to statistic. - std::map _statistics; - // To index current plan. - static const int DEFAULT_SENDER_ID = -1; -}; - -} - -#endif diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h new file mode 100644 index 00000000000000..d9e948199ae88b --- /dev/null +++ b/be/src/runtime/query_statistics.h @@ -0,0 +1,217 @@ +// 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. + +#ifndef DORIS_BE_EXEC_QUERY_STATISTICS_H +#define DORIS_BE_EXEC_QUERY_STATISTICS_H + +#include "common/atomic.h" +#include "gen_cpp/data.pb.h" +#include "util/spinlock.h" + +namespace doris { + +// This is responsible for collecting query statistics, usually it consist of +// two parts, one is current fragment or plan's statistics, the other is sub fragment +// or plan's statistics.Now cpu is measured by number of rows, io is measured by byte. +class QueryStatistics { +public: + + class Statistics { + public: + + Statistics() : process_rows(0), scan_bytes(0) { + } + + void add(const Statistics& other) { + process_rows += other.process_rows; + scan_bytes += other.scan_bytes; + } + + void add(const PQueryStatistics& other) { + process_rows += other.process_rows(); + scan_bytes += other.scan_bytes(); + } + + void reset() { + process_rows = 0; + scan_bytes = 0; + } + + void set_scan_bytes(int64_t scan_bytes) { + this->scan_bytes = scan_bytes; + } + + void add_scan_bytes(int64_t scan_bytes) { + this->scan_bytes += scan_bytes; + } + + long get_scan_bytes() { + return scan_bytes; + } + + void set_process_rows(int64_t process_rows) { + this->process_rows = process_rows; + } + + void add_process_rows(int64_t process_rows) { + this->process_rows += process_rows; + } + + long get_process_rows() { + return process_rows; + } + + void serialize(PQueryStatistics* statistics) { + DCHECK(statistics != nullptr); + statistics->set_process_rows(process_rows); + statistics->set_scan_bytes(scan_bytes); + } + + void deserialize(const PQueryStatistics& statistics) { + process_rows = statistics.process_rows(); + scan_bytes = statistics.scan_bytes(); + } + + private: + + long process_rows; + long scan_bytes; + }; + + QueryStatistics() { + } + + void add(const QueryStatistics& other) { + boost::lock_guard l(_lock); + auto other_iter = other._statistics_map.begin(); + while (other_iter != other._statistics_map.end()) { + auto iter = _statistics_map.find(other_iter->first); + Statistics* statistics = nullptr; + if (iter == _statistics_map.end()) { + statistics = new Statistics(); + _statistics_map[other_iter->first] = statistics; + } else { + statistics = iter->second; + } + Statistics* other_statistic = other_iter->second; + statistics->add(*other_statistic); + other_iter++; + } + } + + void add_process_rows(long process_rows) { + boost::lock_guard l(_lock); + auto statistics = find(DEFAULT_SENDER_ID); + statistics->add_process_rows(process_rows); + } + + void add_scan_bytes(long scan_bytes) { + boost::lock_guard l(_lock); + auto statistics = find(DEFAULT_SENDER_ID); + statistics->add_scan_bytes(scan_bytes); + } + + void deserialize(const PQueryStatistics& other, int sender_id) { + boost::lock_guard l(_lock); + auto statistics = find(sender_id); + statistics->deserialize(other); + } + + void serialize(PQueryStatistics* statistics) { + boost::lock_guard l(_lock); + DCHECK(statistics != nullptr); + Statistics total_statistics = get_total_statistics(); + total_statistics.serialize(statistics); + } + + long get_process_rows() { + boost::lock_guard l(_lock); + Statistics statistics = get_total_statistics(); + return statistics.get_process_rows(); + } + + long get_scan_bytes() { + boost::lock_guard l(_lock); + Statistics statistics = get_total_statistics(); + return statistics.get_scan_bytes(); + } + + long get_process_rows(int sender_id) { + boost::lock_guard l(_lock); + auto statistics = find(sender_id); + return statistics->get_process_rows(); + } + + long get_scan_bytes(int sender_id) { + boost::lock_guard l(_lock); + auto statistics = find(sender_id); + return statistics->get_scan_bytes(); + } + + void clear() { + boost::lock_guard l(_lock); + auto iter = _statistics_map.begin(); + while (iter != _statistics_map.end()) { + iter->second->reset(); + iter++; + } + } + + ~QueryStatistics() { + boost::lock_guard l(_lock); + auto iter = _statistics_map.begin(); + while (iter != _statistics_map.end()) { + delete iter->second; + iter++; + } + } + +private: + + Statistics* find(int sender_id) { + auto iter = _statistics_map.find(sender_id); + Statistics* statistics = nullptr; + if (iter == _statistics_map.end()) { + statistics = new Statistics(); + _statistics_map[sender_id] = statistics; + } else { + statistics = iter->second; + } + return statistics; + } + + Statistics get_total_statistics() { + Statistics total_statistics; + auto iter = _statistics_map.begin(); + while (iter != _statistics_map.end()) { + total_statistics.add(*(iter->second)); + iter++; + } + return total_statistics; + } + + // Map lock. + SpinLock _lock; + // Sender id to statistics. + std::map _statistics_map; + // To index current plan. + static const int DEFAULT_SENDER_ID = -1; +}; + +} + +#endif diff --git a/be/src/runtime/result_sink.cpp b/be/src/runtime/result_sink.cpp index 9f05f50200fb89..b25bd0ef2a3c27 100644 --- a/be/src/runtime/result_sink.cpp +++ b/be/src/runtime/result_sink.cpp @@ -92,8 +92,8 @@ Status ResultSink::close(RuntimeState* state, Status exec_status) { return Status::OK; } -void ResultSink::set_query_statistic(boost::shared_ptr statistic) { - _sender->set_query_statistic(statistic); +void ResultSink::set_query_statistics(std::shared_ptr statistics) { + _sender->set_query_statistics(statistics); } } diff --git a/be/src/runtime/result_sink.h b/be/src/runtime/result_sink.h index f5c1d7cc83d200..f9c861d8a20e1b 100644 --- a/be/src/runtime/result_sink.h +++ b/be/src/runtime/result_sink.h @@ -56,7 +56,7 @@ class ResultSink : public DataSink { return _profile; } - virtual void set_query_statistic(boost::shared_ptr statistic); + virtual void set_query_statistics(std::shared_ptr statistics); private: Status prepare_exprs(RuntimeState* state); diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 97981cc6196147..accd2fb719bef9 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -32,7 +32,6 @@ #include "runtime/descriptors.h" #include "runtime/exec_env.h" #include "runtime/initial_reservations.h" -#include "runtime/query_statistic.h" #include "runtime/runtime_state.h" #include "runtime/load_path_mgr.h" #include "util/cpu_info.h" diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index aa4026fa47a976..b58ef954932035 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -60,7 +60,6 @@ class LoadErrorHub; class ReservationTracker; class InitialReservations; class RowDescriptor; -class QueryStatistic; // A collection of items that are part of the global state of a // query and shared across all execution nodes of that query. diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 1bf219f6b27bcb..23ebeb6ab28770 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -55,15 +55,15 @@ void PInternalServiceImpl::transmit_data(google::protobuf::RpcController* cnt eos ? nullptr : &done); } - if (request->has_query_statistic()) { + if (request->has_query_statistics()) { TUniqueId finst_id; finst_id.__set_hi(request->finst_id().hi()); finst_id.__set_lo(request->finst_id().lo()); - _exec_env->stream_mgr()->update_query_statistic( + _exec_env->stream_mgr()->update_query_statistics( finst_id, request->node_id(), request->sender_id(), - request->query_statistic()); + request->query_statistics()); } if (eos) { diff --git a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryFragmentProcNode.java b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryFragmentProcNode.java index 4d995696fa50da..4136d756fc5682 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryFragmentProcNode.java +++ b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryFragmentProcNode.java @@ -39,7 +39,7 @@ public class CurrentQueryFragmentProcNode implements ProcNodeInterface { private static final Logger LOG = LogManager.getLogger(CurrentQueryFragmentProcNode.class); public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("FragmentId").add("InstanceId").add("Host") - .add("IO").add("CPU").build(); + .add("ScanRawData").add("ProcessRows").build(); private QueryStatisticsItem item; public CurrentQueryFragmentProcNode(QueryStatisticsItem item) { @@ -79,8 +79,8 @@ private ProcResult requestFragmentExecInfos() throws AnalysisException { rowData.add(instanceConsumption.getFragmentId()); rowData.add(instanceConsumption.getInstanceId().toString()); rowData.add(instanceConsumption.getAddress().toString()); - rowData.add(instanceConsumption.getFormattingIo()); - rowData.add(instanceConsumption.getFormattingCpu()); + rowData.add(instanceConsumption.getFormattingScanBytes()); + rowData.add(instanceConsumption.getFormattingProcessRows()); sortedRowDatas.add(rowData); } diff --git a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java index 9827e5ee3ef779..a5922c9263229e 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java +++ b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java @@ -351,13 +351,13 @@ private long getTotalIoConsumption() { return io; } - public String getFormattingCpu() { + public String getFormattingProcessRows() { final StringBuilder builder = new StringBuilder(); builder.append(getTotalCpuConsumption()).append(" Rows"); return builder.toString(); } - public String getFormattingIo() { + public String getFormattingScanBytes() { final Pair pair = DebugUtil.getByteUint(getTotalIoConsumption()); final Formatter fmt = new Formatter(); final StringBuilder builder = new StringBuilder(); diff --git a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryStatisticsProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryStatisticsProcDir.java index 79808423e30417..02257be5673901 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryStatisticsProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryStatisticsProcDir.java @@ -37,7 +37,7 @@ public class CurrentQueryStatisticsProcDir implements ProcDirInterface { private static final Logger LOG = LogManager.getLogger(CurrentQueryStatisticsProcDir.class); public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("ConnectionId").add("QueryId").add("Database").add("User") - .add("IO").add("CPU").add("ExecTime").build(); + .add("ScanRawData").add("ProcessRows").add("ExecTime").build(); private static final int EXEC_TIME_INDEX = 6; @@ -76,8 +76,8 @@ public ProcResult fetchResult() throws AnalysisException { values.add(item.getDb()); values.add(item.getUser()); final CurrentQueryInfoProvider.Consumption consumption = consumptions.get(item.getQueryId()); - values.add(consumption.getFormattingIo()); - values.add(consumption.getFormattingCpu()); + values.add(consumption.getFormattingScanBytes()); + values.add(consumption.getFormattingProcessRows()); values.add(item.getQueryExecTime()); sortedRowData.add(values); } diff --git a/fe/src/main/java/org/apache/doris/planner/Planner.java b/fe/src/main/java/org/apache/doris/planner/Planner.java index e2a501f758b70f..66b676d8431589 100644 --- a/fe/src/main/java/org/apache/doris/planner/Planner.java +++ b/fe/src/main/java/org/apache/doris/planner/Planner.java @@ -248,7 +248,7 @@ public void optimizeTransferQueryStatistic() { } private void optimizeTransferQueryStatistic(PlanFragment fragment, PlanFragment parent) { - if (parent != null && hasLimit(parent.getPlanRoot())) { + if (parent != null && hasLimit(parent.getPlanRoot(), fragment.getPlanRoot())) { fragment.setTransferQueryStatisticWithEveryBatch(true); } for (PlanFragment child : fragment.getChildren()) { @@ -256,16 +256,31 @@ private void optimizeTransferQueryStatistic(PlanFragment fragment, PlanFragment } } - private boolean hasLimit(PlanNode planNode) { - if (planNode.hasLimit()) { - return true; - } - for (PlanNode child : planNode.getChildren()) { - if (hasLimit(child)) { + // Check whether leaf node contains limit. + private boolean hasLimit(PlanNode ancestor, PlanNode successor) { + final List leaves = Lists.newArrayList(); + collectExchangeNode(ancestor, leaves); + for (PlanNode leaf : leaves) { + if (leaf.getChild(0) == successor + && leaf.hasLimit()) { return true; - }; + } } return false; } + + private void collectExchangeNode(PlanNode planNode, List leaves) { + if (planNode instanceof ExchangeNode) { + leaves.add(planNode); + } + + for (PlanNode child : planNode.getChildren()) { + if (child instanceof ExchangeNode) { + leaves.add(child); + } else { + collectExchangeNode(child, leaves); + } + } + } } } diff --git a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 0f82109c50d2d2..de6fbe83249a6f 100644 --- a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -94,15 +94,15 @@ private void handlePing() { } private void auditAfterExec(String origStmt, StatementBase parsedStmt, - StmtExecutor.QueryStatistic queryStatistic) { + StmtExecutor.QueryStatistics statistics) { // slow query long elapseMs = System.currentTimeMillis() - ctx.getStartTime(); // query state log ctx.getAuditBuilder().put("state", ctx.getState()); ctx.getAuditBuilder().put("time", elapseMs); - Preconditions.checkNotNull(queryStatistic); - ctx.getAuditBuilder().put("cpu", queryStatistic.getFormattingCpu()); - ctx.getAuditBuilder().put("io", queryStatistic.getFormattingIo()); + Preconditions.checkNotNull(statistics); + ctx.getAuditBuilder().put("ProcessRows", statistics.getFormattingCpu()); + ctx.getAuditBuilder().put("ScanRawData", statistics.getFormattingIo()); ctx.getAuditBuilder().put("returnRows", ctx.getReturnRows()); ctx.getAuditBuilder().put("stmt_id", ctx.getStmtId()); @@ -183,7 +183,7 @@ private void handleQuery() { // audit after exec // replace '\n' to '\\n' to make string in one line auditAfterExec(stmt.replace("\n", " \\n"), executor.getParsedStmt(), - executor.getQueryStatisticForAuditLog()); + executor.getQueryStatisticsForAuditLog()); } // Get the column definitions of a table diff --git a/fe/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/src/main/java/org/apache/doris/qe/Coordinator.java index 11317ac63b07cd..9c9a66dd5bc824 100644 --- a/fe/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/src/main/java/org/apache/doris/qe/Coordinator.java @@ -1578,7 +1578,7 @@ List toThrift(int backendNum) { params.setBackend_num(backendNum++); params.setQuery_globals(queryGlobals); params.setQuery_options(queryOptions); - params.params.setSend_query_statistic_with_every_batch( + params.params.setSend_query_statistics_with_every_batch( fragment.isTransferQueryStatisticWithEveryBatch()); if (queryOptions.getQuery_type() == TQueryType.LOAD) { LoadErrorHub.Param param = Catalog.getCurrentCatalog().getLoadInstance().getLoadErrorHubInfo(); diff --git a/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java b/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java index 993f47171e3729..dee56553abf9bc 100644 --- a/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java +++ b/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java @@ -92,8 +92,8 @@ public RowBatch getNext(Status status) throws TException { return null; } - if (pResult.queryStatistic != null) { - rowBatch.setQueryStatistic(pResult.queryStatistic); + if (pResult.statistics != null) { + rowBatch.setQueryStatistics(pResult.statistics); } if (packetIdx != pResult.packetSeq) { diff --git a/fe/src/main/java/org/apache/doris/qe/RowBatch.java b/fe/src/main/java/org/apache/doris/qe/RowBatch.java index deeba8bef4920e..98bdfcf10888ab 100644 --- a/fe/src/main/java/org/apache/doris/qe/RowBatch.java +++ b/fe/src/main/java/org/apache/doris/qe/RowBatch.java @@ -17,12 +17,12 @@ package org.apache.doris.qe; -import org.apache.doris.rpc.PQueryStatistic; +import org.apache.doris.rpc.PQueryStatistics; import org.apache.doris.thrift.TResultBatch; public final class RowBatch { private TResultBatch batch; - private PQueryStatistic queryStatistic; + private PQueryStatistics statistics; private boolean eos; public RowBatch() { @@ -37,12 +37,12 @@ public void setBatch(TResultBatch batch) { this.batch = batch; } - public PQueryStatistic getQueryStatistic() { - return queryStatistic; + public PQueryStatistics getQueryStatistics() { + return statistics; } - public void setQueryStatistic(PQueryStatistic queryStatistic) { - this.queryStatistic = queryStatistic; + public void setQueryStatistics(PQueryStatistics statistics) { + this.statistics = statistics; } public boolean isEos() { diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 5884a0e7109b41..57985c7207de15 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -65,7 +65,7 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.planner.Planner; import org.apache.doris.rewrite.ExprRewriter; -import org.apache.doris.rpc.PQueryStatistic; +import org.apache.doris.rpc.PQueryStatistics; import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TQueryOptions; @@ -105,7 +105,7 @@ public class StmtExecutor { private Planner planner; private boolean isProxy; private ShowResultSet proxyResultSet = null; - private QueryStatistic statisticForAuditLog; + private QueryStatistics statisticsForAuditLog; public StmtExecutor(ConnectContext context, String stmt, boolean isProxy) { this.context = context; @@ -565,8 +565,8 @@ private void handleQueryStmt() throws Exception { private void setConsumptionForAuditLog(RowBatch batch) { if (batch != null) { - final PQueryStatistic queryStatistic = batch.getQueryStatistic(); - statisticForAuditLog = new QueryStatistic(queryStatistic.cpu, queryStatistic.io); + final PQueryStatistics statistics = batch.getQueryStatistics(); + statisticsForAuditLog = new QueryStatistics(statistics.processRows, statistics.scanBytes); } } @@ -788,35 +788,35 @@ private void handleExportStmt() throws Exception { context.getCatalog().getExportMgr().addExportJob(exportStmt); } - public QueryStatistic getQueryStatisticForAuditLog() { - if (statisticForAuditLog == null) { - statisticForAuditLog = new QueryStatistic(); + public QueryStatistics getQueryStatisticsForAuditLog() { + if (statisticsForAuditLog == null) { + statisticsForAuditLog = new QueryStatistics(); } - return statisticForAuditLog; + return statisticsForAuditLog; } - public static class QueryStatistic { - private final long cpuByRow; - private final long ioByByte; + public static class QueryStatistics { + private final long processRows; + private final long scanBytes; - public QueryStatistic() { - this.cpuByRow = 0; - this.ioByByte = 0; + public QueryStatistics() { + this.processRows = 0; + this.scanBytes = 0; } - public QueryStatistic(long cpuByRow, long ioByByte) { - this.cpuByRow = cpuByRow; - this.ioByByte = ioByByte; + public QueryStatistics(long processRows, long scanBytes) { + this.processRows = processRows; + this.scanBytes = scanBytes; } - public String getFormattingCpu() { + public String getFormattingProcessRows() { final StringBuilder builder = new StringBuilder(); - builder.append(cpuByRow).append(" Rows"); + builder.append(processRows).append(" Rows"); return builder.toString(); } - public String getFormattingIo() { - final Pair pair = DebugUtil.getByteUint(ioByByte); + public String getFormattingScanBytes() { + final Pair pair = DebugUtil.getByteUint(scanBytes); final Formatter fmt = new Formatter(); final StringBuilder builder = new StringBuilder(); builder.append(fmt.format("%.2f", pair.first)).append(" ").append(pair.second); diff --git a/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java b/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java index f8f46aad8f5f49..389327ea62c766 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java +++ b/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java @@ -29,5 +29,5 @@ public class PFetchDataResult { @Protobuf(order = 3, required = false) public boolean eos; @Protobuf(order = 4, required = false) - public PQueryStatistic queryStatistic; + public PQueryStatistics statistics; } diff --git a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistic.java b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java similarity index 92% rename from fe/src/main/java/org/apache/doris/rpc/PQueryStatistic.java rename to fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java index b95a45e2ff4971..5a328ecc301dea 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistic.java +++ b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java @@ -21,9 +21,9 @@ import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; @ProtobufClass -public class PQueryStatistic { +public class PQueryStatistics { @Protobuf(order = 1, required = false) - public long cpu; + public long processRows; @Protobuf(order = 2, required = false) - public long io; + public long scanBytes; } diff --git a/gensrc/proto/data.proto b/gensrc/proto/data.proto index 49fd79b7d042d9..5bcb34f387c750 100644 --- a/gensrc/proto/data.proto +++ b/gensrc/proto/data.proto @@ -19,9 +19,9 @@ syntax="proto2"; package doris; -message PQueryStatistic { - optional int64 cpu_by_row = 1; - optional int64 io_by_byte = 2; +message PQueryStatistics { + optional int64 process_rows = 1; + optional int64 scan_bytes = 2; } message PRowBatch { diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 255962232055bd..61ea6f3f6ca241 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -41,7 +41,7 @@ message PTransmitDataParams { optional PRowBatch row_batch = 6; // different per packet required int64 packet_seq = 7; - optional PQueryStatistic query_statistic = 8; + optional PQueryStatistics query_statistics = 8; }; message PTransmitDataResult { @@ -130,7 +130,7 @@ message PFetchDataResult { // valid when status is ok optional int64 packet_seq = 2; optional bool eos = 3; - optional PQueryStatistic query_statistic = 4; + optional PQueryStatistics query_statistics = 4; }; message PTriggerProfileReportRequest { diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 1599b7afedabee..25428d880589a7 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -175,7 +175,7 @@ struct TPlanFragmentExecParams { // Id of this fragment in its role as a sender. 9: optional i32 sender_id 10: optional i32 num_senders - 11: optional bool send_query_statistic_with_every_batch + 11: optional bool send_query_statistics_with_every_batch } // Global query parameters assigned by the coordinator. From 40f6069bd03cb7f19a6df44d9212ac0d0ca7ac86 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Tue, 15 Jan 2019 16:29:47 +0800 Subject: [PATCH 03/18] Modify variables name and add some comments --- be/src/exec/data_sink.cpp | 7 ++-- be/src/exec/data_sink.h | 1 + be/src/exec/exchange_node.cpp | 3 +- be/src/exec/exec_node.h | 6 ++-- .../exec/new_partitioned_aggregation_node.cc | 4 +-- be/src/runtime/data_stream_recvr.h | 3 -- be/src/runtime/data_stream_sender.cpp | 2 +- be/src/runtime/plan_fragment_executor.cpp | 12 +++---- be/src/runtime/plan_fragment_executor.h | 6 ++-- be/src/runtime/query_statistics.h | 19 ++++++----- .../common/proc/CurrentQueryInfoProvider.java | 30 ++++++++--------- .../apache/doris/planner/PlanFragment.java | 18 +++++------ .../org/apache/doris/planner/Planner.java | 32 +++++++++---------- .../org/apache/doris/qe/ConnectProcessor.java | 4 +-- .../java/org/apache/doris/qe/Coordinator.java | 2 +- .../org/apache/doris/qe/StmtExecutor.java | 4 +-- .../apache/doris/qe/ConnectProcessorTest.java | 6 ++-- .../org/apache/doris/qe/StmtExecutorTest.java | 1 - 18 files changed, 77 insertions(+), 83 deletions(-) diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index 1ef29f70ccb509..a470702e397206 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -49,11 +49,8 @@ Status DataSink::create_data_sink( if (!thrift_sink.__isset.stream_sink) { return Status("Missing data stream sink."); } - bool send_query_statistics_with_every_batch = false; - if (params.__isset.send_query_statistics_with_every_batch) { - send_query_statistics_with_every_batch = - params.send_query_statistics_with_every_batch; - } + bool send_query_statistics_with_every_batch = params.__isset.send_query_statistics_with_every_batch ? + params.send_query_statistics_with_every_batch : false; // TODO: figure out good buffer size based on size of output row tmp_sink = new DataStreamSender( pool, params.sender_id, row_desc, diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h index 3227e875bdf3e1..942e33bdaaf5fd 100644 --- a/be/src/exec/data_sink.h +++ b/be/src/exec/data_sink.h @@ -88,6 +88,7 @@ class DataSink { bool _closed; std::unique_ptr _expr_mem_tracker; + // Maybe this will be transfered to BufferControlBlock. std::shared_ptr _query_statistics; }; diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp index 0f48b4d9a4a664..7a1cd2b24b7b5a 100644 --- a/be/src/exec/exchange_node.cpp +++ b/be/src/exec/exchange_node.cpp @@ -99,8 +99,7 @@ Status ExchangeNode::open(RuntimeState* state) { Status ExchangeNode::collect_query_statistics(QueryStatistics* statistics) { RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); statistics->add_process_rows(_merge_rows_counter->value()); - QueryStatistics* sub_plan_statistics = _sub_plan_statistics.get(); - statistics->add(*sub_plan_statistics); + statistics->add(_sub_plan_statistics.get()); return Status::OK; } diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index a164efd53d80a1..0e9e265baaab85 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -115,9 +115,9 @@ class ExecNode { // so should be fast. virtual Status reset(RuntimeState* state); - // collect_query_statistics() should be called before close() and after get_next(), it - // responsible for collecting statisticss returned with query result, it can't be called - // when prepare() return error. + // This should be called before close() and after get_next(), it is responsible for + // collecting statistics sent with row batch, it can't be called when prepare() returns + // error. virtual Status collect_query_statistics(QueryStatistics* statistics); // close() will get called for every exec node, regardless of what else is called and diff --git a/be/src/exec/new_partitioned_aggregation_node.cc b/be/src/exec/new_partitioned_aggregation_node.cc index 0dd66165f7857f..2d7cc4a237bff9 100644 --- a/be/src/exec/new_partitioned_aggregation_node.cc +++ b/be/src/exec/new_partitioned_aggregation_node.cc @@ -593,7 +593,7 @@ bool NewPartitionedAggregationNode::ShouldExpandPreaggHashTables() const { ht_rows += ht->size(); } - // Need some rows in tables to have valid statisticss. + // Need some rows in tables to have valid statistics. if (ht_rows == 0) return true; // Find the appropriate reduction factor in our table for the current hash table sizes. @@ -621,7 +621,7 @@ bool NewPartitionedAggregationNode::ShouldExpandPreaggHashTables() const { // set, N is the number of input rows, excluding passed-through rows, and n is the // number of rows inserted or merged into the hash tables. This is a very rough // approximation but is good enough to be useful. - // TODO: consider collecting more statisticss to better estimate reduction. + // TODO: consider collecting more statistics to better estimate reduction. // double estimated_reduction = aggregated_input_rows >= expected_input_rows // ? current_reduction // : 1 + (expected_input_rows / aggregated_input_rows) * (current_reduction - 1); diff --git a/be/src/runtime/data_stream_recvr.h b/be/src/runtime/data_stream_recvr.h index 2d1b807398663e..c59e7b0390b111 100644 --- a/be/src/runtime/data_stream_recvr.h +++ b/be/src/runtime/data_stream_recvr.h @@ -104,9 +104,6 @@ class DataStreamRecvr { _sub_plan_query_statistics->deserialize(statistics, sender_id); } - QueryStatistics* get() { - return _sub_plan_query_statistics; - } private: friend class DataStreamMgr; class SenderQueue; diff --git a/be/src/runtime/data_stream_sender.cpp b/be/src/runtime/data_stream_sender.cpp index db5c475853e679..8845953069e6c1 100644 --- a/be/src/runtime/data_stream_sender.cpp +++ b/be/src/runtime/data_stream_sender.cpp @@ -168,7 +168,7 @@ class DataStreamSender::Channel { palo::PInternalService_Stub* _brpc_stub = nullptr; RefCountClosure* _closure = nullptr; int32_t _brpc_timeout_ms = 500; - // whether the dest can be treated as consumption transfer chain. + // whether the dest can be treated as query statistics transfer chain. bool _is_transfer_chain; bool _send_query_statistics_with_every_batch; }; diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index bc8a329d8af431..19d786d3de0298 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -55,7 +55,7 @@ PlanFragmentExecutor::PlanFragmentExecutor( _closed(false), _has_thread_token(false), _is_report_success(true), - _collect_query_statistics_every_batch(false) { + _collect_query_statistics_with_every_batch(false) { } PlanFragmentExecutor::~PlanFragmentExecutor() { @@ -198,10 +198,8 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) { profile()->add_child(sink_profile, true, NULL); } - if (params.__isset.send_query_statistics_with_every_batch) { - _collect_query_statistics_every_batch = - params.send_query_statistics_with_every_batch; - } + _collect_query_statistics_with_every_batch = params.__isset.send_query_statistics_with_every_batch ? + params.send_query_statistics_with_every_batch : false; } else { _sink.reset(NULL); } @@ -327,7 +325,7 @@ Status PlanFragmentExecutor::open_internal() { SCOPED_TIMER(profile()->total_time_counter()); // Collect this plan and sub plan statisticss, and send to parent plan. - if (_collect_query_statistics_every_batch) { + if (_collect_query_statistics_with_every_batch) { collect_query_statistics(); } RETURN_IF_ERROR(_sink->send(runtime_state(), batch)); @@ -346,7 +344,7 @@ Status PlanFragmentExecutor::open_internal() { // audit the sinks to check that this is ok, or change that behaviour. { SCOPED_TIMER(profile()->total_time_counter()); - collect_query_statistics() + collect_query_statistics(); Status status = _sink->close(runtime_state(), _status); RETURN_IF_ERROR(status); } diff --git a/be/src/runtime/plan_fragment_executor.h b/be/src/runtime/plan_fragment_executor.h index d298ae7e73a3fd..6301940305b529 100644 --- a/be/src/runtime/plan_fragment_executor.h +++ b/be/src/runtime/plan_fragment_executor.h @@ -205,10 +205,10 @@ class PlanFragmentExecutor { // of the execution. RuntimeProfile::Counter* _average_thread_tokens; - // This plan and it's sub plan query statisic. Because plan may have been finished - // when fe fetch data and query statistics, this will be shared with BufferControlBlock. + // This plan and it's sub plan query statisics. Because plan may have been finished + // when fe fetch data and query statistics, this will be shared with ResultSink. std::shared_ptr _query_statistics; - bool _collect_query_statistics_every_batch; + bool _collect_query_statistics_with_every_batch; ObjectPool* obj_pool() { return _runtime_state->obj_pool(); diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index d9e948199ae88b..40cd0b232a7097 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -26,7 +26,7 @@ namespace doris { // This is responsible for collecting query statistics, usually it consist of // two parts, one is current fragment or plan's statistics, the other is sub fragment -// or plan's statistics.Now cpu is measured by number of rows, io is measured by byte. +// or plan's statistics. class QueryStatistics { public: @@ -86,7 +86,7 @@ class QueryStatistics { scan_bytes = statistics.scan_bytes(); } - private: + private: long process_rows; long scan_bytes; @@ -95,10 +95,13 @@ class QueryStatistics { QueryStatistics() { } - void add(const QueryStatistics& other) { + // It can't be called by this and other at the same time in different threads. + // Otherwise it will cause dead lock. + void add(QueryStatistics* other) { boost::lock_guard l(_lock); - auto other_iter = other._statistics_map.begin(); - while (other_iter != other._statistics_map.end()) { + boost::lock_guard other_l(other->_lock); + auto other_iter = other->_statistics_map.begin(); + while (other_iter != other->_statistics_map.end()) { auto iter = _statistics_map.find(other_iter->first); Statistics* statistics = nullptr; if (iter == _statistics_map.end()) { @@ -107,8 +110,8 @@ class QueryStatistics { } else { statistics = iter->second; } - Statistics* other_statistic = other_iter->second; - statistics->add(*other_statistic); + Statistics* other_statistics = other_iter->second; + statistics->add(*other_statistics); other_iter++; } } @@ -132,8 +135,8 @@ class QueryStatistics { } void serialize(PQueryStatistics* statistics) { - boost::lock_guard l(_lock); DCHECK(statistics != nullptr); + boost::lock_guard l(_lock); Statistics total_statistics = get_total_statistics(); total_statistics.serialize(statistics); } diff --git a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java index a5922c9263229e..3cddd59a4ef0f1 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java +++ b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java @@ -338,7 +338,7 @@ private String parsePossibleExecNodeName(String str) { private long getTotalCpuConsumption() { long cpu = 0; for (ConsumptionCalculator consumption : calculators) { - cpu += consumption.getCpu(); + cpu += consumption.getProcessRows(); } return cpu; } @@ -346,7 +346,7 @@ private long getTotalCpuConsumption() { private long getTotalIoConsumption() { long io = 0; for (ConsumptionCalculator consumption : calculators) { - io += consumption.getIo(); + io += consumption.getScanBytes(); } return io; } @@ -403,27 +403,27 @@ public ConsumptionCalculator(List> counterMaps) { this.counterMaps = counterMaps; } - public long getCpu() { + public long getProcessRows() { long cpu = 0; for (Map counters : counterMaps) { - cpu += getCpuByRows(counters); + cpu += getProcessRows(counters); } return cpu; } - public long getIo() { + public long getScanBytes() { long io = 0; for (Map counters : counterMaps) { - io += getIoByByte(counters); + io += getScanBytes(counters); } return io; } - protected long getCpuByRows(Map counters) { + protected long getProcessRows(Map counters) { return 0; } - protected long getIoByByte(Map counters) { + protected long getScanBytes(Map counters) { return 0; } } @@ -434,7 +434,7 @@ public OlapScanNodeConsumptionCalculator(List> counterMaps) } @Override - protected long getIoByByte(Map counters) { + protected long getScanBytes(Map counters) { final Counter counter = counters.get("CompressedBytesRead"); return counter == null ? 0 : counter.getValue(); } @@ -446,7 +446,7 @@ public HashJoinConsumptionCalculator(List> counterMaps) { } @Override - protected long getCpuByRows(Map counters) { + protected long getProcessRows(Map counters) { final Counter probeCounter = counters.get("ProbeRows"); final Counter buildCounter = counters.get("BuildRows"); return probeCounter == null || buildCounter == null ? @@ -460,7 +460,7 @@ public HashAggConsumptionCalculator(List> counterMaps) { } @Override - protected long getCpuByRows(Map counters) { + protected long getProcessRows(Map counters) { final Counter buildCounter = counters.get("BuildRows"); return buildCounter == null ? 0 : buildCounter.getValue(); } @@ -472,7 +472,7 @@ public SortConsumptionCalculator(List> counterMaps) { } @Override - protected long getCpuByRows(Map counters) { + protected long getProcessRows(Map counters) { final Counter sortRowsCounter = counters.get("SortRows"); return sortRowsCounter == null ? 0 : sortRowsCounter.getValue(); } @@ -484,7 +484,7 @@ public WindowsConsumptionCalculator(List> counterMaps) { } @Override - protected long getCpuByRows(Map counters) { + protected long getProcessRows(Map counters) { final Counter processRowsCounter = counters.get("ProcessRows"); return processRowsCounter == null ? 0 : processRowsCounter.getValue(); @@ -497,7 +497,7 @@ public UnionConsumptionCalculator(List> counterMaps) { } @Override - protected long getCpuByRows(Map counters) { + protected long getProcessRows(Map counters) { final Counter materializeRowsCounter = counters.get("MaterializeRows"); return materializeRowsCounter == null ? 0 : materializeRowsCounter.getValue(); } @@ -510,7 +510,7 @@ public ExchangeConsumptionCalculator(List> counterMaps) { } @Override - protected long getCpuByRows(Map counters) { + protected long getProcessRows(Map counters) { final Counter mergeRowsCounter = counters.get("MergeRows"); return mergeRowsCounter == null ? 0 : mergeRowsCounter.getValue(); } diff --git a/fe/src/main/java/org/apache/doris/planner/PlanFragment.java b/fe/src/main/java/org/apache/doris/planner/PlanFragment.java index 93d487b395fbed..29cce7d71c5193 100644 --- a/fe/src/main/java/org/apache/doris/planner/PlanFragment.java +++ b/fe/src/main/java/org/apache/doris/planner/PlanFragment.java @@ -96,10 +96,10 @@ public class PlanFragment extends TreeNode { // if the output is UNPARTITIONED, it is being broadcast private DataPartition outputPartition; - // Whether query statistic is sent with every batch. In order to get the query - // statistic correctly when query contains limit, it is necessary to send query - // statistic in every batch, or only in close. - private boolean transferQueryStatisticWithEveryBatch; + // Whether query statistics is sent with every batch. In order to get the query + // statistics correctly when query contains limit, it is necessary to send query + // statistics with every batch, or only in close. + private boolean transferQueryStatisticsWithEveryBatch; // TODO: SubstitutionMap outputSmap; // substitution map to remap exprs onto the output of this fragment, to be applied @@ -113,7 +113,7 @@ public PlanFragment(PlanFragmentId id, PlanNode root, DataPartition partition) { this.planRoot = root; this.dataPartition = partition; this.outputPartition = DataPartition.UNPARTITIONED; - this.transferQueryStatisticWithEveryBatch = false; + this.transferQueryStatisticsWithEveryBatch = false; setFragmentInPlanTree(planRoot); } @@ -275,11 +275,11 @@ public PlanFragmentId getFragmentId() { return fragmentId; } - public void setTransferQueryStatisticWithEveryBatch(boolean value) { - transferQueryStatisticWithEveryBatch = value; + public void setTransferQueryStatisticsWithEveryBatch(boolean value) { + transferQueryStatisticsWithEveryBatch = value; } - public boolean isTransferQueryStatisticWithEveryBatch() { - return transferQueryStatisticWithEveryBatch; + public boolean isTransferQueryStatisticsWithEveryBatch() { + return transferQueryStatisticsWithEveryBatch; } } diff --git a/fe/src/main/java/org/apache/doris/planner/Planner.java b/fe/src/main/java/org/apache/doris/planner/Planner.java index 66b676d8431589..de6ee206aa3f98 100644 --- a/fe/src/main/java/org/apache/doris/planner/Planner.java +++ b/fe/src/main/java/org/apache/doris/planner/Planner.java @@ -178,8 +178,8 @@ public void createPlanFragments(StatementBase statment, Analyzer analyzer, TQuer // Optimize the transfer of query statistic when query does't contain limit. PlanFragment rootFragment = fragments.get(fragments.size() - 1); - QueryStatisticTransferOptimizer queryStatisticTransferOptimizer = new QueryStatisticTransferOptimizer(rootFragment); - queryStatisticTransferOptimizer.optimizeTransferQueryStatistic(); + QueryStatisticsTransferOptimizer queryStatisticTransferOptimizer = new QueryStatisticsTransferOptimizer(rootFragment); + queryStatisticTransferOptimizer.optimizeQueryStatisticsTransfer(); if (statment instanceof InsertStmt) { InsertStmt insertStmt = (InsertStmt) statment; @@ -235,32 +235,32 @@ private PlanNode addUnassignedConjuncts(Analyzer analyzer, PlanNode root) return selectNode; } - private static class QueryStatisticTransferOptimizer { + private static class QueryStatisticsTransferOptimizer { private final PlanFragment root; - public QueryStatisticTransferOptimizer(PlanFragment root) { + public QueryStatisticsTransferOptimizer(PlanFragment root) { Preconditions.checkNotNull(root); this.root = root; } - public void optimizeTransferQueryStatistic() { - optimizeTransferQueryStatistic(root, null); + public void optimizeQueryStatisticsTransfer() { + optimizeQueryStatisticsTransfer(root, null); } - private void optimizeTransferQueryStatistic(PlanFragment fragment, PlanFragment parent) { + private void optimizeQueryStatisticsTransfer(PlanFragment fragment, PlanFragment parent) { if (parent != null && hasLimit(parent.getPlanRoot(), fragment.getPlanRoot())) { - fragment.setTransferQueryStatisticWithEveryBatch(true); + fragment.setTransferQueryStatisticsWithEveryBatch(true); } for (PlanFragment child : fragment.getChildren()) { - optimizeTransferQueryStatistic(child, fragment); + optimizeQueryStatisticsTransfer(child, fragment); } } // Check whether leaf node contains limit. private boolean hasLimit(PlanNode ancestor, PlanNode successor) { - final List leaves = Lists.newArrayList(); - collectExchangeNode(ancestor, leaves); - for (PlanNode leaf : leaves) { + final List exchangeNodes = Lists.newArrayList(); + collectExchangeNode(ancestor, exchangeNodes); + for (PlanNode leaf : exchangeNodes) { if (leaf.getChild(0) == successor && leaf.hasLimit()) { return true; @@ -269,16 +269,16 @@ private boolean hasLimit(PlanNode ancestor, PlanNode successor) { return false; } - private void collectExchangeNode(PlanNode planNode, List leaves) { + private void collectExchangeNode(PlanNode planNode, List exchangeNodes) { if (planNode instanceof ExchangeNode) { - leaves.add(planNode); + exchangeNodes.add(planNode); } for (PlanNode child : planNode.getChildren()) { if (child instanceof ExchangeNode) { - leaves.add(child); + exchangeNodes.add(child); } else { - collectExchangeNode(child, leaves); + collectExchangeNode(child, exchangeNodes); } } } diff --git a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java index de6fbe83249a6f..f378d26264cb99 100644 --- a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -101,8 +101,8 @@ private void auditAfterExec(String origStmt, StatementBase parsedStmt, ctx.getAuditBuilder().put("state", ctx.getState()); ctx.getAuditBuilder().put("time", elapseMs); Preconditions.checkNotNull(statistics); - ctx.getAuditBuilder().put("ProcessRows", statistics.getFormattingCpu()); - ctx.getAuditBuilder().put("ScanRawData", statistics.getFormattingIo()); + ctx.getAuditBuilder().put("ProcessRows", statistics.getFormattingProcessRows()); + ctx.getAuditBuilder().put("ScanRawData", statistics.getFormattingScanBytes()); ctx.getAuditBuilder().put("returnRows", ctx.getReturnRows()); ctx.getAuditBuilder().put("stmt_id", ctx.getStmtId()); diff --git a/fe/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/src/main/java/org/apache/doris/qe/Coordinator.java index 9c9a66dd5bc824..a51c075fb69a02 100644 --- a/fe/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/src/main/java/org/apache/doris/qe/Coordinator.java @@ -1579,7 +1579,7 @@ List toThrift(int backendNum) { params.setQuery_globals(queryGlobals); params.setQuery_options(queryOptions); params.params.setSend_query_statistics_with_every_batch( - fragment.isTransferQueryStatisticWithEveryBatch()); + fragment.isTransferQueryStatisticsWithEveryBatch()); if (queryOptions.getQuery_type() == TQueryType.LOAD) { LoadErrorHub.Param param = Catalog.getCurrentCatalog().getLoadInstance().getLoadErrorHubInfo(); if (param != null) { diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 57985c7207de15..534675ffb0c061 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -556,14 +556,14 @@ private void handleQueryStmt() throws Exception { } context.updateReturnRows(batch.getBatch().getRows().size()); } - setConsumptionForAuditLog(batch); + setQueryStatisticsForAuditLog(batch); if (!isSendFields) { sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); } context.getState().setEof(); } - private void setConsumptionForAuditLog(RowBatch batch) { + private void setQueryStatisticsForAuditLog(RowBatch batch) { if (batch != null) { final PQueryStatistics statistics = batch.getQueryStatistics(); statisticsForAuditLog = new QueryStatistics(statistics.processRows, statistics.scanBytes); diff --git a/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java b/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java index 43eda0c3ceff1f..1d7a58c442c8cd 100644 --- a/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java @@ -231,7 +231,7 @@ public void testQuery() throws Exception { // Mock statement executor StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); - EasyMock.expect(qe.getQueryStatisticForAuditLog()).andReturn(new StmtExecutor.QueryStatistic()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new StmtExecutor.QueryStatistics()); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(qe); PowerMock.expectNew( @@ -255,7 +255,7 @@ public void testQueryFail() throws Exception { StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); EasyMock.expectLastCall().andThrow(new IOException("Fail")).anyTimes(); - EasyMock.expect(qe.getQueryStatisticForAuditLog()).andReturn(new StmtExecutor.QueryStatistic()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new StmtExecutor.QueryStatistics()); EasyMock.replay(qe); PowerMock.expectNew(StmtExecutor.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(String.class)) .andReturn(qe).anyTimes(); @@ -273,7 +273,7 @@ public void testQueryFail2() throws Exception { // Mock statement executor StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); - EasyMock.expect(qe.getQueryStatisticForAuditLog()).andReturn(new StmtExecutor.QueryStatistic()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new StmtExecutor.QueryStatistics()); EasyMock.expectLastCall().andThrow(new NullPointerException("Fail")).anyTimes(); EasyMock.replay(qe); PowerMock.expectNew(StmtExecutor.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(String.class)) diff --git a/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java b/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java index 0e35a7b7a5facf..2e9973f613e044 100644 --- a/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java @@ -160,7 +160,6 @@ public void testSelect() throws Exception { cood.endProfile(); EasyMock.expectLastCall().anyTimes(); EasyMock.expect(cood.getQueryProfile()).andReturn(new RuntimeProfile()).anyTimes(); - EasyMock.expect(cood.getNext()).andReturn(null).anyTimes(); EasyMock.replay(cood); PowerMock.expectNew(Coordinator.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(Analyzer.class), EasyMock.isA(Planner.class)) From 95c377304e147f9fe3476c4bb86d1111a16afe35 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 11:14:58 +0800 Subject: [PATCH 04/18] Modify QueryStatistics and fix unit test --- be/src/exec/analytic_eval_node.cpp | 6 - be/src/exec/analytic_eval_node.h | 1 - be/src/exec/exchange_node.cpp | 7 +- be/src/exec/exchange_node.h | 6 +- be/src/exec/hash_join_node.cpp | 7 - be/src/exec/hash_join_node.h | 1 - .../exec/new_partitioned_aggregation_node.cc | 6 - .../exec/new_partitioned_aggregation_node.h | 1 - be/src/exec/olap_scan_node.cpp | 1 + be/src/exec/olap_scan_node.h | 2 +- be/src/exec/sort_node.cpp | 7 - be/src/exec/sort_node.h | 1 - be/src/exec/union_node.cpp | 6 - be/src/exec/union_node.h | 1 - be/src/runtime/data_stream_mgr.cpp | 8 +- be/src/runtime/data_stream_mgr.h | 4 +- be/src/runtime/data_stream_recvr.cc | 4 +- be/src/runtime/data_stream_recvr.h | 10 +- be/src/runtime/query_statistics.h | 218 +++++------------- be/src/service/internal_service.cpp | 2 +- .../org/apache/doris/qe/ConnectProcessor.java | 2 +- .../org/apache/doris/qe/StmtExecutor.java | 14 +- .../apache/doris/rpc/PQueryStatistics.java | 2 +- .../org/apache/doris/qe/StmtExecutorTest.java | 1 + gensrc/proto/data.proto | 2 +- 25 files changed, 92 insertions(+), 228 deletions(-) diff --git a/be/src/exec/analytic_eval_node.cpp b/be/src/exec/analytic_eval_node.cpp index 9fae59bac4031c..99aa5e8b51a71d 100644 --- a/be/src/exec/analytic_eval_node.cpp +++ b/be/src/exec/analytic_eval_node.cpp @@ -839,12 +839,6 @@ Status AnalyticEvalNode::get_next(RuntimeState* state, RowBatch* row_batch, bool return Status::OK; } -Status AnalyticEvalNode::collect_query_statistics(QueryStatistics* statistics) { - RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); - statistics->add_process_rows(_process_rows_counter->value()); - return Status::OK; -} - Status AnalyticEvalNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK; diff --git a/be/src/exec/analytic_eval_node.h b/be/src/exec/analytic_eval_node.h index 1c47988c84497b..6b92c37ea57e08 100644 --- a/be/src/exec/analytic_eval_node.h +++ b/be/src/exec/analytic_eval_node.h @@ -68,7 +68,6 @@ class AnalyticEvalNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); protected: diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp index 7a1cd2b24b7b5a..e9b2a8f633cb62 100644 --- a/be/src/exec/exchange_node.cpp +++ b/be/src/exec/exchange_node.cpp @@ -65,14 +65,14 @@ Status ExchangeNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::prepare(state)); _convert_row_batch_timer = ADD_TIMER(runtime_profile(), "ConvertRowBatchTime"); _merge_rows_counter = ADD_COUNTER(runtime_profile(), "MergeRows", TUnit::UNIT); - _sub_plan_statistics.reset(new QueryStatistics()); // TODO: figure out appropriate buffer size DCHECK_GT(_num_senders, 0); + _sub_plan_query_statistics_recvr.reset(new QueryStatisticsRecvr()); _stream_recvr = state->exec_env()->stream_mgr()->create_recvr( state, _input_row_desc, state->fragment_instance_id(), _id, _num_senders, config::exchg_node_buffer_size_bytes, - state->runtime_profile(), _is_merging, _sub_plan_statistics.get()); + state->runtime_profile(), _is_merging, _sub_plan_query_statistics_recvr.get()); if (_is_merging) { RETURN_IF_ERROR(_sort_exec_exprs.prepare( state, _row_descriptor, _row_descriptor, expr_mem_tracker())); @@ -98,8 +98,7 @@ Status ExchangeNode::open(RuntimeState* state) { Status ExchangeNode::collect_query_statistics(QueryStatistics* statistics) { RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); - statistics->add_process_rows(_merge_rows_counter->value()); - statistics->add(_sub_plan_statistics.get()); + _sub_plan_query_statistics_recvr->add_to(statistics); return Status::OK; } diff --git a/be/src/exec/exchange_node.h b/be/src/exec/exchange_node.h index c31951f2f1e769..29d5ae2c1beac6 100644 --- a/be/src/exec/exchange_node.h +++ b/be/src/exec/exchange_node.h @@ -49,7 +49,7 @@ class ExchangeNode : public ExecNode { // Blocks until the first batch is available for consumption via GetNext(). virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistics(QueryStatistics* statistics); + Status collect_query_statistics(QueryStatistics* statistics) override; virtual Status close(RuntimeState* state); // the number of senders needs to be set after the c'tor, because it's not @@ -112,8 +112,8 @@ class ExchangeNode : public ExecNode { RuntimeProfile::Counter* _merge_rows_counter; - // Query statistics from sub plan. - boost::scoped_ptr _sub_plan_statistics; + // Sub plan query statistics receiver. + boost::scoped_ptr _sub_plan_query_statistics_recvr; }; }; diff --git a/be/src/exec/hash_join_node.cpp b/be/src/exec/hash_join_node.cpp index a6a5e7c17ca7cb..e24842af8af619 100644 --- a/be/src/exec/hash_join_node.cpp +++ b/be/src/exec/hash_join_node.cpp @@ -177,13 +177,6 @@ Status HashJoinNode::prepare(RuntimeState* state) { return Status::OK; } -Status HashJoinNode::collect_query_statistics(QueryStatistics* statistics) { - RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); - statistics->add_process_rows(_probe_rows_counter->value()); - statistics->add_process_rows(_build_rows_counter->value()); - return Status::OK; -} - Status HashJoinNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK; diff --git a/be/src/exec/hash_join_node.h b/be/src/exec/hash_join_node.h index a45d120db4c6b6..1f054d24099563 100644 --- a/be/src/exec/hash_join_node.h +++ b/be/src/exec/hash_join_node.h @@ -56,7 +56,6 @@ class HashJoinNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); static const char* _s_llvm_class_name; diff --git a/be/src/exec/new_partitioned_aggregation_node.cc b/be/src/exec/new_partitioned_aggregation_node.cc index 2d7cc4a237bff9..72ba8876e8b3ff 100644 --- a/be/src/exec/new_partitioned_aggregation_node.cc +++ b/be/src/exec/new_partitioned_aggregation_node.cc @@ -671,12 +671,6 @@ Status NewPartitionedAggregationNode::reset(RuntimeState* state) { return ExecNode::reset(state); } -Status NewPartitionedAggregationNode::collect_query_statistics(QueryStatistics* statistics) { - RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); - statistics->add_process_rows(_build_rows_counter->value()); - return Status::OK; -} - Status NewPartitionedAggregationNode::close(RuntimeState* state) { if (is_closed()) return Status::OK; diff --git a/be/src/exec/new_partitioned_aggregation_node.h b/be/src/exec/new_partitioned_aggregation_node.h index f613ef19a6c514..227cbed44baed8 100644 --- a/be/src/exec/new_partitioned_aggregation_node.h +++ b/be/src/exec/new_partitioned_aggregation_node.h @@ -137,7 +137,6 @@ class NewPartitionedAggregationNode : public ExecNode { virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); virtual Status reset(RuntimeState* state); - virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); static const char* LLVM_CLASS_NAME; diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp index 3d98c3a7fa78ac..7de863ecef7968 100644 --- a/be/src/exec/olap_scan_node.cpp +++ b/be/src/exec/olap_scan_node.cpp @@ -309,6 +309,7 @@ Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo Status OlapScanNode::collect_query_statistics(QueryStatistics* statistics) { RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); statistics->add_scan_bytes(_read_compressed_counter->value()); + statistics->add_scan_rows(rows_returned()); return Status::OK; } diff --git a/be/src/exec/olap_scan_node.h b/be/src/exec/olap_scan_node.h index 1a08b176b9ab38..a5227e87e3cbdc 100644 --- a/be/src/exec/olap_scan_node.h +++ b/be/src/exec/olap_scan_node.h @@ -56,7 +56,7 @@ class OlapScanNode : public ScanNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistics(QueryStatistics* statistics); + Status collect_query_statistics(QueryStatistics* statistics) override; virtual Status close(RuntimeState* state); virtual Status set_scan_ranges(const std::vector& scan_ranges); diff --git a/be/src/exec/sort_node.cpp b/be/src/exec/sort_node.cpp index d6a0036d3e8868..3b816e9edcfb59 100644 --- a/be/src/exec/sort_node.cpp +++ b/be/src/exec/sort_node.cpp @@ -117,13 +117,6 @@ Status SortNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) { return Status::OK; } - -Status SortNode::collect_query_statistics(QueryStatistics* statistics) { - RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); - statistics->add_process_rows(_sort_rows_counter->value()); - return Status::OK; -} - Status SortNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK; diff --git a/be/src/exec/sort_node.h b/be/src/exec/sort_node.h index ee52549fbd0d59..68dcfd8edbe430 100644 --- a/be/src/exec/sort_node.h +++ b/be/src/exec/sort_node.h @@ -42,7 +42,6 @@ class SortNode : public ExecNode { virtual Status prepare(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistics(QueryStatistics* statistics); virtual Status close(RuntimeState* state); protected: diff --git a/be/src/exec/union_node.cpp b/be/src/exec/union_node.cpp index 6e0df5fae28126..a1494cc1ef8017 100644 --- a/be/src/exec/union_node.cpp +++ b/be/src/exec/union_node.cpp @@ -337,12 +337,6 @@ Status UnionNode::reset(RuntimeState* state) { } #endif -Status UnionNode::collect_query_statistics(QueryStatistics* statistics) { - RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); - statistics->add_process_rows(_materialize_rows_counter->value()); - return Status::OK; -} - Status UnionNode::close(RuntimeState* state) { if (is_closed()) return Status::OK; _child_batch.reset(); diff --git a/be/src/exec/union_node.h b/be/src/exec/union_node.h index a67aa4ed56e60a..7daf2a623ae645 100644 --- a/be/src/exec/union_node.h +++ b/be/src/exec/union_node.h @@ -48,7 +48,6 @@ class UnionNode : public ExecNode { virtual void codegen(RuntimeState* state); virtual Status open(RuntimeState* state); virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status collect_query_statistics(QueryStatistics* statistics); // virtual Status reset(RuntimeState* state); virtual Status close(RuntimeState* state); diff --git a/be/src/runtime/data_stream_mgr.cpp b/be/src/runtime/data_stream_mgr.cpp index c8781c6d75f1f0..99b44c19f14be0 100644 --- a/be/src/runtime/data_stream_mgr.cpp +++ b/be/src/runtime/data_stream_mgr.cpp @@ -51,14 +51,14 @@ inline uint32_t DataStreamMgr::get_hash_value( shared_ptr DataStreamMgr::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, QueryStatistics* query_statistics) { + bool is_merging, QueryStatisticsRecvr* sub_plan_query_statistics_recvr) { DCHECK(profile != NULL); VLOG_FILE << "creating receiver for fragment=" << fragment_instance_id << ", node=" << dest_node_id; shared_ptr recvr( new DataStreamRecvr(this, state->instance_mem_tracker(), row_desc, fragment_instance_id, dest_node_id, num_senders, is_merging, buffer_size, - profile, query_statistics)); + profile, sub_plan_query_statistics_recvr)); uint32_t hash_value = get_hash_value(fragment_instance_id, dest_node_id); lock_guard l(_lock); _fragment_stream_set.insert(std::make_pair(fragment_instance_id, dest_node_id)); @@ -118,11 +118,11 @@ Status DataStreamMgr::add_data( return Status::OK; } -Status DataStreamMgr::update_query_statistics(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, +Status DataStreamMgr::add_query_statistics(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int sender_id, const PQueryStatistics& query_statistics) { shared_ptr recvr = find_recvr(fragment_instance_id, dest_node_id); if (recvr != NULL) { - recvr->update_sub_plan_statistics(query_statistics, sender_id); + recvr->add_sub_plan_statistics(query_statistics, sender_id); } return Status::OK; } diff --git a/be/src/runtime/data_stream_mgr.h b/be/src/runtime/data_stream_mgr.h index 2c26053ce02d52..fcd2661b798983 100644 --- a/be/src/runtime/data_stream_mgr.h +++ b/be/src/runtime/data_stream_mgr.h @@ -77,7 +77,7 @@ class DataStreamMgr { 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, QueryStatistics* query_statistics); + bool is_merging, QueryStatisticsRecvr* sub_plan_query_statistics_recvr); // Adds a row batch to the recvr identified by fragment_instance_id/dest_node_id // if the recvr has not been cancelled. sender_id identifies the sender instance @@ -99,7 +99,7 @@ class DataStreamMgr { Status close_sender(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int sender_id, int be_number); - Status update_query_statistics(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, + Status add_query_statistics(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int sender_id, const PQueryStatistics& query_statistics); // Closes all receivers registered for fragment_instance_id immediately. diff --git a/be/src/runtime/data_stream_recvr.cc b/be/src/runtime/data_stream_recvr.cc index 4bf10a1c98f9e7..c77099d8130545 100644 --- a/be/src/runtime/data_stream_recvr.cc +++ b/be/src/runtime/data_stream_recvr.cc @@ -351,7 +351,7 @@ DataStreamRecvr::DataStreamRecvr( DataStreamMgr* stream_mgr, MemTracker* parent_tracker, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, bool is_merging, int total_buffer_limit, - RuntimeProfile* profile, QueryStatistics* query_statistics) : + RuntimeProfile* profile, QueryStatisticsRecvr* sub_plan_query_statistics_recvr) : _mgr(stream_mgr), _fragment_instance_id(fragment_instance_id), _dest_node_id(dest_node_id), @@ -360,7 +360,7 @@ DataStreamRecvr::DataStreamRecvr( _is_merging(is_merging), _num_buffered_bytes(0), _profile(profile), - _sub_plan_query_statistics(query_statistics) { + _sub_plan_query_statistics_recvr(sub_plan_query_statistics_recvr) { _mem_tracker.reset(new MemTracker(-1, "DataStreamRecvr", parent_tracker)); // Create one queue per sender if is_merging is true. diff --git a/be/src/runtime/data_stream_recvr.h b/be/src/runtime/data_stream_recvr.h index c59e7b0390b111..226054f8d47cb4 100644 --- a/be/src/runtime/data_stream_recvr.h +++ b/be/src/runtime/data_stream_recvr.h @@ -100,8 +100,8 @@ class DataStreamRecvr { const RowDescriptor& row_desc() const { return _row_desc; } MemTracker* mem_tracker() const { return _mem_tracker.get(); } - void update_sub_plan_statistics(const PQueryStatistics& statistics, int sender_id) { - _sub_plan_query_statistics->deserialize(statistics, sender_id); + void add_sub_plan_statistics(const PQueryStatistics& statistics, int sender_id) { + _sub_plan_query_statistics_recvr->insert(statistics, sender_id); } private: @@ -111,7 +111,7 @@ class DataStreamRecvr { DataStreamRecvr(DataStreamMgr* stream_mgr, MemTracker* parent_tracker, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, bool is_merging, int total_buffer_limit, - RuntimeProfile* profile, QueryStatistics* query_statistics); + RuntimeProfile* profile, QueryStatisticsRecvr* sub_plan_query_statistics_recvr); // If receive queue is full, done is enqueue pending, and return with *done is nullptr void add_batch(const PRowBatch& batch, int sender_id, @@ -199,8 +199,8 @@ class DataStreamRecvr { // Wall time senders spend waiting for the recv buffer to have capacity. RuntimeProfile::Counter* _buffer_full_wall_timer; - // Query statistics returned with batch - QueryStatistics* _sub_plan_query_statistics; + // Sub plan query statistics receiver. + QueryStatisticsRecvr* _sub_plan_query_statistics_recvr; // Total time spent waiting for data to arrive in the recv buffer // RuntimeProfile::Counter* _data_arrival_timer; diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index 40cd0b232a7097..41b921014eb56e 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -18,201 +18,101 @@ #ifndef DORIS_BE_EXEC_QUERY_STATISTICS_H #define DORIS_BE_EXEC_QUERY_STATISTICS_H -#include "common/atomic.h" #include "gen_cpp/data.pb.h" #include "util/spinlock.h" namespace doris { -// This is responsible for collecting query statistics, usually it consist of +// This is responsible for collecting query statistics, usually it consists of // two parts, one is current fragment or plan's statistics, the other is sub fragment -// or plan's statistics. +// or plan's statistics and QueryStatisticsRecvr is responsible for collecting it. class QueryStatistics { public: - class Statistics { - public: - - Statistics() : process_rows(0), scan_bytes(0) { - } - - void add(const Statistics& other) { - process_rows += other.process_rows; - scan_bytes += other.scan_bytes; - } - - void add(const PQueryStatistics& other) { - process_rows += other.process_rows(); - scan_bytes += other.scan_bytes(); - } - - void reset() { - process_rows = 0; - scan_bytes = 0; - } - - void set_scan_bytes(int64_t scan_bytes) { - this->scan_bytes = scan_bytes; - } - - void add_scan_bytes(int64_t scan_bytes) { - this->scan_bytes += scan_bytes; - } - - long get_scan_bytes() { - return scan_bytes; - } - - void set_process_rows(int64_t process_rows) { - this->process_rows = process_rows; - } - - void add_process_rows(int64_t process_rows) { - this->process_rows += process_rows; - } - - long get_process_rows() { - return process_rows; - } - - void serialize(PQueryStatistics* statistics) { - DCHECK(statistics != nullptr); - statistics->set_process_rows(process_rows); - statistics->set_scan_bytes(scan_bytes); - } - - void deserialize(const PQueryStatistics& statistics) { - process_rows = statistics.process_rows(); - scan_bytes = statistics.scan_bytes(); - } - - private: - - long process_rows; - long scan_bytes; - }; - - QueryStatistics() { + QueryStatistics() : scan_rows(0), scan_bytes(0) { } - // It can't be called by this and other at the same time in different threads. - // Otherwise it will cause dead lock. - void add(QueryStatistics* other) { - boost::lock_guard l(_lock); - boost::lock_guard other_l(other->_lock); - auto other_iter = other->_statistics_map.begin(); - while (other_iter != other->_statistics_map.end()) { - auto iter = _statistics_map.find(other_iter->first); - Statistics* statistics = nullptr; - if (iter == _statistics_map.end()) { - statistics = new Statistics(); - _statistics_map[other_iter->first] = statistics; - } else { - statistics = iter->second; - } - Statistics* other_statistics = other_iter->second; - statistics->add(*other_statistics); - other_iter++; - } + void add(const QueryStatistics& other) { + scan_rows += other.scan_rows; + scan_bytes += other.scan_bytes; } - void add_process_rows(long process_rows) { - boost::lock_guard l(_lock); - auto statistics = find(DEFAULT_SENDER_ID); - statistics->add_process_rows(process_rows); - } + void add(const PQueryStatistics& other) { + scan_rows += other.scan_rows(); + scan_bytes += other.scan_bytes(); + } + + void add_scan_rows(long scan_rows) { + this->scan_rows += scan_rows; + } void add_scan_bytes(long scan_bytes) { - boost::lock_guard l(_lock); - auto statistics = find(DEFAULT_SENDER_ID); - statistics->add_scan_bytes(scan_bytes); + this->scan_bytes += scan_bytes; } - void deserialize(const PQueryStatistics& other, int sender_id) { - boost::lock_guard l(_lock); - auto statistics = find(sender_id); - statistics->deserialize(other); + void clear() { + scan_rows = 0; + scan_bytes = 0; } - void serialize(PQueryStatistics* statistics) { + void serialize(PQueryStatistics* statistics) { DCHECK(statistics != nullptr); - boost::lock_guard l(_lock); - Statistics total_statistics = get_total_statistics(); - total_statistics.serialize(statistics); + statistics->set_scan_rows(scan_rows); + statistics->set_scan_bytes(scan_bytes); } - long get_process_rows() { - boost::lock_guard l(_lock); - Statistics statistics = get_total_statistics(); - return statistics.get_process_rows(); + void deserialize(const PQueryStatistics& statistics) { + scan_rows = statistics.scan_rows(); + scan_bytes = statistics.scan_bytes(); } - long get_scan_bytes() { - boost::lock_guard l(_lock); - Statistics statistics = get_total_statistics(); - return statistics.get_scan_bytes(); - } +private: - long get_process_rows(int sender_id) { - boost::lock_guard l(_lock); - auto statistics = find(sender_id); - return statistics->get_process_rows(); - } + long scan_rows; + long scan_bytes; +}; + +// It is used for collecting sub plan query statistics in DataStreamRecvr. +class QueryStatisticsRecvr { +public: - long get_scan_bytes(int sender_id) { + void insert(const PQueryStatistics& statistics, int sender_id) { boost::lock_guard l(_lock); - auto statistics = find(sender_id); - return statistics->get_scan_bytes(); + QueryStatistics* query_statistics = nullptr; + auto iter = _query_statistics.find(sender_id); + if (iter == _query_statistics.end()) { + query_statistics = new QueryStatistics; + _query_statistics[sender_id] = query_statistics; + } else { + query_statistics = iter->second; + } + query_statistics->deserialize(statistics); } - void clear() { + void add_to(QueryStatistics* statistics) { boost::lock_guard l(_lock); - auto iter = _statistics_map.begin(); - while (iter != _statistics_map.end()) { - iter->second->reset(); + auto iter = _query_statistics.begin(); + while (iter != _query_statistics.end()) { + statistics->add(*(iter->second)); iter++; } } - - ~QueryStatistics() { - boost::lock_guard l(_lock); - auto iter = _statistics_map.begin(); - while (iter != _statistics_map.end()) { + + ~QueryStatisticsRecvr() { + // It is unnecessary to lock here, because the destructor will be + // called alter DataStreamRecvr's close in ExchangeNode. + auto iter = _query_statistics.begin(); + while (iter != _query_statistics.end()) { delete iter->second; - iter++; - } - } - + iter++; + } + _query_statistics.clear(); + } + private: - - Statistics* find(int sender_id) { - auto iter = _statistics_map.find(sender_id); - Statistics* statistics = nullptr; - if (iter == _statistics_map.end()) { - statistics = new Statistics(); - _statistics_map[sender_id] = statistics; - } else { - statistics = iter->second; - } - return statistics; - } - - Statistics get_total_statistics() { - Statistics total_statistics; - auto iter = _statistics_map.begin(); - while (iter != _statistics_map.end()) { - total_statistics.add(*(iter->second)); - iter++; - } - return total_statistics; - } - - // Map lock. + + std::map _query_statistics; SpinLock _lock; - // Sender id to statistics. - std::map _statistics_map; - // To index current plan. - static const int DEFAULT_SENDER_ID = -1; }; } diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 23ebeb6ab28770..8033fe2cac4d63 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -59,7 +59,7 @@ void PInternalServiceImpl::transmit_data(google::protobuf::RpcController* cnt TUniqueId finst_id; finst_id.__set_hi(request->finst_id().hi()); finst_id.__set_lo(request->finst_id().lo()); - _exec_env->stream_mgr()->update_query_statistics( + _exec_env->stream_mgr()->add_query_statistics( finst_id, request->node_id(), request->sender_id(), diff --git a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java index f378d26264cb99..03986ada8ba043 100644 --- a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -101,7 +101,7 @@ private void auditAfterExec(String origStmt, StatementBase parsedStmt, ctx.getAuditBuilder().put("state", ctx.getState()); ctx.getAuditBuilder().put("time", elapseMs); Preconditions.checkNotNull(statistics); - ctx.getAuditBuilder().put("ProcessRows", statistics.getFormattingProcessRows()); + ctx.getAuditBuilder().put("ScanRows", statistics.getFormattingScanRows()); ctx.getAuditBuilder().put("ScanRawData", statistics.getFormattingScanBytes()); ctx.getAuditBuilder().put("returnRows", ctx.getReturnRows()); ctx.getAuditBuilder().put("stmt_id", ctx.getStmtId()); diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 534675ffb0c061..1408469f70e801 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -566,7 +566,7 @@ private void handleQueryStmt() throws Exception { private void setQueryStatisticsForAuditLog(RowBatch batch) { if (batch != null) { final PQueryStatistics statistics = batch.getQueryStatistics(); - statisticsForAuditLog = new QueryStatistics(statistics.processRows, statistics.scanBytes); + statisticsForAuditLog = new QueryStatistics(statistics.scanRows, statistics.scanBytes); } } @@ -796,22 +796,22 @@ public QueryStatistics getQueryStatisticsForAuditLog() { } public static class QueryStatistics { - private final long processRows; + private final long scanRows; private final long scanBytes; public QueryStatistics() { - this.processRows = 0; + this.scanRows = 0; this.scanBytes = 0; } - public QueryStatistics(long processRows, long scanBytes) { - this.processRows = processRows; + public QueryStatistics(long scanRows, long scanBytes) { + this.scanRows = scanRows; this.scanBytes = scanBytes; } - public String getFormattingProcessRows() { + public String getFormattingScanRows() { final StringBuilder builder = new StringBuilder(); - builder.append(processRows).append(" Rows"); + builder.append(scanRows).append(" Rows"); return builder.toString(); } diff --git a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java index 5a328ecc301dea..e8ac87478f0592 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java +++ b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java @@ -23,7 +23,7 @@ @ProtobufClass public class PQueryStatistics { @Protobuf(order = 1, required = false) - public long processRows; + public long scanRows; @Protobuf(order = 2, required = false) public long scanBytes; } diff --git a/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java b/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java index 2e9973f613e044..0e35a7b7a5facf 100644 --- a/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java @@ -160,6 +160,7 @@ public void testSelect() throws Exception { cood.endProfile(); EasyMock.expectLastCall().anyTimes(); EasyMock.expect(cood.getQueryProfile()).andReturn(new RuntimeProfile()).anyTimes(); + EasyMock.expect(cood.getNext()).andReturn(null).anyTimes(); EasyMock.replay(cood); PowerMock.expectNew(Coordinator.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(Analyzer.class), EasyMock.isA(Planner.class)) diff --git a/gensrc/proto/data.proto b/gensrc/proto/data.proto index 5bcb34f387c750..d690fb6c58ddd4 100644 --- a/gensrc/proto/data.proto +++ b/gensrc/proto/data.proto @@ -20,7 +20,7 @@ syntax="proto2"; package doris; message PQueryStatistics { - optional int64 process_rows = 1; + optional int64 scan_rows = 1; optional int64 scan_bytes = 2; } From 3a3b73cf2e5b35278228182c42a5200752a93bd9 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 13:06:26 +0800 Subject: [PATCH 05/18] Format and modify ResultSink --- be/src/runtime/result_sink.h | 2 +- be/src/runtime/runtime_state.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/be/src/runtime/result_sink.h b/be/src/runtime/result_sink.h index f9c861d8a20e1b..53739740bd573c 100644 --- a/be/src/runtime/result_sink.h +++ b/be/src/runtime/result_sink.h @@ -56,7 +56,7 @@ class ResultSink : public DataSink { return _profile; } - virtual void set_query_statistics(std::shared_ptr statistics); + void set_query_statistics(std::shared_ptr statistics) override; private: Status prepare_exprs(RuntimeState* state); diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index accd2fb719bef9..1f6fde88eb6a93 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -188,6 +188,7 @@ Status RuntimeState::init( } _db_name = "insert_stmt"; _import_label = print_id(fragment_instance_id); + return Status::OK; } From d4ad20be22b22c9a4bfcad1624e8dc19f05cc2ff Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 13:08:51 +0800 Subject: [PATCH 06/18] Remove ununsed function --- be/src/runtime/query_statistics.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index 41b921014eb56e..b6fbb948966e4c 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -37,11 +37,6 @@ class QueryStatistics { scan_bytes += other.scan_bytes; } - void add(const PQueryStatistics& other) { - scan_rows += other.scan_rows(); - scan_bytes += other.scan_bytes(); - } - void add_scan_rows(long scan_rows) { this->scan_rows += scan_rows; } From f3a1af65a413110b51b168a70d6a7b14d6ecae11 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 13:39:23 +0800 Subject: [PATCH 07/18] Modify boost api to std --- be/src/exec/exchange_node.h | 2 +- be/src/runtime/query_statistics.h | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/be/src/exec/exchange_node.h b/be/src/exec/exchange_node.h index 29d5ae2c1beac6..bfb1d35c4e2278 100644 --- a/be/src/exec/exchange_node.h +++ b/be/src/exec/exchange_node.h @@ -113,7 +113,7 @@ class ExchangeNode : public ExecNode { RuntimeProfile::Counter* _merge_rows_counter; // Sub plan query statistics receiver. - boost::scoped_ptr _sub_plan_query_statistics_recvr; + std::unique_ptr _sub_plan_query_statistics_recvr; }; }; diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index b6fbb948966e4c..6750d0653763b4 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -18,6 +18,8 @@ #ifndef DORIS_BE_EXEC_QUERY_STATISTICS_H #define DORIS_BE_EXEC_QUERY_STATISTICS_H +#include + #include "gen_cpp/data.pb.h" #include "util/spinlock.h" @@ -72,7 +74,7 @@ class QueryStatisticsRecvr { public: void insert(const PQueryStatistics& statistics, int sender_id) { - boost::lock_guard l(_lock); + std::lock_guard l(_lock); QueryStatistics* query_statistics = nullptr; auto iter = _query_statistics.find(sender_id); if (iter == _query_statistics.end()) { @@ -85,7 +87,7 @@ class QueryStatisticsRecvr { } void add_to(QueryStatistics* statistics) { - boost::lock_guard l(_lock); + std::lock_guard l(_lock); auto iter = _query_statistics.begin(); while (iter != _query_statistics.end()) { statistics->add(*(iter->second)); From fae9bc2ce1638e6272dbc5c24e60a8911a6724aa Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 16:33:38 +0800 Subject: [PATCH 08/18] Change some variables name and remove some redundant code --- be/src/exec/exchange_node.cpp | 2 +- be/src/exprs/cast_functions.cpp | 1 + be/src/runtime/CMakeLists.txt | 1 + be/src/runtime/buffer_control_block.cpp | 2 +- be/src/runtime/data_stream_mgr.cpp | 51 +++++------------ be/src/runtime/data_stream_mgr.h | 25 +------- be/src/runtime/data_stream_sender.cpp | 2 +- be/src/runtime/query_statistics.cpp | 55 ++++++++++++++++++ be/src/runtime/query_statistics.h | 57 ++++++------------- be/src/service/internal_service.cpp | 31 +--------- .../org/apache/doris/qe/ConnectProcessor.java | 24 +++++++- .../org/apache/doris/qe/StmtExecutor.java | 47 ++------------- .../apache/doris/rpc/PQueryStatistics.java | 5 ++ 13 files changed, 128 insertions(+), 175 deletions(-) create mode 100644 be/src/runtime/query_statistics.cpp diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp index e9b2a8f633cb62..ea9ef64d3d3164 100644 --- a/be/src/exec/exchange_node.cpp +++ b/be/src/exec/exchange_node.cpp @@ -98,7 +98,7 @@ Status ExchangeNode::open(RuntimeState* state) { Status ExchangeNode::collect_query_statistics(QueryStatistics* statistics) { RETURN_IF_ERROR(ExecNode::collect_query_statistics(statistics)); - _sub_plan_query_statistics_recvr->add_to(statistics); + statistics->merge(_sub_plan_query_statistics_recvr.get()); return Status::OK; } diff --git a/be/src/exprs/cast_functions.cpp b/be/src/exprs/cast_functions.cpp index 065307a344824a..253126cf140241 100644 --- a/be/src/exprs/cast_functions.cpp +++ b/be/src/exprs/cast_functions.cpp @@ -171,6 +171,7 @@ StringVal CastFunctions::cast_to_string_val(FunctionContext* ctx, const LargeInt return sv; \ } \ const FunctionContext::TypeDesc& returnType = ctx->get_return_type(); \ + LOG(INFO) << "chenhao cast_to_string len:" << returnType.len; \ if (returnType.len > 0) { \ sv.len = snprintf(reinterpret_cast(sv.ptr), sv.len, format, val.val); \ DCHECK_GT(sv.len, 0); \ diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt index fd911382e65947..13464da33b01a4 100644 --- a/be/src/runtime/CMakeLists.txt +++ b/be/src/runtime/CMakeLists.txt @@ -93,6 +93,7 @@ add_library(Runtime STATIC initial_reservations.cc snapshot_loader.cpp kafka_consumer_pipe.cpp + query_statistics.cpp ) # This test runs forever so should not be part of 'make test' diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp index 24237f240d29c8..6199d731d62edb 100644 --- a/be/src/runtime/buffer_control_block.cpp +++ b/be/src/runtime/buffer_control_block.cpp @@ -36,7 +36,7 @@ void GetResultBatchCtx::on_close(int64_t packet_seq, Status status; status.to_protobuf(result->mutable_status()); if (statistics != nullptr) { - statistics->serialize(result->mutable_query_statistics()); + statistics->to_pb(result->mutable_query_statistics()); } result->set_packet_seq(packet_seq); result->set_eos(true); diff --git a/be/src/runtime/data_stream_mgr.cpp b/be/src/runtime/data_stream_mgr.cpp index 99b44c19f14be0..3d87293fb0d5d5 100644 --- a/be/src/runtime/data_stream_mgr.cpp +++ b/be/src/runtime/data_stream_mgr.cpp @@ -93,18 +93,14 @@ shared_ptr DataStreamMgr::find_recvr( return shared_ptr(); } -Status DataStreamMgr::add_data( - const PUniqueId& finst_id, int32_t node_id, - const PRowBatch& pb_batch, int32_t sender_id, - int be_number, int64_t packet_seq, - ::google::protobuf::Closure** done) { - VLOG_ROW << "add_data(): fragment_instance_id=" << print_id(finst_id) - << " node=" << node_id; +Status DataStreamMgr::transmit_data(const PTransmitDataParams* request, ::google::protobuf::Closure** done) { + const PUniqueId& finst_id = request->finst_id(); TUniqueId t_finst_id; t_finst_id.hi = finst_id.hi(); t_finst_id.lo = finst_id.lo(); - shared_ptr recvr = find_recvr(t_finst_id, node_id); - if (recvr == NULL) { + shared_ptr recvr = find_recvr(t_finst_id, request->node_id()); + + if (recvr == nullptr) { // The receiver may remove itself from the receiver map via deregister_recvr() // at any time without considering the remaining number of senders. // As a consequence, find_recvr() may return an innocuous NULL if a thread @@ -114,37 +110,20 @@ Status DataStreamMgr::add_data( // errors from receiver-initiated teardowns. return Status::OK; } - recvr->add_batch(pb_batch, sender_id, be_number, packet_seq, done); - return Status::OK; -} -Status DataStreamMgr::add_query_statistics(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, - int sender_id, const PQueryStatistics& query_statistics) { - shared_ptr recvr = find_recvr(fragment_instance_id, dest_node_id); - if (recvr != NULL) { - recvr->add_sub_plan_statistics(query_statistics, sender_id); + bool eos = request->eos(); + if (request->has_row_batch()) { + recvr->add_batch(request->row_batch(), request->sender_id(), + request->be_number(), request->packet_seq(), eos ? nullptr : done); } - return Status::OK; -} -Status DataStreamMgr::close_sender(const TUniqueId& fragment_instance_id, - PlanNodeId dest_node_id, - int sender_id, - int be_number) { - VLOG_FILE << "close_sender(): fragment_instance_id=" << fragment_instance_id - << ", node=" << dest_node_id; - shared_ptr recvr = find_recvr(fragment_instance_id, dest_node_id); - if (recvr == NULL) { - // The receiver may remove itself from the receiver map via deregister_recvr() - // at any time without considering the remaining number of senders. - // As a consequence, find_recvr() may return an innocuous NULL if a thread - // calling deregister_recvr() beat the thread calling find_recvr() - // in acquiring _lock. - // TODO: Rethink the lifecycle of DataStreamRecvr to distinguish - // errors from receiver-initiated teardowns. - return Status::OK; + if (request->has_query_statistics()) { + recvr->add_sub_plan_statistics(request->query_statistics(), request->sender_id()); } - recvr->remove_sender(sender_id, be_number); + + if (eos) { + recvr->remove_sender(request->sender_id(), request->be_number()); + } return Status::OK; } diff --git a/be/src/runtime/data_stream_mgr.h b/be/src/runtime/data_stream_mgr.h index fcd2661b798983..ab730b0b933ac6 100644 --- a/be/src/runtime/data_stream_mgr.h +++ b/be/src/runtime/data_stream_mgr.h @@ -32,6 +32,8 @@ #include "runtime/mem_tracker.h" #include "runtime/query_statistics.h" #include "util/runtime_profile.h" +#include "gen_cpp/internal_service.pb.h" +#include "gen_cpp/palo_internal_service.pb.h" #include "gen_cpp/Types_types.h" // for TUniqueId namespace google { @@ -79,28 +81,7 @@ class DataStreamMgr { int num_senders, int buffer_size, RuntimeProfile* profile, bool is_merging, QueryStatisticsRecvr* sub_plan_query_statistics_recvr); - // Adds a row batch to the recvr identified by fragment_instance_id/dest_node_id - // if the recvr has not been cancelled. sender_id identifies the sender instance - // from which the data came. - // The call blocks if this ends up pushing the stream over its buffering limit; - // it unblocks when the consumer removed enough data to make space for - // row_batch. - // TODO: enforce per-sender quotas (something like 200% of buffer_size/#senders), - // so that a single sender can't flood the buffer and stall everybody else. - // Returns OK if successful, error status otherwise. - Status add_data(const PUniqueId& fragment_instance_id, int32_t node_id, - const PRowBatch& pb_batch, int32_t sender_id, - int32_t be_number, int64_t packet_seq, - ::google::protobuf::Closure** done); - - // Notifies the recvr associated with the fragment/node id that the specified - // sender has closed. - // Returns OK if successful, error status otherwise. - Status close_sender(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, - int sender_id, int be_number); - - Status add_query_statistics(const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, - int sender_id, const PQueryStatistics& query_statistics); + Status transmit_data(const PTransmitDataParams* request, ::google::protobuf::Closure** done); // Closes all receivers registered for fragment_instance_id immediately. void cancel(const TUniqueId& fragment_instance_id); diff --git a/be/src/runtime/data_stream_sender.cpp b/be/src/runtime/data_stream_sender.cpp index 8845953069e6c1..dfe0726c8097d7 100644 --- a/be/src/runtime/data_stream_sender.cpp +++ b/be/src/runtime/data_stream_sender.cpp @@ -213,7 +213,7 @@ Status DataStreamSender::Channel::send_batch(PRowBatch* batch, bool eos) { << " dest_node=" << _dest_node_id; if (_is_transfer_chain && (_send_query_statistics_with_every_batch || eos)) { auto statistic = _brpc_request.mutable_query_statistics(); - _parent->_query_statistics->serialize(statistic); + _parent->_query_statistics->to_pb(statistic); } _brpc_request.set_eos(eos); diff --git a/be/src/runtime/query_statistics.cpp b/be/src/runtime/query_statistics.cpp new file mode 100644 index 00000000000000..2d2fee9e826fc8 --- /dev/null +++ b/be/src/runtime/query_statistics.cpp @@ -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. + +#include "runtime/query_statistics.h" + +namespace doris { + +void QueryStatistics::merge(QueryStatisticsRecvr* recvr) { + std::lock_guard l(recvr->_lock); + auto iter = recvr->_query_statistics.begin(); + while (iter != recvr->_query_statistics.end()) { + merge(*(iter->second)); + iter++; + } +} + +void QueryStatisticsRecvr::insert(const PQueryStatistics& statistics, int sender_id) { + std::lock_guard l(_lock); + QueryStatistics* query_statistics = nullptr; + auto iter = _query_statistics.find(sender_id); + if (iter == _query_statistics.end()) { + query_statistics = new QueryStatistics; + _query_statistics[sender_id] = query_statistics; + } else { + query_statistics = iter->second; + } + query_statistics->from_pb(statistics); +} + +QueryStatisticsRecvr::~QueryStatisticsRecvr() { + // It is unnecessary to lock here, because the destructor will be + // called alter DataStreamRecvr's close in ExchangeNode. + auto iter = _query_statistics.begin(); + while (iter != _query_statistics.end()) { + delete iter->second; + iter++; + } + _query_statistics.clear(); +} + +} diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index 6750d0653763b4..9999dfde9a93b6 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -25,6 +25,8 @@ namespace doris { +class QueryStatisticsRecvr; + // This is responsible for collecting query statistics, usually it consists of // two parts, one is current fragment or plan's statistics, the other is sub fragment // or plan's statistics and QueryStatisticsRecvr is responsible for collecting it. @@ -34,80 +36,55 @@ class QueryStatistics { QueryStatistics() : scan_rows(0), scan_bytes(0) { } - void add(const QueryStatistics& other) { + void merge(const QueryStatistics& other) { scan_rows += other.scan_rows; scan_bytes += other.scan_bytes; } - void add_scan_rows(long scan_rows) { + void add_scan_rows(int64_t scan_rows) { this->scan_rows += scan_rows; } - void add_scan_bytes(long scan_bytes) { + void add_scan_bytes(int64_t scan_bytes) { this->scan_bytes += scan_bytes; } + void merge(QueryStatisticsRecvr* recvr); + void clear() { scan_rows = 0; scan_bytes = 0; } - void serialize(PQueryStatistics* statistics) { + void to_pb(PQueryStatistics* statistics) { DCHECK(statistics != nullptr); statistics->set_scan_rows(scan_rows); statistics->set_scan_bytes(scan_bytes); } - void deserialize(const PQueryStatistics& statistics) { + void from_pb(const PQueryStatistics& statistics) { scan_rows = statistics.scan_rows(); scan_bytes = statistics.scan_bytes(); } private: - long scan_rows; - long scan_bytes; + int64_t scan_rows; + int64_t scan_bytes; }; // It is used for collecting sub plan query statistics in DataStreamRecvr. class QueryStatisticsRecvr { public: - void insert(const PQueryStatistics& statistics, int sender_id) { - std::lock_guard l(_lock); - QueryStatistics* query_statistics = nullptr; - auto iter = _query_statistics.find(sender_id); - if (iter == _query_statistics.end()) { - query_statistics = new QueryStatistics; - _query_statistics[sender_id] = query_statistics; - } else { - query_statistics = iter->second; - } - query_statistics->deserialize(statistics); - } + ~QueryStatisticsRecvr(); + + void insert(const PQueryStatistics& statistics, int sender_id); - void add_to(QueryStatistics* statistics) { - std::lock_guard l(_lock); - auto iter = _query_statistics.begin(); - while (iter != _query_statistics.end()) { - statistics->add(*(iter->second)); - iter++; - } - } - - ~QueryStatisticsRecvr() { - // It is unnecessary to lock here, because the destructor will be - // called alter DataStreamRecvr's close in ExchangeNode. - auto iter = _query_statistics.begin(); - while (iter != _query_statistics.end()) { - delete iter->second; - iter++; - } - _query_statistics.clear(); - } - private: - + +friend class QueryStatistics; + std::map _query_statistics; SpinLock _lock; }; diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 8033fe2cac4d63..07095faaa983d3 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -46,34 +46,9 @@ void PInternalServiceImpl::transmit_data(google::protobuf::RpcController* cnt const PTransmitDataParams* request, PTransmitDataResult* response, google::protobuf::Closure* done) { - bool eos = request->eos(); - if (request->has_row_batch()) { - _exec_env->stream_mgr()->add_data( - request->finst_id(), request->node_id(), - request->row_batch(), request->sender_id(), - request->be_number(), request->packet_seq(), - eos ? nullptr : &done); - } - - if (request->has_query_statistics()) { - TUniqueId finst_id; - finst_id.__set_hi(request->finst_id().hi()); - finst_id.__set_lo(request->finst_id().lo()); - _exec_env->stream_mgr()->add_query_statistics( - finst_id, - request->node_id(), - request->sender_id(), - request->query_statistics()); - } - - if (eos) { - TUniqueId finst_id; - finst_id.__set_hi(request->finst_id().hi()); - finst_id.__set_lo(request->finst_id().lo()); - _exec_env->stream_mgr()->close_sender( - finst_id, request->node_id(), - request->sender_id(), request->be_number()); - } + VLOG_ROW << "transmit data: fragment_instance_id=" << print_id(request->finst_id()) + << " node=" << request->node_id(); + _exec_env->stream_mgr()->transmit_data(request, &done); if (done != nullptr) { done->Run(); } diff --git a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 03986ada8ba043..32458d3785fbaa 100644 --- a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -28,12 +28,15 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.Pair; +import org.apache.doris.common.util.DebugUtil; import org.apache.doris.metric.MetricRepo; import org.apache.doris.mysql.MysqlChannel; import org.apache.doris.mysql.MysqlCommand; import org.apache.doris.mysql.MysqlPacket; import org.apache.doris.mysql.MysqlProto; import org.apache.doris.mysql.MysqlSerializer; +import org.apache.doris.rpc.PQueryStatistics; import org.apache.doris.thrift.TMasterOpRequest; import org.apache.doris.thrift.TMasterOpResult; @@ -47,6 +50,7 @@ import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; import java.nio.channels.AsynchronousCloseException; +import java.util.Formatter; import java.util.List; /** @@ -93,16 +97,30 @@ private void handlePing() { ctx.getState().setOk(); } + public String getFormattingScanRows(PQueryStatistics statistics) { + final StringBuilder builder = new StringBuilder(); + builder.append(statistics.scanRows).append(" Rows"); + return builder.toString(); + } + + public String getFormattingScanBytes(PQueryStatistics statistics) { + final Pair pair = DebugUtil.getByteUint(statistics.scanBytes); + final Formatter fmt = new Formatter(); + final StringBuilder builder = new StringBuilder(); + builder.append(fmt.format("%.2f", pair.first)).append(" ").append(pair.second); + return builder.toString(); + } + private void auditAfterExec(String origStmt, StatementBase parsedStmt, - StmtExecutor.QueryStatistics statistics) { + PQueryStatistics statistics) { // slow query long elapseMs = System.currentTimeMillis() - ctx.getStartTime(); // query state log ctx.getAuditBuilder().put("state", ctx.getState()); ctx.getAuditBuilder().put("time", elapseMs); Preconditions.checkNotNull(statistics); - ctx.getAuditBuilder().put("ScanRows", statistics.getFormattingScanRows()); - ctx.getAuditBuilder().put("ScanRawData", statistics.getFormattingScanBytes()); + ctx.getAuditBuilder().put("ScanRows", getFormattingScanRows(statistics)); + ctx.getAuditBuilder().put("ScanRawData", getFormattingScanBytes(statistics)); ctx.getAuditBuilder().put("returnRows", ctx.getReturnRows()); ctx.getAuditBuilder().put("stmt_id", ctx.getStmtId()); diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 1408469f70e801..3d98e8af4b06e7 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -17,7 +17,6 @@ package org.apache.doris.qe; -import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -53,7 +52,6 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.NotImplementedException; -import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.ProfileManager; @@ -78,7 +76,6 @@ import java.io.IOException; import java.io.StringReader; import java.nio.ByteBuffer; -import java.util.Formatter; import java.util.List; import java.util.Map; import java.util.UUID; @@ -105,7 +102,7 @@ public class StmtExecutor { private Planner planner; private boolean isProxy; private ShowResultSet proxyResultSet = null; - private QueryStatistics statisticsForAuditLog; + private PQueryStatistics statisticsForAuditLog; public StmtExecutor(ConnectContext context, String stmt, boolean isProxy) { this.context = context; @@ -556,20 +553,13 @@ private void handleQueryStmt() throws Exception { } context.updateReturnRows(batch.getBatch().getRows().size()); } - setQueryStatisticsForAuditLog(batch); + statisticsForAuditLog = batch.getQueryStatistics(); if (!isSendFields) { sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); } context.getState().setEof(); } - private void setQueryStatisticsForAuditLog(RowBatch batch) { - if (batch != null) { - final PQueryStatistics statistics = batch.getQueryStatistics(); - statisticsForAuditLog = new QueryStatistics(statistics.scanRows, statistics.scanBytes); - } - } - // Process a select statement. private void handleInsertStmt() throws Exception { // Every time set no send flag and clean all data in buffer @@ -788,39 +778,10 @@ private void handleExportStmt() throws Exception { context.getCatalog().getExportMgr().addExportJob(exportStmt); } - public QueryStatistics getQueryStatisticsForAuditLog() { + public PQueryStatistics getQueryStatisticsForAuditLog() { if (statisticsForAuditLog == null) { - statisticsForAuditLog = new QueryStatistics(); + statisticsForAuditLog = new PQueryStatistics(); } return statisticsForAuditLog; } - - public static class QueryStatistics { - private final long scanRows; - private final long scanBytes; - - public QueryStatistics() { - this.scanRows = 0; - this.scanBytes = 0; - } - - public QueryStatistics(long scanRows, long scanBytes) { - this.scanRows = scanRows; - this.scanBytes = scanBytes; - } - - public String getFormattingScanRows() { - final StringBuilder builder = new StringBuilder(); - builder.append(scanRows).append(" Rows"); - return builder.toString(); - } - - public String getFormattingScanBytes() { - final Pair pair = DebugUtil.getByteUint(scanBytes); - final Formatter fmt = new Formatter(); - final StringBuilder builder = new StringBuilder(); - builder.append(fmt.format("%.2f", pair.first)).append(" ").append(pair.second); - return builder.toString(); - } - } } diff --git a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java index e8ac87478f0592..5b31c9a5fcce4b 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java +++ b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java @@ -26,4 +26,9 @@ public class PQueryStatistics { public long scanRows; @Protobuf(order = 2, required = false) public long scanBytes; + + public PQueryStatistics() { + scanRows = 0; + scanBytes = 0; + } } From 90111dd7959d48c2f3806296b74d05071842edb9 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 16:45:07 +0800 Subject: [PATCH 09/18] Remove ununsed log --- be/src/exprs/cast_functions.cpp | 1 - fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/be/src/exprs/cast_functions.cpp b/be/src/exprs/cast_functions.cpp index 253126cf140241..065307a344824a 100644 --- a/be/src/exprs/cast_functions.cpp +++ b/be/src/exprs/cast_functions.cpp @@ -171,7 +171,6 @@ StringVal CastFunctions::cast_to_string_val(FunctionContext* ctx, const LargeInt return sv; \ } \ const FunctionContext::TypeDesc& returnType = ctx->get_return_type(); \ - LOG(INFO) << "chenhao cast_to_string len:" << returnType.len; \ if (returnType.len > 0) { \ sv.len = snprintf(reinterpret_cast(sv.ptr), sv.len, format, val.val); \ DCHECK_GT(sv.len, 0); \ diff --git a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 32458d3785fbaa..ffa52f7abd1120 100644 --- a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -97,13 +97,13 @@ private void handlePing() { ctx.getState().setOk(); } - public String getFormattingScanRows(PQueryStatistics statistics) { + private String getFormattingScanRows(PQueryStatistics statistics) { final StringBuilder builder = new StringBuilder(); builder.append(statistics.scanRows).append(" Rows"); return builder.toString(); } - public String getFormattingScanBytes(PQueryStatistics statistics) { + private String getFormattingScanBytes(PQueryStatistics statistics) { final Pair pair = DebugUtil.getByteUint(statistics.scanBytes); final Formatter fmt = new Formatter(); final StringBuilder builder = new StringBuilder(); From 2e866aaad89ceabf545c9960353d2b11c53163f5 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 16:55:02 +0800 Subject: [PATCH 10/18] Remove ununsed include --- be/src/runtime/data_stream_mgr.h | 1 - 1 file changed, 1 deletion(-) diff --git a/be/src/runtime/data_stream_mgr.h b/be/src/runtime/data_stream_mgr.h index ab730b0b933ac6..fbfb9fcd0e8905 100644 --- a/be/src/runtime/data_stream_mgr.h +++ b/be/src/runtime/data_stream_mgr.h @@ -32,7 +32,6 @@ #include "runtime/mem_tracker.h" #include "runtime/query_statistics.h" #include "util/runtime_profile.h" -#include "gen_cpp/internal_service.pb.h" #include "gen_cpp/palo_internal_service.pb.h" #include "gen_cpp/Types_types.h" // for TUniqueId From 6d64b35a2cd203a8605d3dfb0082d01202f417c8 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 17:21:00 +0800 Subject: [PATCH 11/18] Fix unit test fail --- fe/src/main/java/org/apache/doris/qe/StmtExecutor.java | 6 +++++- .../java/org/apache/doris/qe/ConnectProcessorTest.java | 7 ++++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 3d98e8af4b06e7..130c7dfbc4c472 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -553,7 +553,11 @@ private void handleQueryStmt() throws Exception { } context.updateReturnRows(batch.getBatch().getRows().size()); } - statisticsForAuditLog = batch.getQueryStatistics(); + + if (batch != null) { + statisticsForAuditLog = batch.getQueryStatistics(); + } + if (!isSendFields) { sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); } diff --git a/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java b/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java index 1d7a58c442c8cd..5a5f1ff999d137 100644 --- a/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java @@ -26,6 +26,7 @@ import org.apache.doris.mysql.MysqlErrPacket; import org.apache.doris.mysql.MysqlOkPacket; import org.apache.doris.mysql.MysqlSerializer; +import org.apache.doris.rpc.PQueryStatistics; import org.easymock.EasyMock; import org.junit.Assert; @@ -231,7 +232,7 @@ public void testQuery() throws Exception { // Mock statement executor StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); - EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new StmtExecutor.QueryStatistics()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new PQueryStatistics()); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(qe); PowerMock.expectNew( @@ -255,7 +256,7 @@ public void testQueryFail() throws Exception { StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); EasyMock.expectLastCall().andThrow(new IOException("Fail")).anyTimes(); - EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new StmtExecutor.QueryStatistics()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new PQueryStatistics()); EasyMock.replay(qe); PowerMock.expectNew(StmtExecutor.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(String.class)) .andReturn(qe).anyTimes(); @@ -273,7 +274,7 @@ public void testQueryFail2() throws Exception { // Mock statement executor StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); - EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new StmtExecutor.QueryStatistics()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new PQueryStatistics()); EasyMock.expectLastCall().andThrow(new NullPointerException("Fail")).anyTimes(); EasyMock.replay(qe); PowerMock.expectNew(StmtExecutor.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(String.class)) From 1c5aac1e8883bc9853860e50a145280cc75334ab Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 19:07:16 +0800 Subject: [PATCH 12/18] Trigger github check --- fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java index 5b31c9a5fcce4b..333ded38c820ca 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java +++ b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java @@ -31,4 +31,5 @@ public PQueryStatistics() { scanRows = 0; scanBytes = 0; } + } From ee3935eb6120021441a3d587ee0837be795ce22a Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 19:53:55 +0800 Subject: [PATCH 13/18] Remove redundant code and modify unit test --- .../java/org/apache/doris/qe/ResultReceiver.java | 4 +--- .../java/org/apache/doris/qe/StmtExecutor.java | 14 ++++++++------ .../java/org/apache/doris/qe/StmtExecutorTest.java | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java b/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java index dee56553abf9bc..ed8a0323f7852a 100644 --- a/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java +++ b/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java @@ -92,9 +92,7 @@ public RowBatch getNext(Status status) throws TException { return null; } - if (pResult.statistics != null) { - rowBatch.setQueryStatistics(pResult.statistics); - } + rowBatch.setQueryStatistics(pResult.statistics); if (packetIdx != pResult.packetSeq) { LOG.warn("receive packet failed, expect={}, receive={}", packetIdx, pResult.packetSeq); diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 130c7dfbc4c472..e26a345d1de110 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -542,21 +542,23 @@ private void handleQueryStmt() throws Exception { RowBatch batch; MysqlChannel channel = context.getMysqlChannel(); boolean isSendFields = false; - - while ((batch = coord.getNext()) != null && !batch.isEos()) { + while (true) { + batch = coord.getNext(); + if (batch.isEos()) { + break; + } if (!isSendFields) { sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); + isSendFields = !isSendFields; } - isSendFields = true; + for (ByteBuffer row : batch.getBatch().getRows()) { channel.sendOnePacket(row); } context.updateReturnRows(batch.getBatch().getRows().size()); } - if (batch != null) { - statisticsForAuditLog = batch.getQueryStatistics(); - } + statisticsForAuditLog = batch.getQueryStatistics(); if (!isSendFields) { sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); diff --git a/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java b/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java index 0e35a7b7a5facf..3b12236e856644 100644 --- a/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java @@ -160,7 +160,7 @@ public void testSelect() throws Exception { cood.endProfile(); EasyMock.expectLastCall().anyTimes(); EasyMock.expect(cood.getQueryProfile()).andReturn(new RuntimeProfile()).anyTimes(); - EasyMock.expect(cood.getNext()).andReturn(null).anyTimes(); + EasyMock.expect(cood.getNext()).andReturn(new RowBatch()).anyTimes(); EasyMock.replay(cood); PowerMock.expectNew(Coordinator.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(Analyzer.class), EasyMock.isA(Planner.class)) From 079ac791a33f9af9a5549d95732ad0043a71761e Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 23:31:27 +0800 Subject: [PATCH 14/18] Add some comments and modify QueryStatistics --- be/src/exec/exchange_node.h | 4 +++- be/src/runtime/buffer_control_block.h | 3 +++ be/src/runtime/plan_fragment_executor.h | 5 +++-- be/src/runtime/query_statistics.cpp | 12 +++--------- be/src/runtime/query_statistics.h | 9 +++++++++ 5 files changed, 21 insertions(+), 12 deletions(-) diff --git a/be/src/exec/exchange_node.h b/be/src/exec/exchange_node.h index bfb1d35c4e2278..e53742b30d65cf 100644 --- a/be/src/exec/exchange_node.h +++ b/be/src/exec/exchange_node.h @@ -112,7 +112,9 @@ class ExchangeNode : public ExecNode { RuntimeProfile::Counter* _merge_rows_counter; - // Sub plan query statistics receiver. + // Sub plan query statistics receiver. It is shared with DataStreamRecvr and will be + // called in two different threads. But their calls are all at different time, there is + // no problem of multithreaded access. std::unique_ptr _sub_plan_query_statistics_recvr; }; diff --git a/be/src/runtime/buffer_control_block.h b/be/src/runtime/buffer_control_block.h index 34a9ea4c899d74..71682287e08368 100644 --- a/be/src/runtime/buffer_control_block.h +++ b/be/src/runtime/buffer_control_block.h @@ -107,6 +107,9 @@ class BufferControlBlock { std::deque _waiting_rpc; + // It is shared with PlanFragmentExecutor and will be called in two different + // threads. But their calls are all at different time, there is no problem of + // multithreaded access. std::shared_ptr _query_statistics; }; diff --git a/be/src/runtime/plan_fragment_executor.h b/be/src/runtime/plan_fragment_executor.h index 6301940305b529..cc5c51d4b250ae 100644 --- a/be/src/runtime/plan_fragment_executor.h +++ b/be/src/runtime/plan_fragment_executor.h @@ -205,8 +205,9 @@ class PlanFragmentExecutor { // of the execution. RuntimeProfile::Counter* _average_thread_tokens; - // This plan and it's sub plan query statisics. Because plan may have been finished - // when fe fetch data and query statistics, this will be shared with ResultSink. + // It is shared with BufferControlBlock and will be called in two different + // threads. But their calls are all at different time, there is no problem of + // multithreaded access. std::shared_ptr _query_statistics; bool _collect_query_statistics_with_every_batch; diff --git a/be/src/runtime/query_statistics.cpp b/be/src/runtime/query_statistics.cpp index 2d2fee9e826fc8..9981b7dc7f1949 100644 --- a/be/src/runtime/query_statistics.cpp +++ b/be/src/runtime/query_statistics.cpp @@ -20,12 +20,7 @@ namespace doris { void QueryStatistics::merge(QueryStatisticsRecvr* recvr) { - std::lock_guard l(recvr->_lock); - auto iter = recvr->_query_statistics.begin(); - while (iter != recvr->_query_statistics.end()) { - merge(*(iter->second)); - iter++; - } + recvr->merge(this); } void QueryStatisticsRecvr::insert(const PQueryStatistics& statistics, int sender_id) { @@ -45,9 +40,8 @@ QueryStatisticsRecvr::~QueryStatisticsRecvr() { // It is unnecessary to lock here, because the destructor will be // called alter DataStreamRecvr's close in ExchangeNode. auto iter = _query_statistics.begin(); - while (iter != _query_statistics.end()) { - delete iter->second; - iter++; + for (auto& pair : _query_statistics) { + delete pair.second; } _query_statistics.clear(); } diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index 9999dfde9a93b6..ff2e61dbd3210d 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -84,6 +84,15 @@ class QueryStatisticsRecvr { private: friend class QueryStatistics; + + void merge(QueryStatistics* statistics) { + std::lock_guard l(_lock); + auto iter = _query_statistics.begin(); + while (iter != _query_statistics.end()) { + statistics->merge(*(iter->second)); + iter++; + } + } std::map _query_statistics; SpinLock _lock; From 3eebc3929c3415adcaf59a0eb9fc8568ce75a65b Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Wed, 16 Jan 2019 23:45:27 +0800 Subject: [PATCH 15/18] Modify QueryStatistics --- be/src/runtime/query_statistics.cpp | 1 - be/src/runtime/query_statistics.h | 6 ++---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/be/src/runtime/query_statistics.cpp b/be/src/runtime/query_statistics.cpp index 9981b7dc7f1949..3865d415815bd2 100644 --- a/be/src/runtime/query_statistics.cpp +++ b/be/src/runtime/query_statistics.cpp @@ -39,7 +39,6 @@ void QueryStatisticsRecvr::insert(const PQueryStatistics& statistics, int sender QueryStatisticsRecvr::~QueryStatisticsRecvr() { // It is unnecessary to lock here, because the destructor will be // called alter DataStreamRecvr's close in ExchangeNode. - auto iter = _query_statistics.begin(); for (auto& pair : _query_statistics) { delete pair.second; } diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index ff2e61dbd3210d..79b911e720a126 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -87,10 +87,8 @@ friend class QueryStatistics; void merge(QueryStatistics* statistics) { std::lock_guard l(_lock); - auto iter = _query_statistics.begin(); - while (iter != _query_statistics.end()) { - statistics->merge(*(iter->second)); - iter++; + for (auto& pair : _query_statistics) { + statistics->merge(*(pair.second)); } } From 5bb1c33d14fbffccb459772e61d7332e66382dc2 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Thu, 17 Jan 2019 10:37:07 +0800 Subject: [PATCH 16/18] Modify handlerQuery in StmtExecutor --- .../org/apache/doris/qe/StmtExecutor.java | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index e26a345d1de110..0cfa28dca88c95 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -544,25 +544,24 @@ private void handleQueryStmt() throws Exception { boolean isSendFields = false; while (true) { batch = coord.getNext(); - if (batch.isEos()) { - break; - } - if (!isSendFields) { - sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); - isSendFields = !isSendFields; + if (batch.getBatch() != null) { + if (!isSendFields) { + sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); + isSendFields = true; + } + + for (ByteBuffer row : batch.getBatch().getRows()) { + channel.sendOnePacket(row); + } + context.updateReturnRows(batch.getBatch().getRows().size()); } - for (ByteBuffer row : batch.getBatch().getRows()) { - channel.sendOnePacket(row); + if (batch.isEos()) { + break; } - context.updateReturnRows(batch.getBatch().getRows().size()); } statisticsForAuditLog = batch.getQueryStatistics(); - - if (!isSendFields) { - sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); - } context.getState().setEof(); } From 8ac2c3d8f2f5ee247564ca19e982ed125f7e1479 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Thu, 17 Jan 2019 10:58:13 +0800 Subject: [PATCH 17/18] Modify handlerQuery in StmtExecutor --- fe/src/main/java/org/apache/doris/qe/StmtExecutor.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 0cfa28dca88c95..bfa494ffbb4616 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -543,13 +543,13 @@ private void handleQueryStmt() throws Exception { MysqlChannel channel = context.getMysqlChannel(); boolean isSendFields = false; while (true) { + if (!isSendFields) { + sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); + isSendFields = true; + } + batch = coord.getNext(); if (batch.getBatch() != null) { - if (!isSendFields) { - sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); - isSendFields = true; - } - for (ByteBuffer row : batch.getBatch().getRows()) { channel.sendOnePacket(row); } From a93e7b98d4828bb0aa5bc266fb1ebc9e78686064 Mon Sep 17 00:00:00 2001 From: chenhao7253886 Date: Thu, 17 Jan 2019 11:27:54 +0800 Subject: [PATCH 18/18] Modify handlerQuery in StmtExecutor --- fe/src/main/java/org/apache/doris/qe/StmtExecutor.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index bfa494ffbb4616..64159dc06636dd 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -541,13 +541,8 @@ private void handleQueryStmt() throws Exception { // send result RowBatch batch; MysqlChannel channel = context.getMysqlChannel(); - boolean isSendFields = false; + sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); while (true) { - if (!isSendFields) { - sendFields(queryStmt.getColLabels(), queryStmt.getResultExprs()); - isSendFields = true; - } - batch = coord.getNext(); if (batch.getBatch() != null) { for (ByteBuffer row : batch.getBatch().getRows()) { @@ -555,7 +550,6 @@ private void handleQueryStmt() throws Exception { } context.updateReturnRows(batch.getBatch().getRows().size()); } - if (batch.isEos()) { break; }