add tablet table update report statis

This commit is contained in:
maosy
2023-01-05 05:08:06 +00:00
committed by ob-robot
parent a046ae912c
commit a2ab33a2dd
2 changed files with 97 additions and 36 deletions

View File

@ -20,6 +20,7 @@
#include "share/ob_tablet_replica_checksum_operator.h" // for ObTabletReplicaChecksumItem
#include "lib/mysqlclient/ob_mysql_transaction.h" // ObMySQLTransaction
#include "lib/mysqlclient/ob_mysql_proxy.h"
#include "lib/thread_local/ob_tsi_factory.h"
#include "share/ob_tablet_meta_table_compaction_operator.h"
namespace oceanbase
@ -30,6 +31,41 @@ using namespace share;
namespace observer
{
void TSITabletTableUpdatStatistics::reset()
{
suc_cnt_ = 0;
fail_cnt_ = 0;
remove_task_cnt_ = 0;
update_task_cnt_ = 0;
total_wait_us_ = 0;
total_exec_us_ = 0;
}
void TSITabletTableUpdatStatistics::calc(
int64_t succ_cnt,
int64_t fail_cnt,
int64_t remove_task_cnt,
int64_t update_task_cnt,
int64_t wait_us,
int64_t exec_us)
{
total_wait_us_ += wait_us;
total_exec_us_ += exec_us;
suc_cnt_ += succ_cnt;
fail_cnt_ += fail_cnt;
remove_task_cnt_ += remove_task_cnt;
update_task_cnt_ += update_task_cnt;
}
void TSITabletTableUpdatStatistics::dump()
{
int64_t total_cnt = suc_cnt_ + fail_cnt_;
FLOG_INFO("[TABLET_TABLE_UPDATE_STATISTIC] dump tablet table update statistics",
K_(suc_cnt), K_(fail_cnt), K_(remove_task_cnt), K_(update_task_cnt),
"avg_wait_us", total_wait_us_ / total_cnt,
"avg_exec_us", total_exec_us_ / total_cnt);
}
ObTabletTableUpdateTask::~ObTabletTableUpdateTask()
{
}
@ -38,23 +74,20 @@ int ObTabletTableUpdateTask::init(
const uint64_t tenant_id,
const ObLSID &ls_id,
const ObTabletID &tablet_id,
const int64_t add_timestamp,
const int64_t add_task_cnt)
const int64_t add_timestamp)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!ls_id.is_valid_with_tenant(tenant_id)
|| !tablet_id.is_valid_with_tenant(tenant_id)
|| 0 >= add_timestamp
|| 0 > add_task_cnt)) {
|| 0 >= add_timestamp)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("task init failed", KR(ret), K(tenant_id), K(ls_id), K(tablet_id),
K(add_timestamp), K(add_task_cnt));
K(add_timestamp));
} else {
tenant_id_ = tenant_id;
ls_id_ = ls_id;
tablet_id_ = tablet_id;
add_timestamp_ = add_timestamp;
add_task_cnt_ = add_task_cnt;
}
return ret;
@ -68,7 +101,6 @@ int ObTabletTableUpdateTask::assign(const ObTabletTableUpdateTask &other)
ls_id_ = other.get_ls_id();
tablet_id_ = other.get_tablet_id();
add_timestamp_ = other.get_add_timestamp();
add_task_cnt_ = other.add_task_cnt_;
}
return ret;
}
@ -92,7 +124,6 @@ void ObTabletTableUpdateTask::reset()
ls_id_.reset();
tablet_id_.reset();
add_timestamp_ = OB_INVALID_TIMESTAMP;
add_task_cnt_ = OB_INVALID_COUNT;
}
bool ObTabletTableUpdateTask::compare_without_version(
@ -112,22 +143,12 @@ bool ObTabletTableUpdateTask::compare_without_version(
void ObTabletTableUpdateTask::check_task_status() const
{
int64_t now = ObTimeUtility::current_time();
int64_t task_execute_time = 1 * 1000 * 1000;//1s
const int64_t safe_interval = TABLET_CHECK_INTERVAL;
// need to print a WARN log if this task is not executed correctly since two minuts ago
if (now - add_timestamp_ > safe_interval) {
LOG_WARN("tablet table update task cost too much time to execute",
FLOG_WARN("tablet table update task cost too much time to execute",
K(*this), K(safe_interval), "cost_time", now - add_timestamp_);
}
//Assuming that each task takes 1s to execute,
//an error is reported if the execution time of the current task minus the time of
//the previous task exceeds 2 minutes.
if (now - add_timestamp_ - task_execute_time * add_task_cnt_ > safe_interval) {
LOG_ERROR("tablet table update task cost too much time to execute",
K(*this), K(safe_interval), "cost_time", now - add_timestamp_,
K(add_task_cnt_));
}
}
bool ObTabletTableUpdateTask::is_valid() const
@ -136,8 +157,7 @@ bool ObTabletTableUpdateTask::is_valid() const
return OB_INVALID_TENANT_ID != tenant_id_
&& ls_id_.is_valid_with_tenant(tenant_id_)
&& tablet_id_.is_valid_with_tenant(tenant_id_)
&& 0 < add_timestamp_
&& 0 <= add_task_cnt_;
&& 0 < add_timestamp_;
}
bool ObTabletTableUpdateTask::is_barrier() const
@ -222,7 +242,6 @@ int ObTabletTableUpdater::async_update(
{
int ret = OB_SUCCESS;
int64_t add_timestamp = ObTimeUtility::current_time();
const int64_t task_cnt = update_queue_.task_count();
ObTabletTableUpdateTask task;
if (OB_UNLIKELY(!is_inited())) {
ret = OB_NOT_INIT;
@ -240,9 +259,9 @@ int ObTabletTableUpdater::async_update(
} else if (OB_FAIL(task.init(tenant_id,
ls_id,
tablet_id,
add_timestamp, task_cnt))) {
add_timestamp))) {
LOG_WARN("set update task failed", KR(ret), K(tenant_id), K(ls_id), K(tablet_id),
K(add_timestamp), K(task_cnt));
K(add_timestamp));
} else if (OB_FAIL(add_task_(task))){
LOG_WARN("fail to add task", KR(ret), K(tenant_id), K(ls_id), K(tablet_id),
K(add_timestamp));
@ -436,6 +455,15 @@ int ObTabletTableUpdater::batch_process_tasks(
uint64_t meta_tenant_id = OB_INVALID_TENANT_ID;
bool tenant_dropped = false;
bool schema_not_ready = false;
int64_t succ_cnt = 0;
int64_t update_task_cnt = 0;
int64_t remove_task_cnt = 0;
int64_t wait_cost = 0;
if (OB_SUCC(ret)) {
for (int64_t i = 0; i < batch_tasks.count(); i++) { // overwrite ret
wait_cost += (start_time - batch_tasks.at(i).get_add_timestamp());
}
}
if (OB_UNLIKELY(!is_inited())) {
ret = OB_NOT_INIT;
LOG_WARN("ObTabletTableUpdater is not inited", KR(ret));
@ -475,12 +503,16 @@ int ObTabletTableUpdater::batch_process_tasks(
"update_tablet_tasks", update_tablet_tasks.count(),
"remove_tablet_tasks", remove_tablet_tasks.count());
} else {
update_task_cnt = update_tablet_replicas.count();
remove_task_cnt = remove_tablet_replicas.count();
tmp_ret = do_batch_update_(start_time, update_tablet_tasks, update_tablet_replicas, update_tablet_checksums);
if (OB_SUCCESS != tmp_ret) {
ret = OB_SUCC(ret) ? tmp_ret : ret;
LOG_WARN("do_batch_update_ failed", KR(tmp_ret), K(start_time),
"tasks count", update_tablet_tasks.count(),
"tablet replicas count", update_tablet_replicas.count());
} else {
succ_cnt += update_task_cnt;
}
if (remove_tablet_tasks.count() > 0) {
tmp_ret = do_batch_remove_(start_time, remove_tablet_tasks, remove_tablet_replicas);
@ -489,9 +521,24 @@ int ObTabletTableUpdater::batch_process_tasks(
LOG_WARN("do_batch_remove_ failed", KR(tmp_ret), K(start_time),
"tasks count", remove_tablet_tasks.count(),
"remove replicas count", remove_tablet_replicas.count());
} else {
succ_cnt += remove_task_cnt;
}
}
}
const int64_t end = ObTimeUtility::current_time();
auto* statistics = GET_TSI(TSITabletTableUpdatStatistics);
if (OB_ISNULL(statistics)) {
LOG_WARN("fail to get statistic", "ret", OB_ERR_UNEXPECTED);
} else {
(void)statistics->calc(succ_cnt, batch_tasks.count() - succ_cnt,
remove_task_cnt, update_task_cnt, wait_cost, end - start_time);
const int64_t interval = 10 * 1000 * 1000; // 1s
if (TC_REACH_TIME_INTERVAL(interval)) {
(void)statistics->dump();
(void)statistics->reset();
}
}
return ret;
}

View File

@ -35,6 +35,26 @@ namespace observer
{
class ObService;
class ObTabletTableUpdater;
struct TSITabletTableUpdatStatistics
{
public:
TSITabletTableUpdatStatistics() { reset(); }
void reset();
void calc(int64_t succ_cnt,
int64_t fail_cnt,
int64_t remove_task_cnt,
int64_t update_task_cnt,
int64_t wait_us,
int64_t exec_us);
void dump();
private:
int64_t suc_cnt_;
int64_t fail_cnt_;
int64_t remove_task_cnt_;
int64_t update_task_cnt_;
int64_t total_wait_us_;
int64_t total_exec_us_;
};
class ObTabletTableUpdateTask : public ObIUniqTaskQueueTask<ObTabletTableUpdateTask>
{
public:
@ -44,26 +64,22 @@ public:
: tenant_id_(OB_INVALID_TENANT_ID),
ls_id_(),
tablet_id_(),
add_timestamp_(OB_INVALID_TIMESTAMP),
add_task_cnt_(OB_INVALID_COUNT) {}
add_timestamp_(OB_INVALID_TIMESTAMP){}
explicit ObTabletTableUpdateTask(
const uint64_t tenant_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
const int64_t add_timestamp,
const int64_t add_task_cnt)
const int64_t add_timestamp)
: tenant_id_(tenant_id),
ls_id_(ls_id),
tablet_id_(tablet_id),
add_timestamp_(add_timestamp),
add_task_cnt_(add_task_cnt) {}
add_timestamp_(add_timestamp) {}
virtual ~ObTabletTableUpdateTask();
int init(
const uint64_t tenant_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
const int64_t add_timestamp,
const int64_t add_task_cnt);
const int64_t add_timestamp);
void reset();
// operator-related functions for ObTabletTableUpdateTask
bool is_valid() const;
@ -76,7 +92,6 @@ public:
inline const share::ObLSID &get_ls_id() const { return ls_id_; }
inline const common::ObTabletID &get_tablet_id() const { return tablet_id_; }
inline int64_t get_add_timestamp() const { return add_timestamp_; }
inline int64_t get_add_task_count() const { return add_task_cnt_; }
// other functions
bool need_process_alone() const { return false; }
@ -93,14 +108,13 @@ public:
// TODO: need to realize barrier related functions
bool is_barrier() const;
TO_STRING_KV(K_(tenant_id), K_(ls_id), K_(tablet_id), K_(add_timestamp), K_(add_task_cnt));
TO_STRING_KV(K_(tenant_id), K_(ls_id), K_(tablet_id), K_(add_timestamp));
private:
const int64_t TABLET_CHECK_INTERVAL = 120l * 1000 * 1000; //2 minutes
const int64_t TABLET_CHECK_INTERVAL = 2 * 3600 * 1000L * 1000L; //2 hour
uint64_t tenant_id_;
share::ObLSID ls_id_;
common::ObTabletID tablet_id_;
int64_t add_timestamp_;
int64_t add_task_cnt_;
};
typedef ObUniqTaskQueue<ObTabletTableUpdateTask, ObTabletTableUpdater> ObTabletTableUpdateTaskQueue;