init push
This commit is contained in:
414
src/sql/monitor/ob_exec_stat.h
Normal file
414
src/sql/monitor/ob_exec_stat.h
Normal file
@ -0,0 +1,414 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifdef EVENT_INFO
|
||||
EVENT_INFO(WAIT_TIME, wait_time)
|
||||
EVENT_INFO(WAIT_COUNT, wait_count)
|
||||
EVENT_INFO(SCHED_TIME, sched_time)
|
||||
EVENT_INFO(USER_IO_TIME, user_io_time)
|
||||
EVENT_INFO(APPLICATION_TIME, application_time)
|
||||
EVENT_INFO(CONCURRENCY_TIME, concurrency_time)
|
||||
EVENT_INFO(IO_READ_COUNT, io_read_count)
|
||||
EVENT_INFO(IO_WRITES_COUNT, io_write_count)
|
||||
EVENT_INFO(IO_READ_BYTES, io_read_bytes)
|
||||
EVENT_INFO(IO_WRITE_BYTES, io_write_bytes)
|
||||
EVENT_INFO(RPC_PACKET_IN, rpc_packet_in)
|
||||
EVENT_INFO(RPC_PACKET_IN_BYTES, rpc_packet_in_bytes)
|
||||
EVENT_INFO(RPC_PACKET_OUT, rpc_packet_out)
|
||||
EVENT_INFO(RPC_PACKET_OUT_BYTES, rpc_packet_out_bytes)
|
||||
EVENT_INFO(ROW_CACHE_HIT, row_cache_hit)
|
||||
EVENT_INFO(ROW_CACHE_MISS, row_cache_miss)
|
||||
EVENT_INFO(BLOCK_INDEX_CACHE_HIT, block_index_cache_hit)
|
||||
EVENT_INFO(BLOCK_INDEX_CACHE_MISS, block_index_cache_miss)
|
||||
EVENT_INFO(BLOCK_CACHE_HIT, block_cache_hit)
|
||||
EVENT_INFO(BLOCK_CACHE_MISS, block_cache_miss)
|
||||
EVENT_INFO(BLOOM_FILTER_FILTS, bloom_filter_filts)
|
||||
EVENT_INFO(LOCATION_CACHE_HIT, location_cache_hit)
|
||||
EVENT_INFO(LOCATION_CACHE_MISS, location_cache_miss)
|
||||
EVENT_INFO(MEMSTORE_READ_LOCK_SUCC_COUNT, memstore_read_lock_succ_count)
|
||||
EVENT_INFO(MEMSTORE_WRITE_LOCK_SUCC_COUNT, memstore_write_lock_succ_count)
|
||||
EVENT_INFO(MEMSTORE_WAIT_READ_LOCK_TIME, memstore_wait_read_lock_time)
|
||||
EVENT_INFO(MEMSTORE_WAIT_WRITE_LOCK_TIME, memstore_wait_write_lock_time)
|
||||
EVENT_INFO(TRANS_COMMIT_LOG_SYNC_TIME, trans_commit_log_sync_time)
|
||||
EVENT_INFO(TRANS_COMMIT_LOG_SYNC_COUNT, trans_commit_log_sync_count)
|
||||
EVENT_INFO(TRANS_COMMIT_LOG_SUBMIT_COUNT, trans_commit_log_submit_count)
|
||||
EVENT_INFO(TRANS_COMMIT_TIME, trans_commit_time)
|
||||
EVENT_INFO(MEMSTORE_READ_ROW_COUNT, memstore_read_row_count)
|
||||
EVENT_INFO(SSSTORE_READ_ROW_COUNT, ssstore_read_row_count)
|
||||
EVENT_INFO(FUSE_ROW_CACHE_HIT, fuse_row_cache_hit)
|
||||
EVENT_INFO(FUSE_ROW_CACHE_MISS, fuse_row_cache_miss)
|
||||
#endif
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_EXEC_STAT_H
|
||||
#define OCEANBASE_SQL_OB_EXEC_STAT_H
|
||||
#include "lib/stat/ob_diagnose_info.h"
|
||||
#include "lib/wait_event/ob_wait_event.h"
|
||||
#include "lib/statistic_event/ob_stat_event.h"
|
||||
#include "lib/net/ob_addr.h"
|
||||
#include "sql/ob_sql_define.h"
|
||||
#include "sql/plan_cache/ob_plan_cache_util.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
struct ObExecRecord {
|
||||
// max wait event during sql exec
|
||||
common::ObWaitEventDesc max_wait_event_;
|
||||
|
||||
#define EVENT_INFO(def, name) \
|
||||
int64_t name##_start_; \
|
||||
int64_t name##_end_; \
|
||||
int64_t name##_;
|
||||
#include "ob_exec_stat.h"
|
||||
#undef EVENT_INFO
|
||||
|
||||
ObExecRecord()
|
||||
{
|
||||
MEMSET(this, 0, sizeof(*this));
|
||||
}
|
||||
|
||||
#define EVENT_INFO(def, name) \
|
||||
int64_t get_##name() const \
|
||||
{ \
|
||||
return name##_end_ - name##_start_; \
|
||||
}
|
||||
#include "ob_exec_stat.h"
|
||||
#undef EVENT_INFO
|
||||
|
||||
#define RECORD(se, di) \
|
||||
do { \
|
||||
oceanbase::common::ObDiagnoseSessionInfo* diag_session_info = \
|
||||
(NULL != di) ? di : oceanbase::common::ObDiagnoseSessionInfo::get_local_diagnose_info(); \
|
||||
if (NULL != diag_session_info) { \
|
||||
io_read_count_##se##_ = EVENT_GET(ObStatEventIds::IO_READ_COUNT, diag_session_info); \
|
||||
io_write_count_##se##_ = EVENT_GET(ObStatEventIds::IO_WRITE_COUNT, diag_session_info); \
|
||||
block_cache_hit_##se##_ = EVENT_GET(ObStatEventIds::BLOCK_CACHE_HIT, diag_session_info); \
|
||||
io_read_bytes_##se##_ = EVENT_GET(ObStatEventIds::IO_READ_BYTES, diag_session_info); \
|
||||
io_write_bytes_##se##_ = EVENT_GET(ObStatEventIds::IO_WRITE_BYTES, diag_session_info); \
|
||||
rpc_packet_in_##se##_ = EVENT_GET(ObStatEventIds::RPC_PACKET_IN, diag_session_info); \
|
||||
rpc_packet_in_bytes_##se##_ = EVENT_GET(ObStatEventIds::RPC_PACKET_IN_BYTES, diag_session_info); \
|
||||
rpc_packet_out_##se##_ = EVENT_GET(ObStatEventIds::RPC_PACKET_OUT, diag_session_info); \
|
||||
rpc_packet_out_bytes_##se##_ = EVENT_GET(ObStatEventIds::RPC_PACKET_OUT_BYTES, diag_session_info); \
|
||||
trans_commit_log_sync_time_##se##_ = EVENT_GET(ObStatEventIds::TRANS_COMMIT_LOG_SYNC_TIME, diag_session_info); \
|
||||
row_cache_hit_##se##_ = EVENT_GET(ObStatEventIds::ROW_CACHE_HIT, diag_session_info); \
|
||||
row_cache_miss_##se##_ = EVENT_GET(ObStatEventIds::ROW_CACHE_MISS, diag_session_info); \
|
||||
block_index_cache_hit_##se##_ = EVENT_GET(ObStatEventIds::BLOCK_INDEX_CACHE_HIT, diag_session_info); \
|
||||
block_index_cache_miss_##se##_ = EVENT_GET(ObStatEventIds::BLOCK_INDEX_CACHE_MISS, diag_session_info); \
|
||||
block_cache_hit_##se##_ = EVENT_GET(ObStatEventIds::BLOCK_CACHE_HIT, diag_session_info); \
|
||||
block_cache_miss_##se##_ = EVENT_GET(ObStatEventIds::BLOCK_CACHE_MISS, diag_session_info); \
|
||||
bloom_filter_filts_##se##_ = EVENT_GET(ObStatEventIds::BLOOM_FILTER_FILTS, diag_session_info); \
|
||||
location_cache_hit_##se##_ = EVENT_GET(ObStatEventIds::LOCATION_CACHE_HIT, diag_session_info); \
|
||||
location_cache_miss_##se##_ = EVENT_GET(ObStatEventIds::LOCATION_CACHE_MISS, diag_session_info); \
|
||||
memstore_read_lock_succ_count_##se##_ = \
|
||||
EVENT_GET(ObStatEventIds::MEMSTORE_READ_LOCK_SUCC_COUNT, diag_session_info); \
|
||||
memstore_write_lock_succ_count_##se##_ = \
|
||||
EVENT_GET(ObStatEventIds::MEMSTORE_WRITE_LOCK_SUCC_COUNT, diag_session_info); \
|
||||
memstore_wait_read_lock_time_##se##_ = \
|
||||
EVENT_GET(ObStatEventIds::MEMSTORE_WAIT_READ_LOCK_TIME, diag_session_info); \
|
||||
memstore_wait_write_lock_time_##se##_ = \
|
||||
EVENT_GET(ObStatEventIds::MEMSTORE_WAIT_WRITE_LOCK_TIME, diag_session_info); \
|
||||
memstore_read_row_count_##se##_ = EVENT_GET(ObStatEventIds::MEMSTORE_READ_ROW_COUNT, diag_session_info); \
|
||||
ssstore_read_row_count_##se##_ = EVENT_GET(ObStatEventIds::SSSTORE_READ_ROW_COUNT, diag_session_info); \
|
||||
fuse_row_cache_hit_##se##_ = EVENT_GET(ObStatEventIds::FUSE_ROW_CACHE_HIT, diag_session_info); \
|
||||
fuse_row_cache_miss_##se##_ = EVENT_GET(ObStatEventIds::FUSE_ROW_CACHE_MISS, diag_session_info); \
|
||||
for (int i = 0; i < oceanbase::common::ObWaitEventIds::WAIT_EVENT_END; ++i) { \
|
||||
if (NULL != diag_session_info->get_event_stats().get(i)) { \
|
||||
uint64_t time = diag_session_info->get_event_stats().get(i)->time_waited_; \
|
||||
switch (oceanbase::common::OB_WAIT_EVENTS[i].wait_class_) { \
|
||||
case oceanbase::common::ObWaitClassIds::USER_IO: { \
|
||||
user_io_time_##se##_ += time; \
|
||||
break; \
|
||||
} \
|
||||
case oceanbase::common::ObWaitClassIds::SCHEDULER: { \
|
||||
sched_time_##se##_ += time; \
|
||||
break; \
|
||||
} \
|
||||
case oceanbase::common::ObWaitClassIds::CONCURRENCY: { \
|
||||
concurrency_time_##se##_ += time; \
|
||||
break; \
|
||||
} \
|
||||
case oceanbase::common::ObWaitClassIds::APPLICATION: { \
|
||||
application_time_##se##_ += time; \
|
||||
break; \
|
||||
} \
|
||||
default: { \
|
||||
/*do nothing*/ \
|
||||
break; \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
} /*for end */ \
|
||||
} \
|
||||
} while (0);
|
||||
|
||||
#define UPDATE_EVENT(event) \
|
||||
do { \
|
||||
event##_ += event##_end_ - event##_start_; \
|
||||
} while (0);
|
||||
|
||||
void record_start(common::ObDiagnoseSessionInfo* di = NULL)
|
||||
{
|
||||
RECORD(start, di);
|
||||
}
|
||||
|
||||
void record_end(common::ObDiagnoseSessionInfo* di = NULL)
|
||||
{
|
||||
RECORD(end, di);
|
||||
}
|
||||
|
||||
void update_stat()
|
||||
{
|
||||
UPDATE_EVENT(wait_time);
|
||||
UPDATE_EVENT(wait_count);
|
||||
UPDATE_EVENT(io_read_count);
|
||||
UPDATE_EVENT(io_write_count);
|
||||
UPDATE_EVENT(block_cache_hit);
|
||||
UPDATE_EVENT(io_read_bytes);
|
||||
UPDATE_EVENT(io_write_bytes);
|
||||
UPDATE_EVENT(rpc_packet_in);
|
||||
UPDATE_EVENT(rpc_packet_in_bytes);
|
||||
UPDATE_EVENT(rpc_packet_out);
|
||||
UPDATE_EVENT(rpc_packet_out_bytes);
|
||||
UPDATE_EVENT(trans_commit_log_sync_time);
|
||||
UPDATE_EVENT(row_cache_hit);
|
||||
UPDATE_EVENT(row_cache_miss);
|
||||
UPDATE_EVENT(block_index_cache_hit);
|
||||
UPDATE_EVENT(block_index_cache_miss);
|
||||
UPDATE_EVENT(bloom_filter_filts);
|
||||
UPDATE_EVENT(location_cache_hit);
|
||||
UPDATE_EVENT(location_cache_miss);
|
||||
UPDATE_EVENT(memstore_read_lock_succ_count);
|
||||
UPDATE_EVENT(memstore_write_lock_succ_count);
|
||||
UPDATE_EVENT(memstore_wait_read_lock_time);
|
||||
UPDATE_EVENT(memstore_wait_write_lock_time);
|
||||
UPDATE_EVENT(user_io_time);
|
||||
UPDATE_EVENT(sched_time);
|
||||
UPDATE_EVENT(concurrency_time);
|
||||
UPDATE_EVENT(application_time);
|
||||
UPDATE_EVENT(memstore_read_row_count);
|
||||
UPDATE_EVENT(ssstore_read_row_count);
|
||||
}
|
||||
};
|
||||
|
||||
enum ExecType { InvalidType = 0, MpQuery, InnerSql, RpcProcessor, PLSql };
|
||||
|
||||
struct ObReqTimestamp {
|
||||
ObReqTimestamp()
|
||||
{
|
||||
MEMSET(this, 0, sizeof(*this));
|
||||
}
|
||||
int64_t receive_timestamp_;
|
||||
int64_t run_timestamp_;
|
||||
int64_t enqueue_timestamp_;
|
||||
};
|
||||
|
||||
struct ObExecTimestamp {
|
||||
ObExecTimestamp()
|
||||
{
|
||||
MEMSET(this, 0, sizeof(*this));
|
||||
}
|
||||
ExecType exec_type_;
|
||||
|
||||
int64_t rpc_send_ts_; // Send rpc timestamp
|
||||
int64_t receive_ts_; // The timestamp of the received request, followed by the net wait time
|
||||
//***The timestamp below needs to be updated after each retry***
|
||||
int64_t enter_queue_ts_; // Enter the queue timestamp
|
||||
int64_t run_ts_; // The timestamp of the start of the run, followed by the decode time
|
||||
int64_t before_process_ts_; // The timestamp of the beginning of the before process
|
||||
int64_t single_process_ts_; // Start timestamp of a single sql do process
|
||||
int64_t process_executor_ts_; // The time stamp when the plan started executing
|
||||
int64_t executor_end_ts_; // The timestamp of the end of plan execution
|
||||
|
||||
int64_t elapsed_t_;
|
||||
//**** The following records only time-consuming during the first execution**
|
||||
int64_t net_t_;
|
||||
int64_t net_wait_t_;
|
||||
//***** The following records are the cumulative time ***
|
||||
int64_t queue_t_;
|
||||
int64_t decode_t_;
|
||||
int64_t get_plan_t_;
|
||||
int64_t executor_t_;
|
||||
|
||||
// Time accumulates when there is a retry
|
||||
void update_stage_time()
|
||||
{
|
||||
elapsed_t_ = executor_end_ts_ - receive_ts_; // Retry does not need to accumulate
|
||||
queue_t_ += run_ts_ - enter_queue_ts_;
|
||||
decode_t_ += before_process_ts_ - run_ts_;
|
||||
get_plan_t_ += process_executor_ts_ - single_process_ts_;
|
||||
executor_t_ += executor_end_ts_ - process_executor_ts_;
|
||||
}
|
||||
};
|
||||
|
||||
class ObSchedInfo {
|
||||
public:
|
||||
ObSchedInfo() : sched_info_(NULL), sched_info_len_(0)
|
||||
{}
|
||||
void reset()
|
||||
{
|
||||
MEMSET(this, 0, sizeof(*this));
|
||||
}
|
||||
inline void assign(char* sched_info, int64_t info_len)
|
||||
{
|
||||
sched_info_ = sched_info;
|
||||
sched_info_len_ = info_len;
|
||||
}
|
||||
int append(common::ObIAllocator& allocator, const char* sched_info, int64_t info_len)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (sched_info_len_ >= 0 && info_len > 0 && sched_info_len_ + info_len <= common::OB_MAX_SCHED_INFO_LENGTH) {
|
||||
void* ptr = NULL;
|
||||
if (OB_UNLIKELY(NULL == (ptr = allocator.alloc(info_len + sched_info_len_)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_MONITOR_LOG(WARN, "fail to alloc sched info", K(ret));
|
||||
} else {
|
||||
char* str = static_cast<char*>(ptr);
|
||||
if (sched_info_ != NULL && sched_info_len_ > 0) {
|
||||
MEMCPY(str, sched_info_, sched_info_len_);
|
||||
}
|
||||
MEMCPY(str + sched_info_len_, sched_info, info_len);
|
||||
assign(str, info_len + sched_info_len_);
|
||||
}
|
||||
} else {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
SQL_MONITOR_LOG(WARN, "sched info len is invalid", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
inline const char* get_ptr() const
|
||||
{
|
||||
return sched_info_;
|
||||
}
|
||||
inline int64_t get_len() const
|
||||
{
|
||||
return sched_info_len_;
|
||||
}
|
||||
|
||||
private:
|
||||
char* sched_info_;
|
||||
int64_t sched_info_len_;
|
||||
};
|
||||
|
||||
struct ObAuditRecordData {
|
||||
ObAuditRecordData() : sched_info_()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
~ObAuditRecordData()
|
||||
{
|
||||
sched_info_.reset();
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
MEMSET(this, 0, sizeof(*this));
|
||||
consistency_level_ = common::INVALID_CONSISTENCY;
|
||||
ps_stmt_id_ = OB_INVALID_STMT_ID;
|
||||
trans_hash_ = 0;
|
||||
request_type_ = EXECUTE_INVALID;
|
||||
is_batched_multi_stmt_ = false;
|
||||
plan_hash_ = 0;
|
||||
trx_lock_for_read_elapse_ = 0;
|
||||
}
|
||||
|
||||
int64_t get_elapsed_time() const
|
||||
{
|
||||
return exec_timestamp_.executor_end_ts_ - exec_timestamp_.receive_ts_;
|
||||
}
|
||||
|
||||
int64_t get_process_time() const
|
||||
{
|
||||
return exec_timestamp_.executor_end_ts_ - exec_timestamp_.single_process_ts_;
|
||||
}
|
||||
|
||||
void update_stage_stat()
|
||||
{
|
||||
exec_timestamp_.update_stage_time();
|
||||
exec_record_.update_stat();
|
||||
const int64_t cpu_time = MAX(exec_timestamp_.elapsed_t_ - exec_record_.wait_time_, 0);
|
||||
const int64_t elapsed_time = MAX(exec_timestamp_.elapsed_t_, 0);
|
||||
if (is_inner_sql_) {
|
||||
EVENT_ADD(SYS_TIME_MODEL_DB_INNER_TIME, elapsed_time);
|
||||
EVENT_ADD(SYS_TIME_MODEL_DB_INNER_CPU, cpu_time);
|
||||
} else {
|
||||
EVENT_ADD(SYS_TIME_MODEL_DB_TIME, elapsed_time);
|
||||
EVENT_ADD(SYS_TIME_MODEL_DB_CPU, cpu_time);
|
||||
}
|
||||
}
|
||||
|
||||
bool is_timeout() const
|
||||
{
|
||||
return common::OB_TIMEOUT == status_ || common::OB_TRANS_STMT_TIMEOUT == status_;
|
||||
}
|
||||
|
||||
int64_t get_extra_size() const
|
||||
{
|
||||
return sql_len_ + tenant_name_len_ + user_name_len_ + db_name_len_;
|
||||
}
|
||||
|
||||
int16_t seq_; // packet->get_packet_header().seq_; always 0 currently
|
||||
int status_; // error code
|
||||
uint64_t trace_id_[2];
|
||||
int64_t request_id_; // set by request_manager automatic when add record
|
||||
int64_t execution_id_; // used to jion v$sql_plan_monitor
|
||||
uint64_t session_id_;
|
||||
uint64_t qc_id_;
|
||||
int64_t dfo_id_;
|
||||
int64_t sqc_id_;
|
||||
int64_t worker_id_;
|
||||
common::ObAddr server_addr_;
|
||||
common::ObAddr client_addr_;
|
||||
common::ObAddr user_client_addr_;
|
||||
int64_t tenant_id_;
|
||||
int64_t effective_tenant_id_;
|
||||
char* tenant_name_;
|
||||
int64_t tenant_name_len_;
|
||||
int64_t user_id_;
|
||||
char* user_name_;
|
||||
int64_t user_name_len_;
|
||||
int user_group_; // The cgroup id that the user belongs to, only displayed on the main thread
|
||||
uint64_t db_id_;
|
||||
char* db_name_;
|
||||
int64_t db_name_len_;
|
||||
char sql_id_[common::OB_MAX_SQL_ID_LENGTH + 1];
|
||||
char* sql_; // The memory is allocated by allocate_ and released when the record is eliminated;
|
||||
int64_t sql_len_;
|
||||
int64_t plan_id_;
|
||||
int64_t affected_rows_; // The number of rows affected by delete, update, insert, and the number of rows selected by
|
||||
// select
|
||||
int64_t return_rows_;
|
||||
int64_t partition_cnt_; // The number of partitions involved in the request
|
||||
int64_t expected_worker_cnt_; // px expected number of threads allocated
|
||||
int64_t used_worker_cnt_; // px actual number of threads allocated
|
||||
int64_t try_cnt_; // Number of attempts to execute
|
||||
ObPhyPlanType plan_type_;
|
||||
bool is_executor_rpc_;
|
||||
bool is_inner_sql_;
|
||||
bool is_hit_plan_cache_;
|
||||
bool is_multi_stmt_; // Is it multi sql
|
||||
bool table_scan_;
|
||||
common::ObConsistencyLevel consistency_level_;
|
||||
int64_t request_memory_used_;
|
||||
ObExecTimestamp exec_timestamp_;
|
||||
ObExecRecord exec_record_;
|
||||
ObTableScanStat table_scan_stat_;
|
||||
ObSchedInfo sched_info_; // px sched info
|
||||
int64_t ps_stmt_id_;
|
||||
int64_t request_type_;
|
||||
uint64_t trans_hash_;
|
||||
bool is_batched_multi_stmt_;
|
||||
ObString ob_trace_info_;
|
||||
uint64_t plan_hash_;
|
||||
int64_t trx_lock_for_read_elapse_;
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif
|
||||
178
src/sql/monitor/ob_exec_stat_collector.cpp
Normal file
178
src/sql/monitor/ob_exec_stat_collector.cpp
Normal file
@ -0,0 +1,178 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_MONITOR
|
||||
#include "sql/monitor/ob_exec_stat_collector.h"
|
||||
#include "sql/monitor/ob_phy_operator_monitor_info.h"
|
||||
#include "sql/monitor/ob_phy_plan_monitor_info.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/ob_phy_operator.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "observer/ob_inner_sql_result.h"
|
||||
#include "sql/ob_sql.h"
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::observer;
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
template <class T>
|
||||
int ObExecStatCollector::add_stat(const T* value)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(value)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(value));
|
||||
} else if (OB_FAIL(serialization::encode_vi32(extend_buf_, MAX_STAT_BUF_COUNT, length_, value->get_type()))) {
|
||||
LOG_WARN("fail to encode type", K(ret), K(value));
|
||||
} else if (OB_FAIL(value->serialize(extend_buf_, MAX_STAT_BUF_COUNT, length_))) {
|
||||
LOG_WARN("fail to serialize value", K(ret), K(value));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExecStatCollector::add_raw_stat(const common::ObString& str)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (length_ + str.length() >= MAX_STAT_BUF_COUNT) {
|
||||
ret = OB_BUF_NOT_ENOUGH;
|
||||
LOG_DEBUG("buffer size not enough", K(ret), K(length_), K(str.length()));
|
||||
} else {
|
||||
MEMCPY(extend_buf_ + length_, str.ptr(), str.length());
|
||||
length_ += str.length();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExecStatCollector::get_extend_info(ObIAllocator& allocator, ObString& str)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObString tmp_str(length_, extend_buf_);
|
||||
if (OB_FAIL(ob_write_string(allocator, tmp_str, str))) {
|
||||
LOG_WARN("fail to write string", K(tmp_str), K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
void ObExecStatCollector::reset()
|
||||
{
|
||||
length_ = 0;
|
||||
}
|
||||
|
||||
int ObExecStatCollector::collect_plan_monitor_info(
|
||||
uint64_t job_id, uint64_t task_id, ObPhyPlanMonitorInfo* monitor_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(monitor_info)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
SQL_MONITOR_LOG(WARN, "invalid argument", K(ret), K(monitor_info));
|
||||
} else {
|
||||
ObPhyOperatorMonitorInfo* op_info = NULL;
|
||||
for (int64_t i = 0; i < monitor_info->get_operator_count() && OB_SUCC(ret); i++) {
|
||||
if (OB_FAIL(monitor_info->get_operator_info_by_index(i, op_info))) {
|
||||
SQL_MONITOR_LOG(WARN, "fail to get operator info by index", K(ret), K(i));
|
||||
} else if (OB_ISNULL(op_info)) {
|
||||
SQL_MONITOR_LOG(WARN, "get invalie op_info", K(ret), K(op_info));
|
||||
} else if (OB_FAIL(op_info->set_job_id(job_id))) {
|
||||
SQL_MONITOR_LOG(WARN, "fail to set job id", K(ret), K(job_id));
|
||||
} else if (OB_FAIL(op_info->set_task_id(task_id))) {
|
||||
SQL_MONITOR_LOG(WARN, "fail to to set task id", K(ret), K(task_id));
|
||||
} else if (OB_FAIL(add_stat<ObPhyOperatorMonitorInfo>(op_info))) {
|
||||
SQL_MONITOR_LOG(WARN, "fail to add value", K(ret), K(i));
|
||||
} else {
|
||||
LOG_DEBUG("collect plan monitor info", K(*op_info));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExecStatCollector::collect_monitor_info(uint64_t job_id, uint64_t task_id, ObPhyOperatorMonitorInfo& op_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(op_info.set_job_id(job_id))) {
|
||||
SQL_MONITOR_LOG(WARN, "fail to set job id", K(ret), K(job_id));
|
||||
} else if (OB_FAIL(op_info.set_task_id(task_id))) {
|
||||
SQL_MONITOR_LOG(WARN, "fail to to set task id", K(ret), K(task_id));
|
||||
} else if (OB_FAIL(add_stat<ObPhyOperatorMonitorInfo>(&op_info))) {
|
||||
SQL_MONITOR_LOG(WARN, "fail to add value", K(ret));
|
||||
} else {
|
||||
LOG_DEBUG("add monitor info", K(op_info));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
//////////////////////////////////////
|
||||
int ObExecStatDispatch::set_extend_info(const ObString& stat_buf)
|
||||
{
|
||||
stat_str_.assign_ptr(stat_buf.ptr(), stat_buf.length());
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
int ObExecStatDispatch::dispatch(
|
||||
bool need_add_monitor, ObPhyPlanMonitorInfo* monitor_info, bool need_update_plan, ObPhysicalPlan* plan)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
StatType type = OB_INVALID_STAT_TYPE;
|
||||
if ((need_add_monitor && OB_ISNULL(monitor_info)) || OB_ISNULL(plan)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(monitor_info), K(plan));
|
||||
}
|
||||
while (OB_SUCC(ret) && OB_SUCC(get_next_type(type))) {
|
||||
switch (type) {
|
||||
case PLAN_MONITOR_INFO: {
|
||||
ObPhyOperatorMonitorInfo op_info;
|
||||
if (OB_FAIL(get_value<ObPhyOperatorMonitorInfo>(&op_info))) {
|
||||
LOG_WARN("fail to get value", K(ret));
|
||||
} else if (need_add_monitor && OB_FAIL(monitor_info->add_operator_info(op_info))) {
|
||||
LOG_WARN("fail to add operator info", K(ret), K(op_info));
|
||||
} else if (need_update_plan && OB_FAIL(plan->op_stats_.add_op_stat(op_info))) {
|
||||
LOG_WARN("fail to add operatgor info", K(ret), K(op_info));
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
ret = OB_UNKNOWN_OBJ;
|
||||
LOG_WARN("unknown type", K(ret), K(type));
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExecStatDispatch::get_next_type(StatType& type)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t type_value = 0;
|
||||
if (pos_ == stat_str_.length()) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_FAIL(serialization::decode_vi32(stat_str_.ptr(), stat_str_.length(), pos_, &type_value))) {
|
||||
LOG_WARN("fail to decode type", K(ret), K(pos_));
|
||||
} else {
|
||||
type = static_cast<StatType>(type_value);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template <class T>
|
||||
int ObExecStatDispatch::get_value(T* value)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(value)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(value));
|
||||
} else if (OB_FAIL(value->deserialize(stat_str_.ptr(), stat_str_.length(), pos_))) {
|
||||
LOG_WARN("fail to deserialize value", K(ret), K(pos_));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
117
src/sql/monitor/ob_exec_stat_collector.h
Normal file
117
src/sql/monitor/ob_exec_stat_collector.h
Normal file
@ -0,0 +1,117 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_EXEC_STAT_COLLECTOR_H_
|
||||
#define OCEANBASE_SQL_OB_EXEC_STAT_COLLECTOR_H_
|
||||
|
||||
#include "lib/utility/ob_macro_utils.h"
|
||||
#include "lib/string/ob_string.h"
|
||||
#include "lib/trace/ob_trace_event.h"
|
||||
#include "sql/monitor/ob_phy_operator_monitor_info.h"
|
||||
#include "sql/monitor/ob_exec_stat.h"
|
||||
#include "lib/net/ob_addr.h"
|
||||
namespace oceanbase {
|
||||
namespace observer {
|
||||
class ObInnerSQLResult;
|
||||
}
|
||||
namespace sql {
|
||||
class ObExecContext;
|
||||
class ObPhyPlanMonitorInfo;
|
||||
class ObPhyOperatorMonitorInfo;
|
||||
class ObSql;
|
||||
class ObPhysicalPlan;
|
||||
class ObSQLSessionInfo;
|
||||
class ObExecStatCollector {
|
||||
public:
|
||||
ObExecStatCollector() : length_(0)
|
||||
{}
|
||||
~ObExecStatCollector()
|
||||
{}
|
||||
int collect_monitor_info(uint64_t job_id, uint64_t task_id, ObPhyOperatorMonitorInfo& op_info);
|
||||
|
||||
int collect_plan_monitor_info(uint64_t job_id, uint64_t task_id, ObPhyPlanMonitorInfo* monitor_info);
|
||||
int add_raw_stat(const common::ObString& str);
|
||||
int get_extend_info(common::ObIAllocator& allocator, common::ObString& str);
|
||||
void reset();
|
||||
|
||||
private:
|
||||
template <class T>
|
||||
int add_stat(const T* value);
|
||||
|
||||
/* functions */
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExecStatCollector);
|
||||
static const int64_t MAX_STAT_BUF_COUNT = 10240;
|
||||
char extend_buf_[MAX_STAT_BUF_COUNT];
|
||||
int64_t length_;
|
||||
};
|
||||
|
||||
class ObExecStatDispatch {
|
||||
public:
|
||||
ObExecStatDispatch() : stat_str_(), pos_(0){};
|
||||
~ObExecStatDispatch(){};
|
||||
int set_extend_info(const common::ObString& str);
|
||||
int dispatch(bool need_add_monitor, ObPhyPlanMonitorInfo* monitor_info, bool need_update_plan, ObPhysicalPlan* plan);
|
||||
|
||||
private:
|
||||
int get_next_type(StatType& type);
|
||||
template <class T>
|
||||
int get_value(T* value);
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExecStatDispatch);
|
||||
common::ObString stat_str_;
|
||||
int64_t pos_;
|
||||
};
|
||||
|
||||
class ObExecStatUtils {
|
||||
public:
|
||||
template <class T>
|
||||
OB_INLINE static void record_exec_timestamp(const T& process,
|
||||
bool is_first, // fir execution, not retry
|
||||
ObExecTimestamp& exec_timestamp)
|
||||
{
|
||||
exec_timestamp.rpc_send_ts_ = process.get_send_timestamp();
|
||||
exec_timestamp.receive_ts_ = process.get_receive_timestamp();
|
||||
exec_timestamp.enter_queue_ts_ = process.get_enqueue_timestamp();
|
||||
exec_timestamp.run_ts_ = process.get_run_timestamp();
|
||||
exec_timestamp.before_process_ts_ = process.get_process_timestamp();
|
||||
exec_timestamp.single_process_ts_ = process.get_single_process_timestamp();
|
||||
exec_timestamp.process_executor_ts_ = process.get_exec_start_timestamp();
|
||||
exec_timestamp.executor_end_ts_ = process.get_exec_end_timestamp();
|
||||
|
||||
if (is_first) {
|
||||
/* packet event order:
|
||||
* send -> receive -> enter_queue -> run -> before_process -> single process -> executor
|
||||
*/
|
||||
exec_timestamp.net_t_ = exec_timestamp.receive_ts_ - exec_timestamp.rpc_send_ts_;
|
||||
exec_timestamp.net_wait_t_ = exec_timestamp.enter_queue_ts_ - exec_timestamp.receive_ts_;
|
||||
}
|
||||
// SQL_MONITOR_LOG(DEBUG, "set exec timestamp", K(exec_timestamp.exec_type_),
|
||||
// K(exec_timestamp.before_process_ts_),
|
||||
// K(exec_timestamp.process_executor_ts_),
|
||||
// K(exec_timestamp.executor_end_ts_),
|
||||
// K(exec_timestamp.receive_ts_),
|
||||
// K(exec_timestamp.enter_queue_ts_),
|
||||
// K(exec_timestamp.run_ts_),
|
||||
// K(exec_timestamp.single_process_ts_));
|
||||
}
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExecStatUtils);
|
||||
ObExecStatUtils();
|
||||
~ObExecStatUtils();
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif /* OCEANBASE_COMMON_STAT_OB_EXEC_STAT_COLLECTOR_H_ */
|
||||
//// end of header file
|
||||
37
src/sql/monitor/ob_i_collect_value.h
Normal file
37
src/sql/monitor/ob_i_collect_value.h
Normal file
@ -0,0 +1,37 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_I_COLLECT_VALUE_H
|
||||
#define OCEANBASE_SQL_OB_I_COLLECT_VALUE_H
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
enum StatType { OB_INVALID_STAT_TYPE = 0, PLAN_MONITOR_INFO };
|
||||
|
||||
class ObIValue {
|
||||
public:
|
||||
explicit ObIValue(StatType type)
|
||||
{
|
||||
value_type_ = type;
|
||||
}
|
||||
virtual ~ObIValue()
|
||||
{}
|
||||
StatType get_type() const
|
||||
{
|
||||
return value_type_;
|
||||
}
|
||||
|
||||
protected:
|
||||
StatType value_type_;
|
||||
};
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif
|
||||
38
src/sql/monitor/ob_monitor_info_elimination_task.cpp
Normal file
38
src/sql/monitor/ob_monitor_info_elimination_task.cpp
Normal file
@ -0,0 +1,38 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#include "sql/monitor/ob_monitor_info_elimination_task.h"
|
||||
#include "sql/monitor/ob_monitor_info_manager.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
int ObMonitorInfoEliminationTask::init(ObMonitorInfoManager* info)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (OB_ISNULL(info)) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
SQL_MONITOR_LOG(WARN, "invalid argument", K(ret), K(info));
|
||||
} else {
|
||||
monitor_info_ = info;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
void ObMonitorInfoEliminationTask::runTimerTask()
|
||||
{
|
||||
if (OB_ISNULL(monitor_info_)) {
|
||||
SQL_MONITOR_LOG(ERROR, "invalid history info", K(monitor_info_));
|
||||
} else {
|
||||
monitor_info_->print_memory_size();
|
||||
monitor_info_->gc();
|
||||
}
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
37
src/sql/monitor/ob_monitor_info_elimination_task.h
Normal file
37
src/sql/monitor/ob_monitor_info_elimination_task.h
Normal file
@ -0,0 +1,37 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_MONITOR_INFO_ELIMINATION_TASK_H
|
||||
#define OCEANBASE_SQL_OB_MONITOR_INFO_ELIMINATION_TASK_H
|
||||
#include "lib/task/ob_timer.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObMonitorInfoManager;
|
||||
class ObMonitorInfoEliminationTask : public common::ObTimerTask {
|
||||
public:
|
||||
ObMonitorInfoEliminationTask() : monitor_info_(NULL)
|
||||
{}
|
||||
~ObMonitorInfoEliminationTask()
|
||||
{}
|
||||
int init(ObMonitorInfoManager* monitor_info);
|
||||
virtual void runTimerTask();
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMonitorInfoEliminationTask);
|
||||
|
||||
private:
|
||||
ObMonitorInfoManager* monitor_info_;
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif
|
||||
236
src/sql/monitor/ob_monitor_info_manager.cpp
Normal file
236
src/sql/monitor/ob_monitor_info_manager.cpp
Normal file
@ -0,0 +1,236 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_MONITOR
|
||||
#include "sql/monitor/ob_monitor_info_manager.h"
|
||||
#include "sql/monitor/ob_phy_plan_monitor_info.h"
|
||||
using namespace oceanbase::common;
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
ObMonitorInfoManager::ObMonitorInfoManager()
|
||||
: allocator_(),
|
||||
slow_query_queue_(),
|
||||
timer_(),
|
||||
elimination_task_(),
|
||||
plan_execution_time_map_(),
|
||||
max_push_interval_(OB_MAX_PUSH_INTERVAL),
|
||||
operator_info_size_(0)
|
||||
{
|
||||
memory_limit_ = min(
|
||||
MAX_MEMORY_SIZE, static_cast<int64_t>(static_cast<double>(GCONF.get_server_memory_avail()) * MONITOR_MEM_FACTOR));
|
||||
}
|
||||
|
||||
ObMonitorInfoManager::~ObMonitorInfoManager()
|
||||
{
|
||||
destroy();
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::init()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t delay = 5 * 1000 * 1000;
|
||||
if (OB_FAIL(allocator_.init(memory_limit_, memory_limit_, PAGE_SIZE))) {
|
||||
LOG_WARN("fail to init allocator", K(ret));
|
||||
} else if (OB_FAIL(slow_query_queue_.init(ObModIds::OB_SQL_PLAN_MONITOR, OB_MAX_QUEUE_SIZE))) {
|
||||
LOG_WARN("fail to init history info", K(ret));
|
||||
} else if (OB_FAIL(timer_.init("MonInfoEvict"))) {
|
||||
LOG_WARN("fail to init timer", K(ret));
|
||||
} else if (OB_FAIL(elimination_task_.init(this))) {
|
||||
LOG_WARN("fail to init elimination task", K(ret));
|
||||
} else if (OB_FAIL(plan_execution_time_map_.init(ObModIds::OB_SQL_PLAN_MONITOR))) {
|
||||
LOG_WARN("fail to init plan execution time map", K(ret));
|
||||
} else {
|
||||
allocator_.set_label(ObModIds::OB_SQL_PLAN_MONITOR);
|
||||
if (timer_.schedule(elimination_task_, delay, true)) {
|
||||
LOG_WARN("fail to schedule timer task", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMonitorInfoManager::destroy()
|
||||
{
|
||||
IGNORE_RETURN plan_execution_time_map_.destroy();
|
||||
timer_.destroy();
|
||||
clear_queue(INT64_MAX);
|
||||
slow_query_queue_.destroy();
|
||||
allocator_.destroy();
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::get_by_request_id(
|
||||
int64_t request_id, int64_t& index, ObPhyPlanMonitorInfo*& plan_info, Ref* ref)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
index = -1;
|
||||
if (OB_UNLIKELY(request_id <= 0)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid agument", K(ret), K(request_id));
|
||||
} else {
|
||||
int64_t start_idx = slow_query_queue_.get_pop_idx();
|
||||
int64_t end_idx = slow_query_queue_.get_push_idx();
|
||||
void* tmp_info = NULL;
|
||||
ret = OB_ERROR;
|
||||
for (int64_t i = start_idx; i <= end_idx && OB_ERROR == ret; i++) {
|
||||
if (NULL == (tmp_info = slow_query_queue_.get(i, ref))) {
|
||||
ret = OB_ENTRY_NOT_EXIST;
|
||||
} else if (request_id == static_cast<ObPhyPlanMonitorInfo*>(tmp_info)->get_request_id()) {
|
||||
plan_info = static_cast<ObPhyPlanMonitorInfo*>(tmp_info);
|
||||
index = i;
|
||||
ret = OB_SUCCESS;
|
||||
break;
|
||||
}
|
||||
if (NULL != ref) {
|
||||
slow_query_queue_.revert(ref);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::get_by_index(int64_t index, ObPhyPlanMonitorInfo*& plan_info, Ref* ref)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
plan_info = NULL;
|
||||
if (OB_UNLIKELY(index < 0)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid arguemnt", K(ret), K(index));
|
||||
} else if (index >= get_start_index() + get_size()) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (NULL == (plan_info = static_cast<ObPhyPlanMonitorInfo*>(slow_query_queue_.get(index, ref)))) {
|
||||
ret = OB_ENTRY_NOT_EXIST;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::is_info_nearly_duplicated(const ObAddr& addr, int64_t plan_id, bool& is_duplicated)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
is_duplicated = false;
|
||||
int64_t last_execution_time = 0;
|
||||
int64_t current_time = common::ObTimeUtility::current_time();
|
||||
PlanKey key;
|
||||
key.plan_id_ = plan_id;
|
||||
key.addr_ = addr;
|
||||
if (OB_FAIL(plan_execution_time_map_.get(key, last_execution_time))) {
|
||||
if (OB_ENTRY_NOT_EXIST == ret) {
|
||||
if (OB_FAIL(plan_execution_time_map_.insert(key, current_time))) {
|
||||
LOG_WARN("fail to insert into plan execution time map", K(ret), K(key.plan_id_));
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("fail to get last execution time", K(ret));
|
||||
}
|
||||
} else {
|
||||
if (current_time - last_execution_time < max_push_interval_) {
|
||||
is_duplicated = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::add_monitor_info(ObPhyPlanMonitorInfo* info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(info)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid agument", K(ret), K(info));
|
||||
} else {
|
||||
int64_t retry_times = 3;
|
||||
while (retry_times > 0) {
|
||||
retry_times--;
|
||||
int64_t req_id = 0;
|
||||
if (OB_FAIL(slow_query_queue_.push((void*)info, req_id))) {
|
||||
if (OB_SIZE_OVERFLOW == ret) {
|
||||
clear_queue(OB_BATCH_GC_COUNT);
|
||||
}
|
||||
} else {
|
||||
info->set_request_id(req_id);
|
||||
operator_info_size_ += info->get_operator_info_memory_size();
|
||||
LOG_DEBUG("add monitor info", K(*info));
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret) && NULL != info) {
|
||||
free(info);
|
||||
info = NULL;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMonitorInfoManager::clear_queue(int64_t limit)
|
||||
{
|
||||
int64_t pop_cnt = 0;
|
||||
ObPhyPlanMonitorInfo* poped = NULL;
|
||||
while (pop_cnt++ < limit && NULL != (poped = (ObPhyPlanMonitorInfo*)slow_query_queue_.pop())) {
|
||||
operator_info_size_ -= poped->get_operator_info_memory_size();
|
||||
poped->destroy();
|
||||
}
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::alloc(int64_t request_id, ObPhyPlanMonitorInfo*& info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void* ptr = NULL;
|
||||
if (OB_UNLIKELY(request_id < 0)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid agument", K(ret), K(request_id));
|
||||
} else if (OB_ISNULL(ptr = allocator_.alloc(sizeof(ObPhyPlanMonitorInfo)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_ERROR("fail to alloc memory", K(ret), K(request_id), K(allocator_.allocated()), K_(memory_limit));
|
||||
} else {
|
||||
info = new (ptr) ObPhyPlanMonitorInfo(allocator_);
|
||||
info->set_request_id(request_id);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::free(ObPhyPlanMonitorInfo*& info)
|
||||
{
|
||||
if (OB_ISNULL(info)) {
|
||||
} else {
|
||||
info->~ObPhyPlanMonitorInfo();
|
||||
allocator_.free(info);
|
||||
info = NULL;
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::reclain_map()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t timestamp = ObTimeUtility::current_time();
|
||||
ReclainCond cond(timestamp, max_push_interval_);
|
||||
if (OB_FAIL(plan_execution_time_map_.remove_if(cond))) {
|
||||
LOG_WARN("fail to remove map", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitorInfoManager::gc()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t allocated_size = allocator_.allocated() + operator_info_size_;
|
||||
int64_t mem_limit = memory_limit_ * 8 / 10;
|
||||
if (mem_limit < allocated_size) {
|
||||
int64_t pop_count = 3;
|
||||
while (mem_limit / 2 < allocator_.allocated() + operator_info_size_ && pop_count > 0) {
|
||||
clear_queue(OB_BATCH_GC_COUNT);
|
||||
pop_count--;
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(reclain_map())) {
|
||||
LOG_WARN("fail to reclain map", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
128
src/sql/monitor/ob_monitor_info_manager.h
Normal file
128
src/sql/monitor/ob_monitor_info_manager.h
Normal file
@ -0,0 +1,128 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_MONITOR_INFO_MANAGER_H
|
||||
#define OCEANBASE_SQL_OB_MONITOR_INFO_MANAGER_H
|
||||
#include "lib/task/ob_timer.h"
|
||||
#include "lib/atomic/ob_atomic.h"
|
||||
#include "lib/hash/ob_linear_hash_map.h"
|
||||
#include "lib/allocator/ob_concurrent_fifo_allocator.h"
|
||||
#include "sql/monitor/ob_monitor_info_elimination_task.h"
|
||||
#include "observer/mysql/ob_mysql_request_manager.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
struct PlanKey {
|
||||
uint64_t hash() const
|
||||
{
|
||||
uint64_t value = 0;
|
||||
value = murmurhash(&plan_id_, sizeof(plan_id_), value);
|
||||
value = murmurhash(&addr_, sizeof(addr_), value);
|
||||
return value;
|
||||
}
|
||||
bool operator==(const PlanKey& other) const
|
||||
{
|
||||
return (plan_id_ == other.plan_id_) && (addr_ == other.addr_);
|
||||
}
|
||||
int64_t plan_id_;
|
||||
common::ObAddr addr_;
|
||||
};
|
||||
class ObMonitorInfoManager {
|
||||
friend class TestMonitorInfoManager_tets_duplicated_Test;
|
||||
struct ReclainCond {
|
||||
public:
|
||||
ReclainCond(int64_t curr_timestamp, int64_t max_remain_interval)
|
||||
: curr_timestamp_(curr_timestamp), max_remain_interval_(max_remain_interval)
|
||||
{}
|
||||
int64_t curr_timestamp_;
|
||||
int64_t max_remain_interval_;
|
||||
bool operator()(const PlanKey& plan_id, int64_t& execution_time)
|
||||
{
|
||||
UNUSED(plan_id);
|
||||
bool bret = false;
|
||||
if (curr_timestamp_ - execution_time < max_remain_interval_) {
|
||||
} else {
|
||||
bret = true;
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
};
|
||||
typedef common::ObRaQueue::Ref Ref;
|
||||
|
||||
public:
|
||||
ObMonitorInfoManager();
|
||||
~ObMonitorInfoManager();
|
||||
int init();
|
||||
void destroy();
|
||||
int get_by_request_id(int64_t request_id, int64_t& index, ObPhyPlanMonitorInfo*& plan_info, Ref* ref);
|
||||
int get_by_index(int64_t index, ObPhyPlanMonitorInfo*& plan_info, Ref* ref);
|
||||
int revert(Ref* req)
|
||||
{
|
||||
slow_query_queue_.revert(req);
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
int add_monitor_info(ObPhyPlanMonitorInfo* plan_info);
|
||||
int alloc(int64_t request_id, ObPhyPlanMonitorInfo*& plan_info);
|
||||
int free(ObPhyPlanMonitorInfo*& plan_info);
|
||||
int64_t get_start_index()
|
||||
{
|
||||
return slow_query_queue_.get_pop_idx();
|
||||
}
|
||||
int64_t get_size()
|
||||
{
|
||||
return slow_query_queue_.get_size();
|
||||
}
|
||||
int64_t get_count()
|
||||
{
|
||||
return slow_query_queue_.get_size();
|
||||
}
|
||||
int is_info_nearly_duplicated(const common::ObAddr& addr, int64_t plan_id, bool& is_duplicated);
|
||||
void print_memory_size()
|
||||
{
|
||||
SQL_MONITOR_LOG(INFO,
|
||||
"print memory size, ",
|
||||
K(get_start_index()),
|
||||
K(get_size()),
|
||||
K(allocator_.allocated()),
|
||||
K(allocator_.hold()),
|
||||
K(operator_info_size_));
|
||||
}
|
||||
int gc();
|
||||
static const int64_t OB_MAX_QUEUE_SIZE = 100000;
|
||||
static const int64_t MAX_MEMORY_SIZE = 500 * 1024 * 1024L;
|
||||
static const int64_t PAGE_SIZE = common::OB_MALLOC_BIG_BLOCK_SIZE;
|
||||
static const int64_t OB_BATCH_GC_COUNT = 2000;
|
||||
static const int64_t OB_MAX_PUSH_INTERVAL = 10 * 1000 * 1000;
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMonitorInfoManager);
|
||||
void clear_queue(int64_t limit);
|
||||
int reclain_map();
|
||||
void set_max_push_interval(int64_t time)
|
||||
{
|
||||
max_push_interval_ = time;
|
||||
}
|
||||
|
||||
private:
|
||||
common::ObConcurrentFIFOAllocator allocator_;
|
||||
common::ObRaQueue slow_query_queue_;
|
||||
common::ObTimer timer_; // for timing task for cleaning data.
|
||||
ObMonitorInfoEliminationTask elimination_task_;
|
||||
common::ObLinearHashMap<PlanKey, int64_t> plan_execution_time_map_; // last execution time of plan.
|
||||
int64_t max_push_interval_;
|
||||
int64_t memory_limit_;
|
||||
int64_t operator_info_size_; // the memory of operator_infos_ need stat separately,
|
||||
// because it is not alloced from allocator.
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif
|
||||
268
src/sql/monitor/ob_phy_operator_monitor_info.cpp
Normal file
268
src/sql/monitor/ob_phy_operator_monitor_info.cpp
Normal file
@ -0,0 +1,268 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_MONITOR
|
||||
#include "sql/monitor/ob_phy_operator_monitor_info.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/ob_name_def.h"
|
||||
#include "share/ob_time_utility2.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::share;
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
ObPhyOperatorMonitorInfo::ObPhyOperatorMonitorInfo()
|
||||
: ObIValue(PLAN_MONITOR_INFO), op_id_(-1), job_id_(0), task_id_(0), op_type_(PHY_INVALID)
|
||||
{
|
||||
memset(info_array_, 0, OB_MAX_INFORMATION_COUNT * sizeof(int64_t));
|
||||
}
|
||||
|
||||
ObPhyOperatorMonitorInfo::~ObPhyOperatorMonitorInfo()
|
||||
{}
|
||||
|
||||
int ObPhyOperatorMonitorInfo::set_operator_id(int64_t op_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (op_id < 0) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument.", K(op_id));
|
||||
} else {
|
||||
op_id_ = op_id;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhyOperatorMonitorInfo::set_job_id(int64_t job_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (job_id < 0) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument.", K(job_id));
|
||||
} else {
|
||||
job_id_ = job_id;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhyOperatorMonitorInfo::set_task_id(int64_t task_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (task_id < 0) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument.", K(task_id));
|
||||
} else {
|
||||
task_id_ = task_id;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhyOperatorMonitorInfo::assign(const ObPhyOperatorMonitorInfo& other)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
op_id_ = other.op_id_;
|
||||
job_id_ = other.job_id_;
|
||||
task_id_ = other.task_id_;
|
||||
op_type_ = other.op_type_;
|
||||
for (int64_t i = 0; i < OB_MAX_INFORMATION_COUNT; i++) {
|
||||
info_array_[i] = other.info_array_[i];
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObPhyOperatorMonitorInfo::operator=(const ObPhyOperatorMonitorInfo& other)
|
||||
{
|
||||
if (OB_SUCCESS != assign(other)) {
|
||||
LOG_ERROR("fail to assign", K(&other));
|
||||
}
|
||||
}
|
||||
|
||||
ObPhyOperatorMonitorInfo::ObPhyOperatorMonitorInfo(const ObPhyOperatorMonitorInfo& other)
|
||||
: ObIValue(PLAN_MONITOR_INFO),
|
||||
op_id_(other.op_id_),
|
||||
job_id_(other.job_id_),
|
||||
task_id_(other.task_id_),
|
||||
op_type_(other.op_type_)
|
||||
{
|
||||
for (int64_t i = 0; i < OB_MAX_INFORMATION_COUNT; i++) {
|
||||
info_array_[i] = other.info_array_[i];
|
||||
}
|
||||
}
|
||||
|
||||
int64_t ObPhyOperatorMonitorInfo::to_string(char* buf, int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
J_KV(N_OP_ID, op_id_, N_JOB_ID, job_id_, N_TASK_ID, task_id_, N_OP, op_type_);
|
||||
for (int64_t i = 0; i < OB_MAX_INFORMATION_COUNT; i++) {
|
||||
if (info_array_[i] != 0 && NULL != OB_OPERATOR_MONITOR_INFOS[i].info_name_) {
|
||||
J_OBJ_START();
|
||||
J_KV(OB_OPERATOR_MONITOR_INFOS[i].info_name_, info_array_[i]);
|
||||
J_OBJ_END();
|
||||
J_COMMA();
|
||||
}
|
||||
}
|
||||
J_OBJ_END();
|
||||
return pos;
|
||||
}
|
||||
|
||||
bool ObPhyOperatorMonitorInfo::is_timestamp(int64_t i) const
|
||||
{
|
||||
bool bret = false;
|
||||
if (OPEN_TIME == i || FIRST_ROW_TIME == i || LAST_ROW_TIME == i || CLOSE_TIME == i) {
|
||||
bret = true;
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
|
||||
int64_t ObPhyOperatorMonitorInfo::print_info(char* buf, int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
bool first_cell = true;
|
||||
const int64_t time_buf_len = 128;
|
||||
char timebuf[time_buf_len];
|
||||
int64_t time_buf_pos = 0;
|
||||
|
||||
for (int64_t i = 0; i < OB_MAX_INFORMATION_COUNT; i++) {
|
||||
time_buf_pos = 0;
|
||||
if (info_array_[i] != 0 && NULL != OB_OPERATOR_MONITOR_INFOS[i].info_name_) {
|
||||
if (first_cell) {
|
||||
first_cell = false;
|
||||
} else {
|
||||
J_COMMA();
|
||||
}
|
||||
J_OBJ_START();
|
||||
if (is_timestamp(i)) {
|
||||
if (OB_SUCCESS != ObTimeUtility2::usec_to_str(info_array_[i], timebuf, time_buf_len, time_buf_pos)) {
|
||||
LOG_WARN("fail to print time as str", K(i));
|
||||
J_KV(OB_OPERATOR_MONITOR_INFOS[i].info_name_, info_array_[i]);
|
||||
} else {
|
||||
timebuf[time_buf_pos] = '\0';
|
||||
J_KV(OB_OPERATOR_MONITOR_INFOS[i].info_name_, timebuf);
|
||||
}
|
||||
} else {
|
||||
J_KV(OB_OPERATOR_MONITOR_INFOS[i].info_name_, info_array_[i]);
|
||||
}
|
||||
J_OBJ_END();
|
||||
}
|
||||
}
|
||||
J_OBJ_END();
|
||||
return pos;
|
||||
}
|
||||
|
||||
void ObPhyOperatorMonitorInfo::set_value(ObOperatorMonitorInfoIds index, int64_t value)
|
||||
{
|
||||
if (index >= 0 && index < OB_MAX_INFORMATION_COUNT) {
|
||||
info_array_[index] = value;
|
||||
} else {
|
||||
LOG_WARN("invalid index", K(index), K(value));
|
||||
}
|
||||
}
|
||||
|
||||
void ObPhyOperatorMonitorInfo::increase_value(ObOperatorMonitorInfoIds index)
|
||||
{
|
||||
if (index >= 0 && index < OB_MAX_INFORMATION_COUNT) {
|
||||
info_array_[index]++;
|
||||
} else {
|
||||
LOG_WARN("invalid index", K(index));
|
||||
}
|
||||
}
|
||||
|
||||
void ObPhyOperatorMonitorInfo::get_value(ObOperatorMonitorInfoIds index, int64_t& value)
|
||||
{
|
||||
if (index >= 0 && index < OB_MAX_INFORMATION_COUNT) {
|
||||
value = info_array_[index];
|
||||
} else {
|
||||
LOG_WARN("invalid index", K(index));
|
||||
}
|
||||
}
|
||||
|
||||
int64_t ObPhyOperatorMonitorInfo::get_valid_info_count() const
|
||||
{
|
||||
int64_t count = 0;
|
||||
for (int64_t i = 0; i < OB_MAX_INFORMATION_COUNT; i++) {
|
||||
if (info_array_[i] != 0) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
OB_DEF_SERIALIZE(ObPhyOperatorMonitorInfo)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, op_id_);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, job_id_);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, task_id_);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, op_type_);
|
||||
int64_t valid_count = get_valid_info_count();
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, valid_count);
|
||||
for (int64_t i = 0; i < OB_MAX_INFORMATION_COUNT && OB_SUCC(ret); i++) {
|
||||
if (info_array_[i] != 0) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, i))) {
|
||||
LOG_WARN("fail to encode vi64", K(ret), K(i), K(buf), K(buf_len), K(pos));
|
||||
} else if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, info_array_[i]))) {
|
||||
LOG_WARN("fail to encode vi64", K(ret), K(i), K(buf), K(buf_len), K(pos), K(info_array_[i]));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObPhyOperatorMonitorInfo)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t valid_count = 0;
|
||||
LST_DO_CODE(OB_UNIS_DECODE, op_id_);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, job_id_);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, task_id_);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, op_type_);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, valid_count);
|
||||
int64_t index = 0;
|
||||
int64_t value = 0;
|
||||
for (int64_t i = 0; i < valid_count && OB_SUCC(ret); i++) {
|
||||
if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &index))) {
|
||||
LOG_WARN("fail to decode index", K(ret), K(pos));
|
||||
} else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &value))) {
|
||||
LOG_WARN("fail to decode value", K(ret), K(pos));
|
||||
} else {
|
||||
if (index >= OB_MAX_INFORMATION_COUNT) {
|
||||
} else if (index < 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("invalid index", K(index), K(i), K(value), K(ret));
|
||||
} else {
|
||||
set_value(static_cast<ObOperatorMonitorInfoIds>(index), value);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObPhyOperatorMonitorInfo)
|
||||
{
|
||||
int64_t len = 0;
|
||||
int64_t valid_count = get_valid_info_count();
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, op_id_);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, job_id_);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, task_id_);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, op_type_);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, valid_count);
|
||||
for (int64_t i = 0; i < OB_MAX_INFORMATION_COUNT; i++) {
|
||||
if (info_array_[i] != 0) {
|
||||
len += serialization::encoded_length_vi64(i);
|
||||
len += serialization::encoded_length_vi64(info_array_[i]);
|
||||
}
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
102
src/sql/monitor/ob_phy_operator_monitor_info.h
Normal file
102
src/sql/monitor/ob_phy_operator_monitor_info.h
Normal file
@ -0,0 +1,102 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifdef OPERATOR_MONITOR_INFO_DEF
|
||||
OPERATOR_MONITOR_INFO_DEF(OPEN_TIME, open_time)
|
||||
OPERATOR_MONITOR_INFO_DEF(FIRST_ROW_TIME, first_row)
|
||||
OPERATOR_MONITOR_INFO_DEF(LAST_ROW_TIME, last_row)
|
||||
OPERATOR_MONITOR_INFO_DEF(CLOSE_TIME, close_time)
|
||||
OPERATOR_MONITOR_INFO_DEF(RESCAN_TIMES, rescan_times)
|
||||
OPERATOR_MONITOR_INFO_DEF(INPUT_ROW_COUNT, input_row_count)
|
||||
OPERATOR_MONITOR_INFO_DEF(OUTPUT_ROW_COUNT, output_row_count)
|
||||
OPERATOR_MONITOR_INFO_DEF(MEMORY_USED, memory_used)
|
||||
OPERATOR_MONITOR_INFO_DEF(DISK_READ_COUNT, disk_read_count)
|
||||
OPERATOR_MONITOR_INFO_DEF(MONITOR_INFO_END, monitor_end)
|
||||
#endif
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_PHY_OPERATOR_MONITOR_INFORMATION_H
|
||||
#define OCEANBASE_SQL_OB_PHY_OPERATOR_MONITOR_INFORMATION_H
|
||||
#include "share/ob_define.h"
|
||||
#include "lib/utility/ob_unify_serialize.h"
|
||||
#include "sql/monitor/ob_i_collect_value.h"
|
||||
#include "sql/engine/ob_phy_operator_type.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
struct MonitorName {
|
||||
int64_t id_;
|
||||
const char* const info_name_;
|
||||
};
|
||||
enum ObOperatorMonitorInfoIds {
|
||||
#define OPERATOR_MONITOR_INFO_DEF(def, name) def,
|
||||
#include "ob_phy_operator_monitor_info.h"
|
||||
#undef OPERATOR_MONITOR_INFO_DEF
|
||||
};
|
||||
static const MonitorName OB_OPERATOR_MONITOR_INFOS[] = {
|
||||
#define OPERATOR_MONITOR_INFO_DEF(def, name) {def, #name},
|
||||
#include "ob_phy_operator_monitor_info.h"
|
||||
#undef OPERATOR_MONITOR_INFO_DEF
|
||||
};
|
||||
|
||||
class ObPhyOperatorMonitorInfo : public ObIValue {
|
||||
OB_UNIS_VERSION(1);
|
||||
|
||||
public:
|
||||
ObPhyOperatorMonitorInfo();
|
||||
virtual ~ObPhyOperatorMonitorInfo();
|
||||
int set_operator_id(int64_t op_id);
|
||||
int set_job_id(int64_t job_id);
|
||||
int set_task_id(int64_t task_id);
|
||||
void set_operator_type(ObPhyOperatorType type)
|
||||
{
|
||||
op_type_ = type;
|
||||
}
|
||||
int64_t get_op_id() const
|
||||
{
|
||||
return op_id_;
|
||||
}
|
||||
int64_t get_job_id() const
|
||||
{
|
||||
return job_id_;
|
||||
}
|
||||
int64_t get_task_id() const
|
||||
{
|
||||
return task_id_;
|
||||
}
|
||||
ObPhyOperatorType get_operator_type() const
|
||||
{
|
||||
return op_type_;
|
||||
}
|
||||
int assign(const ObPhyOperatorMonitorInfo& info);
|
||||
void operator=(const ObPhyOperatorMonitorInfo& other);
|
||||
ObPhyOperatorMonitorInfo(const ObPhyOperatorMonitorInfo& other);
|
||||
int64_t to_string(char* buf, int64_t buf_len) const;
|
||||
virtual int64_t print_info(char* buf, int64_t buf_len) const;
|
||||
void set_value(ObOperatorMonitorInfoIds index, int64_t value);
|
||||
void get_value(ObOperatorMonitorInfoIds index, int64_t& value);
|
||||
void increase_value(ObOperatorMonitorInfoIds index);
|
||||
static const int64_t OB_MAX_INFORMATION_COUNT = MONITOR_INFO_END;
|
||||
|
||||
private:
|
||||
int64_t get_valid_info_count() const;
|
||||
virtual bool is_timestamp(int64_t index) const;
|
||||
|
||||
protected:
|
||||
int64_t op_id_;
|
||||
int64_t job_id_; // Effective in distributed execution plan
|
||||
int64_t task_id_; // Effective in distributed execution plan
|
||||
private:
|
||||
ObPhyOperatorType op_type_;
|
||||
uint64_t info_array_[OB_MAX_INFORMATION_COUNT];
|
||||
};
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif
|
||||
121
src/sql/monitor/ob_phy_operator_stats.cpp
Normal file
121
src/sql/monitor/ob_phy_operator_stats.cpp
Normal file
@ -0,0 +1,121 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_MONITOR
|
||||
#include "sql/monitor/ob_phy_operator_stats.h"
|
||||
#include "sql/monitor/ob_phy_operator_monitor_info.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/plan_cache/ob_plan_cache_util.h"
|
||||
#include "lib/allocator/ob_allocator.h"
|
||||
#include "lib/atomic/ob_atomic.h"
|
||||
#include "lib/utility/utility.h"
|
||||
using namespace oceanbase::common;
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
int ObPhyOperatorStats::init(ObIAllocator* alloc, int64_t op_count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void* ptr = NULL;
|
||||
if (OB_ISNULL(alloc) || OB_UNLIKELY(op_count < 0)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(alloc), K(op_count));
|
||||
} else {
|
||||
array_size_ = op_count * StatId::MAX_STAT * COPY_COUNT;
|
||||
if (OB_ISNULL(ptr = alloc->alloc(sizeof(int64_t) * array_size_))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_ERROR("fail to alloc memory for aray", K(ret), K_(array_size));
|
||||
} else {
|
||||
op_stats_array_ = static_cast<int64_t*>(ptr);
|
||||
memset(op_stats_array_, 0, sizeof(int64_t) * array_size_);
|
||||
op_count_ = op_count;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/*
|
||||
*
|
||||
* |- COPY1 -||- COPY2 -|
|
||||
* |- op1 -||- op2 -||- op1
|
||||
* |---------------------------------------------------------+---------------------------|
|
||||
* | E0 | I0 | O0 | R0 || E'1 | I'1 | O'1 | R'1 || ...
|
||||
* |-------------------------------------------------------------------------------------|
|
||||
*
|
||||
* E: EXEC_COUNT
|
||||
* I: INPUT_ROWS
|
||||
* O: OUTPUT_ROWS
|
||||
* R: RESCAN_TIMES
|
||||
* COPY = 10
|
||||
*/
|
||||
|
||||
int ObPhyOperatorStats::add_op_stat(ObPhyOperatorMonitorInfo& info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const int64_t COPY_SIZE = op_count_ * StatId::MAX_STAT;
|
||||
int64_t copy_start_index = (get_cpu_id() % COPY_COUNT) * COPY_SIZE;
|
||||
int64_t stat_start_index = copy_start_index + info.get_op_id() * StatId::MAX_STAT;
|
||||
if (stat_start_index < 0 || stat_start_index + StatId::MAX_STAT > array_size_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invaild array index", K(stat_start_index), K(array_size_));
|
||||
} else {
|
||||
int64_t last_input_rows = 0;
|
||||
int64_t last_output_rows = 0;
|
||||
int64_t rescan_times = 0;
|
||||
info.get_value(INPUT_ROW_COUNT, last_input_rows);
|
||||
info.get_value(OUTPUT_ROW_COUNT, last_output_rows);
|
||||
info.get_value(RESCAN_TIMES, rescan_times);
|
||||
ATOMIC_AAF(&(op_stats_array_[stat_start_index + StatId::INPUT_ROWS]), last_input_rows);
|
||||
ATOMIC_AAF(&(op_stats_array_[stat_start_index + StatId::OUTPUT_ROWS]), last_output_rows);
|
||||
ATOMIC_AAF(&(op_stats_array_[stat_start_index + StatId::RESCAN_TIMES]), rescan_times);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhyOperatorStats::get_op_stat_accumulation(ObPhysicalPlan* plan, int64_t op_id, ObOperatorStat& stat)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(plan)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(plan));
|
||||
} else {
|
||||
int64_t exec_times = 0;
|
||||
int64_t op_first_index = op_id * StatId::MAX_STAT;
|
||||
int64_t retry_times = 3;
|
||||
const int64_t COPY_SIZE = op_count_ * StatId::MAX_STAT;
|
||||
int64_t copy_start_index = 0;
|
||||
int64_t max_index = op_first_index + (COPY_COUNT - 1) * COPY_SIZE + StatId::MAX_STAT;
|
||||
if (op_first_index < 0 || max_index > array_size_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid array index", K(op_first_index), K(max_index), K(array_size_));
|
||||
} else {
|
||||
do {
|
||||
exec_times = ATOMIC_LOAD(&(execution_times_));
|
||||
stat.init();
|
||||
stat.execute_times_ = exec_times;
|
||||
retry_times--;
|
||||
for (int64_t i = 0; i < COPY_COUNT; i++) {
|
||||
copy_start_index = op_first_index + i * COPY_SIZE;
|
||||
stat.input_rows_ += ATOMIC_LOAD(&(op_stats_array_[copy_start_index + StatId::INPUT_ROWS]));
|
||||
stat.rescan_times_ += ATOMIC_LOAD(&(op_stats_array_[copy_start_index + StatId::RESCAN_TIMES]));
|
||||
stat.output_rows_ += ATOMIC_LOAD(&(op_stats_array_[copy_start_index + StatId::OUTPUT_ROWS]));
|
||||
}
|
||||
} while (exec_times != ATOMIC_LOAD(&(execution_times_)) && retry_times > 0);
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
stat.operation_id_ = op_id;
|
||||
stat.plan_id_ = plan->get_plan_id();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
64
src/sql/monitor/ob_phy_operator_stats.h
Normal file
64
src/sql/monitor/ob_phy_operator_stats.h
Normal file
@ -0,0 +1,64 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_PHY_OPERATOR_STATS_H
|
||||
#define OCEANBASE_SQL_OB_PHY_OPERATOR_STATS_H
|
||||
#include "share/ob_define.h"
|
||||
#include "lib/atomic/ob_atomic.h"
|
||||
#include "lib/utility/ob_macro_utils.h"
|
||||
namespace oceanbase {
|
||||
namespace common {
|
||||
class ObIAllocator;
|
||||
}
|
||||
namespace sql {
|
||||
class ObPhyOperatorMonitorInfo;
|
||||
class ObOperatorStat;
|
||||
class ObPhysicalPlan;
|
||||
namespace StatId {
|
||||
enum StatId { INPUT_ROWS = 0, RESCAN_TIMES, OUTPUT_ROWS, MAX_STAT };
|
||||
}
|
||||
class ObPhyOperatorStats {
|
||||
public:
|
||||
friend class TestPhyOperatorStats_init_Test;
|
||||
friend class TestPhyOperatorStats_test_add_Test;
|
||||
friend class TestPhyOperatorStats_test_accumulation_Test;
|
||||
ObPhyOperatorStats() : op_stats_array_(NULL), op_count_(0), array_size_(0), execution_times_(0)
|
||||
{}
|
||||
~ObPhyOperatorStats()
|
||||
{}
|
||||
int init(common::ObIAllocator* alloc, int64_t op_count);
|
||||
int add_op_stat(ObPhyOperatorMonitorInfo& info);
|
||||
int get_op_stat_accumulation(ObPhysicalPlan* plan, int64_t op_id, ObOperatorStat& stat);
|
||||
static const int64_t COPY_COUNT = 10;
|
||||
int64_t count()
|
||||
{
|
||||
return op_count_;
|
||||
}
|
||||
void inc_execution_times()
|
||||
{
|
||||
ATOMIC_INC(&execution_times_);
|
||||
}
|
||||
int64_t get_execution_times() const
|
||||
{
|
||||
return execution_times_;
|
||||
}
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObPhyOperatorStats);
|
||||
int64_t* op_stats_array_;
|
||||
int64_t op_count_;
|
||||
int64_t array_size_;
|
||||
int64_t execution_times_;
|
||||
};
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif
|
||||
169
src/sql/monitor/ob_phy_plan_exec_info.h
Normal file
169
src/sql/monitor/ob_phy_plan_exec_info.h
Normal file
@ -0,0 +1,169 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifdef PLAN_MONITOR_INFO_DEF
|
||||
PLAN_MONITOR_INFO_DEF(QUERY_TYPE, query_type)
|
||||
PLAN_MONITOR_INFO_DEF(TOTAL_WAIT_TIME, total_wait_time)
|
||||
PLAN_MONITOR_INFO_DEF(TOTAL_WAIT_COUNT, total_wait_count)
|
||||
// timestamp
|
||||
PLAN_MONITOR_INFO_DEF(RPC_SEND_TIME, rpc_send_time)
|
||||
PLAN_MONITOR_INFO_DEF(RECEIVE_TIME, receive_time)
|
||||
PLAN_MONITOR_INFO_DEF(ENTER_QUEUE_TIME, enter_queue_time)
|
||||
PLAN_MONITOR_INFO_DEF(RUN_TIME, run_time)
|
||||
PLAN_MONITOR_INFO_DEF(BEFORE_PROCESS_TIME, before_process_time)
|
||||
PLAN_MONITOR_INFO_DEF(SINGLE_PROCESS_TIME, single_process_time)
|
||||
PLAN_MONITOR_INFO_DEF(PROCESS_EXECUTOR_TIME, process_executor_time)
|
||||
PLAN_MONITOR_INFO_DEF(EXECUTOR_END_TIME, executor_end_time)
|
||||
#endif
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_PHY_PLAN_EXEC_INFO_H
|
||||
#define OCEANBASE_SQL_OB_PHY_PLAN_EXEC_INFO_H
|
||||
#include "share/ob_time_utility2.h"
|
||||
#include "sql/monitor/ob_phy_operator_monitor_info.h"
|
||||
#include "sql/monitor/ob_exec_stat.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
enum ObPlanMonitorInfoIds {
|
||||
#define PLAN_MONITOR_INFO_DEF(def, name) def,
|
||||
#include "ob_phy_plan_exec_info.h"
|
||||
#undef PLAN_MONITOR_INFO_DEF
|
||||
#define EVENT_INFO(def, name) def,
|
||||
#include "ob_exec_stat.h"
|
||||
#undef EVENT_INFO
|
||||
MAX_EVENT_ID
|
||||
};
|
||||
|
||||
static const MonitorName OB_PLAN_MONITOR_INFOS[] = {
|
||||
#define PLAN_MONITOR_INFO_DEF(def, name) {def, #name},
|
||||
#include "ob_phy_plan_exec_info.h"
|
||||
#undef PLAN_MONITOR_INFO_DEF
|
||||
#define EVENT_INFO(def, name) {def, #name},
|
||||
#include "ob_exec_stat.h"
|
||||
#undef EVENT_INFO
|
||||
{MAX_EVENT_ID, "max_event"}};
|
||||
|
||||
class ObPhyPlanExecInfo final : public ObPhyOperatorMonitorInfo {
|
||||
public:
|
||||
ObPhyPlanExecInfo()
|
||||
{}
|
||||
inline virtual int64_t print_info(char* buf, int64_t buf_len) const;
|
||||
inline int add_exec_record(const ObExecRecord& exec_record);
|
||||
inline int add_exec_timestamp(const ObExecTimestamp& exec_timestamp);
|
||||
int64_t to_string(char* buf, int64_t buf_len) const
|
||||
{
|
||||
return print_info(buf, buf_len);
|
||||
}
|
||||
|
||||
private:
|
||||
void set_value(ObPlanMonitorInfoIds index, int64_t value)
|
||||
{
|
||||
plan_info_array_[index] = value;
|
||||
}
|
||||
bool is_timestamp(int64_t index) const
|
||||
{
|
||||
return (index >= RPC_SEND_TIME) && (index <= EXECUTOR_END_TIME);
|
||||
}
|
||||
DISALLOW_COPY_AND_ASSIGN(ObPhyPlanExecInfo);
|
||||
|
||||
private:
|
||||
common::ObWaitEventDesc max_wait_event_;
|
||||
int64_t plan_info_array_[MAX_EVENT_ID];
|
||||
};
|
||||
|
||||
int64_t ObPhyPlanExecInfo::print_info(char* buf, int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
const int64_t time_buf_len = 128;
|
||||
char timebuf[time_buf_len];
|
||||
int64_t time_buf_pos = 0;
|
||||
|
||||
J_OBJ_START();
|
||||
J_KV(N_MAX_WAIT_EVENT, max_wait_event_);
|
||||
J_OBJ_END();
|
||||
J_COMMA();
|
||||
bool first_cell = true;
|
||||
for (int64_t i = 0; i < MAX_EVENT_ID; i++) {
|
||||
time_buf_pos = 0;
|
||||
if (plan_info_array_[i] != 0) {
|
||||
if (first_cell) {
|
||||
first_cell = false;
|
||||
} else {
|
||||
J_COMMA();
|
||||
}
|
||||
J_OBJ_START();
|
||||
if (is_timestamp(i)) {
|
||||
if (common::OB_SUCCESS !=
|
||||
share::ObTimeUtility2::usec_to_str(plan_info_array_[i], timebuf, time_buf_len, time_buf_pos)) {
|
||||
SQL_MONITOR_LOG(WARN, "fail to print time as str", K(i));
|
||||
J_KV(OB_PLAN_MONITOR_INFOS[i].info_name_, plan_info_array_[i]);
|
||||
} else {
|
||||
timebuf[time_buf_pos] = '\0';
|
||||
J_KV(OB_PLAN_MONITOR_INFOS[i].info_name_, timebuf);
|
||||
}
|
||||
|
||||
} else {
|
||||
J_KV(OB_PLAN_MONITOR_INFOS[i].info_name_, plan_info_array_[i]);
|
||||
}
|
||||
J_OBJ_END();
|
||||
}
|
||||
}
|
||||
return pos;
|
||||
}
|
||||
|
||||
int ObPhyPlanExecInfo::add_exec_record(const ObExecRecord& exec_record)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
op_id_ = -1;
|
||||
job_id_ = -1;
|
||||
task_id_ = -1;
|
||||
max_wait_event_ = exec_record.max_wait_event_;
|
||||
set_value(TOTAL_WAIT_TIME, exec_record.wait_time_end_);
|
||||
set_value(TOTAL_WAIT_COUNT, exec_record.wait_count_end_);
|
||||
|
||||
#define EVENT_INFO(def, name) set_value(def, exec_record.get_##name());
|
||||
#include "ob_exec_stat.h"
|
||||
#undef EVENT_INFO
|
||||
return ret;
|
||||
}
|
||||
int ObPhyPlanExecInfo::add_exec_timestamp(const ObExecTimestamp& exec_timestamp)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
SQL_MONITOR_LOG(DEBUG,
|
||||
"add exec timestamp",
|
||||
K(exec_timestamp.exec_type_),
|
||||
K(exec_timestamp.before_process_ts_),
|
||||
K(exec_timestamp.process_executor_ts_),
|
||||
K(exec_timestamp.executor_end_ts_),
|
||||
K(exec_timestamp.receive_ts_),
|
||||
K(exec_timestamp.enter_queue_ts_),
|
||||
K(exec_timestamp.run_ts_),
|
||||
K(exec_timestamp.single_process_ts_));
|
||||
set_value(QUERY_TYPE, exec_timestamp.exec_type_);
|
||||
set_value(BEFORE_PROCESS_TIME, exec_timestamp.before_process_ts_);
|
||||
set_value(PROCESS_EXECUTOR_TIME, exec_timestamp.process_executor_ts_);
|
||||
set_value(EXECUTOR_END_TIME, exec_timestamp.executor_end_ts_);
|
||||
if (InnerSql != exec_timestamp.exec_type_) {
|
||||
set_value(RECEIVE_TIME, exec_timestamp.receive_ts_);
|
||||
set_value(ENTER_QUEUE_TIME, exec_timestamp.enter_queue_ts_);
|
||||
set_value(RUN_TIME, exec_timestamp.run_ts_);
|
||||
}
|
||||
if (MpQuery == exec_timestamp.exec_type_) {
|
||||
set_value(SINGLE_PROCESS_TIME, exec_timestamp.single_process_ts_);
|
||||
}
|
||||
if (RpcProcessor == exec_timestamp.exec_type_) {
|
||||
set_value(RPC_SEND_TIME, exec_timestamp.rpc_send_ts_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif
|
||||
77
src/sql/monitor/ob_phy_plan_monitor_info.cpp
Normal file
77
src/sql/monitor/ob_phy_plan_monitor_info.cpp
Normal file
@ -0,0 +1,77 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_MONITOR
|
||||
#include "sql/monitor/ob_phy_plan_monitor_info.h"
|
||||
using namespace oceanbase::common;
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
OB_SERIALIZE_MEMBER(ObPhyPlanMonitorInfo, request_id_, plan_id_, plan_info_, operator_infos_, execution_time_);
|
||||
|
||||
ObPhyPlanMonitorInfo::ObPhyPlanMonitorInfo(common::ObConcurrentFIFOAllocator& allocator)
|
||||
: allocator_(allocator),
|
||||
request_id_(OB_INVALID_ID),
|
||||
scheduler_addr_(),
|
||||
plan_id_(OB_INVALID_ID),
|
||||
execution_time_(0),
|
||||
operator_infos_(OB_MALLOC_NORMAL_BLOCK_SIZE),
|
||||
exec_trace_(false, ObLatchIds::TRACE_RECORDER_LOCK)
|
||||
{}
|
||||
|
||||
int ObPhyPlanMonitorInfo::add_operator_info(const ObPhyOperatorMonitorInfo& info)
|
||||
{
|
||||
return operator_infos_.push_back(info);
|
||||
}
|
||||
|
||||
int ObPhyPlanMonitorInfo::get_operator_info(int64_t op_id, ObPhyOperatorMonitorInfo& info) const
|
||||
{
|
||||
int ret = OB_ENTRY_NOT_EXIST;
|
||||
ARRAY_FOREACH_NORET(operator_infos_, idx)
|
||||
{
|
||||
if (operator_infos_.at(idx).get_op_id() == op_id) {
|
||||
if (OB_FAIL(info.assign(operator_infos_.at(idx)))) {
|
||||
LOG_WARN("fail to assgin to phy_operator info", K(ret));
|
||||
} else {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhyPlanMonitorInfo::get_operator_info_by_index(int64_t index, ObPhyOperatorMonitorInfo*& info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (0 > index) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid operator index", K(index));
|
||||
} else if (index > operator_infos_.count()) {
|
||||
ret = OB_ITER_END;
|
||||
LOG_WARN("fail to get operator info", K(ret), K(index));
|
||||
} else if (index < operator_infos_.count()) {
|
||||
info = &(operator_infos_.at(index));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhyPlanMonitorInfo::set_plan_exec_record(const ObExecRecord& exec_record)
|
||||
{
|
||||
return plan_info_.add_exec_record(exec_record);
|
||||
}
|
||||
int ObPhyPlanMonitorInfo::set_plan_exec_timestamp(const ObExecTimestamp& exec_timestamp)
|
||||
{
|
||||
execution_time_ = exec_timestamp.run_ts_;
|
||||
return plan_info_.add_exec_timestamp(exec_timestamp);
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
125
src/sql/monitor/ob_phy_plan_monitor_info.h
Normal file
125
src/sql/monitor/ob_phy_plan_monitor_info.h
Normal file
@ -0,0 +1,125 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_PHY_PLAN_MONITOR_INFORMATION_H
|
||||
#define OCEANBASE_SQL_OB_PHY_PLAN_MONITOR_INFORMATION_H
|
||||
#include "lib/container/ob_se_array.h"
|
||||
#include "lib/allocator/ob_concurrent_fifo_allocator.h"
|
||||
#include "sql/monitor/ob_phy_operator_monitor_info.h"
|
||||
#include "sql/monitor/ob_phy_plan_exec_info.h"
|
||||
#include "observer/mysql/ob_mysql_request_manager.h"
|
||||
#include "lib/trace/ob_trace_event.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObPhyPlanMonitorInfo final {
|
||||
public:
|
||||
OB_UNIS_VERSION(1);
|
||||
|
||||
public:
|
||||
const static int OPERATOR_LOCAL_COUNT = 8;
|
||||
explicit ObPhyPlanMonitorInfo(common::ObConcurrentFIFOAllocator& allocator);
|
||||
virtual void destroy()
|
||||
{
|
||||
reset();
|
||||
allocator_.free(this);
|
||||
}
|
||||
void reset()
|
||||
{
|
||||
operator_infos_.reset();
|
||||
}
|
||||
|
||||
int add_operator_info(const ObPhyOperatorMonitorInfo& info);
|
||||
int64_t get_operator_count()
|
||||
{
|
||||
return operator_infos_.count();
|
||||
}
|
||||
int set_plan_exec_record(const ObExecRecord& exec_record);
|
||||
int set_plan_exec_timestamp(const ObExecTimestamp& exec_timestamp);
|
||||
int get_operator_info(int64_t op_id, ObPhyOperatorMonitorInfo& info) const;
|
||||
int get_operator_info_by_index(int64_t index, ObPhyOperatorMonitorInfo*& info);
|
||||
int get_plan_info(ObPhyPlanExecInfo*& info)
|
||||
{
|
||||
info = &plan_info_;
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
int set_trace(const common::ObTraceEventRecorder& trace)
|
||||
{
|
||||
return exec_trace_.assign(trace);
|
||||
}
|
||||
const common::ObTraceEventRecorder& get_trace() const
|
||||
{
|
||||
return exec_trace_;
|
||||
}
|
||||
int64_t to_string(char* buf, int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
J_KV(N_QID, request_id_);
|
||||
J_KV(N_PLAN_ID, plan_id_);
|
||||
J_KV(N_EXECUTION_TIME, execution_time_);
|
||||
J_KV(N_PLAN_MONITOR_INFO, plan_info_);
|
||||
for (int64_t i = 0; i < operator_infos_.count(); i++) {
|
||||
J_KV(N_OPERATOR_MONITOR_INFO, operator_infos_.at(i));
|
||||
}
|
||||
J_OBJ_END();
|
||||
return pos;
|
||||
}
|
||||
int64_t get_operator_info_memory_size()
|
||||
{
|
||||
// Local info memory is counted in alloctor_, so only non-local memory needs to be calculated
|
||||
return operator_infos_.count() < OPERATOR_LOCAL_COUNT ? 0 : operator_infos_.get_data_size();
|
||||
}
|
||||
void set_request_id(int64_t request_id)
|
||||
{
|
||||
request_id_ = request_id;
|
||||
}
|
||||
void set_plan_id(int64_t plan_id)
|
||||
{
|
||||
plan_id_ = plan_id;
|
||||
}
|
||||
int64_t get_request_id() const
|
||||
{
|
||||
return request_id_;
|
||||
}
|
||||
int64_t get_plan_id() const
|
||||
{
|
||||
return plan_id_;
|
||||
}
|
||||
int64_t get_execution_time() const
|
||||
{
|
||||
return execution_time_;
|
||||
}
|
||||
void set_address(common::ObAddr addr)
|
||||
{
|
||||
scheduler_addr_ = addr;
|
||||
}
|
||||
common::ObAddr get_address() const
|
||||
{
|
||||
return scheduler_addr_;
|
||||
}
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObPhyPlanMonitorInfo);
|
||||
|
||||
private:
|
||||
common::ObConcurrentFIFOAllocator& allocator_;
|
||||
int64_t request_id_;
|
||||
common::ObAddr scheduler_addr_;
|
||||
int64_t plan_id_;
|
||||
int64_t execution_time_;
|
||||
common::ObSEArray<ObPhyOperatorMonitorInfo, OPERATOR_LOCAL_COUNT> operator_infos_;
|
||||
ObPhyPlanExecInfo plan_info_;
|
||||
common::ObTraceEventRecorder exec_trace_;
|
||||
};
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif
|
||||
Reference in New Issue
Block a user