add tablet table update report statis
This commit is contained in:
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
Reference in New Issue
Block a user