[CP] [FEAT MERGE]checkpoint diagnose virtual tablet
Co-authored-by: LiefB <954800091@qq.com>
This commit is contained in:
parent
68a433ea37
commit
7a8c39abcb
@ -151,7 +151,7 @@ TEST_F(TestLockMemtableCheckpoint, replay_disorder)
|
||||
LOG_INFO("TestLockMemtableCheckpoint::replay_disorder 5");
|
||||
ret = memtable_->recover_obj_lock(DEFAULT_OUT_TRANS_UNLOCK_OP);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
ASSERT_EQ(OB_SUCCESS, checkpoint->flush(share::SCN::max_scn()));
|
||||
ASSERT_EQ(OB_SUCCESS, checkpoint->flush(share::SCN::max_scn(), 0));
|
||||
commit_version.val_ = 1;
|
||||
commit_scn.val_ = 1;
|
||||
ret = memtable_->update_lock_status(DEFAULT_OUT_TRANS_UNLOCK_OP,
|
||||
|
@ -76,7 +76,7 @@ void minor_freeze_tx_ctx_memtable(ObLS *ls)
|
||||
checkpoint_executor->handlers_[logservice::TRANS_SERVICE_LOG_BASE_TYPE])
|
||||
->common_checkpoints_[ObCommonCheckpointType::TX_CTX_MEMTABLE_TYPE]);
|
||||
ASSERT_EQ(true, tx_ctx_memtable->is_active_memtable());
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn()));
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn(), -1));
|
||||
|
||||
// // TODO(handora.qc): use more graceful wait
|
||||
// usleep(10 * 1000 * 1000);
|
||||
|
@ -638,7 +638,7 @@ public:
|
||||
ASSERT_EQ(OB_SUCCESS, ((ObTxDataMemtableMgr *)(memtable_mgr))->freeze());
|
||||
share::SCN start_log_ts = tx_data_memtable->get_start_scn();
|
||||
share::SCN end_log_ts = tx_data_memtable->get_end_scn();
|
||||
ASSERT_EQ(OB_SUCCESS, tx_data_memtable->flush());
|
||||
ASSERT_EQ(OB_SUCCESS, tx_data_memtable->flush(0));
|
||||
|
||||
wait_freeze_tx_table_finish(start_log_ts, end_log_ts);
|
||||
|
||||
|
@ -182,7 +182,7 @@ void ObLSBeforeRestartTest::minor_freeze_tx_ctx_table()
|
||||
= dynamic_cast<ObTxCtxMemtable *>(dynamic_cast<ObLSTxService *>(checkpoint_executor
|
||||
->handlers_[logservice::TRANS_SERVICE_LOG_BASE_TYPE])
|
||||
->common_checkpoints_[ObCommonCheckpointType::TX_CTX_MEMTABLE_TYPE]);
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn()));
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn(), 0));
|
||||
int retry_time = 0;
|
||||
while (tx_ctx_memtable->is_frozen_memtable()) {
|
||||
usleep(1000 * 1000);
|
||||
|
@ -406,7 +406,7 @@ void ObMinorFreezeTest::batch_tablet_freeze()
|
||||
|
||||
const int64_t start = ObTimeUtility::current_time();
|
||||
while (ObTimeUtility::current_time() - start <= freeze_duration_) {
|
||||
ASSERT_EQ(OB_SUCCESS, ls_handles_.at(0).get_ls()->batch_tablet_freeze(tablet_ids_, (i % 2 == 0) ? true : false));
|
||||
ASSERT_EQ(OB_SUCCESS, ls_handles_.at(0).get_ls()->batch_tablet_freeze(0, tablet_ids_, (i % 2 == 0) ? true : false));
|
||||
i = i + 1;
|
||||
}
|
||||
}
|
||||
|
@ -112,7 +112,7 @@ public:
|
||||
void minor_freeze_data_memtable(ObLS *ls)
|
||||
{
|
||||
TRANS_LOG(INFO, "minor_freeze_data_memtable begin");
|
||||
ASSERT_EQ(OB_SUCCESS, ls->ls_freezer_.logstream_freeze());
|
||||
ASSERT_EQ(OB_SUCCESS, ls->ls_freezer_.logstream_freeze(0));
|
||||
|
||||
// TODO(handora.qc): use more graceful wait
|
||||
usleep(10 * 1000 * 1000);
|
||||
@ -131,7 +131,7 @@ public:
|
||||
dynamic_cast<ObLSTxService *>(
|
||||
checkpoint_executor->handlers_[logservice::TRANS_SERVICE_LOG_BASE_TYPE])
|
||||
->common_checkpoints_[ObCommonCheckpointType::TX_CTX_MEMTABLE_TYPE]);
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn()));
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn(), 0));
|
||||
|
||||
// TODO(handora.qc): use more graceful wait
|
||||
usleep(10 * 1000 * 1000);
|
||||
|
@ -112,7 +112,7 @@ public:
|
||||
void minor_freeze_data_memtable(ObLS *ls)
|
||||
{
|
||||
TRANS_LOG(INFO, "minor_freeze_data_memtable begin");
|
||||
ASSERT_EQ(OB_SUCCESS, ls->ls_freezer_.logstream_freeze());
|
||||
ASSERT_EQ(OB_SUCCESS, ls->ls_freezer_.logstream_freeze(0));
|
||||
|
||||
// TODO(handora.qc): use more graceful wait
|
||||
usleep(10 * 1000 * 1000);
|
||||
@ -131,7 +131,7 @@ public:
|
||||
dynamic_cast<ObLSTxService *>(
|
||||
checkpoint_executor->handlers_[logservice::TRANS_SERVICE_LOG_BASE_TYPE])
|
||||
->common_checkpoints_[ObCommonCheckpointType::TX_CTX_MEMTABLE_TYPE]);
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn()));
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn(), 0));
|
||||
|
||||
// TODO(handora.qc): use more graceful wait
|
||||
usleep(10 * 1000 * 1000);
|
||||
|
@ -112,7 +112,7 @@ public:
|
||||
void minor_freeze_data_memtable(ObLS *ls)
|
||||
{
|
||||
TRANS_LOG(INFO, "minor_freeze_data_memtable begin");
|
||||
ASSERT_EQ(OB_SUCCESS, ls->ls_freezer_.logstream_freeze());
|
||||
ASSERT_EQ(OB_SUCCESS, ls->ls_freezer_.logstream_freeze(0));
|
||||
|
||||
// TODO(handora.qc): use more graceful wait
|
||||
usleep(10 * 1000 * 1000);
|
||||
@ -131,7 +131,7 @@ public:
|
||||
dynamic_cast<ObLSTxService *>(
|
||||
checkpoint_executor->handlers_[logservice::TRANS_SERVICE_LOG_BASE_TYPE])
|
||||
->common_checkpoints_[ObCommonCheckpointType::TX_CTX_MEMTABLE_TYPE]);
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn()));
|
||||
ASSERT_EQ(OB_SUCCESS, tx_ctx_memtable->flush(share::SCN::max_scn(), 0));
|
||||
|
||||
// TODO(handora.qc): use more graceful wait
|
||||
usleep(10 * 1000 * 1000);
|
||||
|
@ -403,6 +403,9 @@ ob_set_subtarget(ob_server virtual_table
|
||||
virtual_table/ob_all_virtual_cgroup_config.cpp
|
||||
virtual_table/ob_virtual_flt_config.cpp
|
||||
virtual_table/ob_all_virtual_kv_connection.cpp
|
||||
virtual_table/ob_all_virtual_activity_metrics.cpp
|
||||
virtual_table/ob_all_virtual_checkpoint_diagnose_info.cpp
|
||||
virtual_table/ob_all_virtual_checkpoint_diagnose_memtable_info.cpp
|
||||
)
|
||||
|
||||
ob_server_add_target(ob_server)
|
||||
|
@ -158,6 +158,7 @@
|
||||
#include "storage/tenant_snapshot/ob_tenant_snapshot_service.h"
|
||||
#include "share/index_usage/ob_index_usage_info_mgr.h"
|
||||
#include "rootserver/mview/ob_mview_maintenance_service.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::lib;
|
||||
@ -571,6 +572,7 @@ int ObMultiTenant::init(ObAddr myaddr,
|
||||
MTL_BIND2(mtl_new_default, ObIndexUsageInfoMgr::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
|
||||
MTL_BIND2(mtl_new_default, storage::ObTabletMemtableMgrPool::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default);
|
||||
MTL_BIND2(mtl_new_default, rootserver::ObMViewMaintenanceService::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
|
||||
MTL_BIND2(mtl_new_default, ObCheckpointDiagnoseMgr::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default);
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
@ -1314,6 +1316,9 @@ int ObMultiTenant::update_tenant_config(uint64_t tenant_id)
|
||||
if (OB_TMP_FAIL(update_throttle_config_(tenant_id))) {
|
||||
LOG_WARN("update throttle config failed", K(ret), K(tenant_id));
|
||||
}
|
||||
if (OB_TMP_FAIL(update_checkpoint_diagnose_config())) {
|
||||
LOG_WARN("failed to update tenant ddl config", K(tmp_ret), K(tenant_id));
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG_INFO("update_tenant_config success", K(tenant_id));
|
||||
@ -1365,6 +1370,21 @@ int ObMultiTenant::update_tenant_ddl_config()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMultiTenant::update_checkpoint_diagnose_config()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObCheckpointDiagnoseMgr *cdm = MTL(ObCheckpointDiagnoseMgr*);
|
||||
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID()));
|
||||
const int64_t checkpoint_diagnose_preservation_count = tenant_config->_checkpoint_diagnose_preservation_count;
|
||||
if (OB_ISNULL(cdm)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("cdm should not be null", K(ret));
|
||||
} else if(OB_FAIL(cdm->update_max_trace_info_size(checkpoint_diagnose_preservation_count))) {
|
||||
LOG_WARN("failed to update_max_trace_info_size", K(ret), K(checkpoint_diagnose_preservation_count));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMultiTenant::update_tenant_freezer_config_()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
@ -118,6 +118,7 @@ public:
|
||||
int update_palf_config();
|
||||
int update_tenant_dag_scheduler_config();
|
||||
int update_tenant_ddl_config();
|
||||
int update_checkpoint_diagnose_config();
|
||||
int get_tenant(const uint64_t tenant_id, ObTenant *&tenant) const;
|
||||
int get_tenant_with_tenant_lock(const uint64_t tenant_id, common::ObLDHandle &handle, ObTenant *&tenant) const;
|
||||
int get_active_tenant_with_tenant_lock(const uint64_t tenant_id, common::ObLDHandle &handle, ObTenant *&tenant) const;
|
||||
|
183
src/observer/virtual_table/ob_all_virtual_activity_metrics.cpp
Normal file
183
src/observer/virtual_table/ob_all_virtual_activity_metrics.cpp
Normal file
@ -0,0 +1,183 @@
|
||||
/**
|
||||
* 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 "observer/virtual_table/ob_all_virtual_activity_metrics.h"
|
||||
#include "observer/ob_server.h"
|
||||
#include "share/rc/ob_tenant_base.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace observer
|
||||
{
|
||||
|
||||
ObAllVirtualActivityMetric::ObAllVirtualActivityMetric()
|
||||
: ObVirtualTableScannerIterator(),
|
||||
current_pos_(0),
|
||||
length_(0),
|
||||
addr_(),
|
||||
ip_buffer_()
|
||||
{
|
||||
}
|
||||
|
||||
ObAllVirtualActivityMetric::~ObAllVirtualActivityMetric()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
void ObAllVirtualActivityMetric::reset()
|
||||
{
|
||||
current_pos_ = 0;
|
||||
length_ = 0;
|
||||
addr_.reset();
|
||||
ip_buffer_[0] = '\0';
|
||||
omt::ObMultiTenantOperator::reset();
|
||||
ObVirtualTableScannerIterator::reset();
|
||||
}
|
||||
|
||||
bool ObAllVirtualActivityMetric::is_need_process(uint64_t tenant_id)
|
||||
{
|
||||
if (!is_virtual_tenant_id(tenant_id) &&
|
||||
(is_sys_tenant(effective_tenant_id_) || tenant_id == effective_tenant_id_)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void ObAllVirtualActivityMetric::release_last_tenant()
|
||||
{
|
||||
current_pos_ = 0;
|
||||
length_ = 0;
|
||||
ip_buffer_[0] = '\0';
|
||||
}
|
||||
|
||||
int ObAllVirtualActivityMetric::inner_get_next_row(ObNewRow *&row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(execute(row))) {
|
||||
SERVER_LOG(WARN, "execute fail", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAllVirtualActivityMetric::get_next_freezer_stat_(ObTenantFreezerStat& stat)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
storage::ObTenantFreezer *freezer = MTL(storage::ObTenantFreezer *);
|
||||
|
||||
if (current_pos_ < length_) {
|
||||
(void)freezer->get_freezer_stat_from_history(current_pos_, stat);
|
||||
current_pos_++;
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAllVirtualActivityMetric::prepare_start_to_read_()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
storage::ObTenantFreezer *freezer = MTL(storage::ObTenantFreezer *);
|
||||
|
||||
(void)freezer->get_freezer_stat_history_snapshot(length_);
|
||||
current_pos_ = 0;
|
||||
start_to_read_ = true;
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAllVirtualActivityMetric::process_curr_tenant(ObNewRow *&row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTenantFreezerStat stat;
|
||||
|
||||
if (NULL == allocator_) {
|
||||
ret = OB_NOT_INIT;
|
||||
SERVER_LOG(WARN, "allocator_ shouldn't be NULL", K(allocator_), K(ret));
|
||||
} else if (!start_to_read_ && OB_FAIL(prepare_start_to_read_())) {
|
||||
SERVER_LOG(WARN, "prepare start to read", K(allocator_), K(ret));
|
||||
} else if (OB_FAIL(get_next_freezer_stat_(stat))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
SERVER_LOG(WARN, "get next freezer stat failed", KR(ret));
|
||||
}
|
||||
} else {
|
||||
ObObj *cells = NULL;
|
||||
// allocator_ is allocator of PageArena type, no need to free
|
||||
if (NULL == (cells = cur_row_.cells_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(ERROR, "cur row cell is NULL", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < output_column_ids_.count(); ++i) {
|
||||
uint64_t col_id = output_column_ids_.at(i);
|
||||
switch (col_id) {
|
||||
case SERVER_IP:
|
||||
if (!addr_.ip_to_string(ip_buffer_, common::OB_IP_STR_BUFF)) {
|
||||
STORAGE_LOG(ERROR, "ip to string failed");
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
} else {
|
||||
cells[i].set_varchar(ip_buffer_);
|
||||
cells[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
}
|
||||
break;
|
||||
case SERVER_PORT:
|
||||
cells[i].set_int(addr_.get_port());
|
||||
break;
|
||||
case TENANT_ID:
|
||||
cells[i].set_int(MTL_ID());
|
||||
break;
|
||||
case ACTIVITY_TIMESTAMP:
|
||||
cells[i].set_timestamp(stat.last_captured_timestamp_);
|
||||
break;
|
||||
case MODIFICATION_SIZE:
|
||||
cells[i].set_int(stat.captured_data_size_);
|
||||
break;
|
||||
case FREEZE_TIMES:
|
||||
cells[i].set_int(stat.captured_freeze_times_);
|
||||
break;
|
||||
case MINI_MERGE_COST:
|
||||
cells[i].set_int(stat.captured_merge_time_cost_[storage::ObTenantFreezerStat::ObFreezerMergeType::MINI_MERGE]);
|
||||
break;
|
||||
case MINI_MERGE_TIMES:
|
||||
cells[i].set_int(stat.captured_merge_times_[storage::ObTenantFreezerStat::ObFreezerMergeType::MINI_MERGE]);
|
||||
break;
|
||||
case MINOR_MERGE_COST:
|
||||
cells[i].set_int(stat.captured_merge_time_cost_[storage::ObTenantFreezerStat::ObFreezerMergeType::MINOR_MERGE]);
|
||||
break;
|
||||
case MINOR_MERGE_TIMES:
|
||||
cells[i].set_int(stat.captured_merge_times_[storage::ObTenantFreezerStat::ObFreezerMergeType::MINOR_MERGE]);
|
||||
break;
|
||||
case MAJOR_MERGE_COST:
|
||||
cells[i].set_int(stat.captured_merge_time_cost_[storage::ObTenantFreezerStat::ObFreezerMergeType::MAJOR_MERGE]);
|
||||
break;
|
||||
case MAJOR_MERGE_TIMES:
|
||||
cells[i].set_int(stat.captured_merge_times_[storage::ObTenantFreezerStat::ObFreezerMergeType::MAJOR_MERGE]);
|
||||
break;
|
||||
default:
|
||||
// abnormal column id
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "unexpected column id", K(ret));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
row = &cur_row_;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namepace observer
|
||||
} // namespace observer
|
73
src/observer/virtual_table/ob_all_virtual_activity_metrics.h
Normal file
73
src/observer/virtual_table/ob_all_virtual_activity_metrics.h
Normal file
@ -0,0 +1,73 @@
|
||||
/**
|
||||
* Copyright (c) 2023 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 OB_ALL_VIRTUAL_ACTIVITY_METRICS_H_
|
||||
#define OB_ALL_VIRTUAL_ACTIVITY_METRICS_H_
|
||||
|
||||
#include "share/ob_virtual_table_scanner_iterator.h"
|
||||
#include "storage/tx_storage/ob_tenant_freezer.h"
|
||||
#include "observer/omt/ob_multi_tenant_operator.h"
|
||||
#include "share/ob_scanner.h"
|
||||
#include "common/row/ob_row.h"
|
||||
#include "share/ob_define.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace common
|
||||
{
|
||||
class ObAddr;
|
||||
}
|
||||
namespace observer
|
||||
{
|
||||
class ObAllVirtualActivityMetric : public common::ObVirtualTableScannerIterator,
|
||||
public omt::ObMultiTenantOperator
|
||||
{
|
||||
enum ACTIVITY_METRIC_COLUMN {
|
||||
SERVER_IP = common::OB_APP_MIN_COLUMN_ID,
|
||||
SERVER_PORT,
|
||||
TENANT_ID,
|
||||
ACTIVITY_TIMESTAMP,
|
||||
MODIFICATION_SIZE,
|
||||
FREEZE_TIMES,
|
||||
MINI_MERGE_COST,
|
||||
MINI_MERGE_TIMES,
|
||||
MINOR_MERGE_COST,
|
||||
MINOR_MERGE_TIMES,
|
||||
MAJOR_MERGE_COST,
|
||||
MAJOR_MERGE_TIMES,
|
||||
};
|
||||
public:
|
||||
ObAllVirtualActivityMetric();
|
||||
virtual ~ObAllVirtualActivityMetric();
|
||||
public:
|
||||
virtual int inner_get_next_row(common::ObNewRow *&row);
|
||||
virtual void reset();
|
||||
inline void set_addr(common::ObAddr &addr) { addr_ = addr; }
|
||||
private:
|
||||
bool is_need_process(uint64_t tenant_id) override;
|
||||
int process_curr_tenant(common::ObNewRow *&row) override;
|
||||
void release_last_tenant() override;
|
||||
int get_next_freezer_stat_(storage::ObTenantFreezerStat& stat);
|
||||
int prepare_start_to_read_();
|
||||
|
||||
private:
|
||||
int64_t current_pos_;
|
||||
int64_t length_;
|
||||
common::ObAddr addr_;
|
||||
char ip_buffer_[common::OB_IP_STR_BUFF];
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObAllVirtualActivityMetric);
|
||||
};
|
||||
|
||||
} // namespace observer
|
||||
} // namespace oceanbase
|
||||
#endif /* OB_ALL_VIRTUAL_ACTIVITY_METRICS_H */
|
@ -0,0 +1,132 @@
|
||||
/**
|
||||
* 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 STORAGE
|
||||
|
||||
#include "ob_all_virtual_checkpoint_diagnose_info.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
using namespace share;
|
||||
using namespace storage;
|
||||
using namespace common;
|
||||
using namespace omt;
|
||||
using namespace checkpoint;
|
||||
namespace observer
|
||||
{
|
||||
|
||||
bool ObAllVirtualCheckpointDiagnoseInfo::is_need_process(uint64_t tenant_id)
|
||||
{
|
||||
if (is_sys_tenant(effective_tenant_id_) || tenant_id == effective_tenant_id_) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
int GenerateTraceRow::operator()(const ObTraceInfo &trace_info) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char ip_buf[common::OB_IP_STR_BUFF];
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < virtual_table_.output_column_ids_.count(); ++i) {
|
||||
uint64_t col_id = virtual_table_.output_column_ids_.at(i);
|
||||
switch (col_id) {
|
||||
// tenant_id
|
||||
case OB_APP_MIN_COLUMN_ID:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(MTL_ID());
|
||||
break;
|
||||
// svr_ip
|
||||
case OB_APP_MIN_COLUMN_ID + 1:
|
||||
MEMSET(ip_buf, 0, common::OB_IP_STR_BUFF);
|
||||
if (virtual_table_.addr_.ip_to_string(ip_buf, sizeof(ip_buf))) {
|
||||
virtual_table_.cur_row_.cells_[i].set_varchar(ip_buf);
|
||||
virtual_table_.cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "fail to execute ip_to_string", KR(ret));
|
||||
}
|
||||
break;
|
||||
// svr_port
|
||||
case OB_APP_MIN_COLUMN_ID + 2:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(virtual_table_.addr_.get_port());
|
||||
break;
|
||||
// ls_id
|
||||
case OB_APP_MIN_COLUMN_ID + 3:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(trace_info.ls_id_.id());
|
||||
break;
|
||||
// trace_id
|
||||
case OB_APP_MIN_COLUMN_ID + 4:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(trace_info.trace_id_);
|
||||
break;
|
||||
// freeze_clock
|
||||
case OB_APP_MIN_COLUMN_ID + 5:
|
||||
virtual_table_.cur_row_.cells_[i].set_uint32(trace_info.freeze_clock_);
|
||||
break;
|
||||
// checkpoint_thread_name
|
||||
case OB_APP_MIN_COLUMN_ID + 6:
|
||||
virtual_table_.cur_row_.cells_[i].set_varchar(trace_info.thread_name_);
|
||||
virtual_table_.cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
break;
|
||||
// checkpoint_start_time
|
||||
case OB_APP_MIN_COLUMN_ID + 7:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(trace_info.checkpoint_start_time_);
|
||||
break;
|
||||
default:
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "unexpected type", KR(ret), K(col_id));
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(virtual_table_.scanner_.add_row(virtual_table_.cur_row_))) {
|
||||
SERVER_LOG(WARN, "failed to add row", K(ret), K(virtual_table_.cur_row_));
|
||||
if (OB_SIZE_OVERFLOW == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAllVirtualCheckpointDiagnoseInfo::process_curr_tenant(common::ObNewRow *&row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!start_to_read_) {
|
||||
SERVER_LOG(INFO, "__all_virtual_checkpoint_diagnose_info start");
|
||||
if(OB_FAIL(MTL(ObCheckpointDiagnoseMgr*)->read_trace_info(GenerateTraceRow(*this)))) {
|
||||
SERVER_LOG(WARN, "failed to read trace info", K(ret), K(cur_row_));
|
||||
|
||||
} else {
|
||||
scanner_it_ = scanner_.begin();
|
||||
start_to_read_ = true;
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(scanner_it_.get_next_row(cur_row_))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
SERVER_LOG(WARN, "fail to get next row", K(ret));
|
||||
}
|
||||
} else {
|
||||
row = &cur_row_;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObAllVirtualCheckpointDiagnoseInfo::release_last_tenant()
|
||||
{
|
||||
scanner_.reuse();
|
||||
start_to_read_ = false;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
@ -0,0 +1,59 @@
|
||||
/**
|
||||
* 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 OB_ALL_VIRTUAL_CHEKPOINT_DIAGNOSE_INFO_H
|
||||
#define OB_ALL_VIRTUAL_CHEKPOINT_DIAGNOSE_INFO_H
|
||||
|
||||
#include "share/ob_virtual_table_scanner_iterator.h"
|
||||
#include "observer/omt/ob_multi_tenant_operator.h"
|
||||
#include "observer/omt/ob_multi_tenant.h"
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace observer
|
||||
{
|
||||
|
||||
class ObAllVirtualCheckpointDiagnoseInfo : public common::ObVirtualTableScannerIterator,
|
||||
public omt::ObMultiTenantOperator
|
||||
{
|
||||
friend class GenerateTraceRow;
|
||||
public:
|
||||
virtual ~ObAllVirtualCheckpointDiagnoseInfo() { omt::ObMultiTenantOperator::reset(); }
|
||||
virtual int inner_get_next_row(common::ObNewRow *&row) { return execute(row); }
|
||||
inline void set_addr(common::ObAddr &addr) { addr_ = addr; }
|
||||
|
||||
private:
|
||||
virtual bool is_need_process(uint64_t tenant_id) override;
|
||||
virtual int process_curr_tenant(common::ObNewRow *&row) override;
|
||||
virtual void release_last_tenant() override;
|
||||
|
||||
common::ObAddr addr_;
|
||||
};
|
||||
|
||||
struct GenerateTraceRow
|
||||
{
|
||||
public:
|
||||
GenerateTraceRow() = delete;
|
||||
GenerateTraceRow(const GenerateTraceRow&) = delete;
|
||||
GenerateTraceRow& operator=(const GenerateTraceRow&) = delete;
|
||||
GenerateTraceRow(ObAllVirtualCheckpointDiagnoseInfo &virtual_table)
|
||||
: virtual_table_(virtual_table)
|
||||
{}
|
||||
int operator()(const storage::checkpoint::ObTraceInfo &trace_info) const;
|
||||
|
||||
private:
|
||||
ObAllVirtualCheckpointDiagnoseInfo &virtual_table_;
|
||||
};
|
||||
|
||||
|
||||
} // observer
|
||||
} // oceanbase
|
||||
#endif
|
@ -0,0 +1,371 @@
|
||||
/**
|
||||
* 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 STORAGE
|
||||
|
||||
#include "ob_all_virtual_checkpoint_diagnose_memtable_info.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
using namespace share;
|
||||
using namespace storage;
|
||||
using namespace common;
|
||||
using namespace omt;
|
||||
using namespace checkpoint;
|
||||
namespace observer
|
||||
{
|
||||
int ObAllVirtualCheckpointDiagnoseMemtableInfo::get_primary_key_()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (key_ranges_.count() != 1) {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_USER_ERROR(OB_NOT_SUPPORTED, "only support select a single trace, multiple range select is ");
|
||||
SERVER_LOG(WARN, "invalid key ranges", KR(ret));
|
||||
} else {
|
||||
ObNewRange &key_range = key_ranges_.at(0);
|
||||
if (OB_UNLIKELY(key_range.get_start_key().get_obj_cnt() != 2 || key_range.get_end_key().get_obj_cnt() != 2)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(ERROR,
|
||||
"unexpected key_ranges_ of rowkey columns",
|
||||
KR(ret),
|
||||
"size of start key",
|
||||
key_range.get_start_key().get_obj_cnt(),
|
||||
"size of end key",
|
||||
key_range.get_end_key().get_obj_cnt());
|
||||
} else {
|
||||
ObObj tenant_obj_low = (key_range.get_start_key().get_obj_ptr()[0]);
|
||||
ObObj tenant_obj_high = (key_range.get_end_key().get_obj_ptr()[0]);
|
||||
ObObj trace_obj_low = (key_range.get_start_key().get_obj_ptr()[1]);
|
||||
ObObj trace_obj_high = (key_range.get_end_key().get_obj_ptr()[1]);
|
||||
|
||||
uint64_t tenant_low = tenant_obj_low.is_min_value() ? 0 : tenant_obj_low.get_uint64();
|
||||
uint64_t tenant_high = tenant_obj_high.is_max_value() ? UINT64_MAX : tenant_obj_high.get_uint64();
|
||||
uint64_t trace_id_low = trace_obj_low.is_min_value() ? 0 : trace_obj_low.get_int();
|
||||
uint64_t trace_id_high = trace_obj_high.is_max_value() ? 0 : trace_obj_high.get_int();
|
||||
if (tenant_low != tenant_high || trace_id_low != trace_id_high) {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant id and trace_id must be specified. range select is ");
|
||||
SERVER_LOG(WARN,
|
||||
"only support point select.",
|
||||
KR(ret),
|
||||
K(tenant_low),
|
||||
K(tenant_high),
|
||||
K(trace_id_low),
|
||||
K(trace_id_high));
|
||||
} else {
|
||||
trace_id_ = trace_id_low;
|
||||
if (is_sys_tenant(effective_tenant_id_)
|
||||
|| effective_tenant_id_ == tenant_low) {
|
||||
tenant_id_ = tenant_low;
|
||||
} else {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant id must not be specified to other tenant in user tenant. this is ");
|
||||
SERVER_LOG(WARN,
|
||||
"tenant id must not be specified to other tenant in user tenant",
|
||||
KR(ret),
|
||||
K(effective_tenant_id_),
|
||||
K(trace_id_low),
|
||||
K(tenant_low));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int GenerateMemtableRow::operator()(const ObTraceInfo &trace_info,
|
||||
const ObCheckpointDiagnoseKey &key,
|
||||
const ObMemtableDiagnoseInfo &memtable_diagnose_info) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char ip_buf[common::OB_IP_STR_BUFF];
|
||||
char ptr_buf[16];
|
||||
int64_t pos = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < virtual_table_.output_column_ids_.count(); ++i) {
|
||||
uint64_t col_id = virtual_table_.output_column_ids_.at(i);
|
||||
switch (col_id) {
|
||||
// tenant_id
|
||||
case OB_APP_MIN_COLUMN_ID:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(MTL_ID());
|
||||
break;
|
||||
// trace_id
|
||||
case OB_APP_MIN_COLUMN_ID + 1:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(trace_info.trace_id_);
|
||||
break;
|
||||
// svr_ip
|
||||
case OB_APP_MIN_COLUMN_ID + 2:
|
||||
MEMSET(ip_buf, 0, common::OB_IP_STR_BUFF);
|
||||
if (virtual_table_.addr_.ip_to_string(ip_buf, sizeof(ip_buf))) {
|
||||
virtual_table_.cur_row_.cells_[i].set_varchar(ip_buf);
|
||||
virtual_table_.cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "fail to execute ip_to_string", KR(ret));
|
||||
}
|
||||
break;
|
||||
// svr_port
|
||||
case OB_APP_MIN_COLUMN_ID + 3:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(virtual_table_.addr_.get_port());
|
||||
break;
|
||||
// ls_id
|
||||
case OB_APP_MIN_COLUMN_ID + 4:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(trace_info.ls_id_.id());
|
||||
break;
|
||||
// checkpoint_thread_name
|
||||
case OB_APP_MIN_COLUMN_ID + 5:
|
||||
virtual_table_.cur_row_.cells_[i].set_varchar(trace_info.thread_name_);
|
||||
virtual_table_.cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
break;
|
||||
// checkpoint_start_time
|
||||
case OB_APP_MIN_COLUMN_ID + 6:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(trace_info.checkpoint_start_time_);
|
||||
break;
|
||||
// tablet_id
|
||||
case OB_APP_MIN_COLUMN_ID + 7:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(key.tablet_id_.id());
|
||||
break;
|
||||
// ptr
|
||||
case OB_APP_MIN_COLUMN_ID + 8:
|
||||
MEMSET(ptr_buf, 0, 16);
|
||||
databuff_print_obj(ptr_buf, 16, pos, key.checkpoint_unit_ptr_);
|
||||
virtual_table_.cur_row_.cells_[i].set_varchar(ptr_buf);
|
||||
virtual_table_.cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
break;
|
||||
// start_scn
|
||||
case OB_APP_MIN_COLUMN_ID + 9:
|
||||
virtual_table_.cur_row_.cells_[i].set_uint64(memtable_diagnose_info.start_scn_.get_val_for_inner_table_field());
|
||||
break;
|
||||
// end_scn
|
||||
case OB_APP_MIN_COLUMN_ID + 10:
|
||||
virtual_table_.cur_row_.cells_[i].set_uint64(memtable_diagnose_info.end_scn_.get_val_for_inner_table_field());
|
||||
break;
|
||||
// rec_scn
|
||||
case OB_APP_MIN_COLUMN_ID + 11:
|
||||
virtual_table_.cur_row_.cells_[i].set_uint64(memtable_diagnose_info.rec_scn_.get_val_for_inner_table_field());
|
||||
break;
|
||||
// create_flush_dag_time
|
||||
case OB_APP_MIN_COLUMN_ID + 12:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(memtable_diagnose_info.create_flush_dag_time_);
|
||||
break;
|
||||
// merge_finish_time
|
||||
case OB_APP_MIN_COLUMN_ID + 13:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(memtable_diagnose_info.merge_finish_time_);
|
||||
break;
|
||||
// start_gc_time
|
||||
case OB_APP_MIN_COLUMN_ID + 14:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(memtable_diagnose_info.start_gc_time_);
|
||||
break;
|
||||
// frozen_finish_time
|
||||
case OB_APP_MIN_COLUMN_ID + 15:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(memtable_diagnose_info.frozen_finish_time_);
|
||||
break;
|
||||
// merge_start_time
|
||||
case OB_APP_MIN_COLUMN_ID + 16:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(memtable_diagnose_info.merge_start_time_);
|
||||
break;
|
||||
// release_time
|
||||
case OB_APP_MIN_COLUMN_ID + 17:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(memtable_diagnose_info.release_time_);
|
||||
break;
|
||||
// memtable_occupy_size
|
||||
case OB_APP_MIN_COLUMN_ID + 18:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(memtable_diagnose_info.memtable_occupy_size_);
|
||||
break;
|
||||
// occupy_size
|
||||
case OB_APP_MIN_COLUMN_ID + 19:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(memtable_diagnose_info.occupy_size_);
|
||||
break;
|
||||
// concurrent_cnt
|
||||
case OB_APP_MIN_COLUMN_ID + 20:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(memtable_diagnose_info.concurrent_cnt_);
|
||||
break;
|
||||
default:
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "unexpected type", KR(ret), K(col_id));
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(virtual_table_.scanner_.add_row(virtual_table_.cur_row_))) {
|
||||
SERVER_LOG(WARN, "failed to add row", K(ret), K(virtual_table_.cur_row_));
|
||||
if (OB_SIZE_OVERFLOW == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAllVirtualCheckpointDiagnoseMemtableInfo::inner_get_next_row(common::ObNewRow *&row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!start_to_read_) {
|
||||
SERVER_LOG(INFO, "__all_virtual_checkpoint_diagnose_memtable_info start");
|
||||
if (OB_FAIL(get_primary_key_())) {
|
||||
SERVER_LOG(WARN, "failed to get_primary_key", KR(ret));
|
||||
} else {
|
||||
MTL_SWITCH(tenant_id_)
|
||||
{
|
||||
if (OB_FAIL(MTL(ObCheckpointDiagnoseMgr*)->read_diagnose_info
|
||||
<ObMemtableDiagnoseInfo>(trace_id_, GenerateMemtableRow(*this)))) {
|
||||
SERVER_LOG(WARN, "failed to read trace info", K(ret), K(cur_row_));
|
||||
} else {
|
||||
scanner_it_ = scanner_.begin();
|
||||
start_to_read_ = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(scanner_it_.get_next_row(cur_row_))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
SERVER_LOG(WARN, "fail to get next row", K(ret));
|
||||
}
|
||||
} else {
|
||||
row = &cur_row_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int GenerateCheckpointUnitRow::operator()(const storage::checkpoint::ObTraceInfo &trace_info,
|
||||
const storage::checkpoint::ObCheckpointDiagnoseKey &key,
|
||||
const storage::checkpoint::ObCheckpointUnitDiagnoseInfo &checkpoint_unit_diagnose_info) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char ip_buf[common::OB_IP_STR_BUFF];
|
||||
char ptr_buf[16];
|
||||
int64_t pos = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < virtual_table_.output_column_ids_.count(); ++i) {
|
||||
uint64_t col_id = virtual_table_.output_column_ids_.at(i);
|
||||
switch (col_id) {
|
||||
// tenant_id
|
||||
case OB_APP_MIN_COLUMN_ID:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(MTL_ID());
|
||||
break;
|
||||
// trace_id
|
||||
case OB_APP_MIN_COLUMN_ID + 1:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(trace_info.trace_id_);
|
||||
break;
|
||||
// svr_ip
|
||||
case OB_APP_MIN_COLUMN_ID + 2:
|
||||
MEMSET(ip_buf, 0, common::OB_IP_STR_BUFF);
|
||||
if (virtual_table_.addr_.ip_to_string(ip_buf, sizeof(ip_buf))) {
|
||||
virtual_table_.cur_row_.cells_[i].set_varchar(ip_buf);
|
||||
virtual_table_.cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "fail to execute ip_to_string", KR(ret));
|
||||
}
|
||||
break;
|
||||
// svr_port
|
||||
case OB_APP_MIN_COLUMN_ID + 3:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(virtual_table_.addr_.get_port());
|
||||
break;
|
||||
// ls_id
|
||||
case OB_APP_MIN_COLUMN_ID + 4:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(trace_info.ls_id_.id());
|
||||
break;
|
||||
// checkpoint_thread_name
|
||||
case OB_APP_MIN_COLUMN_ID + 5:
|
||||
virtual_table_.cur_row_.cells_[i].set_varchar(trace_info.thread_name_);
|
||||
virtual_table_.cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
break;
|
||||
// checkpoint_start_time
|
||||
case OB_APP_MIN_COLUMN_ID + 6:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(trace_info.checkpoint_start_time_);
|
||||
break;
|
||||
// tablet_id
|
||||
case OB_APP_MIN_COLUMN_ID + 7:
|
||||
virtual_table_.cur_row_.cells_[i].set_int(key.tablet_id_.id());
|
||||
break;
|
||||
// ptr
|
||||
case OB_APP_MIN_COLUMN_ID + 8:
|
||||
MEMSET(ptr_buf, 0, 16);
|
||||
databuff_print_obj(ptr_buf, 16, pos, key.checkpoint_unit_ptr_);
|
||||
virtual_table_.cur_row_.cells_[i].set_varchar(ptr_buf);
|
||||
virtual_table_.cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
|
||||
break;
|
||||
// start_scn
|
||||
case OB_APP_MIN_COLUMN_ID + 9:
|
||||
virtual_table_.cur_row_.cells_[i].set_uint64(checkpoint_unit_diagnose_info.start_scn_.get_val_for_inner_table_field());
|
||||
break;
|
||||
// end_scn
|
||||
case OB_APP_MIN_COLUMN_ID + 10:
|
||||
virtual_table_.cur_row_.cells_[i].set_uint64(checkpoint_unit_diagnose_info.end_scn_.get_val_for_inner_table_field());
|
||||
break;
|
||||
// rec_scn
|
||||
case OB_APP_MIN_COLUMN_ID + 11:
|
||||
virtual_table_.cur_row_.cells_[i].set_uint64(checkpoint_unit_diagnose_info.rec_scn_.get_val_for_inner_table_field());
|
||||
break;
|
||||
// create_flush_dag_time
|
||||
case OB_APP_MIN_COLUMN_ID + 12:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(checkpoint_unit_diagnose_info.create_flush_dag_time_);
|
||||
break;
|
||||
// merge_finish_time
|
||||
case OB_APP_MIN_COLUMN_ID + 13:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(checkpoint_unit_diagnose_info.merge_finish_time_);
|
||||
break;
|
||||
// start_gc_time
|
||||
case OB_APP_MIN_COLUMN_ID + 14:
|
||||
virtual_table_.cur_row_.cells_[i].set_timestamp(checkpoint_unit_diagnose_info.start_gc_time_);
|
||||
break;
|
||||
default:
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "unexpected type", KR(ret), K(col_id));
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(virtual_table_.scanner_.add_row(virtual_table_.cur_row_))) {
|
||||
SERVER_LOG(WARN, "failed to add row", K(ret), K(virtual_table_.cur_row_));
|
||||
if (OB_SIZE_OVERFLOW == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int ObAllVirtualCheckpointDiagnoseCheckpointUnitInfo::inner_get_next_row(common::ObNewRow *&row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!start_to_read_) {
|
||||
SERVER_LOG(INFO, "__all_virtual_checkpoint_diagnose_checkpoint_unit_info start");
|
||||
if (OB_FAIL(get_primary_key_())) {
|
||||
SERVER_LOG(WARN, "failed to get_primary_key", KR(ret));
|
||||
} else {
|
||||
MTL_SWITCH(tenant_id_)
|
||||
{
|
||||
if (OB_FAIL(MTL(ObCheckpointDiagnoseMgr*)->read_diagnose_info
|
||||
<ObCheckpointUnitDiagnoseInfo>(trace_id_, GenerateCheckpointUnitRow(*this)))) {
|
||||
SERVER_LOG(WARN, "failed to read trace info", K(ret), K(cur_row_));
|
||||
} else {
|
||||
scanner_it_ = scanner_.begin();
|
||||
start_to_read_ = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(scanner_it_.get_next_row(cur_row_))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
SERVER_LOG(WARN, "fail to get next row", K(ret));
|
||||
}
|
||||
} else {
|
||||
row = &cur_row_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
@ -0,0 +1,80 @@
|
||||
/**
|
||||
* 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 OB_ALL_VIRTUAL_CHEKPOINT_DIAGNOSE_MEMTABLE_INFO_H
|
||||
#define OB_ALL_VIRTUAL_CHEKPOINT_DIAGNOSE_MEMTABLE_INFO_H
|
||||
|
||||
#include "share/ob_virtual_table_scanner_iterator.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace observer
|
||||
{
|
||||
|
||||
class ObAllVirtualCheckpointDiagnoseMemtableInfo : public common::ObVirtualTableScannerIterator
|
||||
{
|
||||
friend class GenerateMemtableRow;
|
||||
public:
|
||||
virtual int inner_get_next_row(common::ObNewRow *&row);
|
||||
inline void set_addr(common::ObAddr &addr) { addr_ = addr; }
|
||||
|
||||
protected:
|
||||
int get_primary_key_();
|
||||
common::ObAddr addr_;
|
||||
uint64_t tenant_id_;
|
||||
int64_t trace_id_;
|
||||
};
|
||||
|
||||
class ObAllVirtualCheckpointDiagnoseCheckpointUnitInfo : public ObAllVirtualCheckpointDiagnoseMemtableInfo
|
||||
{
|
||||
friend class GenerateCheckpointUnitRow;
|
||||
virtual int inner_get_next_row(common::ObNewRow *&row);
|
||||
};
|
||||
|
||||
struct GenerateMemtableRow
|
||||
{
|
||||
public:
|
||||
GenerateMemtableRow() = delete;
|
||||
GenerateMemtableRow(const GenerateMemtableRow&) = delete;
|
||||
GenerateMemtableRow& operator=(const GenerateMemtableRow&) = delete;
|
||||
GenerateMemtableRow(ObAllVirtualCheckpointDiagnoseMemtableInfo &virtual_table)
|
||||
: virtual_table_(virtual_table)
|
||||
{}
|
||||
int operator()(const storage::checkpoint::ObTraceInfo &trace_info,
|
||||
const storage::checkpoint::ObCheckpointDiagnoseKey &key,
|
||||
const storage::checkpoint::ObMemtableDiagnoseInfo &memtable_diagnose_info) const;
|
||||
|
||||
private:
|
||||
ObAllVirtualCheckpointDiagnoseMemtableInfo &virtual_table_;
|
||||
};
|
||||
|
||||
struct GenerateCheckpointUnitRow
|
||||
{
|
||||
public:
|
||||
GenerateCheckpointUnitRow() = delete;
|
||||
GenerateCheckpointUnitRow(const GenerateCheckpointUnitRow&) = delete;
|
||||
GenerateCheckpointUnitRow& operator=(const GenerateCheckpointUnitRow&) = delete;
|
||||
GenerateCheckpointUnitRow(ObAllVirtualCheckpointDiagnoseCheckpointUnitInfo &virtual_table)
|
||||
: virtual_table_(virtual_table)
|
||||
{}
|
||||
int operator()(const storage::checkpoint::ObTraceInfo &trace_info,
|
||||
const storage::checkpoint::ObCheckpointDiagnoseKey &key,
|
||||
const storage::checkpoint::ObCheckpointUnitDiagnoseInfo &checkpoint_unit_diagnose_info) const;
|
||||
|
||||
private:
|
||||
ObAllVirtualCheckpointDiagnoseCheckpointUnitInfo &virtual_table_;
|
||||
};
|
||||
|
||||
} // observer
|
||||
} // oceanbase
|
||||
#endif
|
@ -216,6 +216,10 @@
|
||||
#include "observer/virtual_table/ob_virtual_flt_config.h"
|
||||
#include "observer/virtual_table/ob_all_virtual_tenant_snapshot_ls_replica.h"
|
||||
#include "observer/virtual_table/ob_all_virtual_tenant_snapshot_ls_replica_history.h"
|
||||
#include "observer/virtual_table/ob_all_virtual_activity_metrics.h"
|
||||
#include "observer/virtual_table/ob_all_virtual_checkpoint_diagnose_info.h"
|
||||
#include "observer/virtual_table/ob_all_virtual_checkpoint_diagnose_memtable_info.h"
|
||||
|
||||
#include "observer/virtual_table/ob_all_virtual_kv_connection.h"
|
||||
#include "observer/virtual_table/ob_tenant_show_restore_preview.h"
|
||||
|
||||
@ -2604,6 +2608,46 @@ int ObVTIterCreator::create_vt_iter(ObVTableScanParam ¶ms,
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OB_ALL_VIRTUAL_DATA_ACTIVITY_METRICS_TID: {
|
||||
ObAllVirtualActivityMetric *all_virtual_activity_metrics = NULL;
|
||||
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObAllVirtualActivityMetric, all_virtual_activity_metrics))) {
|
||||
SERVER_LOG(ERROR, "ObAllVirtualActivityMetric construct failed", K(ret));
|
||||
} else {
|
||||
all_virtual_activity_metrics->set_addr(addr_);
|
||||
vt_iter = static_cast<ObVirtualTableIterator *>(all_virtual_activity_metrics);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_INFO_TID: {
|
||||
ObAllVirtualCheckpointDiagnoseInfo *checkpoint_diagnose_info = NULL;
|
||||
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObAllVirtualCheckpointDiagnoseInfo, checkpoint_diagnose_info))) {
|
||||
SERVER_LOG(ERROR, "ObAllVirtualCheckpointDiagnoseInfo construct fail", K(ret));
|
||||
} else {
|
||||
checkpoint_diagnose_info->set_addr(addr_);
|
||||
vt_iter = static_cast<ObAllVirtualCheckpointDiagnoseInfo*>(checkpoint_diagnose_info);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_MEMTABLE_INFO_TID: {
|
||||
ObAllVirtualCheckpointDiagnoseMemtableInfo *checkpoint_diagnose_memtable_info = NULL;
|
||||
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObAllVirtualCheckpointDiagnoseMemtableInfo, checkpoint_diagnose_memtable_info))) {
|
||||
SERVER_LOG(ERROR, "ObAllVirtualCheckpointDiagnoseMemtableInfo construct fail", K(ret));
|
||||
} else {
|
||||
checkpoint_diagnose_memtable_info->set_addr(addr_);
|
||||
vt_iter = static_cast<ObAllVirtualCheckpointDiagnoseMemtableInfo*>(checkpoint_diagnose_memtable_info);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_CHECKPOINT_UNIT_INFO_TID: {
|
||||
ObAllVirtualCheckpointDiagnoseCheckpointUnitInfo *checkpoint_diagnose_checkpoint_unit_info = NULL;
|
||||
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObAllVirtualCheckpointDiagnoseCheckpointUnitInfo, checkpoint_diagnose_checkpoint_unit_info))) {
|
||||
SERVER_LOG(ERROR, "ObAllVirtualCheckpointDiagnoseCheckpointUnitInfo construct fail", K(ret));
|
||||
} else {
|
||||
checkpoint_diagnose_checkpoint_unit_info->set_addr(addr_);
|
||||
vt_iter = static_cast<ObAllVirtualCheckpointDiagnoseCheckpointUnitInfo*>(checkpoint_diagnose_checkpoint_unit_info);
|
||||
}
|
||||
break;
|
||||
}
|
||||
END_CREATE_VT_ITER_SWITCH_LAMBDA
|
||||
|
||||
#define AGENT_VIRTUAL_TABLE_CREATE_ITER
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -989,11 +989,15 @@ public:
|
||||
static int all_virtual_import_table_job_history_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_import_table_task_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_import_table_task_history_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_data_activity_metrics_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_column_group_mapping_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_column_group_history_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_column_group_mapping_history_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_clone_job_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_clone_job_history_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_checkpoint_diagnose_memtable_info_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_checkpoint_diagnose_checkpoint_unit_info_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_checkpoint_diagnose_info_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_aux_stat_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_tenant_snapshot_job_schema(share::schema::ObTableSchema &table_schema);
|
||||
static int all_virtual_ls_snapshot_schema(share::schema::ObTableSchema &table_schema);
|
||||
@ -3571,11 +3575,15 @@ const schema_create_func virtual_table_schema_creators [] = {
|
||||
ObInnerTableSchema::all_virtual_import_table_job_history_schema,
|
||||
ObInnerTableSchema::all_virtual_import_table_task_schema,
|
||||
ObInnerTableSchema::all_virtual_import_table_task_history_schema,
|
||||
ObInnerTableSchema::all_virtual_data_activity_metrics_schema,
|
||||
ObInnerTableSchema::all_virtual_column_group_mapping_schema,
|
||||
ObInnerTableSchema::all_virtual_column_group_history_schema,
|
||||
ObInnerTableSchema::all_virtual_column_group_mapping_history_schema,
|
||||
ObInnerTableSchema::all_virtual_clone_job_schema,
|
||||
ObInnerTableSchema::all_virtual_clone_job_history_schema,
|
||||
ObInnerTableSchema::all_virtual_checkpoint_diagnose_memtable_info_schema,
|
||||
ObInnerTableSchema::all_virtual_checkpoint_diagnose_checkpoint_unit_info_schema,
|
||||
ObInnerTableSchema::all_virtual_checkpoint_diagnose_info_schema,
|
||||
ObInnerTableSchema::all_virtual_aux_stat_schema,
|
||||
ObInnerTableSchema::all_virtual_tenant_snapshot_job_schema,
|
||||
ObInnerTableSchema::all_virtual_ls_snapshot_schema,
|
||||
@ -5368,6 +5376,9 @@ const uint64_t tenant_space_tables [] = {
|
||||
OB_ALL_VIRTUAL_COLUMN_GROUP_MAPPING_HISTORY_TID,
|
||||
OB_ALL_VIRTUAL_CLONE_JOB_TID,
|
||||
OB_ALL_VIRTUAL_CLONE_JOB_HISTORY_TID,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_MEMTABLE_INFO_TID,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_CHECKPOINT_UNIT_INFO_TID,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_INFO_TID,
|
||||
OB_ALL_VIRTUAL_TENANT_SNAPSHOT_JOB_TID,
|
||||
OB_ALL_VIRTUAL_LS_SNAPSHOT_TID,
|
||||
OB_ALL_VIRTUAL_TENANT_SNAPSHOT_LS_REPLICA_HISTORY_TID,
|
||||
@ -7822,6 +7833,9 @@ const char* const tenant_space_table_names [] = {
|
||||
OB_ALL_VIRTUAL_COLUMN_GROUP_MAPPING_HISTORY_TNAME,
|
||||
OB_ALL_VIRTUAL_CLONE_JOB_TNAME,
|
||||
OB_ALL_VIRTUAL_CLONE_JOB_HISTORY_TNAME,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_MEMTABLE_INFO_TNAME,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_CHECKPOINT_UNIT_INFO_TNAME,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_INFO_TNAME,
|
||||
OB_ALL_VIRTUAL_TENANT_SNAPSHOT_JOB_TNAME,
|
||||
OB_ALL_VIRTUAL_LS_SNAPSHOT_TNAME,
|
||||
OB_ALL_VIRTUAL_TENANT_SNAPSHOT_LS_REPLICA_HISTORY_TNAME,
|
||||
@ -9651,6 +9665,9 @@ const uint64_t tenant_distributed_vtables [] = {
|
||||
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TID,
|
||||
OB_ALL_VIRTUAL_LS_LOG_RESTORE_STATUS_TID,
|
||||
OB_ALL_VIRTUAL_CGROUP_CONFIG_TID,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_MEMTABLE_INFO_TID,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_CHECKPOINT_UNIT_INFO_TID,
|
||||
OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_INFO_TID,
|
||||
OB_ALL_VIRTUAL_LS_SNAPSHOT_TID,
|
||||
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID,
|
||||
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TID,
|
||||
@ -12282,11 +12299,11 @@ static inline int get_sys_table_lob_aux_schema(const uint64_t tid,
|
||||
|
||||
const int64_t OB_CORE_TABLE_COUNT = 4;
|
||||
const int64_t OB_SYS_TABLE_COUNT = 279;
|
||||
const int64_t OB_VIRTUAL_TABLE_COUNT = 776;
|
||||
const int64_t OB_VIRTUAL_TABLE_COUNT = 780;
|
||||
const int64_t OB_SYS_VIEW_COUNT = 840;
|
||||
const int64_t OB_SYS_TENANT_TABLE_COUNT = 1900;
|
||||
const int64_t OB_SYS_TENANT_TABLE_COUNT = 1904;
|
||||
const int64_t OB_CORE_SCHEMA_VERSION = 1;
|
||||
const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1903;
|
||||
const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1907;
|
||||
|
||||
} // end namespace share
|
||||
} // end namespace oceanbase
|
||||
|
@ -725,11 +725,15 @@ const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TID = 12424; // "__all_virtual_im
|
||||
const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TID = 12425; // "__all_virtual_import_table_job_history"
|
||||
const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TID = 12426; // "__all_virtual_import_table_task"
|
||||
const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TID = 12427; // "__all_virtual_import_table_task_history"
|
||||
const uint64_t OB_ALL_VIRTUAL_DATA_ACTIVITY_METRICS_TID = 12429; // "__all_virtual_data_activity_metrics"
|
||||
const uint64_t OB_ALL_VIRTUAL_COLUMN_GROUP_MAPPING_TID = 12430; // "__all_virtual_column_group_mapping"
|
||||
const uint64_t OB_ALL_VIRTUAL_COLUMN_GROUP_HISTORY_TID = 12431; // "__all_virtual_column_group_history"
|
||||
const uint64_t OB_ALL_VIRTUAL_COLUMN_GROUP_MAPPING_HISTORY_TID = 12432; // "__all_virtual_column_group_mapping_history"
|
||||
const uint64_t OB_ALL_VIRTUAL_CLONE_JOB_TID = 12435; // "__all_virtual_clone_job"
|
||||
const uint64_t OB_ALL_VIRTUAL_CLONE_JOB_HISTORY_TID = 12436; // "__all_virtual_clone_job_history"
|
||||
const uint64_t OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_MEMTABLE_INFO_TID = 12437; // "__all_virtual_checkpoint_diagnose_memtable_info"
|
||||
const uint64_t OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_CHECKPOINT_UNIT_INFO_TID = 12438; // "__all_virtual_checkpoint_diagnose_checkpoint_unit_info"
|
||||
const uint64_t OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_INFO_TID = 12439; // "__all_virtual_checkpoint_diagnose_info"
|
||||
const uint64_t OB_ALL_VIRTUAL_AUX_STAT_TID = 12447; // "__all_virtual_aux_stat"
|
||||
const uint64_t OB_ALL_VIRTUAL_TENANT_SNAPSHOT_JOB_TID = 12453; // "__all_virtual_tenant_snapshot_job"
|
||||
const uint64_t OB_ALL_VIRTUAL_LS_SNAPSHOT_TID = 12458; // "__all_virtual_ls_snapshot"
|
||||
@ -3291,11 +3295,15 @@ const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TNAME = "__all_virtual_import_
|
||||
const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TNAME = "__all_virtual_import_table_job_history";
|
||||
const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TNAME = "__all_virtual_import_table_task";
|
||||
const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TNAME = "__all_virtual_import_table_task_history";
|
||||
const char *const OB_ALL_VIRTUAL_DATA_ACTIVITY_METRICS_TNAME = "__all_virtual_data_activity_metrics";
|
||||
const char *const OB_ALL_VIRTUAL_COLUMN_GROUP_MAPPING_TNAME = "__all_virtual_column_group_mapping";
|
||||
const char *const OB_ALL_VIRTUAL_COLUMN_GROUP_HISTORY_TNAME = "__all_virtual_column_group_history";
|
||||
const char *const OB_ALL_VIRTUAL_COLUMN_GROUP_MAPPING_HISTORY_TNAME = "__all_virtual_column_group_mapping_history";
|
||||
const char *const OB_ALL_VIRTUAL_CLONE_JOB_TNAME = "__all_virtual_clone_job";
|
||||
const char *const OB_ALL_VIRTUAL_CLONE_JOB_HISTORY_TNAME = "__all_virtual_clone_job_history";
|
||||
const char *const OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_MEMTABLE_INFO_TNAME = "__all_virtual_checkpoint_diagnose_memtable_info";
|
||||
const char *const OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_CHECKPOINT_UNIT_INFO_TNAME = "__all_virtual_checkpoint_diagnose_checkpoint_unit_info";
|
||||
const char *const OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_INFO_TNAME = "__all_virtual_checkpoint_diagnose_info";
|
||||
const char *const OB_ALL_VIRTUAL_AUX_STAT_TNAME = "__all_virtual_aux_stat";
|
||||
const char *const OB_ALL_VIRTUAL_TENANT_SNAPSHOT_JOB_TNAME = "__all_virtual_tenant_snapshot_job";
|
||||
const char *const OB_ALL_VIRTUAL_LS_SNAPSHOT_TNAME = "__all_virtual_ls_snapshot";
|
||||
|
@ -13630,7 +13630,30 @@ def_table_schema(**gen_iterate_private_virtual_table_def(
|
||||
|
||||
# 12428: __all_virtual_import_stmt_exec_history
|
||||
|
||||
# 12429: __all_virtual_data_activity_metrics
|
||||
def_table_schema(
|
||||
owner = 'handora.qc',
|
||||
table_name = '__all_virtual_data_activity_metrics',
|
||||
table_id = '12429',
|
||||
table_type = 'VIRTUAL_TABLE',
|
||||
gm_columns = [],
|
||||
rowkey_columns = [
|
||||
('svr_ip', 'varchar:MAX_IP_ADDR_LENGTH'),
|
||||
('svr_port', 'int'),
|
||||
('tenant_id', 'int'),
|
||||
('activity_timestamp', 'timestamp'),
|
||||
],
|
||||
|
||||
normal_columns = [
|
||||
('modification_size', 'int'),
|
||||
('freeze_times', 'int'),
|
||||
('mini_merge_cost', 'int'),
|
||||
('mini_merge_times', 'int'),
|
||||
('minor_merge_cost', 'int'),
|
||||
('minor_merge_times', 'int'),
|
||||
('major_merge_cost', 'int'),
|
||||
('major_merge_times', 'int'),
|
||||
]
|
||||
)
|
||||
|
||||
def_table_schema(**gen_iterate_virtual_table_def(
|
||||
table_id = '12430',
|
||||
@ -13663,9 +13686,95 @@ def_table_schema(**gen_iterate_private_virtual_table_def(
|
||||
in_tenant_space = True,
|
||||
keywords = all_def_keywords['__all_clone_job_history']))
|
||||
|
||||
# 12437: __all_virtual_checkpoint_diagnose_memtable_info
|
||||
# 12438: __all_virtual_checkpoint_diagnose_checkpoint_unit_info
|
||||
# 12439: __all_virtual_checkpoint_diagnose_info
|
||||
def_table_schema(
|
||||
owner = 'zk250686',
|
||||
table_name = '__all_virtual_checkpoint_diagnose_memtable_info',
|
||||
table_id = '12437',
|
||||
table_type = 'VIRTUAL_TABLE',
|
||||
in_tenant_space = True,
|
||||
gm_columns = [],
|
||||
rowkey_columns = [
|
||||
('tenant_id', 'int'),
|
||||
('trace_id', 'int'),
|
||||
],
|
||||
normal_columns = [
|
||||
('svr_ip', 'varchar:MAX_IP_ADDR_LENGTH'),
|
||||
('svr_port', 'int'),
|
||||
('ls_id', 'int'),
|
||||
('checkpoint_thread_name', 'varchar:OB_THREAD_NAME_BUF_LEN'),
|
||||
('checkpoint_start_time', 'timestamp'),
|
||||
('tablet_id', 'int'),
|
||||
('ptr', 'varchar:128'),
|
||||
('start_scn', 'uint'),
|
||||
('end_scn', 'uint'),
|
||||
('rec_scn', 'uint'),
|
||||
('create_flush_dag_time', 'timestamp'),
|
||||
('merge_finish_time', 'timestamp'),
|
||||
('release_time', 'timestamp'),
|
||||
('frozen_finish_time', 'timestamp'),
|
||||
('merge_start_time', 'timestamp'),
|
||||
('start_gc_time', 'timestamp'),
|
||||
('memtable_occupy_size', 'int'),
|
||||
('occupy_size', 'int'),
|
||||
('concurrent_cnt', 'int'),
|
||||
],
|
||||
partition_columns = ['svr_ip', 'svr_port'],
|
||||
vtable_route_policy = 'distributed',
|
||||
)
|
||||
|
||||
def_table_schema(
|
||||
owner = 'zk250686',
|
||||
table_name = '__all_virtual_checkpoint_diagnose_checkpoint_unit_info',
|
||||
table_id = '12438',
|
||||
table_type = 'VIRTUAL_TABLE',
|
||||
in_tenant_space = True,
|
||||
gm_columns = [],
|
||||
rowkey_columns = [
|
||||
('tenant_id', 'int'),
|
||||
('trace_id', 'int'),
|
||||
],
|
||||
normal_columns = [
|
||||
('svr_ip', 'varchar:MAX_IP_ADDR_LENGTH'),
|
||||
('svr_port', 'int'),
|
||||
('ls_id', 'int'),
|
||||
('checkpoint_thread_name', 'varchar:OB_THREAD_NAME_BUF_LEN'),
|
||||
('checkpoint_start_time', 'timestamp'),
|
||||
('tablet_id', 'int'),
|
||||
('ptr', 'varchar:128'),
|
||||
('start_scn', 'uint'),
|
||||
('end_scn', 'uint'),
|
||||
('rec_scn', 'uint'),
|
||||
('create_flush_dag_time', 'timestamp'),
|
||||
('merge_finish_time', 'timestamp'),
|
||||
('start_gc_time', 'timestamp'),
|
||||
],
|
||||
partition_columns = ['svr_ip', 'svr_port'],
|
||||
vtable_route_policy = 'distributed',
|
||||
)
|
||||
|
||||
def_table_schema(
|
||||
owner = 'zk250686',
|
||||
table_name = '__all_virtual_checkpoint_diagnose_info',
|
||||
table_id = '12439',
|
||||
table_type = 'VIRTUAL_TABLE',
|
||||
in_tenant_space = True,
|
||||
gm_columns = [],
|
||||
rowkey_columns = [
|
||||
],
|
||||
normal_columns = [
|
||||
('tenant_id', 'int'),
|
||||
('svr_ip', 'varchar:MAX_IP_ADDR_LENGTH'),
|
||||
('svr_port', 'int'),
|
||||
('ls_id', 'int'),
|
||||
('trace_id', 'int'),
|
||||
('freeze_clock', 'uint32'),
|
||||
('checkpoint_thread_name', 'varchar:OB_THREAD_NAME_BUF_LEN'),
|
||||
('checkpoint_start_time', 'timestamp'),
|
||||
],
|
||||
partition_columns = ['svr_ip', 'svr_port'],
|
||||
vtable_route_policy = 'distributed',
|
||||
)
|
||||
|
||||
# 12440: __all_virtual_wr_system_event
|
||||
# 12441: __all_virtual_wr_event_name
|
||||
# 12442: __all_tenant_scheduler_running_job
|
||||
|
@ -1036,6 +1036,7 @@
|
||||
# 12426: __all_import_table_task # BASE_TABLE_NAME
|
||||
# 12427: __all_virtual_import_table_task_history
|
||||
# 12427: __all_import_table_task_history # BASE_TABLE_NAME
|
||||
# 12429: __all_virtual_data_activity_metrics
|
||||
# 12430: __all_virtual_column_group_mapping
|
||||
# 12430: __all_column_group_mapping # BASE_TABLE_NAME
|
||||
# 12431: __all_virtual_column_group_history
|
||||
@ -1048,6 +1049,9 @@
|
||||
# 12435: __all_clone_job # BASE_TABLE_NAME
|
||||
# 12436: __all_virtual_clone_job_history
|
||||
# 12436: __all_clone_job_history # BASE_TABLE_NAME
|
||||
# 12437: __all_virtual_checkpoint_diagnose_memtable_info
|
||||
# 12438: __all_virtual_checkpoint_diagnose_checkpoint_unit_info
|
||||
# 12439: __all_virtual_checkpoint_diagnose_info
|
||||
# 12447: __all_virtual_aux_stat
|
||||
# 12447: __all_aux_stat # BASE_TABLE_NAME
|
||||
# 12453: __all_virtual_tenant_snapshot_job
|
||||
|
@ -283,6 +283,15 @@ public:
|
||||
total_cost += common::ObTimeUtility::current_time() - last_click_ts_;
|
||||
return total_cost > warn_threshold_;
|
||||
}
|
||||
int64_t get_total_time() const
|
||||
{
|
||||
int64_t total_cost = 0;
|
||||
for (int64_t idx = 0; idx < idx_; ++idx) {
|
||||
total_cost += click_poinsts_[idx];
|
||||
}
|
||||
total_cost += common::ObTimeUtility::fast_current_time() - last_click_ts_;
|
||||
return total_cost;
|
||||
}
|
||||
bool click(const uint16_t line)
|
||||
{
|
||||
if (OB_LIKELY(idx_ < CAPACITY)) {
|
||||
|
@ -1860,10 +1860,9 @@ DEF_BOOL(enable_rpc_authentication_bypass, OB_CLUSTER_PARAMETER, "True",
|
||||
"cluster and provide service when rpc authentication is turned on.",
|
||||
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
|
||||
// take up space on the master for checkpoint diagnose in 42x
|
||||
// DEF_INT(_checkpoint_diagnose_preservation_count, OB_TENANT_PARAMETER, "100", "[0,1000]",
|
||||
// "the count of checkpoint diagnose info preservation",
|
||||
// ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
DEF_INT(_checkpoint_diagnose_preservation_count, OB_TENANT_PARAMETER, "100", "[0,800]",
|
||||
"the count of checkpoint diagnose info preservation",
|
||||
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
DEF_INT(max_partition_num, OB_TENANT_PARAMETER, "8192", "[8192, 65536]",
|
||||
"set max partition num in mysql mode",
|
||||
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
|
@ -84,6 +84,7 @@ class ObTenantMdsService;
|
||||
namespace checkpoint {
|
||||
class ObCheckPointService;
|
||||
class ObTabletGCService;
|
||||
class ObCheckpointDiagnoseMgr;
|
||||
}
|
||||
class ObLobManager;
|
||||
class ObTransferService;
|
||||
@ -352,7 +353,8 @@ using ObTableScanIteratorObjPool = common::ObServerObjectPool<oceanbase::storage
|
||||
rootserver::ObCloneScheduler*, \
|
||||
share::ObIndexUsageInfoMgr*, \
|
||||
storage::ObTabletMemtableMgrPool*, \
|
||||
rootserver::ObMViewMaintenanceService* \
|
||||
rootserver::ObMViewMaintenanceService*, \
|
||||
storage::checkpoint::ObCheckpointDiagnoseMgr* \
|
||||
)
|
||||
|
||||
|
||||
|
@ -272,7 +272,9 @@ bool ObDbmsStatsUtils::is_no_stat_virtual_table(const int64_t table_id)
|
||||
table_id == share::OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA_TID ||
|
||||
table_id == share::OB_ALL_VIRTUAL_TRANS_LOCK_STAT_ORA_TID ||
|
||||
table_id == share::OB_ALL_VIRTUAL_TRANS_SCHEDULER_ORA_TID ||
|
||||
table_id == share::OB_ALL_VIRTUAL_MDS_NODE_STAT_TID;
|
||||
table_id == share::OB_ALL_VIRTUAL_MDS_NODE_STAT_TID ||
|
||||
table_id == share::OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_MEMTABLE_INFO_TID ||
|
||||
table_id == share::OB_ALL_VIRTUAL_CHECKPOINT_DIAGNOSE_CHECKPOINT_UNIT_INFO_TID;
|
||||
}
|
||||
|
||||
bool ObDbmsStatsUtils::is_virtual_index_table(const int64_t table_id)
|
||||
|
@ -567,6 +567,7 @@ ob_set_subtarget(ob_storage common_mixed
|
||||
checkpoint/ob_data_checkpoint.cpp
|
||||
checkpoint/ob_freeze_checkpoint.cpp
|
||||
meta_mem/ob_tablet_leak_checker.cpp
|
||||
checkpoint/ob_checkpoint_diagnose.cpp
|
||||
meta_mem/ob_meta_obj_struct.cpp
|
||||
meta_mem/ob_tablet_handle.cpp
|
||||
meta_mem/ob_tablet_map_key.cpp
|
||||
|
416
src/storage/checkpoint/ob_checkpoint_diagnose.cpp
Normal file
416
src/storage/checkpoint/ob_checkpoint_diagnose.cpp
Normal file
@ -0,0 +1,416 @@
|
||||
/**
|
||||
* 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 STORAGE
|
||||
|
||||
#include "ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace storage
|
||||
{
|
||||
namespace checkpoint
|
||||
{
|
||||
|
||||
void ObTraceInfo::init(const int64_t trace_id,
|
||||
const share::ObLSID &ls_id,
|
||||
const int64_t checkpoint_start_time)
|
||||
{
|
||||
SpinWLockGuard lock(lock_);
|
||||
reset_without_lock_();
|
||||
trace_id_ = trace_id;
|
||||
ls_id_ = ls_id;
|
||||
strncpy(thread_name_, ob_get_tname(), oceanbase::OB_THREAD_NAME_BUF_LEN);
|
||||
checkpoint_start_time_ = checkpoint_start_time;
|
||||
}
|
||||
|
||||
void ObTraceInfo::reset_without_lock_()
|
||||
{
|
||||
TRANS_LOG(INFO, "trace info reset", KPC(this));
|
||||
trace_id_ = INVALID_TRACE_ID;
|
||||
freeze_clock_ = 0;
|
||||
ls_id_.reset();
|
||||
checkpoint_unit_diagnose_info_map_.reuse();
|
||||
memtable_diagnose_info_map_.reuse();
|
||||
checkpoint_start_time_ = 0;
|
||||
memset(thread_name_, 0, oceanbase::OB_THREAD_NAME_BUF_LEN);
|
||||
allocator_.clear();
|
||||
}
|
||||
|
||||
bool ObTraceInfo::check_trace_id_(const int64_t trace_id)
|
||||
{
|
||||
bool ret = true;
|
||||
if (trace_id != trace_id_) {
|
||||
LOG_WARN("trace_id not match", K(trace_id), KPC(this), K(lbt()));
|
||||
ret = false;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObTraceInfo::check_trace_id_(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
bool ret = true;
|
||||
if (param.is_freeze_clock_) {
|
||||
if (param.freeze_clock_ != freeze_clock_ - 1
|
||||
|| param.ls_id_ != ls_id_.id()) {
|
||||
LOG_WARN("freeze_clock not match", K(param), KPC(this), K(lbt()));
|
||||
ret = false;
|
||||
}
|
||||
} else {
|
||||
if (param.trace_id_ != trace_id_) {
|
||||
LOG_WARN("trace_id not match", K(param), KPC(this), K(lbt()));
|
||||
ret = false;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObTraceInfo::update_freeze_clock(const int64_t trace_id,
|
||||
const uint32_t freeze_clock)
|
||||
{
|
||||
SpinWLockGuard lock(lock_);
|
||||
if (check_trace_id_(trace_id)) {
|
||||
freeze_clock_ = freeze_clock;
|
||||
}
|
||||
}
|
||||
|
||||
int ObCheckpointDiagnoseMgr::init()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_INIT) {
|
||||
ret = OB_INIT_TWICE;
|
||||
LOG_WARN("init ObCheckpointDiagnoseMgr twice", KR(ret));
|
||||
} else {
|
||||
for (int i = 0; OB_SUCC(ret) && i < MAX_TRACE_INFO_ARR_SIZE; i++) {
|
||||
const int64_t bucket_count = hash::cal_next_prime(100);
|
||||
if (OB_FAIL(trace_info_arr_[i].memtable_diagnose_info_map_.create(bucket_count, "CkptDgnMem", "CkptDgnMemNode", MTL_ID()))) {
|
||||
LOG_WARN("failed to create map", KR(ret));
|
||||
} else if (OB_FAIL(trace_info_arr_[i].checkpoint_unit_diagnose_info_map_.create(bucket_count, "CkptDgnMemCU", "CkptDgnCUNode", MTL_ID()))) {
|
||||
LOG_WARN("failed to create map", KR(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
is_inited_ = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCheckpointDiagnoseMgr::acquire_trace_id(const share::ObLSID &ls_id,
|
||||
int64_t &trace_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
trace_id = INVALID_TRACE_ID;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", K(ret));
|
||||
} else if (!ls_id.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("ls_id is invalid", KR(ret));
|
||||
} else {
|
||||
SpinWLockGuard lock(pos_lock_);
|
||||
if (max_trace_info_size_ > 0) {
|
||||
const int64_t start_time = ObTimeUtility::current_time();
|
||||
trace_id = ++last_pos_;
|
||||
trace_info_arr_[trace_id % MAX_TRACE_INFO_ARR_SIZE].init(trace_id, ls_id, start_time);
|
||||
reset_old_trace_infos_without_pos_lock_();
|
||||
LOG_INFO("acquire_trace_id", K(trace_id), K(ls_id));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCheckpointDiagnoseMgr::update_freeze_clock(const share::ObLSID &ls_id,
|
||||
const int64_t trace_id,
|
||||
const int logstream_clock)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", K(ret));
|
||||
} else if (!ls_id.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("ls_id is invalid", KR(ret));
|
||||
} else {
|
||||
// if trace_id is invalid, gen trace_id for checkpoint diagnose.
|
||||
int64_t tmp_trace_id = trace_id;
|
||||
if (checkpoint::INVALID_TRACE_ID == tmp_trace_id) {
|
||||
MTL(checkpoint::ObCheckpointDiagnoseMgr*)->acquire_trace_id(ls_id, tmp_trace_id);
|
||||
}
|
||||
if (checkpoint::INVALID_TRACE_ID != tmp_trace_id) {
|
||||
trace_info_arr_[tmp_trace_id % MAX_TRACE_INFO_ARR_SIZE].update_freeze_clock(tmp_trace_id, logstream_clock);
|
||||
LOG_INFO("update_freeze_clock", K(trace_info_arr_[tmp_trace_id % MAX_TRACE_INFO_ARR_SIZE]));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void UpdateScheduleDagInfo::operator()(ObCheckpointUnitDiagnoseInfo &info) const
|
||||
{
|
||||
const int64_t start_time = ObTimeUtility::current_time();
|
||||
info.rec_scn_ = rec_scn_;
|
||||
info.start_scn_ = start_scn_;
|
||||
info.end_scn_ = end_scn_;
|
||||
info.create_flush_dag_time_ = start_time;
|
||||
TRANS_LOG(INFO, "update_schedule_dag_info", K(info), K(param_));
|
||||
}
|
||||
int ObCheckpointDiagnoseMgr::update_schedule_dag_info(const ObCheckpointDiagnoseParam ¶m,
|
||||
const share::SCN &rec_scn,
|
||||
const share::SCN &start_scn,
|
||||
const share::SCN &end_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", K(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
if (OB_FAIL(trace_info_arr_[param.trace_id_ % MAX_TRACE_INFO_ARR_SIZE].update_diagnose_info
|
||||
<ObCheckpointUnitDiagnoseInfo>(param, UpdateScheduleDagInfo(param, rec_scn, start_scn, end_scn)))) {
|
||||
LOG_WARN("failed to add_checkpoint_unit_diagnose_info", KR(ret), K(param));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
DEF_UPDATE_TIME_FUNCTOR(UpdateMergeInfoForCheckpointUnit, ObCheckpointUnitDiagnoseInfo, merge_finish_time)
|
||||
int ObCheckpointDiagnoseMgr::update_merge_info_for_checkpoint_unit(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
if (OB_FAIL(trace_info_arr_[param.trace_id_ % MAX_TRACE_INFO_ARR_SIZE].update_diagnose_info
|
||||
<ObCheckpointUnitDiagnoseInfo>(param, UpdateMergeInfoForCheckpointUnit(param)))) {
|
||||
LOG_WARN("failed to update_checkpoint_unit_diagnose_info", KR(ret), K(param));
|
||||
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
|
||||
}
|
||||
|
||||
DEF_UPDATE_TIME_FUNCTOR(UpdateStartGcTimeForCheckpointUnit, ObCheckpointUnitDiagnoseInfo, start_gc_time)
|
||||
int ObCheckpointDiagnoseMgr::update_start_gc_time_for_checkpoint_unit(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
if (OB_FAIL(trace_info_arr_[param.trace_id_ % MAX_TRACE_INFO_ARR_SIZE].update_diagnose_info
|
||||
<ObCheckpointUnitDiagnoseInfo>(param, UpdateStartGcTimeForCheckpointUnit(param)))) {
|
||||
LOG_WARN("failed to update_checkpoint_unit_diagnose_info", KR(ret), K(param));
|
||||
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void UpdateFreezeInfo::operator()(ObMemtableDiagnoseInfo &info) const
|
||||
{
|
||||
const int64_t start_time = ObTimeUtility::current_time();
|
||||
info.rec_scn_ = rec_scn_;
|
||||
info.start_scn_ = start_scn_;
|
||||
info.end_scn_ = end_scn_;
|
||||
info.occupy_size_ = occupy_size_;
|
||||
info.frozen_finish_time_ = start_time;
|
||||
TRANS_LOG(INFO, "update_freeze_info", K(info), K(param_));
|
||||
}
|
||||
int ObCheckpointDiagnoseMgr::update_freeze_info(const ObCheckpointDiagnoseParam ¶m,
|
||||
const share::SCN &rec_scn,
|
||||
const share::SCN &start_scn,
|
||||
const share::SCN &end_scn,
|
||||
const int64_t occupy_size)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
ObTraceInfo *trace_info_ptr = get_trace_info_for_memtable(param);
|
||||
if (OB_ISNULL(trace_info_ptr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("trace_info is NULL", KR(ret), K(param));
|
||||
} else if (OB_FAIL(trace_info_ptr->update_diagnose_info<ObMemtableDiagnoseInfo>(param,
|
||||
UpdateFreezeInfo(param, rec_scn, start_scn, end_scn, occupy_size)))) {
|
||||
LOG_WARN("failed to add_memtable_diagnose_info", KR(ret), K(param));
|
||||
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
DEF_UPDATE_TIME_FUNCTOR(UpdateScheduleDagTime, ObMemtableDiagnoseInfo, create_flush_dag_time)
|
||||
int ObCheckpointDiagnoseMgr::update_schedule_dag_time(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
ObTraceInfo *trace_info_ptr = get_trace_info_for_memtable(param);
|
||||
if (OB_ISNULL(trace_info_ptr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("trace_info is NULL", KR(ret), K(param));
|
||||
} else if (OB_FAIL(trace_info_ptr->update_diagnose_info<ObMemtableDiagnoseInfo>(param,
|
||||
UpdateScheduleDagTime(param)))) {
|
||||
LOG_WARN("failed to update_memtable_diagnose_info", KR(ret), K(param));
|
||||
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
DEF_UPDATE_TIME_FUNCTOR(UpdateReleaseTime, ObMemtableDiagnoseInfo, release_time)
|
||||
int ObCheckpointDiagnoseMgr::update_release_time(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
ObTraceInfo *trace_info_ptr = get_trace_info_for_memtable(param);
|
||||
if (OB_ISNULL(trace_info_ptr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("trace_info is NULL", KR(ret), K(param));
|
||||
} else if (OB_FAIL(trace_info_ptr->update_diagnose_info<ObMemtableDiagnoseInfo>(param,
|
||||
UpdateReleaseTime(param)))) {
|
||||
LOG_WARN("failed to update_memtable_diagnose_info", KR(ret), K(param));
|
||||
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
DEF_UPDATE_TIME_FUNCTOR(UpdateStartGCTimeForMemtable, ObMemtableDiagnoseInfo, start_gc_time)
|
||||
int ObCheckpointDiagnoseMgr::update_start_gc_time_for_memtable(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
ObTraceInfo *trace_info_ptr = get_trace_info_for_memtable(param);
|
||||
if (OB_ISNULL(trace_info_ptr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("trace_info is NULL", KR(ret), K(param));
|
||||
} else if (OB_FAIL(trace_info_ptr->update_diagnose_info<ObMemtableDiagnoseInfo>(param,
|
||||
UpdateStartGCTimeForMemtable(param)))) {
|
||||
LOG_WARN("failed to update_memtable_diagnose_info", KR(ret), K(param));
|
||||
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void UpdateMergeInfoForMemtable::operator()(ObMemtableDiagnoseInfo &info) const
|
||||
{
|
||||
info.merge_start_time_ = merge_start_time_;
|
||||
info.merge_finish_time_ = merge_finish_time_;
|
||||
info.occupy_size_ = occupy_size_;
|
||||
info.concurrent_cnt_ = concurrent_cnt_;
|
||||
TRANS_LOG(INFO, "update_merge_info", K(info), K(param_));
|
||||
}
|
||||
int ObCheckpointDiagnoseMgr::update_merge_info_for_memtable(const ObCheckpointDiagnoseParam ¶m,
|
||||
int64_t merge_start_time,
|
||||
int64_t merge_finish_time,
|
||||
int64_t occupy_size,
|
||||
int64_t concurrent_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
ObTraceInfo *trace_info_ptr = get_trace_info_for_memtable(param);
|
||||
if (OB_ISNULL(trace_info_ptr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("trace_info is NULL", KR(ret), K(param));
|
||||
} else if (OB_FAIL(trace_info_ptr->update_diagnose_info<ObMemtableDiagnoseInfo>(param,
|
||||
UpdateMergeInfoForMemtable(param, merge_start_time, merge_finish_time, occupy_size, concurrent_cnt)))) {
|
||||
LOG_WARN("failed to update_memtable_diagnose_info", KR(ret), K(param));
|
||||
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int GetTraceInfoForMemtable::operator()(ObTraceInfo &trace_info) const
|
||||
{
|
||||
if (param_.freeze_clock_ == trace_info.freeze_clock_ - 1
|
||||
&& param_.ls_id_ == trace_info.ls_id_.id()) {
|
||||
ret_ = &trace_info;
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
ObTraceInfo* ObCheckpointDiagnoseMgr::get_trace_info_for_memtable(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
ObTraceInfo *ret = NULL;
|
||||
if (param.is_freeze_clock_) {
|
||||
SpinRLockGuard lock(pos_lock_);
|
||||
for (int64_t i = first_pos_; i <= last_pos_; i++) {
|
||||
trace_info_arr_[i % MAX_TRACE_INFO_ARR_SIZE].read_trace_info(INVALID_TRACE_ID,
|
||||
GetTraceInfoForMemtable(param, ret));
|
||||
|
||||
if (OB_NOT_NULL(ret)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ret = &(trace_info_arr_[param.trace_id_ % MAX_TRACE_INFO_ARR_SIZE]);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCheckpointDiagnoseMgr::update_max_trace_info_size(int64_t max_trace_info_size)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (max_trace_info_size < 0 || max_trace_info_size > MAX_TRACE_INFO_ARR_SIZE) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
TRANS_LOG(WARN, "max_trace_info_size invalid", K(ret), K(max_trace_info_size));
|
||||
} else {
|
||||
SpinWLockGuard lock(pos_lock_);
|
||||
max_trace_info_size_ = max_trace_info_size;
|
||||
reset_old_trace_infos_without_pos_lock_();
|
||||
TRANS_LOG(INFO, "max_trace_info_size update.", KPC(this));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
593
src/storage/checkpoint/ob_checkpoint_diagnose.h
Normal file
593
src/storage/checkpoint/ob_checkpoint_diagnose.h
Normal file
@ -0,0 +1,593 @@
|
||||
/**
|
||||
* 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_STORAGE_OB_CHECKPOINT_DIAGNOSE_H_
|
||||
#define OCEANBASE_STORAGE_OB_CHECKPOINT_DIAGNOSE_H_
|
||||
|
||||
#include "ob_common_checkpoint.h"
|
||||
#include "common/ob_tablet_id.h"
|
||||
#include "share/ob_ls_id.h"
|
||||
#include "share/rc/ob_tenant_base.h"
|
||||
#include "share/ob_errno.h"
|
||||
#include "share/ob_define.h"
|
||||
|
||||
#define ADD_CHECKPOINT_DIAGNOSE_INFO_AND_SET_TRACE_ID(T, trace_id) \
|
||||
checkpoint::ObCheckpointDiagnoseMgr *cdm = MTL(checkpoint::ObCheckpointDiagnoseMgr*); \
|
||||
if (OB_NOT_NULL(cdm)) { \
|
||||
if (checkpoint::INVALID_TRACE_ID != trace_id) { \
|
||||
trace_id_ = trace_id; \
|
||||
checkpoint::ObCheckpointDiagnoseParam param(trace_id, get_tablet_id(), (void*)this); \
|
||||
cdm->add_diagnose_info<T>(param); \
|
||||
} \
|
||||
}
|
||||
|
||||
#define REPORT_CHECKPOINT_DIAGNOSE_INFO(func, unit_ptr, arg...) \
|
||||
checkpoint::ObCheckpointDiagnoseMgr *cdm = MTL(checkpoint::ObCheckpointDiagnoseMgr*); \
|
||||
if (OB_NOT_NULL(cdm)) { \
|
||||
if (checkpoint::INVALID_TRACE_ID != unit_ptr->get_trace_id()) { \
|
||||
checkpoint::ObCheckpointDiagnoseParam param(unit_ptr->get_trace_id(), unit_ptr->get_tablet_id(), unit_ptr); \
|
||||
cdm->func(param, ##arg); \
|
||||
} \
|
||||
}
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace storage
|
||||
{
|
||||
namespace checkpoint
|
||||
{
|
||||
|
||||
// +-------------------------+-------------------------+-------------------------
|
||||
// | tenant 1 | tenant 2 | tenant 3 |
|
||||
// | ObCheckpointDiagnoseMgr | ObCheckpointDiagnoseMgr | ObCheckpointDiagnoseMgr|
|
||||
// +-------------------------+------------+------------+-------------------------
|
||||
// |
|
||||
// V
|
||||
// +-------------+-------------+-------------+
|
||||
// | ObTraceInfo | ObTraceInfo | ObTraceInfo |
|
||||
// +-------------+------+------+-------------+
|
||||
// +-----------------------------------------------------------+
|
||||
// | |
|
||||
// V V
|
||||
// +-----------------------------------------------------=-+ +-------------------------------------------------+
|
||||
// | ObCheckpointUnitDiagnoseInfoMap | | ObMemtableDiagnoseInfoMap |
|
||||
// +-------------------------------------------------------+ +-------------------------------------------------+
|
||||
// | ObCheckpointDiagnoseKey|ObCheckpointUnitDiagnoseInfo* | | ObCheckpointDiagnoseKey|ObMemtableDiagnoseInfo* |
|
||||
// +-------------------------------------------------------+ +-------------------------------------------------+
|
||||
// | ObCheckpointDiagnoseKey|ObCheckpointUnitDiagnoseInfo* | | ObCheckpointDiagnoseKey|ObMemtableDiagnoseInfo* |
|
||||
// +-------------------------------------------------------+ +-------------------------------------------------+
|
||||
// | ObCheckpointDiagnoseKey|ObCheckpointUnitDiagnoseInfo* | | ObCheckpointDiagnoseKey|ObMemtableDiagnoseInfo* |
|
||||
// +-------------------------------------------------------+ +-------------------------------------------------+
|
||||
//
|
||||
// 1. every tenant has a ObCheckpointDiagnoseMgr.
|
||||
// 2. every checkpoint loop has a ObTraceInfo.
|
||||
// 3. ObTraceInfo has two map(ObCheckpointUnitDiagnoseInfoMap/ObMemtableDiagnoseInfoMap).
|
||||
// 4. memtable diagnose information is reported to ObMemtableDiagnoseInfoMap.
|
||||
// 4. other checkpoint_unit diagnose information is reported to ObCheckpointUnitDiagnoseInfoMap.
|
||||
|
||||
|
||||
static const uint64_t INVALID_TRACE_ID = -1;
|
||||
|
||||
struct ObCheckpointDiagnoseKey
|
||||
{
|
||||
ObCheckpointDiagnoseKey(const ObTabletID &tablet_id,
|
||||
void *checkpoint_unit_ptr)
|
||||
: tablet_id_(tablet_id),
|
||||
checkpoint_unit_ptr_(checkpoint_unit_ptr) {}
|
||||
ObCheckpointDiagnoseKey()
|
||||
: tablet_id_(),
|
||||
checkpoint_unit_ptr_(NULL) {}
|
||||
|
||||
int hash(uint64_t &hash_val) const
|
||||
{
|
||||
hash_val = tablet_id_.hash();
|
||||
hash_val = murmurhash(&checkpoint_unit_ptr_, sizeof(checkpoint_unit_ptr_), hash_val);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
bool operator==(const ObCheckpointDiagnoseKey &other) const
|
||||
{ return tablet_id_ == other.tablet_id_ && checkpoint_unit_ptr_ == other.checkpoint_unit_ptr_; }
|
||||
bool is_valid() const { return tablet_id_.is_valid() && OB_NOT_NULL(checkpoint_unit_ptr_); }
|
||||
TO_STRING_KV(K_(tablet_id), K_(checkpoint_unit_ptr));
|
||||
|
||||
ObTabletID tablet_id_;
|
||||
void *checkpoint_unit_ptr_;
|
||||
};
|
||||
|
||||
struct ObCheckpointDiagnoseParam
|
||||
{
|
||||
ObCheckpointDiagnoseParam(const int64_t trace_id,
|
||||
const ObTabletID &tablet_id,
|
||||
void *checkpoint_unit_ptr)
|
||||
: trace_id_(trace_id),
|
||||
key_(tablet_id, checkpoint_unit_ptr),
|
||||
is_freeze_clock_(false) {}
|
||||
// for the memtable of logstream freeze
|
||||
ObCheckpointDiagnoseParam(const int64_t ls_id,
|
||||
const uint32_t freeze_clock,
|
||||
const ObTabletID &tablet_id,
|
||||
void *checkpoint_unit_ptr)
|
||||
: ls_id_(ls_id),
|
||||
freeze_clock_(freeze_clock),
|
||||
key_(tablet_id, checkpoint_unit_ptr),
|
||||
is_freeze_clock_(true) {}
|
||||
TO_STRING_KV(K_(trace_id), K_(ls_id), K_(freeze_clock), K_(key), K_(is_freeze_clock));
|
||||
union
|
||||
{
|
||||
int64_t trace_id_;
|
||||
struct {
|
||||
int64_t ls_id_;
|
||||
uint32_t freeze_clock_;
|
||||
};
|
||||
};
|
||||
bool is_valid() const
|
||||
{
|
||||
if (is_freeze_clock_) {
|
||||
return key_.is_valid() && ls_id_ > 0;
|
||||
} else {
|
||||
return key_.is_valid() && trace_id_ != INVALID_TRACE_ID;
|
||||
}
|
||||
}
|
||||
ObCheckpointDiagnoseKey key_;
|
||||
bool is_freeze_clock_;
|
||||
};
|
||||
|
||||
struct ObCheckpointUnitDiagnoseInfo
|
||||
{
|
||||
const static int64_t MAX_TIME = 253402271999999999;
|
||||
friend class ObTraceInfo;
|
||||
ObCheckpointUnitDiagnoseInfo()
|
||||
: start_scn_(),
|
||||
end_scn_(),
|
||||
rec_scn_(),
|
||||
create_flush_dag_time_(MAX_TIME),
|
||||
merge_finish_time_(MAX_TIME),
|
||||
start_gc_time_(MAX_TIME) {}
|
||||
|
||||
share::SCN start_scn_;
|
||||
share::SCN end_scn_;
|
||||
share::SCN rec_scn_;
|
||||
int64_t create_flush_dag_time_;
|
||||
int64_t merge_finish_time_;
|
||||
int64_t start_gc_time_;
|
||||
TO_STRING_KV(K_(start_scn), K_(end_scn), K_(rec_scn), K_(create_flush_dag_time),
|
||||
K_(merge_finish_time), K_(start_gc_time));
|
||||
private:
|
||||
// lock/unlock in ObTraceInfo
|
||||
common::SpinRWLock lock_;
|
||||
};
|
||||
|
||||
struct ObMemtableDiagnoseInfo : public ObCheckpointUnitDiagnoseInfo
|
||||
{
|
||||
ObMemtableDiagnoseInfo()
|
||||
: frozen_finish_time_(MAX_TIME),
|
||||
memtable_occupy_size_(0),
|
||||
merge_start_time_(MAX_TIME),
|
||||
occupy_size_(0),
|
||||
concurrent_cnt_(0),
|
||||
release_time_(MAX_TIME) {}
|
||||
int64_t frozen_finish_time_;
|
||||
int64_t memtable_occupy_size_;
|
||||
int64_t merge_start_time_;
|
||||
int64_t occupy_size_;
|
||||
int64_t concurrent_cnt_;
|
||||
int64_t release_time_;
|
||||
INHERIT_TO_STRING_KV("ObCheckpointUnitDiagnoseInfo", ObCheckpointUnitDiagnoseInfo,
|
||||
K_(frozen_finish_time), K_(memtable_occupy_size), K_(merge_start_time), K_(occupy_size),
|
||||
K_(concurrent_cnt), K_(release_time));
|
||||
};
|
||||
|
||||
typedef common::hash::ObHashMap<ObCheckpointDiagnoseKey, ObCheckpointUnitDiagnoseInfo*> ObCheckpointUnitDiagnoseInfoMap;
|
||||
typedef common::hash::ObHashMap<ObCheckpointDiagnoseKey, ObMemtableDiagnoseInfo*> ObMemtableDiagnoseInfoMap;
|
||||
|
||||
class ObTraceInfo
|
||||
{
|
||||
const static int64_t MALLOC_BLOCK_SIZE = (1LL << 12) - 128; // 4k
|
||||
public:
|
||||
ObTraceInfo()
|
||||
: trace_id_(INVALID_TRACE_ID),
|
||||
freeze_clock_(0),
|
||||
ls_id_(),
|
||||
checkpoint_start_time_(0),
|
||||
arena_("CkpDgn", MALLOC_BLOCK_SIZE
|
||||
#ifndef UNITTEST
|
||||
, MTL_ID()
|
||||
#endif
|
||||
),
|
||||
allocator_(arena_),
|
||||
checkpoint_unit_diagnose_info_map_(),
|
||||
memtable_diagnose_info_map_(),
|
||||
lock_()
|
||||
{ memset(thread_name_, 0, oceanbase::OB_THREAD_NAME_BUF_LEN); }
|
||||
virtual ~ObTraceInfo() { reset(); }
|
||||
void init(const int64_t trace_id,
|
||||
const share::ObLSID &ls_id,
|
||||
const int64_t checkpoint_start_time);
|
||||
void reset()
|
||||
{
|
||||
SpinWLockGuard lock(lock_);
|
||||
reset_without_lock_();
|
||||
}
|
||||
void reset_without_lock_();
|
||||
void update_freeze_clock(const int64_t trace_id,
|
||||
const uint32_t logstream_clock);
|
||||
bool check_trace_id_(const int64_t trace_id);
|
||||
bool check_trace_id_(const ObCheckpointDiagnoseParam ¶m);
|
||||
|
||||
template <typename T>
|
||||
int add_diagnose_info(const ObCheckpointDiagnoseParam ¶m);
|
||||
template <typename T, typename OP>
|
||||
int update_diagnose_info(const ObCheckpointDiagnoseParam ¶m,
|
||||
const OP &op);
|
||||
template <typename T, typename OP>
|
||||
int read_diagnose_info(const int64_t trace_id,
|
||||
const OP &op);
|
||||
|
||||
template <typename OP>
|
||||
int read_trace_info(const int64_t trace_id,
|
||||
const OP &op);
|
||||
|
||||
template<typename T>
|
||||
common::hash::ObHashMap<ObCheckpointDiagnoseKey, T*> &get_diagnose_info_map_();
|
||||
template<>
|
||||
ObCheckpointUnitDiagnoseInfoMap &get_diagnose_info_map_()
|
||||
{
|
||||
return checkpoint_unit_diagnose_info_map_;
|
||||
}
|
||||
template<>
|
||||
ObMemtableDiagnoseInfoMap &get_diagnose_info_map_()
|
||||
{
|
||||
return memtable_diagnose_info_map_;
|
||||
}
|
||||
TO_STRING_KV(K_(trace_id), K_(freeze_clock), K_(ls_id), K_(thread_name), K(checkpoint_unit_diagnose_info_map_.size()), K(memtable_diagnose_info_map_.size()));
|
||||
|
||||
int64_t trace_id_;
|
||||
uint32_t freeze_clock_;
|
||||
share::ObLSID ls_id_;
|
||||
char thread_name_[oceanbase::OB_THREAD_NAME_BUF_LEN];
|
||||
int64_t checkpoint_start_time_;
|
||||
ObArenaAllocator arena_;
|
||||
ObSafeArenaAllocator allocator_;
|
||||
ObCheckpointUnitDiagnoseInfoMap checkpoint_unit_diagnose_info_map_;
|
||||
ObMemtableDiagnoseInfoMap memtable_diagnose_info_map_;
|
||||
common::SpinRWLock lock_;
|
||||
};
|
||||
|
||||
#define DEF_UPDATE_TIME_FUNCTOR(function, diagnose_info_type, diagnose_info_time) \
|
||||
struct function \
|
||||
{ \
|
||||
public: \
|
||||
function(const ObCheckpointDiagnoseParam ¶m) : param_(param) {} \
|
||||
function& operator=(const function&) = delete; \
|
||||
void operator()(diagnose_info_type &info) const \
|
||||
{ \
|
||||
const int64_t start_time = ObTimeUtility::current_time(); \
|
||||
info.diagnose_info_time##_ = start_time; \
|
||||
TRANS_LOG(INFO, ""#function"", K(info), K(param_)); \
|
||||
} \
|
||||
private: \
|
||||
const ObCheckpointDiagnoseParam ¶m_; \
|
||||
};
|
||||
|
||||
struct UpdateScheduleDagInfo
|
||||
{
|
||||
public:
|
||||
UpdateScheduleDagInfo(const ObCheckpointDiagnoseParam ¶m,
|
||||
const share::SCN &rec_scn,
|
||||
const share::SCN &start_scn,
|
||||
const share::SCN &end_scn)
|
||||
: param_(param),
|
||||
rec_scn_(rec_scn),
|
||||
start_scn_(start_scn),
|
||||
end_scn_(end_scn)
|
||||
{}
|
||||
UpdateScheduleDagInfo& operator=(const UpdateScheduleDagInfo&) = delete;
|
||||
void operator()(ObCheckpointUnitDiagnoseInfo &info) const;
|
||||
private:
|
||||
const ObCheckpointDiagnoseParam ¶m_;
|
||||
const share::SCN &rec_scn_;
|
||||
const share::SCN &start_scn_;
|
||||
const share::SCN &end_scn_;
|
||||
};
|
||||
|
||||
struct UpdateFreezeInfo
|
||||
{
|
||||
public:
|
||||
UpdateFreezeInfo(const ObCheckpointDiagnoseParam ¶m,
|
||||
const share::SCN &rec_scn,
|
||||
const share::SCN &start_scn,
|
||||
const share::SCN &end_scn,
|
||||
int64_t occupy_size)
|
||||
: param_(param),
|
||||
rec_scn_(rec_scn),
|
||||
start_scn_(start_scn),
|
||||
end_scn_(end_scn),
|
||||
occupy_size_(occupy_size)
|
||||
{}
|
||||
UpdateFreezeInfo& operator=(const UpdateFreezeInfo&) = delete;
|
||||
void operator()(ObMemtableDiagnoseInfo &info) const;
|
||||
private:
|
||||
const ObCheckpointDiagnoseParam ¶m_;
|
||||
const share::SCN &rec_scn_;
|
||||
const share::SCN &start_scn_;
|
||||
const share::SCN &end_scn_;
|
||||
int64_t occupy_size_;
|
||||
};
|
||||
|
||||
struct UpdateMergeInfoForMemtable
|
||||
{
|
||||
public:
|
||||
UpdateMergeInfoForMemtable(const ObCheckpointDiagnoseParam ¶m,
|
||||
int64_t merge_start_time,
|
||||
int64_t merge_finish_time,
|
||||
int64_t occupy_size,
|
||||
int64_t concurrent_cnt)
|
||||
: param_(param),
|
||||
merge_start_time_(merge_start_time),
|
||||
merge_finish_time_(merge_finish_time),
|
||||
occupy_size_(occupy_size),
|
||||
concurrent_cnt_(concurrent_cnt)
|
||||
{}
|
||||
UpdateMergeInfoForMemtable& operator=(const UpdateMergeInfoForMemtable&) = delete;
|
||||
void operator()(ObMemtableDiagnoseInfo &info) const;
|
||||
private:
|
||||
const ObCheckpointDiagnoseParam ¶m_;
|
||||
int64_t merge_start_time_;
|
||||
int64_t merge_finish_time_;
|
||||
int64_t occupy_size_;
|
||||
int64_t concurrent_cnt_;
|
||||
};
|
||||
|
||||
struct GetTraceInfoForMemtable
|
||||
{
|
||||
public:
|
||||
GetTraceInfoForMemtable(const ObCheckpointDiagnoseParam ¶m,
|
||||
ObTraceInfo *&ret)
|
||||
: param_(param),
|
||||
ret_(ret)
|
||||
{}
|
||||
GetTraceInfoForMemtable& operator=(const GetTraceInfoForMemtable&) = delete;
|
||||
int operator()(ObTraceInfo &trace_info) const;
|
||||
private:
|
||||
const ObCheckpointDiagnoseParam ¶m_;
|
||||
ObTraceInfo *&ret_;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
int ObTraceInfo::add_diagnose_info(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
SpinRLockGuard lock(lock_);
|
||||
if (check_trace_id_(param)) {
|
||||
void *ptr = NULL;
|
||||
if (OB_ISNULL(ptr = allocator_.alloc(sizeof(T)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
TRANS_LOG(WARN, "failed to alloc", KR(ret));
|
||||
} else {
|
||||
T *diagnose_info = new (ptr)T();
|
||||
SpinWLockGuard lock(diagnose_info->lock_);
|
||||
if (OB_FAIL(get_diagnose_info_map_<T>().set_refactored(param.key_, diagnose_info,
|
||||
0 /* not overwrite */ ))) {
|
||||
TRANS_LOG(WARN, "failed to add diagnose info", KR(ret), K(param.key_), KPC(diagnose_info), KPC(this));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename T, typename OP>
|
||||
int ObTraceInfo::update_diagnose_info(const ObCheckpointDiagnoseParam ¶m,
|
||||
const OP &op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
SpinRLockGuard lock(lock_);
|
||||
|
||||
if (check_trace_id_(param)) {
|
||||
T *diagnose_info = NULL;
|
||||
if (OB_FAIL(get_diagnose_info_map_<T>().get_refactored(param.key_, diagnose_info))) {
|
||||
TRANS_LOG(WARN, "failed to get diagnose info", KR(ret), K(param.key_), KPC(this));
|
||||
} else if (OB_ISNULL(diagnose_info)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
TRANS_LOG(WARN, "diagnose info is NULL", KR(ret), K(param.key_), KPC(this));
|
||||
} else {
|
||||
SpinWLockGuard lock(diagnose_info->lock_);
|
||||
op(*diagnose_info);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename T, typename OP>
|
||||
int ObTraceInfo::read_diagnose_info(const int64_t trace_id,
|
||||
const OP &op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
TRANS_LOG(INFO, "read_diagenose_info in ObTraceInfo", KPC(this), K(trace_id));
|
||||
SpinRLockGuard lock(lock_);
|
||||
if (check_trace_id_(trace_id)) {
|
||||
FOREACH(iter, get_diagnose_info_map_<T>()) {
|
||||
ObCheckpointDiagnoseKey key = iter->first;
|
||||
T *diagnose_info = iter->second;
|
||||
if (OB_ISNULL(diagnose_info)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
TRANS_LOG(WARN, "diagnose_info is NULL", KR(ret), K(key));
|
||||
} else {
|
||||
SpinRLockGuard diagnose_lock(diagnose_info->lock_);
|
||||
if (OB_FAIL(op(*this, key, *diagnose_info))) {
|
||||
TRANS_LOG(WARN, "failed to op in read_diagnose_info", KR(ret), K(key), K(diagnose_info));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename OP>
|
||||
int ObTraceInfo::read_trace_info(const int64_t trace_id, const OP &op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
SpinRLockGuard lock(lock_);
|
||||
if (INVALID_TRACE_ID == trace_id
|
||||
|| check_trace_id_(trace_id)) {
|
||||
if (OB_FAIL(op(*this))) {
|
||||
TRANS_LOG(WARN, "failed to op in read_trace_info", KR(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
class ObCheckpointDiagnoseMgr
|
||||
{
|
||||
private:
|
||||
const static int64_t MAX_TRACE_INFO_ARR_SIZE = 800;
|
||||
public:
|
||||
ObCheckpointDiagnoseMgr(int64_t max_trace_info_size = 100)
|
||||
: first_pos_(0),
|
||||
last_pos_(-1),
|
||||
max_trace_info_size_(max_trace_info_size),
|
||||
is_inited_(false)
|
||||
{}
|
||||
virtual ~ObCheckpointDiagnoseMgr() {}
|
||||
static int mtl_init(ObCheckpointDiagnoseMgr* &m) { return m->init(); }
|
||||
int init();
|
||||
void destroy() { is_inited_ = false; };
|
||||
int update_max_trace_info_size(int64_t max_trace_info_size);
|
||||
|
||||
// trace_info func
|
||||
int acquire_trace_id(const share::ObLSID &ls_id,
|
||||
int64_t &trace_id);
|
||||
int update_freeze_clock(const share::ObLSID &ls_id,
|
||||
const int64_t trace_id,
|
||||
const int logstream_clock);
|
||||
|
||||
template <typename OP>
|
||||
int add_diagnose_info(const ObCheckpointDiagnoseParam ¶m);
|
||||
// checkpoint_unit func
|
||||
int update_schedule_dag_info(const ObCheckpointDiagnoseParam ¶m,
|
||||
const share::SCN &rec_scn,
|
||||
const share::SCN &start_scn,
|
||||
const share::SCN &end_scn);
|
||||
int update_merge_info_for_checkpoint_unit(const ObCheckpointDiagnoseParam ¶m);
|
||||
int update_start_gc_time_for_checkpoint_unit(const ObCheckpointDiagnoseParam ¶m);
|
||||
|
||||
// memtable func
|
||||
int update_freeze_info(const ObCheckpointDiagnoseParam ¶m,
|
||||
const share::SCN &rec_scn,
|
||||
const share::SCN &start_scn,
|
||||
const share::SCN &end_scn,
|
||||
const int64_t occupy_size);
|
||||
int update_schedule_dag_time(const ObCheckpointDiagnoseParam ¶m);
|
||||
int update_release_time(const ObCheckpointDiagnoseParam ¶m);
|
||||
int update_merge_info_for_memtable(const ObCheckpointDiagnoseParam ¶m,
|
||||
int64_t merge_start_time,
|
||||
int64_t merge_finish_time,
|
||||
int64_t occupy_size,
|
||||
int64_t concurrent_cnt);
|
||||
int update_start_gc_time_for_memtable(const ObCheckpointDiagnoseParam ¶m);
|
||||
|
||||
// virtual table iter
|
||||
template <typename OP>
|
||||
int read_trace_info(const OP &op);
|
||||
template<typename T, typename OP>
|
||||
int read_diagnose_info(const int64_t trace_id,
|
||||
const OP &op);
|
||||
|
||||
int64_t get_trace_info_count() { return last_pos_ - first_pos_ + 1; }
|
||||
ObTraceInfo* get_trace_info_for_memtable(const ObCheckpointDiagnoseParam ¶m);
|
||||
TO_STRING_KV(K_(first_pos), K_(last_pos), K_(max_trace_info_size));
|
||||
|
||||
private:
|
||||
void reset_old_trace_infos_without_pos_lock_()
|
||||
{
|
||||
while (last_pos_ - first_pos_ >= max_trace_info_size_) {
|
||||
trace_info_arr_[first_pos_++ % MAX_TRACE_INFO_ARR_SIZE].reset();
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
int64_t first_pos_;
|
||||
int64_t last_pos_;
|
||||
ObTraceInfo trace_info_arr_[MAX_TRACE_INFO_ARR_SIZE];
|
||||
common::SpinRWLock pos_lock_;
|
||||
// max count of trace_info, others in trace_info_arr will be reset
|
||||
int64_t max_trace_info_size_;
|
||||
bool is_inited_;
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
int ObCheckpointDiagnoseMgr::add_diagnose_info(const ObCheckpointDiagnoseParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
TRANS_LOG(WARN, "ObCheckpointDiagnoseMgr not inited.", K(ret));
|
||||
} else if (!param.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
TRANS_LOG(WARN, "param is invalid", KR(ret), K(param));
|
||||
} else {
|
||||
ObTraceInfo *trace_info_ptr = get_trace_info_for_memtable(param);
|
||||
if (OB_ISNULL(trace_info_ptr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
TRANS_LOG(WARN, "trace_info is NULL", KR(ret), K(param));
|
||||
} else if (OB_FAIL(trace_info_ptr->add_diagnose_info<T>(param))) {
|
||||
TRANS_LOG(WARN, "failed to add diagnose_info", KR(ret), K(param));
|
||||
} else {
|
||||
TRANS_LOG(INFO, "add_diagnose_info", KR(ret), K(param));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template<typename T, typename OP>
|
||||
int ObCheckpointDiagnoseMgr::read_diagnose_info(const int64_t trace_id,
|
||||
const OP &op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
TRANS_LOG(INFO, "read_diagnose_info in ObCheckpointDiagnoseMgr", K(first_pos_), K(last_pos_), K(get_trace_info_count()));
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
TRANS_LOG(WARN, "ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else if (INVALID_TRACE_ID == trace_id) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
TRANS_LOG(WARN, "trace_id is invalid", KR(ret));
|
||||
} else {
|
||||
if (get_trace_info_count() > 0) {
|
||||
if (OB_FAIL(trace_info_arr_[trace_id % MAX_TRACE_INFO_ARR_SIZE].read_diagnose_info<T>(trace_id, op))) {
|
||||
TRANS_LOG(WARN, "failed to read_diagnose_info", KR(ret), K(trace_id));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename OP>
|
||||
int ObCheckpointDiagnoseMgr::read_trace_info(const OP &op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
TRANS_LOG(WARN, "ObCheckpointDiagnoseMgr not inited.", KR(ret));
|
||||
} else {
|
||||
SpinRLockGuard lock(pos_lock_);
|
||||
if (get_trace_info_count() > 0) {
|
||||
for (int64_t i = first_pos_; OB_SUCC(ret) && i <= last_pos_; i++) {
|
||||
ret = trace_info_arr_[i % MAX_TRACE_INFO_ARR_SIZE].read_trace_info(i, op);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#endif // OCEANBASE_STORAGE_OB_CHECKPOINT_DIAGNOSE_H_
|
@ -18,6 +18,7 @@
|
||||
#include "storage/checkpoint/ob_data_checkpoint.h"
|
||||
#include "share/ob_force_print_log.h"
|
||||
#include "logservice/ob_log_base_type.h"
|
||||
#include "observer/ob_server_event_history_table_operator.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -33,7 +34,9 @@ ObCheckpointExecutor::ObCheckpointExecutor()
|
||||
update_checkpoint_enabled_(false),
|
||||
reuse_recycle_scn_times_(0),
|
||||
prev_clog_checkpoint_lsn_(),
|
||||
prev_recycle_scn_()
|
||||
prev_recycle_scn_(),
|
||||
update_clog_checkpoint_times_(0),
|
||||
last_add_server_history_time_(0)
|
||||
{
|
||||
reset();
|
||||
}
|
||||
@ -53,6 +56,8 @@ void ObCheckpointExecutor::reset()
|
||||
reuse_recycle_scn_times_ = 0;
|
||||
prev_clog_checkpoint_lsn_.reset();
|
||||
prev_recycle_scn_.set_invalid();
|
||||
update_clog_checkpoint_times_ = 0;
|
||||
last_add_server_history_time_ = 0;
|
||||
}
|
||||
|
||||
int ObCheckpointExecutor::register_handler(const ObLogBaseType &type,
|
||||
@ -137,6 +142,23 @@ inline void get_min_rec_scn_service_type_by_index_(int index, char* service_type
|
||||
}
|
||||
}
|
||||
|
||||
void ObCheckpointExecutor::add_server_event_history_for_update_clog_checkpoint(
|
||||
const SCN &checkpoint_scn,
|
||||
const char *service_type)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (update_clog_checkpoint_times_ > 0) {
|
||||
int64_t cur_time = ObClockGenerator::getClock();
|
||||
if (cur_time - last_add_server_history_time_ > ADD_SERVER_HISTORY_INTERVAL) {
|
||||
const uint64_t tenant_id = MTL_ID();
|
||||
const int64_t ls_id = ls_->get_ls_id().id();
|
||||
last_add_server_history_time_ = cur_time;
|
||||
SERVER_EVENT_ADD("checkpoint", "update_clog_checkpoint", K(tenant_id), K(ls_id), K(checkpoint_scn), K(service_type), K_(update_clog_checkpoint_times));
|
||||
update_clog_checkpoint_times_ = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
int ObCheckpointExecutor::update_clog_checkpoint()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -187,10 +209,12 @@ int ObCheckpointExecutor::update_clog_checkpoint()
|
||||
} else if (OB_FAIL(ls_->set_clog_checkpoint(clog_checkpoint_lsn, checkpoint_scn))) {
|
||||
STORAGE_LOG(WARN, "set clog checkpoint failed", K(ret), K(clog_checkpoint_lsn), K(checkpoint_scn), K(ls_id));
|
||||
} else {
|
||||
update_clog_checkpoint_times_++;
|
||||
FLOG_INFO("[CHECKPOINT] update clog checkpoint successfully",
|
||||
K(clog_checkpoint_lsn), K(checkpoint_scn), K(ls_id),
|
||||
K(service_type));
|
||||
}
|
||||
add_server_event_history_for_update_clog_checkpoint(checkpoint_scn, service_type);
|
||||
}
|
||||
} else {
|
||||
STORAGE_LOG(WARN, "freezer should not null", K(ls_->get_ls_id()));
|
||||
|
@ -72,6 +72,9 @@ public:
|
||||
|
||||
// calculate clog checkpoint and update in ls_meta
|
||||
int update_clog_checkpoint();
|
||||
void add_server_event_history_for_update_clog_checkpoint(
|
||||
const SCN &checkpoint_scn,
|
||||
const char *service_type);
|
||||
|
||||
// the service will flush and advance checkpoint
|
||||
// after flush, checkpoint_scn will be equal or greater than recycle_scn
|
||||
@ -94,6 +97,7 @@ private:
|
||||
|
||||
private:
|
||||
static const int64_t CLOG_GC_PERCENT = 60;
|
||||
static const int64_t ADD_SERVER_HISTORY_INTERVAL = 10 * 60 * 1000 * 1000; // 10 min
|
||||
|
||||
ObLS *ls_;
|
||||
logservice::ObILogHandler *loghandler_;
|
||||
@ -112,6 +116,9 @@ private:
|
||||
|
||||
palf::LSN prev_clog_checkpoint_lsn_;
|
||||
share::SCN prev_recycle_scn_;
|
||||
|
||||
int64_t update_clog_checkpoint_times_;
|
||||
int64_t last_add_server_history_time_;
|
||||
};
|
||||
|
||||
|
||||
|
@ -89,7 +89,7 @@ class ObCommonCheckpoint
|
||||
{
|
||||
public:
|
||||
virtual share::SCN get_rec_scn() = 0;
|
||||
virtual int flush(share::SCN recycle_scn, bool need_freeze = true) = 0;
|
||||
virtual int flush(share::SCN recycle_scn, const int64_t trace_id, bool need_freeze = true) = 0;
|
||||
|
||||
virtual ObTabletID get_tablet_id() const = 0;
|
||||
virtual share::SCN get_rec_scn(ObTabletID &tablet_id) {
|
||||
|
@ -44,7 +44,7 @@ void ObCheckpointDList::reset()
|
||||
}
|
||||
}
|
||||
|
||||
bool ObCheckpointDList::is_empty()
|
||||
bool ObCheckpointDList::is_empty() const
|
||||
{
|
||||
return checkpoint_list_.is_empty();
|
||||
}
|
||||
@ -197,7 +197,7 @@ int ObDataCheckpoint::safe_to_destroy(bool &is_safe_destroy)
|
||||
// avoid start ls_freeze again after waiting ls_freeze finish
|
||||
is_inited_ = false;
|
||||
// wait until ls_freeze finish
|
||||
while(is_flushing()) {
|
||||
while(!ls_freeze_finished_) {
|
||||
ob_usleep(1000 * 1000);
|
||||
if (REACH_TIME_INTERVAL(10 * 1000L * 1000L)) {
|
||||
STORAGE_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "ls freeze cost too much time", K(ls_->get_ls_id()));
|
||||
@ -256,7 +256,7 @@ SCN ObDataCheckpoint::get_active_rec_scn()
|
||||
return min_active_rec_scn;
|
||||
}
|
||||
|
||||
int ObDataCheckpoint::flush(SCN recycle_scn, bool need_freeze)
|
||||
int ObDataCheckpoint::flush(SCN recycle_scn, int64_t trace_id, bool need_freeze)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (need_freeze) {
|
||||
@ -266,9 +266,9 @@ int ObDataCheckpoint::flush(SCN recycle_scn, bool need_freeze)
|
||||
"skip flush data checkpoint cause active_rec_scn is larger than recycle_scn",
|
||||
K(active_rec_scn),
|
||||
K(recycle_scn));
|
||||
} else if (OB_FAIL(freeze_base_on_needs_(recycle_scn))) {
|
||||
} else if (OB_FAIL(freeze_base_on_needs_(trace_id, recycle_scn))) {
|
||||
STORAGE_LOG(WARN, "freeze_base_on_needs failed",
|
||||
K(ret), K(ls_->get_ls_id()), K(recycle_scn));
|
||||
K(ret), K(ls_->get_ls_id()), K(recycle_scn), K(trace_id));
|
||||
}
|
||||
} else if (OB_FAIL(traversal_flush_())) {
|
||||
STORAGE_LOG(WARN, "traversal_flush failed", K(ret), K(ls_->get_ls_id()));
|
||||
@ -306,7 +306,7 @@ ObTabletID ObDataCheckpoint::get_tablet_id() const
|
||||
|
||||
bool ObDataCheckpoint::is_flushing() const
|
||||
{
|
||||
return !ls_freeze_finished_;
|
||||
return !ls_freeze_finished_ || !prepare_list_.is_empty();
|
||||
}
|
||||
|
||||
bool ObDataCheckpoint::is_empty()
|
||||
@ -365,10 +365,13 @@ void ObDataCheckpoint::road_to_flush(SCN rec_scn)
|
||||
last = active_list_.get_first_greater(rec_scn);
|
||||
}
|
||||
pop_active_list_to_ls_frozen_(last);
|
||||
last_time = common::ObTimeUtility::fast_current_time();
|
||||
STORAGE_LOG(INFO, "[Freezer] active_list to ls_frozen_list success",
|
||||
K(ls_->get_ls_id()));
|
||||
// add diagnose info
|
||||
add_diagnose_info_for_ls_frozen_();
|
||||
|
||||
// ls_frozen_list -> prepare_list
|
||||
last_time = common::ObTimeUtility::fast_current_time();
|
||||
ls_frozen_to_prepare_(last_time);
|
||||
STORAGE_LOG(INFO, "[Freezer] road_to_flush end", K(ls_->get_ls_id()));
|
||||
}
|
||||
@ -495,7 +498,7 @@ void ObDataCheckpoint::ls_frozen_to_prepare_(int64_t &last_time)
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
auto ob_freeze_checkpoint = iterator.get_next();
|
||||
if (ob_freeze_checkpoint->ready_for_flush()) {
|
||||
if (OB_FAIL(finish_freeze(ob_freeze_checkpoint))) {
|
||||
if (OB_FAIL(ob_freeze_checkpoint->finish_freeze())) {
|
||||
STORAGE_LOG(WARN, "finish freeze failed", K(ret));
|
||||
}
|
||||
} else if (ob_freeze_checkpoint->is_active_checkpoint()) {
|
||||
@ -886,33 +889,52 @@ int ObDataCheckpoint::get_need_flush_tablets_(const share::SCN recycle_scn,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDataCheckpoint::freeze_base_on_needs_(share::SCN recycle_scn)
|
||||
int ObDataCheckpoint::freeze_base_on_needs_(const int64_t trace_id,
|
||||
share::SCN recycle_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (is_tenant_freeze() || !is_flushing()) {
|
||||
int64_t wait_flush_num = new_create_list_.checkpoint_list_.get_size() + active_list_.checkpoint_list_.get_size();
|
||||
bool logstream_freeze = true;
|
||||
ObSArray<ObTabletID> need_flush_tablets;
|
||||
if (wait_flush_num > MAX_FREEZE_CHECKPOINT_NUM) {
|
||||
if (OB_FAIL(get_need_flush_tablets_(recycle_scn, need_flush_tablets))) {
|
||||
// do nothing
|
||||
} else {
|
||||
int need_flush_num = need_flush_tablets.count();
|
||||
logstream_freeze = need_flush_num * 100 / wait_flush_num > TABLET_FREEZE_PERCENT;
|
||||
if (get_rec_scn() <= recycle_scn) {
|
||||
if (is_tenant_freeze() || !is_flushing()) {
|
||||
int64_t wait_flush_num =
|
||||
new_create_list_.checkpoint_list_.get_size()
|
||||
+ active_list_.checkpoint_list_.get_size();
|
||||
bool logstream_freeze = true;
|
||||
ObSArray<ObTabletID> need_flush_tablets;
|
||||
if (wait_flush_num > MAX_FREEZE_CHECKPOINT_NUM) {
|
||||
if (OB_FAIL(get_need_flush_tablets_(recycle_scn, need_flush_tablets))) {
|
||||
// do nothing
|
||||
} else {
|
||||
int need_flush_num = need_flush_tablets.count();
|
||||
logstream_freeze =
|
||||
need_flush_num * 100 / wait_flush_num > TABLET_FREEZE_PERCENT;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (logstream_freeze) {
|
||||
if (OB_FAIL(ls_->logstream_freeze(false /* !is_sync */))) {
|
||||
STORAGE_LOG(WARN, "minor freeze failed", K(ret), K(ls_->get_ls_id()));
|
||||
if (logstream_freeze) {
|
||||
if (OB_FAIL(ls_->logstream_freeze(trace_id, false /* !is_sync */))) {
|
||||
STORAGE_LOG(WARN, "minor freeze failed", K(ret), K(ls_->get_ls_id()));
|
||||
}
|
||||
} else if (OB_FAIL(ls_->batch_tablet_freeze(trace_id, need_flush_tablets, false /* !is_sync */))) {
|
||||
STORAGE_LOG(WARN, "batch tablet freeze failed",
|
||||
K(ret), K(ls_->get_ls_id()), K(need_flush_tablets));
|
||||
}
|
||||
} else if (OB_FAIL(ls_->batch_tablet_freeze(need_flush_tablets, false /* !is_sync */))) {
|
||||
STORAGE_LOG(WARN, "batch tablet freeze failed", K(ret), K(ls_->get_ls_id()), K(need_flush_tablets));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObDataCheckpoint::add_diagnose_info_for_ls_frozen_()
|
||||
{
|
||||
ObCheckpointIterator iterator;
|
||||
RLOCK(LS_FROZEN);
|
||||
ls_frozen_list_.get_iterator(iterator);
|
||||
while (iterator.has_next()) {
|
||||
memtable::ObMemtable *memtable = static_cast<memtable::ObMemtable*>(iterator.get_next());
|
||||
if (!memtable->is_active_checkpoint()) {
|
||||
memtable->report_memtable_diagnose_info(memtable::ObMemtable::AddCheckpointDiagnoseInfoForMemtable());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace checkpoint
|
||||
} // namespace storage
|
||||
} // namespace oceanbase
|
||||
|
@ -37,7 +37,7 @@ struct ObCheckpointDList
|
||||
~ObCheckpointDList() {}
|
||||
|
||||
void reset();
|
||||
bool is_empty();
|
||||
bool is_empty() const;
|
||||
ObFreezeCheckpoint *get_header();
|
||||
int unlink(ObFreezeCheckpoint *ob_freeze_checkpoint);
|
||||
int insert(ObFreezeCheckpoint *ob_freeze_checkpoint, bool ordered = true);
|
||||
@ -117,7 +117,7 @@ public:
|
||||
share::SCN get_active_rec_scn();
|
||||
// if min_rec_scn <= the input rec_scn
|
||||
// logstream freeze
|
||||
int flush(share::SCN recycle_scn, bool need_freeze = true);
|
||||
int flush(share::SCN recycle_scn, int64_t trace_id, bool need_freeze = true);
|
||||
// if min_rec_scn <= the input rec_scn
|
||||
// add ls_freeze task
|
||||
// logstream freeze optimization
|
||||
@ -172,11 +172,12 @@ private:
|
||||
void pop_new_create_to_ls_frozen_();
|
||||
void ls_frozen_to_active_(int64_t &last_time);
|
||||
void ls_frozen_to_prepare_(int64_t &last_time);
|
||||
void add_diagnose_info_for_ls_frozen_();
|
||||
void print_list_(ObCheckpointDList &list);
|
||||
void set_ls_freeze_finished_(bool is_finished);
|
||||
int get_need_flush_tablets_(const share::SCN recycle_scn,
|
||||
common::ObIArray<ObTabletID> &flush_tablets);
|
||||
int freeze_base_on_needs_(share::SCN recycle_scn);
|
||||
int freeze_base_on_needs_(const int64_t trace_id, share::SCN recycle_scn);
|
||||
int decide_freeze_clock_(ObFreezeCheckpoint *ob_freeze_checkpoint);
|
||||
|
||||
static const int64_t LOOP_TRAVERSAL_INTERVAL_US = 1000L * 50; // 50ms
|
||||
|
@ -107,7 +107,7 @@ public:
|
||||
bool is_in_prepare_list_of_data_checkpoint();
|
||||
// after checkpoint ready_for_flush
|
||||
// move to prepare_list in data_checkpoint
|
||||
int finish_freeze();
|
||||
virtual int finish_freeze();
|
||||
// for empty memtable
|
||||
virtual int set_frozen() { return OB_SUCCESS; }
|
||||
|
||||
|
@ -18,6 +18,11 @@
|
||||
#include "storage/tablet/ob_tablet_common.h"
|
||||
#include "storage/tx_storage/ob_ls_service.h"
|
||||
#include "storage/compaction/ob_partition_merge_progress.h"
|
||||
#include "storage/tx_storage/ob_tenant_freezer.h"
|
||||
#include "ob_tenant_compaction_progress.h"
|
||||
#include "ob_compaction_diagnose.h"
|
||||
#include "ob_compaction_suggestion.h"
|
||||
#include "ob_partition_merge_progress.h"
|
||||
#include "storage/ddl/ob_ddl_merge_task.h"
|
||||
#include "storage/compaction/ob_schedule_dag_func.h"
|
||||
#include "storage/compaction/ob_tenant_tablet_scheduler.h"
|
||||
@ -35,6 +40,7 @@
|
||||
#include "storage/compaction/ob_basic_tablet_merge_ctx.h"
|
||||
#include "storage/compaction/ob_tenant_compaction_progress.h"
|
||||
#include "storage/tx_storage/ob_tenant_freezer.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -971,6 +977,36 @@ int ObTabletMergeFinishTask::init()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletMergeFinishTask::report_checkpoint_diagnose_info(ObTabletMergeCtx &ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObITable *table = nullptr;
|
||||
storage::ObTablesHandleArray &tables_handle = ctx.static_param_.tables_handle_;
|
||||
for (int64_t i = 0; i < tables_handle.get_count() && OB_SUCC(ret); i++) {
|
||||
if (OB_ISNULL(table = tables_handle.get_table(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("table is null", K(ret), K(tables_handle), KP(table));
|
||||
} else if (OB_UNLIKELY(!table->is_memtable())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("table is not memtable", K(ret), K(tables_handle), KPC(table));
|
||||
} else {
|
||||
const ObSSTableMergeInfo &sstable_merge_info = ctx.get_merge_info().get_sstable_merge_info();
|
||||
if (table->is_data_memtable()) {
|
||||
ObMemtable *memtable = nullptr;
|
||||
memtable = static_cast<ObMemtable*>(table);
|
||||
memtable->report_memtable_diagnose_info(ObMemtable::UpdateMergeInfoForMemtable(
|
||||
sstable_merge_info.merge_start_time_, sstable_merge_info.merge_finish_time_,
|
||||
sstable_merge_info.occupy_size_, sstable_merge_info.concurrent_cnt_));
|
||||
} else {
|
||||
ObIMemtable *memtable = nullptr;
|
||||
memtable = static_cast<ObIMemtable*>(table);
|
||||
REPORT_CHECKPOINT_DIAGNOSE_INFO(update_merge_info_for_checkpoint_unit, memtable)
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletMergeFinishTask::process()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -999,6 +1035,13 @@ int ObTabletMergeFinishTask::process()
|
||||
KPC(sstable), "mem_peak", ctx_ptr->mem_ctx_.get_total_mem_peak(), "compat_mode", merge_dag_->get_compat_mode(),
|
||||
"time_guard", ctx_ptr->info_collector_.time_guard_);
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (is_mini_merge(ctx_ptr->static_param_.get_merge_type())
|
||||
&& OB_TMP_FAIL(report_checkpoint_diagnose_info(*ctx_ptr))) {
|
||||
LOG_WARN("failed to report_checkpoint_diagnose_info", K(tmp_ret), KPC(ctx_ptr));
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
/*
|
||||
|
@ -181,6 +181,7 @@ public:
|
||||
ObTabletMergeFinishTask();
|
||||
virtual ~ObTabletMergeFinishTask();
|
||||
int init();
|
||||
int report_checkpoint_diagnose_info(ObTabletMergeCtx &ctx);
|
||||
virtual int process() override;
|
||||
private:
|
||||
bool is_inited_;
|
||||
|
@ -1313,7 +1313,7 @@ int ObTenantTabletScheduler::schedule_ls_minor_merge(
|
||||
} // end of while
|
||||
|
||||
if (FALSE_IT(start_time_us = common::ObTimeUtility::current_time())) {
|
||||
} else if (OB_TMP_FAIL(ls.batch_tablet_freeze(need_fast_freeze_tablets, true/*is_sync*/))) {
|
||||
} else if (OB_TMP_FAIL(ls.batch_tablet_freeze(checkpoint::INVALID_TRACE_ID, need_fast_freeze_tablets, true/*is_sync*/))) {
|
||||
LOG_WARN("failt to batch freeze tablet", KR(tmp_ret), K(ls_id), K(need_fast_freeze_tablets));
|
||||
} else {
|
||||
LOG_INFO("fast freeze by batch_tablet_freeze finish", KR(tmp_ret),
|
||||
|
@ -1074,7 +1074,7 @@ int ObTransferHandler::wait_tablet_write_end_(
|
||||
LOG_WARN("failed to wait tx_write end", KR(ret), K(task_info));
|
||||
} else if (OB_FAIL(ls->get_tx_svr()->traverse_trans_to_submit_redo_log(failed_tx_id))) {
|
||||
LOG_WARN("failed to submit tx log", KR(ret), K(task_info));
|
||||
} else if (OB_FAIL(ls->batch_tablet_freeze(tablet_list, true))) {
|
||||
} else if (OB_FAIL(ls->batch_tablet_freeze(checkpoint::INVALID_TRACE_ID, tablet_list, true))) {
|
||||
LOG_WARN("batch tablet freeze failed", KR(ret), KPC(ls), K(task_info));
|
||||
} else if (OB_FAIL(ls->check_tablet_no_active_memtable(tablet_list, has_active_memtable))) {
|
||||
LOG_WARN("check tablet has active memtable failed", KR(ret), KPC(ls), K(task_info));
|
||||
|
@ -479,7 +479,8 @@ ObLSWRSHandler* ObFreezer::get_ls_wrs_handler()
|
||||
}
|
||||
|
||||
/* logstream freeze */
|
||||
int ObFreezer::logstream_freeze(ObFuture<int> *result)
|
||||
int ObFreezer::logstream_freeze(int64_t trace_id,
|
||||
ObFuture<int> *result)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
SCN freeze_snapshot_version;
|
||||
@ -516,6 +517,11 @@ int ObFreezer::logstream_freeze(ObFuture<int> *result)
|
||||
ObTabletID(ObTabletID::INVALID_TABLET_ID),
|
||||
false/*need_rewrite_meta*/))) {
|
||||
TRANS_LOG(WARN, "[Freezer] fail to begin_set_freeze_stat", K(ret), K(ls_id));
|
||||
}
|
||||
|
||||
MTL(checkpoint::ObCheckpointDiagnoseMgr*)->update_freeze_clock(ls_id, trace_id, get_freeze_clock());
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(inner_logstream_freeze(result))) {
|
||||
TRANS_LOG(WARN, "[Freezer] logstream_freeze failure", K(ret), K(ls_id));
|
||||
stat_.end_set_freeze_stat(ObFreezeState::FINISH, ObTimeUtility::current_time(), ret);
|
||||
@ -1056,7 +1062,7 @@ int ObFreezer::handle_frozen_memtable_for_replace_tablet_meta(const ObTabletID &
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObFreezer::batch_tablet_freeze(const ObIArray<ObTabletID> &tablet_ids, ObFuture<int> *result)
|
||||
int ObFreezer::batch_tablet_freeze(const int64_t trace_id, const ObIArray<ObTabletID> &tablet_ids, ObFuture<int> *result)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
share::ObLSID ls_id = get_ls_id();
|
||||
@ -1096,7 +1102,7 @@ int ObFreezer::batch_tablet_freeze(const ObIArray<ObTabletID> &tablet_ids, ObFut
|
||||
ObTabletID(ObTabletID::INVALID_TABLET_ID),
|
||||
false/*need_rewrite_meta*/))) {
|
||||
TRANS_LOG(WARN, "[Freezer] fail to begin_set_freeze_stat", K(ret), K(ls_id));
|
||||
} else if (OB_FAIL(batch_tablet_freeze_(tablet_ids, result, need_freeze))) {
|
||||
} else if (OB_FAIL(batch_tablet_freeze_(trace_id, tablet_ids, result, need_freeze))) {
|
||||
TRANS_LOG(WARN, "[Freezer] batch_tablet_freeze failed", K(ret), K(ls_id), K(tablet_ids));
|
||||
}
|
||||
if (OB_FAIL(ret) || !need_freeze) {
|
||||
@ -1109,8 +1115,9 @@ int ObFreezer::batch_tablet_freeze(const ObIArray<ObTabletID> &tablet_ids, ObFut
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObFreezer::batch_tablet_freeze_(const ObIArray<ObTabletID> &tablet_ids, ObFuture<int> *result, bool &need_freeze)
|
||||
int ObFreezer::batch_tablet_freeze_(const int64_t trace_id, const ObIArray<ObTabletID> &tablet_ids, ObFuture<int> *result, bool &need_freeze)
|
||||
{
|
||||
const int64_t start = ObTimeUtility::current_time();
|
||||
// succeed to set freeze flag
|
||||
int ret = OB_SUCCESS;
|
||||
ObTableHandleArray memtable_handles;
|
||||
@ -1131,7 +1138,7 @@ int ObFreezer::batch_tablet_freeze_(const ObIArray<ObTabletID> &tablet_ids, ObFu
|
||||
} else if (FALSE_IT(tablet = handle.get_obj())) {
|
||||
} else if (OB_FAIL(tablet->get_protected_memtable_mgr_handle(protected_handle))) {
|
||||
LOG_WARN("failed to get_protected_memtable_mgr_handle", K(ret), KPC(tablet));
|
||||
} else if (OB_FAIL(protected_handle->set_is_tablet_freeze_for_active_memtable(frozen_memtable_handle))) {
|
||||
} else if (OB_FAIL(protected_handle->set_is_tablet_freeze_for_active_memtable(frozen_memtable_handle, trace_id))) {
|
||||
if (ret == OB_ENTRY_NOT_EXIST) {
|
||||
ret = OB_SUCCESS;
|
||||
TRANS_LOG(INFO, "[Freezer] no need to freeze since there is no active memtable", K(ret),
|
||||
@ -1160,7 +1167,7 @@ int ObFreezer::batch_tablet_freeze_(const ObIArray<ObTabletID> &tablet_ids, ObFu
|
||||
} else if (OB_FAIL(submit_batch_tablet_freeze_task(memtable_handles, result))) {
|
||||
TRANS_LOG(WARN, "[Freezer] fail to submit batch_tablet_freeze task", K(ret));
|
||||
} else {
|
||||
TRANS_LOG(INFO, "[Freezer] succeed to start batch_tablet_freeze task", K(ret), K(tablet_ids));
|
||||
TRANS_LOG(INFO, "[Freezer] succeed to start batch_tablet_freeze task", K(ret), K(get_ls_id()), K(tablet_ids));
|
||||
}
|
||||
|
||||
return ret;
|
||||
|
@ -178,12 +178,12 @@ public:
|
||||
|
||||
public:
|
||||
/* freeze */
|
||||
int logstream_freeze(ObFuture<int> *result = nullptr);
|
||||
int logstream_freeze(const int64_t trace_id, ObFuture<int> *result = nullptr);
|
||||
int tablet_freeze(const ObTabletID &tablet_id, ObFuture<int> *result = nullptr);
|
||||
int tablet_freeze_with_rewrite_meta(const ObTabletID &tablet_id);
|
||||
int tablet_freeze_for_replace_tablet_meta(const ObTabletID &tablet_id, ObTableHandleV2 &handle);
|
||||
int handle_frozen_memtable_for_replace_tablet_meta(const ObTabletID &tablet_id, ObTableHandleV2 &handle);
|
||||
int batch_tablet_freeze(const ObIArray<ObTabletID> &tablet_ids, ObFuture<int> *result = nullptr);
|
||||
int batch_tablet_freeze(const int64_t trace_id, const ObIArray<ObTabletID> &tablet_ids, ObFuture<int> *result = nullptr);
|
||||
|
||||
/* freeze_flag */
|
||||
bool is_freeze(uint32_t is_freeze=UINT32_MAX) const;
|
||||
@ -282,7 +282,7 @@ private:
|
||||
int check_ls_state(); // must be used under the protection of ls_lock
|
||||
int freeze_normal_tablet_(const ObTabletID &tablet_id, ObFuture<int> *result = nullptr);
|
||||
int freeze_ls_inner_tablet_(const ObTabletID &tablet_id);
|
||||
int batch_tablet_freeze_(const ObIArray<ObTabletID> &tablet_ids, ObFuture<int> *result, bool &need_freeze);
|
||||
int batch_tablet_freeze_(const int64_t trace_id, const ObIArray<ObTabletID> &tablet_ids, ObFuture<int> *result, bool &need_freeze);
|
||||
int submit_batch_tablet_freeze_task(const ObTableHandleArray &tables_array, ObFuture<int> *result);
|
||||
int batch_tablet_freeze_task(ObTableHandleArray tables_array);
|
||||
int finish_freeze_with_ls_lock(memtable::ObMemtable *memtable);
|
||||
|
@ -1810,7 +1810,7 @@ int ObLS::replay_get_tablet(
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLS::logstream_freeze(const bool is_sync, const int64_t abs_timeout_ts)
|
||||
int ObLS::logstream_freeze(const int64_t trace_id, const bool is_sync, const int64_t abs_timeout_ts)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObFuture<int> result;
|
||||
@ -1828,7 +1828,7 @@ int ObLS::logstream_freeze(const bool is_sync, const int64_t abs_timeout_ts)
|
||||
} else if (OB_UNLIKELY(is_offline())) {
|
||||
ret = OB_MINOR_FREEZE_NOT_ALLOW;
|
||||
LOG_WARN("offline ls not allowed freeze", K(ret), K_(ls_meta));
|
||||
} else if (OB_FAIL(ls_freezer_.logstream_freeze(&result))) {
|
||||
} else if (OB_FAIL(ls_freezer_.logstream_freeze(trace_id, &result))) {
|
||||
LOG_WARN("logstream freeze failed", K(ret), K_(ls_meta));
|
||||
} else {
|
||||
// do nothing
|
||||
@ -1899,9 +1899,10 @@ int ObLS::tablet_freeze_with_rewrite_meta(const ObTabletID &tablet_id, const int
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLS::batch_tablet_freeze(const ObIArray<ObTabletID> &tablet_ids,
|
||||
const bool is_sync,
|
||||
const int64_t abs_timeout_ts)
|
||||
int ObLS::batch_tablet_freeze(const int64_t trace_id,
|
||||
const ObIArray<ObTabletID> &tablet_ids,
|
||||
const bool is_sync,
|
||||
const int64_t abs_timeout_ts)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObFuture<int> result;
|
||||
@ -1919,7 +1920,7 @@ int ObLS::batch_tablet_freeze(const ObIArray<ObTabletID> &tablet_ids,
|
||||
} else if (OB_UNLIKELY(is_offline())) {
|
||||
ret = OB_MINOR_FREEZE_NOT_ALLOW;
|
||||
LOG_WARN("offline ls not allowed freeze", K(ret), K_(ls_meta));
|
||||
} else if (OB_FAIL(ls_freezer_.batch_tablet_freeze(tablet_ids, &result))) {
|
||||
} else if (OB_FAIL(ls_freezer_.batch_tablet_freeze(trace_id, tablet_ids, &result))) {
|
||||
LOG_WARN("batch tablet freeze failed", K(ret));
|
||||
} else {
|
||||
// do nothing
|
||||
|
@ -854,9 +854,11 @@ public:
|
||||
|
||||
// ObFreezer interface:
|
||||
// freeze the data of ls:
|
||||
// @param [in] trace_id, for checkpoint diagnose
|
||||
// @param [in] is_sync, only used for wait_freeze_finished()
|
||||
// @param [in] abs_timeout_ts, wait until timeout if lock conflict
|
||||
int logstream_freeze(const bool is_sync = false,
|
||||
int logstream_freeze(const int64_t trace_id,
|
||||
const bool is_sync = false,
|
||||
const int64_t abs_timeout_ts = INT64_MAX);
|
||||
// tablet freeze
|
||||
// @param [in] is_sync, only used for wait_freeze_finished()
|
||||
@ -872,9 +874,10 @@ public:
|
||||
// @param [in] tablet_ids
|
||||
// @param [in] is_sync
|
||||
// @param [in] abs_timeout_ts, wait until timeout if lock conflict
|
||||
int batch_tablet_freeze(const ObIArray<ObTabletID> &tablet_ids,
|
||||
const bool is_sync = false,
|
||||
const int64_t abs_timeout_ts = INT64_MAX);
|
||||
int batch_tablet_freeze(const int64_t trace_id,
|
||||
const ObIArray<ObTabletID> &tablet_ids,
|
||||
const bool is_sync = false,
|
||||
const int64_t abs_timeout_ts = INT64_MAX);
|
||||
|
||||
// ObTxTable interface
|
||||
DELEGATE_WITH_RET(tx_table_, get_tx_table_guard, int);
|
||||
|
@ -6320,7 +6320,7 @@ int ObLSTabletService::flush_mds_table(int64_t recycle_scn)
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ls tablet service is not init", KR(ret), KPC(this));
|
||||
} else if (OB_FAIL(mds_table_mgr_.flush(SCN::max_scn(), true))) {
|
||||
} else if (OB_FAIL(mds_table_mgr_.flush(SCN::max_scn(), checkpoint::INVALID_TRACE_ID, true))) {
|
||||
LOG_WARN("flush mds table failed", KR(ret), KPC(this));
|
||||
}
|
||||
LOG_INFO("finish flush mds table", KR(ret), K(recycle_scn));
|
||||
|
@ -26,6 +26,7 @@
|
||||
#include "logservice/ob_log_base_header.h"
|
||||
#include "share/scn.h"
|
||||
#include "storage/tx_storage/ob_ls_service.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -605,12 +606,21 @@ int ObLSTxService::flush(SCN &recycle_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
bool has_gen_diagnose_trace = false;
|
||||
RLockGuard guard(rwlock_);
|
||||
int64_t trace_id = INVALID_TRACE_ID;
|
||||
for (int i = 1; i < ObCommonCheckpointType::MAX_BASE_TYPE; i++) {
|
||||
// only flush the common_checkpoint that whose clog need recycle
|
||||
if (OB_NOT_NULL(common_checkpoints_[i]) && recycle_scn >= common_checkpoints_[i]->get_rec_scn()) {
|
||||
if (OB_SUCCESS != (tmp_ret = common_checkpoints_[i]->flush(recycle_scn))) {
|
||||
TRANS_LOG(WARN, "obCommonCheckpoint flush failed", K(tmp_ret), K(i), K(common_checkpoints_[i]));
|
||||
if (OB_NOT_NULL(common_checkpoints_[i])
|
||||
&& !common_checkpoints_[i]->is_flushing()
|
||||
&& recycle_scn >= common_checkpoints_[i]->get_rec_scn()) {
|
||||
if (!has_gen_diagnose_trace) {
|
||||
has_gen_diagnose_trace = true;
|
||||
MTL(ObCheckpointDiagnoseMgr*)->acquire_trace_id(ls_id_, trace_id);
|
||||
}
|
||||
TRANS_LOG(INFO, "common_checkpoints flush", K(trace_id), K(ls_id_), K(has_gen_diagnose_trace), K(common_checkpoints_[i]));
|
||||
if (OB_SUCCESS != (tmp_ret = common_checkpoints_[i]->flush(recycle_scn, trace_id))) {
|
||||
TRANS_LOG(WARN, "obCommonCheckpoint flush failed", K(tmp_ret), K(common_checkpoints_[i]));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -720,7 +730,7 @@ int ObLSTxService::traversal_flush()
|
||||
RLockGuard guard(rwlock_);
|
||||
for (int i = 1; i < ObCommonCheckpointType::MAX_BASE_TYPE; i++) {
|
||||
if (OB_NOT_NULL(common_checkpoints_[i]) &&
|
||||
OB_SUCCESS != (tmp_ret = common_checkpoints_[i]->flush(SCN::max_scn(), false))) {
|
||||
OB_SUCCESS != (tmp_ret = common_checkpoints_[i]->flush(SCN::max_scn(), checkpoint::INVALID_TRACE_ID, false))) {
|
||||
TRANS_LOG(WARN, "obCommonCheckpoint flush failed", K(tmp_ret), KP(common_checkpoints_[i]));
|
||||
}
|
||||
}
|
||||
|
@ -281,6 +281,7 @@ void ObMemtable::destroy()
|
||||
contain_hotspot_row_ = false;
|
||||
snapshot_version_.set_max();
|
||||
encrypt_meta_ = nullptr;
|
||||
reset_trace_id();
|
||||
}
|
||||
|
||||
int ObMemtable::safe_to_destroy(bool &is_safe)
|
||||
@ -2157,7 +2158,7 @@ void ObMemtable::print_ready_for_flush()
|
||||
K(ret), K(bool_ret),
|
||||
K(frozen_memtable_flag), K(write_ref),
|
||||
K(current_right_boundary), K(end_scn),
|
||||
K(logstream_freeze_clock), K(memtable_freeze_clock));
|
||||
K(logstream_freeze_clock), K(memtable_freeze_clock), K_(trace_id));
|
||||
}
|
||||
|
||||
// The freeze_snapshot_version is needed for mini merge, which represents that
|
||||
@ -2351,6 +2352,7 @@ void ObMemtable::set_freeze_state(const int64_t state)
|
||||
}
|
||||
}
|
||||
|
||||
DEF_REPORT_CHEKCPOINT_DIAGNOSE_INFO(UpdateScheduleDagTime, update_schedule_dag_time)
|
||||
int ObMemtable::flush(share::ObLSID ls_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -2373,6 +2375,7 @@ int ObMemtable::flush(share::ObLSID ls_id)
|
||||
}
|
||||
} else {
|
||||
mt_stat_.create_flush_dag_time_ = cur_time;
|
||||
report_memtable_diagnose_info(UpdateScheduleDagTime());
|
||||
TRANS_LOG(INFO, "schedule tablet merge dag successfully", K(ret), K(param), KPC(this));
|
||||
}
|
||||
|
||||
@ -3348,5 +3351,16 @@ storage::ObTabletMemtableMgr *ObMemtable::get_memtable_mgr_()
|
||||
return static_cast<ObTabletMemtableMgr *>(memtable_mgr_handle_.get_memtable_mgr());
|
||||
}
|
||||
|
||||
int ObMemtable::finish_freeze()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObFreezeCheckpoint::finish_freeze())) {
|
||||
TRANS_LOG(WARN, "fail to finish_freeze", KR(ret));
|
||||
} else {
|
||||
report_memtable_diagnose_info(UpdateFreezeInfo(*this));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace memtable
|
||||
} // namespace ocenabase
|
||||
|
@ -29,6 +29,7 @@
|
||||
#include "storage/checkpoint/ob_freeze_checkpoint.h"
|
||||
#include "storage/compaction/ob_medium_compaction_mgr.h"
|
||||
#include "storage/tx_storage/ob_ls_handle.h" //ObLSHandle
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -189,6 +190,71 @@ class ObMemtable : public ObIMemtable, public storage::checkpoint::ObFreezeCheck
|
||||
public:
|
||||
using ObMvccRowAndWriteResults = common::ObSEArray<ObMvccRowAndWriteResult, 16>;
|
||||
typedef share::ObMemstoreAllocator::AllocHandle ObSingleMemstoreAllocator;
|
||||
|
||||
#define DEF_REPORT_CHEKCPOINT_DIAGNOSE_INFO(function, update_function) \
|
||||
struct function \
|
||||
{ \
|
||||
public: \
|
||||
function() {} \
|
||||
function(const function&) = delete; \
|
||||
function& operator=(const function&) = delete; \
|
||||
void operator()(const checkpoint::ObCheckpointDiagnoseParam& param) const \
|
||||
{ \
|
||||
checkpoint::ObCheckpointDiagnoseMgr *cdm = MTL(checkpoint::ObCheckpointDiagnoseMgr*); \
|
||||
if (OB_NOT_NULL(cdm)) { \
|
||||
cdm->update_function(param); \
|
||||
} \
|
||||
} \
|
||||
};
|
||||
DEF_REPORT_CHEKCPOINT_DIAGNOSE_INFO(UpdateStartGCTimeForMemtable, update_start_gc_time_for_memtable)
|
||||
DEF_REPORT_CHEKCPOINT_DIAGNOSE_INFO(AddCheckpointDiagnoseInfoForMemtable, add_diagnose_info<checkpoint::ObMemtableDiagnoseInfo>)
|
||||
|
||||
struct UpdateFreezeInfo
|
||||
{
|
||||
public:
|
||||
UpdateFreezeInfo(ObMemtable &memtable) : memtable_(memtable) {}
|
||||
UpdateFreezeInfo& operator=(const UpdateFreezeInfo&) = delete;
|
||||
void operator()(const checkpoint::ObCheckpointDiagnoseParam& param) const
|
||||
{
|
||||
checkpoint::ObCheckpointDiagnoseMgr *cdm = MTL(checkpoint::ObCheckpointDiagnoseMgr*);
|
||||
if (OB_NOT_NULL(cdm)) {
|
||||
cdm->update_freeze_info(param, memtable_.get_rec_scn(),
|
||||
memtable_.get_start_scn(), memtable_.get_end_scn(), memtable_.get_btree_alloc_memory());
|
||||
}
|
||||
}
|
||||
private:
|
||||
ObMemtable &memtable_;
|
||||
};
|
||||
|
||||
struct UpdateMergeInfoForMemtable
|
||||
{
|
||||
public:
|
||||
UpdateMergeInfoForMemtable(int64_t merge_start_time,
|
||||
int64_t merge_finish_time,
|
||||
int64_t occupy_size,
|
||||
int64_t concurrent_cnt)
|
||||
: merge_start_time_(merge_start_time),
|
||||
merge_finish_time_(merge_finish_time),
|
||||
occupy_size_(occupy_size),
|
||||
concurrent_cnt_(concurrent_cnt)
|
||||
{}
|
||||
UpdateMergeInfoForMemtable& operator=(const UpdateMergeInfoForMemtable&) = delete;
|
||||
void operator()(const checkpoint::ObCheckpointDiagnoseParam& param) const
|
||||
{
|
||||
checkpoint::ObCheckpointDiagnoseMgr *cdm = MTL(checkpoint::ObCheckpointDiagnoseMgr*);
|
||||
if (OB_NOT_NULL(cdm)) {
|
||||
cdm->update_merge_info_for_memtable(param, merge_start_time_, merge_finish_time_,
|
||||
occupy_size_, concurrent_cnt_);
|
||||
}
|
||||
}
|
||||
private:
|
||||
int64_t merge_start_time_;
|
||||
int64_t merge_finish_time_;
|
||||
int64_t occupy_size_;
|
||||
int64_t concurrent_cnt_;
|
||||
};
|
||||
|
||||
public:
|
||||
ObMemtable();
|
||||
virtual ~ObMemtable();
|
||||
public:
|
||||
@ -505,6 +571,38 @@ public:
|
||||
int dump2text(const char *fname);
|
||||
// TODO(handora.qc) ready_for_flush interface adjustment
|
||||
bool is_can_flush() { return ObMemtableFreezeState::READY_FOR_FLUSH == freeze_state_ && share::SCN::max_scn() != get_end_scn(); }
|
||||
virtual int finish_freeze();
|
||||
|
||||
virtual int64_t dec_ref()
|
||||
{
|
||||
int64_t ref_cnt = ObITable::dec_ref();
|
||||
if (0 == ref_cnt) {
|
||||
report_memtable_diagnose_info(UpdateStartGCTimeForMemtable());
|
||||
}
|
||||
return ref_cnt;
|
||||
}
|
||||
|
||||
template<class OP>
|
||||
void report_memtable_diagnose_info(const OP &op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
// logstream freeze
|
||||
if (!get_is_tablet_freeze()) {
|
||||
share::ObLSID ls_id;
|
||||
if (OB_FAIL(get_ls_id(ls_id))) {
|
||||
TRANS_LOG(WARN, "failed to get ls id", KPC(this));
|
||||
} else {
|
||||
checkpoint::ObCheckpointDiagnoseParam param(ls_id.id(), get_freeze_clock(), get_tablet_id(), (void*)this);
|
||||
op(param);
|
||||
}
|
||||
}
|
||||
// batch tablet freeze
|
||||
else if (checkpoint::INVALID_TRACE_ID != get_trace_id()) {
|
||||
checkpoint::ObCheckpointDiagnoseParam param(trace_id_, get_tablet_id(), (void*)this);
|
||||
op(param);
|
||||
}
|
||||
}
|
||||
|
||||
INHERIT_TO_STRING_KV("ObITable", ObITable, KP(this), K_(timestamp), K_(state),
|
||||
K_(freeze_clock), K_(max_schema_version), K_(max_data_schema_version), K_(max_column_cnt),
|
||||
K_(write_ref_cnt), K_(local_allocator), K_(unsubmitted_cnt),
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include "storage/ob_i_table.h"
|
||||
#include "storage/memtable/mvcc/ob_mvcc_ctx.h"
|
||||
#include "storage/tx/ob_trans_define.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -139,10 +140,14 @@ struct ObMergePriorityInfo
|
||||
class ObIMemtable: public storage::ObITable
|
||||
{
|
||||
public:
|
||||
ObIMemtable() : ls_id_(), snapshot_version_(share::SCN::max_scn())
|
||||
ObIMemtable()
|
||||
: ls_id_(),
|
||||
snapshot_version_(share::SCN::max_scn()),
|
||||
trace_id_(checkpoint::INVALID_TRACE_ID)
|
||||
{}
|
||||
virtual ~ObIMemtable() {}
|
||||
virtual share::ObLSID &get_ls_id() { return ls_id_;}
|
||||
virtual ObTabletID get_tablet_id() const = 0;
|
||||
virtual int get(
|
||||
const storage::ObTableIterParam ¶m,
|
||||
storage::ObTableAccessContext &context,
|
||||
@ -209,9 +214,34 @@ public:
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
virtual int64_t dec_ref()
|
||||
{
|
||||
int64_t ref_cnt = ObITable::dec_ref();
|
||||
checkpoint::ObCheckpointDiagnoseMgr *cdm = MTL(checkpoint::ObCheckpointDiagnoseMgr*);
|
||||
if (0 == ref_cnt) {
|
||||
if (get_tablet_id().is_ls_inner_tablet()) {
|
||||
REPORT_CHECKPOINT_DIAGNOSE_INFO(update_start_gc_time_for_checkpoint_unit, this)
|
||||
}
|
||||
}
|
||||
return ref_cnt;
|
||||
}
|
||||
|
||||
void set_trace_id(const int64_t trace_id)
|
||||
{
|
||||
if (get_tablet_id().is_ls_inner_tablet()) {
|
||||
ADD_CHECKPOINT_DIAGNOSE_INFO_AND_SET_TRACE_ID(checkpoint::ObCheckpointUnitDiagnoseInfo, trace_id);
|
||||
} else {
|
||||
ADD_CHECKPOINT_DIAGNOSE_INFO_AND_SET_TRACE_ID(checkpoint::ObMemtableDiagnoseInfo, trace_id);
|
||||
}
|
||||
}
|
||||
void reset_trace_id() { ATOMIC_STORE(&trace_id_, checkpoint::INVALID_TRACE_ID); }
|
||||
int64_t get_trace_id() const { return ATOMIC_LOAD(&trace_id_); }
|
||||
protected:
|
||||
share::ObLSID ls_id_;
|
||||
share::SCN snapshot_version_;
|
||||
// a round tablet freeze identifier for checkpoint diagnose
|
||||
int64_t trace_id_;
|
||||
};
|
||||
|
||||
////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||
|
@ -26,6 +26,7 @@ bool FlushOp::operator()(const ObTabletID &, MdsTableBase *&mds_table)
|
||||
if (mds_table->is_switched_to_empty_shell()) {
|
||||
MDS_LOG_RET(INFO, ret, "skip empty shell tablet mds_table flush",
|
||||
KPC(mds_table), K(scan_mds_table_cnt_), K_(max_consequent_callbacked_scn));
|
||||
} else if (checkpoint::INVALID_TRACE_ID != trace_id_ && FALSE_IT(mds_table->set_trace_id(trace_id_))) {
|
||||
} else if (OB_FAIL(mds_table->flush(do_flush_limit_scn_, max_consequent_callbacked_scn_))) {
|
||||
MDS_LOG_RET(WARN, ret, "flush mds table failed",
|
||||
KR(ret), KPC(mds_table), K_(scan_mds_table_cnt), K_(max_consequent_callbacked_scn));
|
||||
@ -40,4 +41,4 @@ bool FlushOp::operator()(const ObTabletID &, MdsTableBase *&mds_table)
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -14,6 +14,7 @@
|
||||
#define STORAGE_MULTI_DATA_SOURCE_MDS_FOR_EACH_MAP_FLUSH_OPERATION_H
|
||||
#include "share/scn.h"
|
||||
#include "common/ob_tablet_id.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace storage
|
||||
@ -22,20 +23,22 @@ namespace mds
|
||||
{
|
||||
class MdsTableBase;
|
||||
struct FlushOp {
|
||||
FlushOp(share::SCN do_flush_limit_scn, int64_t &scan_mds_table_cnt, share::SCN max_consequent_callbacked_scn)
|
||||
FlushOp(share::SCN do_flush_limit_scn, int64_t &scan_mds_table_cnt, share::SCN max_consequent_callbacked_scn, int64_t trace_id = checkpoint::INVALID_TRACE_ID)
|
||||
: do_flush_limit_scn_(do_flush_limit_scn),
|
||||
scan_mds_table_cnt_(scan_mds_table_cnt),
|
||||
max_consequent_callbacked_scn_(max_consequent_callbacked_scn),
|
||||
is_dag_full_(false) {}
|
||||
is_dag_full_(false),
|
||||
trace_id_(trace_id) {}
|
||||
bool operator()(const ObTabletID &, MdsTableBase *&mds_table);
|
||||
bool dag_full() const { return is_dag_full_; }
|
||||
share::SCN do_flush_limit_scn_;
|
||||
int64_t &scan_mds_table_cnt_;
|
||||
share::SCN max_consequent_callbacked_scn_;
|
||||
bool is_dag_full_;
|
||||
int64_t trace_id_;
|
||||
};
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
#endif
|
||||
#endif
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include "storage/multi_data_source/mds_table_mgr.h"
|
||||
#include "observer/virtual_table/ob_mds_event_buffer.h"
|
||||
#include "storage/multi_data_source/runtime_utility/list_helper.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -107,8 +108,12 @@ public:
|
||||
rec_scn_(share::SCN::max_scn()),
|
||||
total_node_cnt_(0),
|
||||
construct_sequence_(0),
|
||||
lock_() { construct_sequence_ = ObMdsGlobalSequencer::generate_senquence(); }
|
||||
virtual ~MdsTableBase() {}
|
||||
lock_(),
|
||||
trace_id_(checkpoint::INVALID_TRACE_ID) { construct_sequence_ = ObMdsGlobalSequencer::generate_senquence(); }
|
||||
virtual ~MdsTableBase()
|
||||
{
|
||||
REPORT_CHECKPOINT_DIAGNOSE_INFO(update_start_gc_time_for_checkpoint_unit, this);
|
||||
}
|
||||
int init(const ObTabletID tablet_id,
|
||||
const share::ObLSID ls_id,
|
||||
ObTabletPointer *pointer,
|
||||
@ -175,6 +180,11 @@ public:
|
||||
bool is_switched_to_empty_shell() const;
|
||||
bool is_removed_from_t3m() const;
|
||||
int64_t get_removed_from_t3m_ts() const;
|
||||
int64_t get_trace_id() const { return trace_id_; }
|
||||
void set_trace_id(int64_t trace_id)
|
||||
{
|
||||
ADD_CHECKPOINT_DIAGNOSE_INFO_AND_SET_TRACE_ID(checkpoint::ObCheckpointUnitDiagnoseInfo, trace_id);
|
||||
}
|
||||
VIRTUAL_TO_STRING_KV(KP(this));
|
||||
protected:
|
||||
void inc_valid_node_cnt();
|
||||
@ -231,6 +241,9 @@ protected:
|
||||
tablet_id_);
|
||||
observer::ObMdsEventBuffer::append(key, event, file, line, function_name);
|
||||
}
|
||||
|
||||
REPORT_CHECKPOINT_DIAGNOSE_INFO(update_schedule_dag_info, this, rec_scn_, rec_scn_, flushing_scn_);
|
||||
|
||||
}
|
||||
template <int N>
|
||||
void report_on_flush_event_(const char (&event_str)[N],
|
||||
@ -254,6 +267,8 @@ protected:
|
||||
tablet_id_);
|
||||
observer::ObMdsEventBuffer::append(key, event, file, line, function_name);
|
||||
}
|
||||
|
||||
REPORT_CHECKPOINT_DIAGNOSE_INFO(update_merge_info_for_checkpoint_unit, this);
|
||||
}
|
||||
void report_recycle_event_(share::SCN recycle_scn,
|
||||
const char *file = __builtin_FILE(),
|
||||
@ -309,6 +324,8 @@ protected:
|
||||
int64_t construct_sequence_;// To filter invalid dump DAG
|
||||
MdsTableMgrHandle mgr_handle_;
|
||||
mutable MdsLock lock_;
|
||||
// a round checkpoint identifier for checkpoint diagnose
|
||||
int64_t trace_id_;
|
||||
};
|
||||
|
||||
bool check_node_scn_beflow_flush(const MdsNode &node, const share::SCN &flush_scn);
|
||||
@ -316,4 +333,4 @@ bool check_node_scn_beflow_flush(const MdsNode &node, const share::SCN &flush_sc
|
||||
}
|
||||
}
|
||||
}
|
||||
#endif
|
||||
#endif
|
||||
|
@ -148,7 +148,7 @@ struct OrderOp {
|
||||
}
|
||||
Flusher &flusher_;
|
||||
};
|
||||
int ObMdsTableMgr::flush(SCN recycle_scn, bool need_freeze)
|
||||
int ObMdsTableMgr::flush(SCN recycle_scn, int64_t trace_id, bool need_freeze)
|
||||
{
|
||||
#define PRINT_WRAPPER KR(ret), K(ls_->get_ls_id()), K(recycle_scn), K(need_freeze), K(order_flusher_for_some),\
|
||||
K(max_consequent_callbacked_scn), K(*this)
|
||||
@ -190,7 +190,7 @@ int ObMdsTableMgr::flush(SCN recycle_scn, bool need_freeze)
|
||||
MDS_LOG_FREEZE(INFO, "freezing_scn decline to max_consequent_callbacked_scn");
|
||||
}
|
||||
if (order_flusher_for_some.min_key().rec_scn_ <= freezing_scn_) {
|
||||
order_flush_(order_flusher_for_some, freezing_scn_, max_consequent_callbacked_scn);
|
||||
order_flush_(order_flusher_for_some, freezing_scn_, max_consequent_callbacked_scn, trace_id);
|
||||
} else {
|
||||
MDS_LOG_FREEZE(INFO, "no need do flush cause min_rec_scn is larger than freezing scn");
|
||||
}
|
||||
@ -201,7 +201,8 @@ int ObMdsTableMgr::flush(SCN recycle_scn, bool need_freeze)
|
||||
|
||||
void ObMdsTableMgr::order_flush_(FlusherForSome &order_flusher_for_some,
|
||||
share::SCN freezing_scn,
|
||||
share::SCN max_consequent_callbacked_scn)
|
||||
share::SCN max_consequent_callbacked_scn,
|
||||
int64_t trace_id)
|
||||
{
|
||||
#define PRINT_WRAPPER KR(ret), K(ls_->get_ls_id()), K(freezing_scn), K(order_flusher_for_some),\
|
||||
K(third_sacn_mds_table_cnt), K(max_consequent_callbacked_scn), K(order_flusher_for_all.count()),\
|
||||
@ -210,7 +211,7 @@ void ObMdsTableMgr::order_flush_(FlusherForSome &order_flusher_for_some,
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t third_sacn_mds_table_cnt = 0;
|
||||
FlusherForAll order_flusher_for_all;
|
||||
FlushOp flush_op(freezing_scn, third_sacn_mds_table_cnt, max_consequent_callbacked_scn);
|
||||
FlushOp flush_op(freezing_scn, third_sacn_mds_table_cnt, max_consequent_callbacked_scn, trace_id);
|
||||
if (!order_flusher_for_some.full() || order_flusher_for_some.max_key().rec_scn_ > freezing_scn_) {
|
||||
// than means all mds_tables needed be flushed is included in order_flusher_for_some
|
||||
order_flusher_for_some.flush_by_order(mds_table_map_, freezing_scn_, max_consequent_callbacked_scn);
|
||||
@ -336,4 +337,4 @@ DEF_TO_STRING(ObMdsTableMgr)
|
||||
|
||||
} // namespace mds
|
||||
} // namespace storage
|
||||
} // namespace oceanbase
|
||||
} // namespace oceanbase
|
||||
|
@ -109,7 +109,7 @@ public: // derived from ObCommonCheckpoint
|
||||
share::SCN get_freezing_scn() const;
|
||||
virtual share::SCN get_rec_scn() override;
|
||||
virtual share::SCN get_rec_scn(ObTabletID &tablet_id) override;
|
||||
virtual int flush(share::SCN recycle_scn, bool need_freeze = true) override;
|
||||
virtual int flush(share::SCN recycle_scn, const int64_t trace_id, bool need_freeze = true) override;
|
||||
virtual ObTabletID get_tablet_id() const override { return ObTabletID(0); }
|
||||
virtual bool is_flushing() const override { return false; }
|
||||
|
||||
@ -118,10 +118,27 @@ public: // getter and setter
|
||||
void dec_ref() { ATOMIC_DEC(&ref_cnt_); };
|
||||
int64_t get_ref() { return ATOMIC_LOAD(&ref_cnt_); }
|
||||
|
||||
struct ObFlushOp {
|
||||
public:
|
||||
ObFlushOp(int64_t trace_id,
|
||||
share::SCN &do_flush_limit_scn,
|
||||
int64_t &scan_mds_table_cnt)
|
||||
: trace_id_(trace_id),
|
||||
do_flush_limit_scn_(do_flush_limit_scn),
|
||||
scan_mds_table_cnt_(scan_mds_table_cnt)
|
||||
{}
|
||||
ObFlushOp& operator=(const ObFlushOp&) = delete;
|
||||
bool operator()(const common::ObTabletID &tablet_id, MdsTableBase *&mds_table) const;
|
||||
private:
|
||||
int64_t trace_id_;
|
||||
share::SCN &do_flush_limit_scn_;
|
||||
int64_t &scan_mds_table_cnt_;
|
||||
};
|
||||
private:
|
||||
void order_flush_(FlusherForSome &order_flusher_for_some,
|
||||
share::SCN freezing_scn,
|
||||
share::SCN max_consequent_callbacked_scn);
|
||||
share::SCN max_consequent_callbacked_scn,
|
||||
int64_t trace_id);
|
||||
private:
|
||||
bool is_inited_;
|
||||
bool is_freezing_;
|
||||
@ -172,4 +189,4 @@ private:
|
||||
} // namespace storage
|
||||
} // namespace oceanbase
|
||||
|
||||
#endif
|
||||
#endif
|
||||
|
@ -290,7 +290,7 @@ public:
|
||||
}
|
||||
virtual int reset_storage_recorder() { return common::OB_SUCCESS; }
|
||||
virtual int set_frozen_for_all_memtables() { return common::OB_SUCCESS; }
|
||||
virtual int set_is_tablet_freeze_for_active_memtable(ObTableHandleV2 &handle) { return OB_NOT_SUPPORTED; }
|
||||
virtual int set_is_tablet_freeze_for_active_memtable(ObTableHandleV2 &handle, const int64_t trace_id = checkpoint::INVALID_TRACE_ID) { return OB_NOT_SUPPORTED; }
|
||||
virtual int get_last_frozen_memtable(ObTableHandleV2 &handle) const { return OB_NOT_SUPPORTED; }
|
||||
DECLARE_VIRTUAL_TO_STRING;
|
||||
protected:
|
||||
|
@ -112,6 +112,7 @@ void ObLockMemtable::reset()
|
||||
need_check_tablet_status_ = false;
|
||||
freezer_ = nullptr;
|
||||
is_inited_ = false;
|
||||
reset_trace_id();
|
||||
}
|
||||
|
||||
// RX + S = RX + SRX | S + RX = S + SRX
|
||||
@ -923,6 +924,7 @@ bool ObLockMemtable::is_active_memtable() const
|
||||
}
|
||||
|
||||
int ObLockMemtable::flush(SCN recycle_scn,
|
||||
int64_t trace_id,
|
||||
bool need_freeze)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -967,11 +969,14 @@ int ObLockMemtable::flush(SCN recycle_scn,
|
||||
param.tablet_id_ = LS_LOCK_TABLET;
|
||||
param.merge_type_ = compaction::MINI_MERGE;
|
||||
param.merge_version_ = ObVersion::MIN_VERSION;
|
||||
set_trace_id(trace_id);
|
||||
if (OB_FAIL(compaction::ObScheduleDagFunc::schedule_tx_table_merge_dag(param))) {
|
||||
if (OB_EAGAIN != ret && OB_SIZE_OVERFLOW != ret) {
|
||||
LOG_WARN("failed to schedule lock_memtable merge dag", K(ret), K(this));
|
||||
}
|
||||
} else {
|
||||
REPORT_CHECKPOINT_DIAGNOSE_INFO(update_schedule_dag_info, this, get_rec_scn(),
|
||||
get_start_scn(), get_end_scn());
|
||||
LOG_INFO("schedule lock_memtable merge_dag successfully", K(ls_id_), K(freeze_scn_));
|
||||
}
|
||||
}
|
||||
|
@ -126,7 +126,7 @@ public:
|
||||
|
||||
// =========== INHERITED FROM ObCommonCheckPoint ==========
|
||||
virtual share::SCN get_rec_scn();
|
||||
virtual int flush(share::SCN recycle_scn, bool need_freeze = true);
|
||||
virtual int flush(share::SCN recycle_scn, int64_t trace_id, bool need_freeze = true);
|
||||
|
||||
virtual ObTabletID get_tablet_id() const;
|
||||
|
||||
|
@ -553,7 +553,9 @@ int ObTabletMemtableMgr::unset_logging_blocked_for_active_memtable(memtable::ObI
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletMemtableMgr::set_is_tablet_freeze_for_active_memtable(ObTableHandleV2 &handle)
|
||||
int ObTabletMemtableMgr::set_is_tablet_freeze_for_active_memtable(
|
||||
ObTableHandleV2 &handle,
|
||||
const int64_t trace_id)
|
||||
{
|
||||
handle.reset();
|
||||
memtable::ObIMemtable *active_memtable = nullptr;
|
||||
@ -574,6 +576,9 @@ int ObTabletMemtableMgr::set_is_tablet_freeze_for_active_memtable(ObTableHandleV
|
||||
} else if (FALSE_IT(memtable = static_cast<ObMemtable*>(active_memtable))) {
|
||||
} else if (memtable->allow_freeze()) {
|
||||
memtable->set_is_tablet_freeze();
|
||||
if (checkpoint::INVALID_TRACE_ID != trace_id) {
|
||||
memtable->set_trace_id(trace_id);
|
||||
}
|
||||
} else {
|
||||
handle.reset();
|
||||
ret = OB_ENTRY_NOT_EXIST;
|
||||
@ -679,6 +684,7 @@ int ObTabletMemtableMgr::get_all_memtables(ObTableHdlArray &handle)
|
||||
return ret;
|
||||
}
|
||||
|
||||
DEF_REPORT_CHEKCPOINT_DIAGNOSE_INFO(UpdateReleaseTime, update_release_time)
|
||||
int ObTabletMemtableMgr::release_head_memtable_(memtable::ObIMemtable *imemtable,
|
||||
const bool force)
|
||||
{
|
||||
@ -709,12 +715,14 @@ int ObTabletMemtableMgr::release_head_memtable_(memtable::ObIMemtable *imemtable
|
||||
memtable->set_is_flushed();
|
||||
memtable->set_freeze_state(ObMemtableFreezeState::RELEASED);
|
||||
memtable->set_frozen();
|
||||
memtable->report_memtable_diagnose_info(UpdateReleaseTime());
|
||||
release_head_memtable();
|
||||
memtable::ObMemtable *active_memtable = get_active_memtable_();
|
||||
if (OB_NOT_NULL(active_memtable) && !active_memtable->allow_freeze()) {
|
||||
active_memtable->set_allow_freeze(true);
|
||||
FLOG_INFO("allow active memtable to be freezed", K(ls_id), KPC(active_memtable));
|
||||
}
|
||||
|
||||
FLOG_INFO("succeed to release head data memtable", K(ret), K(ls_id), K(tablet_id_));
|
||||
}
|
||||
}
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include "storage/compaction/ob_medium_compaction_mgr.h"
|
||||
#include "storage/multi_data_source/mds_table_handle.h"
|
||||
#include "storage/multi_data_source/mds_table_mgr.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -84,7 +85,8 @@ public:
|
||||
const bool include_active_memtable = true);
|
||||
int get_memtables_nolock(ObTableHdlArray &handle);
|
||||
int get_first_frozen_memtable(ObTableHandleV2 &handle) const;
|
||||
int set_is_tablet_freeze_for_active_memtable(ObTableHandleV2 &handle);
|
||||
int set_is_tablet_freeze_for_active_memtable(ObTableHandleV2 &handle,
|
||||
const int64_t trace_id = checkpoint::INVALID_TRACE_ID);
|
||||
|
||||
ObStorageSchemaRecorder &get_storage_schema_recorder() { return schema_recorder_; }
|
||||
compaction::ObTabletMediumCompactionInfoRecorder &get_medium_info_recorder() { return medium_info_recorder_; }
|
||||
|
@ -49,7 +49,11 @@ public:
|
||||
update_enabled_(true),
|
||||
is_inited_(false)
|
||||
{}
|
||||
~ObTabletGCHandler() { reset(); }
|
||||
~ObTabletGCHandler() {
|
||||
int ret = 0;
|
||||
STORAGE_LOG(WARN, "failed to alloc", KR(ret));
|
||||
reset();
|
||||
}
|
||||
void reset()
|
||||
{
|
||||
ls_ = NULL;
|
||||
|
@ -47,8 +47,12 @@ ObTenantFreezer::ObTenantFreezer()
|
||||
freeze_thread_pool_(),
|
||||
freeze_thread_pool_lock_(common::ObLatchIds::FREEZE_THREAD_POOL_LOCK),
|
||||
exist_ls_freezing_(false),
|
||||
last_update_ts_(0)
|
||||
{}
|
||||
last_update_ts_(0),
|
||||
freezer_stat_(),
|
||||
freezer_history_()
|
||||
{
|
||||
freezer_stat_.reset();
|
||||
}
|
||||
|
||||
ObTenantFreezer::~ObTenantFreezer()
|
||||
{
|
||||
@ -64,6 +68,8 @@ void ObTenantFreezer::destroy()
|
||||
svr_rpc_proxy_ = nullptr;
|
||||
common_rpc_proxy_ = nullptr;
|
||||
rs_mgr_ = nullptr;
|
||||
freezer_stat_.reset();
|
||||
freezer_history_.reset();
|
||||
|
||||
is_inited_ = false;
|
||||
}
|
||||
@ -105,6 +111,8 @@ int ObTenantFreezer::init()
|
||||
common_rpc_proxy_ = GCTX.rs_rpc_proxy_;
|
||||
rs_mgr_ = GCTX.rs_mgr_;
|
||||
tenant_info_.tenant_id_ = MTL_ID();
|
||||
freezer_stat_.reset();
|
||||
freezer_history_.reset();
|
||||
is_inited_ = true;
|
||||
}
|
||||
return ret;
|
||||
@ -121,6 +129,7 @@ int ObTenantFreezer::start()
|
||||
FREEZE_TRIGGER_INTERVAL,
|
||||
[this]() {
|
||||
LOG_INFO("====== tenant freeze timer task ======");
|
||||
this->do_freeze_diagnose();
|
||||
this->check_and_do_freeze();
|
||||
return false; // TODO: false means keep running, true means won't run again
|
||||
}))) {
|
||||
@ -214,7 +223,7 @@ int ObTenantFreezer::ls_freeze_(ObLS *ls,
|
||||
do {
|
||||
need_retry = false;
|
||||
retry_times++;
|
||||
if (OB_SUCC(ls->logstream_freeze(is_sync, abs_timeout_ts))) {
|
||||
if (OB_SUCC(ls->logstream_freeze(-1, is_sync, abs_timeout_ts))) {
|
||||
} else {
|
||||
current_ts = ObTimeUtil::current_time();
|
||||
is_timeout = (current_ts >= abs_timeout_ts);
|
||||
@ -383,6 +392,11 @@ int ObTenantFreezer::tenant_freeze()
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
freezer_stat_.add_freeze_event();
|
||||
}
|
||||
|
||||
LOG_INFO("tenant_freeze finished", KR(ret), K(abs_timeout_ts));
|
||||
|
||||
return ret;
|
||||
@ -688,6 +702,37 @@ int ObTenantFreezer::check_and_freeze_mds_table_()
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
int ObTenantFreezer::do_freeze_diagnose()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMemstoreAllocator &tenant_allocator = MTL(ObSharedMemAllocMgr *)->memstore_allocator();
|
||||
const int64_t current_time = ObTimeUtility::current_time();
|
||||
const int64_t capture_time_interval = 1_min;
|
||||
const uint64_t tenant_id = MTL_ID();
|
||||
|
||||
if (current_time - freezer_stat_.last_captured_timestamp_ >= 30 * 1_min) {
|
||||
int64_t current_retire_clock = tenant_allocator.get_retire_clock();
|
||||
|
||||
if (freezer_stat_.last_captured_timestamp_ == 0) {
|
||||
// The first time we start capturing
|
||||
ATOMIC_SET(&freezer_stat_.last_captured_retire_clock_, current_retire_clock);
|
||||
} else {
|
||||
ATOMIC_FAA(&freezer_stat_.captured_data_size_, current_retire_clock
|
||||
- ATOMIC_LOAD(&freezer_stat_.last_captured_retire_clock_));
|
||||
ATOMIC_SET(&freezer_stat_.last_captured_retire_clock_, current_retire_clock);
|
||||
|
||||
(void)freezer_stat_.print_activity_metrics();
|
||||
(void)freezer_history_.add_activity_metric(freezer_stat_);
|
||||
}
|
||||
|
||||
freezer_stat_.last_captured_timestamp_ = current_time;
|
||||
freezer_stat_.refresh();
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTenantFreezer::check_and_do_freeze()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -1509,6 +1554,10 @@ int ObTenantFreezer::do_minor_freeze_(const ObTenantFreezeCtx &ctx)
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
freezer_stat_.add_freeze_event();
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -1603,5 +1652,152 @@ void ObTenantFreezer::halt_prewarm_if_need_(const ObTenantFreezeCtx &ctx)
|
||||
}
|
||||
}
|
||||
|
||||
void ObTenantFreezer::get_freezer_stat_history_snapshot(int64_t &length)
|
||||
{
|
||||
length = freezer_history_.length_;
|
||||
}
|
||||
|
||||
void ObTenantFreezer::get_freezer_stat_from_history(int64_t pos, ObTenantFreezerStat& stat)
|
||||
{
|
||||
stat = freezer_history_.history_[(freezer_history_.start_ + pos)
|
||||
% ObTenantFreezerStatHistory::MAX_HISTORY_LENGTH];
|
||||
}
|
||||
|
||||
ObTenantFreezerStat::ObFreezerMergeType ObTenantFreezerStat::switch_to_freezer_merge_type(const compaction::ObMergeType type)
|
||||
{
|
||||
ObFreezerMergeType ret_merge_type = ObFreezerMergeType::UNNECESSARY_TYPE;
|
||||
|
||||
if (is_major_merge(type)) {
|
||||
ret_merge_type = ObFreezerMergeType::MAJOR_MERGE;
|
||||
} else if (is_minor_merge(type)) {
|
||||
ret_merge_type = ObFreezerMergeType::MINOR_MERGE;
|
||||
} else if (is_mini_merge(type)) {
|
||||
ret_merge_type = ObFreezerMergeType::MINI_MERGE;
|
||||
} else {
|
||||
ret_merge_type = ObFreezerMergeType::UNNECESSARY_TYPE;
|
||||
}
|
||||
|
||||
return ret_merge_type;
|
||||
}
|
||||
|
||||
const char *ObTenantFreezerStat::freezer_merge_type_to_str(const ObFreezerMergeType merge_type)
|
||||
{
|
||||
const char *str = "";
|
||||
if (ObFreezerMergeType::UNNECESSARY_TYPE == merge_type) {
|
||||
str = "unnecessary_merge_type";
|
||||
} else if (ObFreezerMergeType::MINI_MERGE == merge_type) {
|
||||
str = "mini_merge";
|
||||
} else if (ObFreezerMergeType::MINOR_MERGE == merge_type) {
|
||||
str = "minor_merge";
|
||||
} else if (ObFreezerMergeType::MAJOR_MERGE == merge_type) {
|
||||
str = "major_merge";
|
||||
} else {
|
||||
str = "invalid_merge_type";
|
||||
}
|
||||
return str;
|
||||
}
|
||||
|
||||
bool ObTenantFreezerStat::is_useful_freezer_merge_type(const ObFreezerMergeType merge_type)
|
||||
{
|
||||
if (merge_type > ObFreezerMergeType::UNNECESSARY_TYPE &&
|
||||
merge_type < ObFreezerMergeType::MAX_MERGE_TYPE) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
void ObTenantFreezerStat::reset(int64_t retire_clock)
|
||||
{
|
||||
ATOMIC_SET(&last_captured_timestamp_, 0);
|
||||
ATOMIC_SET(&captured_data_size_, 0);
|
||||
ATOMIC_SET(&captured_freeze_times_, 0);
|
||||
for (int64_t i = 0; i < ObFreezerMergeType::MAX_MERGE_TYPE; i++) {
|
||||
ATOMIC_SET(&(captured_merge_time_cost_[i]), 0);
|
||||
ATOMIC_SET(&(captured_merge_times_[i]), 0);
|
||||
}
|
||||
|
||||
ATOMIC_SET(&last_captured_retire_clock_, retire_clock);
|
||||
}
|
||||
|
||||
void ObTenantFreezerStat::refresh()
|
||||
{
|
||||
ATOMIC_SET(&captured_data_size_, 0);
|
||||
ATOMIC_SET(&captured_freeze_times_, 0);
|
||||
for (int64_t i = 0; i < ObFreezerMergeType::MAX_MERGE_TYPE; i++) {
|
||||
ATOMIC_SET(&(captured_merge_time_cost_[i]), 0);
|
||||
ATOMIC_SET(&(captured_merge_times_[i]), 0);
|
||||
}
|
||||
}
|
||||
|
||||
void ObTenantFreezerStat::add_freeze_event()
|
||||
{
|
||||
ATOMIC_FAA(&captured_freeze_times_, 1);
|
||||
}
|
||||
|
||||
void ObTenantFreezerStat::add_merge_event(const compaction::ObMergeType type, const int64_t cost)
|
||||
{
|
||||
ObFreezerMergeType real_merge_type = switch_to_freezer_merge_type(type);
|
||||
if (is_useful_freezer_merge_type(real_merge_type)) {
|
||||
ATOMIC_FAA(&(captured_merge_time_cost_[real_merge_type]), cost);
|
||||
ATOMIC_FAA(&(captured_merge_times_[real_merge_type]), 1);
|
||||
}
|
||||
}
|
||||
|
||||
void ObTenantFreezerStat::print_activity_metrics()
|
||||
{
|
||||
TRANS_LOG(INFO, "[TENANT_FREEZER_EVENT] print captured event", KPC(this));
|
||||
|
||||
for (int64_t i = 0; i < ObFreezerMergeType::MAX_MERGE_TYPE; i++) {
|
||||
int64_t captured_merge_time_cost = ATOMIC_LOAD(&(captured_merge_time_cost_[i]));
|
||||
int64_t captured_merge_times = ATOMIC_LOAD(&(captured_merge_times_[i]));
|
||||
const ObFreezerMergeType type = (ObFreezerMergeType)i;
|
||||
|
||||
TRANS_LOG(INFO, "[TENANT_FREEZER_EVENT] print merge event",
|
||||
K(freezer_merge_type_to_str(type)),
|
||||
K(captured_merge_times),
|
||||
K(captured_merge_time_cost));
|
||||
}
|
||||
}
|
||||
|
||||
void ObTenantFreezerStat::assign(const ObTenantFreezerStat stat)
|
||||
{
|
||||
last_captured_timestamp_ = stat.last_captured_timestamp_;
|
||||
captured_data_size_ = stat.captured_data_size_;
|
||||
captured_freeze_times_ = stat.captured_freeze_times_;
|
||||
|
||||
for (int64_t i = 0; i < ObFreezerMergeType::MAX_MERGE_TYPE; i++) {
|
||||
captured_merge_time_cost_[i] = stat.captured_merge_time_cost_[i];
|
||||
captured_merge_times_[i] = stat.captured_merge_times_[i];
|
||||
}
|
||||
|
||||
last_captured_retire_clock_ = stat.last_captured_retire_clock_;
|
||||
}
|
||||
|
||||
void ObTenantFreezerStatHistory::add_activity_metric(const ObTenantFreezerStat stat)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
if (start_ < 0 || start_ >= MAX_HISTORY_LENGTH) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
TRANS_LOG(ERROR, "unexpected start position", K(start_), K(length_));
|
||||
} else if (length_ == MAX_HISTORY_LENGTH) {
|
||||
(void)history_[start_].assign(stat);
|
||||
start_ = (start_ + 1) % MAX_HISTORY_LENGTH;
|
||||
} else if (length_ < MAX_HISTORY_LENGTH && 0 == start_) {
|
||||
(void)history_[start_ + length_].assign(stat);
|
||||
length_++;
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
TRANS_LOG(ERROR, "unexpected history length", K(start_), K(length_));
|
||||
}
|
||||
}
|
||||
|
||||
void ObTenantFreezerStatHistory::reset()
|
||||
{
|
||||
start_ = 0;
|
||||
length_ = 0;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include "share/ob_tenant_mgr.h"
|
||||
#include "storage/tx_storage/ob_tenant_freezer_rpc.h"
|
||||
#include "storage/multi_data_source/runtime_utility/mds_factory.h"
|
||||
#include "storage/compaction/ob_compaction_util.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -31,6 +32,71 @@ namespace storage
|
||||
class ObTenantFreezer;
|
||||
class ObTenantTxDataFreezeGuard;
|
||||
|
||||
class ObTenantFreezerStat
|
||||
{
|
||||
public:
|
||||
static const int64_t MAX_FREEZER_MERGE_TYPE = 3;
|
||||
enum ObFreezerMergeType
|
||||
{
|
||||
UNNECESSARY_TYPE = -1,
|
||||
MINI_MERGE = 0,
|
||||
MINOR_MERGE = 1,
|
||||
MAJOR_MERGE = 2,
|
||||
MAX_MERGE_TYPE = 3
|
||||
};
|
||||
ObTenantFreezerStat() { reset(); }
|
||||
~ObTenantFreezerStat() {}
|
||||
public:
|
||||
int64_t last_captured_timestamp_;
|
||||
// captured data size from last captured time
|
||||
int64_t captured_data_size_;
|
||||
int64_t captured_freeze_times_;
|
||||
|
||||
int64_t captured_merge_time_cost_[ObFreezerMergeType::MAX_MERGE_TYPE];
|
||||
int64_t captured_merge_times_[ObFreezerMergeType::MAX_MERGE_TYPE];
|
||||
|
||||
int64_t last_captured_retire_clock_;
|
||||
|
||||
ObFreezerMergeType switch_to_freezer_merge_type(const compaction::ObMergeType type);
|
||||
|
||||
const char *freezer_merge_type_to_str(const ObFreezerMergeType merge_type);
|
||||
|
||||
bool is_useful_freezer_merge_type(const ObFreezerMergeType merge_type);
|
||||
|
||||
void reset(int64_t retire_clock = 0);
|
||||
|
||||
void refresh();
|
||||
|
||||
void add_freeze_event();
|
||||
|
||||
void add_merge_event(const compaction::ObMergeType type, const int64_t cost);
|
||||
|
||||
void print_activity_metrics();
|
||||
|
||||
void assign(const ObTenantFreezerStat stat);
|
||||
|
||||
TO_STRING_KV(K_(last_captured_timestamp),
|
||||
K_(captured_data_size),
|
||||
K_(captured_freeze_times),
|
||||
K_(last_captured_retire_clock));
|
||||
};
|
||||
|
||||
class ObTenantFreezerStatHistory
|
||||
{
|
||||
public:
|
||||
// 5(day in working week) * 24(hour in day) * 2(half of an hour in an hour)
|
||||
static const int64_t MAX_HISTORY_LENGTH = 5 * 24 * 2;
|
||||
ObTenantFreezerStatHistory(): start_(0), length_(0) {}
|
||||
|
||||
void add_activity_metric(const ObTenantFreezerStat stat);
|
||||
|
||||
void reset();
|
||||
public:
|
||||
int64_t start_;
|
||||
int64_t length_;
|
||||
ObTenantFreezerStat history_[MAX_HISTORY_LENGTH];
|
||||
};
|
||||
|
||||
// this is used for tenant freeze, all the freeze task should call the function of this unit.
|
||||
class ObTenantFreezer
|
||||
{
|
||||
@ -78,6 +144,8 @@ public:
|
||||
// check if this tenant's memstore is out of range, and trigger minor/major freeze.
|
||||
int check_and_do_freeze();
|
||||
|
||||
int do_freeze_diagnose();
|
||||
|
||||
// used for replay to check whether can enqueue another replay task
|
||||
bool is_replay_pending_log_too_large(const int64_t pending_size);
|
||||
// If the tenant's freeze process is slowed, we will only freeze one time every
|
||||
@ -145,6 +213,17 @@ public:
|
||||
}
|
||||
static int64_t get_freeze_trigger_interval() { return FREEZE_TRIGGER_INTERVAL; }
|
||||
bool exist_ls_freezing();
|
||||
|
||||
// freezer stat collector and generator
|
||||
void add_merge_event(const compaction::ObMergeType type, const int64_t cost)
|
||||
{
|
||||
freezer_stat_.add_merge_event(type, cost);
|
||||
}
|
||||
|
||||
void get_freezer_stat_history_snapshot(int64_t &length);
|
||||
|
||||
void get_freezer_stat_from_history(int64_t pos, ObTenantFreezerStat& stat);
|
||||
|
||||
private:
|
||||
int check_memstore_full_(bool &last_result,
|
||||
int64_t &last_check_timestamp,
|
||||
@ -220,6 +299,11 @@ private:
|
||||
ObSpinLock freeze_thread_pool_lock_;
|
||||
bool exist_ls_freezing_;
|
||||
int64_t last_update_ts_;
|
||||
|
||||
// diagnose only, we capture the freeze stats every 30 minutes
|
||||
ObTenantFreezerStat freezer_stat_;
|
||||
// diagnose only, we capture the freeze history in one monthes
|
||||
ObTenantFreezerStatHistory freezer_history_;
|
||||
};
|
||||
|
||||
class ObTenantTxDataFreezeGuard
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include "storage/tx/ob_trans_part_ctx.h"
|
||||
#include "storage/compaction/ob_schedule_dag_func.h"
|
||||
#include "storage/compaction/ob_tablet_merge_task.h"
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -45,6 +46,7 @@ void ObTxCtxMemtable::reset()
|
||||
ObITable::reset();
|
||||
is_frozen_ = false;
|
||||
is_inited_ = false;
|
||||
reset_trace_id();
|
||||
}
|
||||
|
||||
int ObTxCtxMemtable::init(const ObITable::TableKey &table_key,
|
||||
@ -209,7 +211,7 @@ bool ObTxCtxMemtable::is_active_memtable() const
|
||||
return !ATOMIC_LOAD(&is_frozen_);
|
||||
}
|
||||
|
||||
int ObTxCtxMemtable::flush(SCN recycle_scn, bool need_freeze)
|
||||
int ObTxCtxMemtable::flush(SCN recycle_scn, const int64_t trace_id, bool need_freeze)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSpinLockGuard guard(flush_lock_);
|
||||
@ -238,11 +240,13 @@ int ObTxCtxMemtable::flush(SCN recycle_scn, bool need_freeze)
|
||||
param.tablet_id_ = LS_TX_CTX_TABLET;
|
||||
param.merge_type_ = compaction::MINI_MERGE;
|
||||
param.merge_version_ = ObVersionRange::MIN_VERSION;
|
||||
set_trace_id(trace_id);
|
||||
if (OB_FAIL(compaction::ObScheduleDagFunc::schedule_tx_table_merge_dag(param))) {
|
||||
if (OB_EAGAIN != ret && OB_SIZE_OVERFLOW != ret) {
|
||||
TRANS_LOG(WARN, "failed to schedule tablet merge dag", K(ret));
|
||||
}
|
||||
} else {
|
||||
REPORT_CHECKPOINT_DIAGNOSE_INFO(update_schedule_dag_info, this, get_rec_scn(), get_start_scn(), get_end_scn());
|
||||
TRANS_LOG(INFO, "tx ctx memtable flush successfully", KPC(this), K(ls_id_));
|
||||
}
|
||||
}
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
|
||||
// ================ INHERITED FROM ObCommonCheckpoint ===============
|
||||
virtual share::SCN get_rec_scn();
|
||||
virtual int flush(share::SCN recycle_scn, bool need_freeze = true);
|
||||
virtual int flush(share::SCN recycle_scn, const int64_t trace_id, bool need_freeze = true);
|
||||
|
||||
virtual ObTabletID get_tablet_id() const override;
|
||||
virtual bool is_flushing() const override;
|
||||
|
@ -152,6 +152,7 @@ void ObTxDataMemtable::reset()
|
||||
DEBUG_last_start_scn_ = SCN::min_scn();
|
||||
stat_change_ts_.reset();
|
||||
is_inited_ = false;
|
||||
reset_trace_id();
|
||||
}
|
||||
|
||||
int ObTxDataMemtable::insert(ObTxData *tx_data)
|
||||
@ -911,7 +912,7 @@ bool ObTxDataMemtable::ready_for_flush()
|
||||
return bool_ret;
|
||||
}
|
||||
|
||||
int ObTxDataMemtable::flush()
|
||||
int ObTxDataMemtable::flush(const int64_t trace_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
compaction::ObTabletMergeDagParam param;
|
||||
@ -919,11 +920,14 @@ int ObTxDataMemtable::flush()
|
||||
param.tablet_id_ = key_.tablet_id_;
|
||||
param.merge_type_ = compaction::MINI_MERGE;
|
||||
param.merge_version_ = ObVersionRange::MIN_VERSION;
|
||||
set_trace_id(trace_id);
|
||||
if (OB_FAIL(compaction::ObScheduleDagFunc::schedule_tx_table_merge_dag(param))) {
|
||||
if (OB_EAGAIN != ret && OB_SIZE_OVERFLOW != ret) {
|
||||
STORAGE_LOG(WARN, "failed to schedule tablet merge dag", K(ret));
|
||||
}
|
||||
} else {
|
||||
REPORT_CHECKPOINT_DIAGNOSE_INFO(update_schedule_dag_info, this, get_rec_scn(),
|
||||
get_start_scn(), get_end_scn());
|
||||
stat_change_ts_.create_flush_dag_time_ = ObTimeUtil::fast_current_time();
|
||||
STORAGE_LOG(INFO,
|
||||
"[TX DATA MERGE]schedule flush tx data memtable task done",
|
||||
|
@ -305,6 +305,7 @@ public: /* derived from ObIMemtable */
|
||||
storage::ObTableAccessContext &context,
|
||||
const blocksstable::ObDatumRowkey &rowkey,
|
||||
blocksstable::ObDatumRow &row) override;
|
||||
virtual ObTabletID get_tablet_id() const { return LS_TX_DATA_TABLET; }
|
||||
|
||||
public: // checkpoint
|
||||
share::SCN get_rec_scn()
|
||||
@ -312,7 +313,7 @@ public: // checkpoint
|
||||
return get_min_tx_scn();
|
||||
}
|
||||
|
||||
int flush();
|
||||
int flush(const int64_t trace_id);
|
||||
|
||||
/**
|
||||
* @brief Because of the random order of clog callbacks, the tx data in a freezing tx data
|
||||
|
@ -460,7 +460,8 @@ SCN ObTxDataMemtableMgr::get_rec_scn()
|
||||
return rec_scn;
|
||||
}
|
||||
|
||||
int ObTxDataMemtableMgr::flush_all_frozen_memtables_(ObTableHdlArray &memtable_handles)
|
||||
int ObTxDataMemtableMgr::flush_all_frozen_memtables_(ObTableHdlArray &memtable_handles,
|
||||
const int64_t trace_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
@ -474,14 +475,14 @@ int ObTxDataMemtableMgr::flush_all_frozen_memtables_(ObTableHdlArray &memtable_h
|
||||
&& !memtable->ready_for_flush()) {
|
||||
// on need return error
|
||||
STORAGE_LOG(INFO, "the tx data memtable is not frozen", KPC(memtable));
|
||||
} else if (OB_FAIL(memtable->flush())) {
|
||||
} else if (OB_FAIL(memtable->flush(trace_id))) {
|
||||
STORAGE_LOG(WARN, "the tx data memtable flush failed", KR(ret), KPC(memtable));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTxDataMemtableMgr::flush(SCN recycle_scn, bool need_freeze)
|
||||
int ObTxDataMemtableMgr::flush(SCN recycle_scn, const int64_t trace_id, bool need_freeze)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
@ -510,7 +511,7 @@ int ObTxDataMemtableMgr::flush(SCN recycle_scn, bool need_freeze)
|
||||
STORAGE_LOG(WARN, "get all memtables failed", KR(ret), KP(this));
|
||||
} else if (memtable_handles.count() == 0) {
|
||||
STORAGE_LOG(INFO, "memtable handles is empty. skip flush once.");
|
||||
} else if (OB_FAIL(flush_all_frozen_memtables_(memtable_handles))) {
|
||||
} else if (OB_FAIL(flush_all_frozen_memtables_(memtable_handles, trace_id))) {
|
||||
STORAGE_LOG(WARN, "flush all frozen memtables failed", KR(ret), KP(this));
|
||||
} else if (OB_NOT_NULL(tx_data_table_) && OB_FAIL(tx_data_table_->update_memtables_cache())) {
|
||||
STORAGE_LOG(WARN, "update memtables cache failed.", KR(ret), KP(this));
|
||||
@ -526,7 +527,7 @@ ObTabletID ObTxDataMemtableMgr::get_tablet_id() const
|
||||
|
||||
bool ObTxDataMemtableMgr::is_flushing() const
|
||||
{
|
||||
return memtable_tail_ - 1 != memtable_head_;
|
||||
return memtable_tail_ - memtable_head_ > 1;
|
||||
}
|
||||
|
||||
int ObTxDataMemtableMgr::get_memtable_range(int64_t &memtable_head, int64_t &memtable_tail)
|
||||
|
@ -144,7 +144,7 @@ public: // ObTxDataMemtableMgr
|
||||
// ================ INHERITED FROM ObCommonCheckpoint ===============
|
||||
virtual share::SCN get_rec_scn() override;
|
||||
|
||||
virtual int flush(share::SCN recycle_scn, bool need_freeze = true) override;
|
||||
virtual int flush(share::SCN recycle_scn, const int64_t trace_id, bool need_freeze = true) override;
|
||||
|
||||
virtual ObTabletID get_tablet_id() const override;
|
||||
|
||||
@ -179,7 +179,8 @@ private: // ObTxDataMemtableMgr
|
||||
|
||||
int get_all_memtables_(ObTableHdlArray &handles);
|
||||
|
||||
int flush_all_frozen_memtables_(ObTableHdlArray &memtable_handles);
|
||||
int flush_all_frozen_memtables_(ObTableHdlArray &memtable_handles,
|
||||
const int64_t trace_id);
|
||||
|
||||
ObTxDataMemtable *get_tx_data_memtable_(const int64_t pos) const;
|
||||
|
||||
|
@ -809,7 +809,7 @@ int ObTxDataTable::self_freeze_task()
|
||||
|
||||
STORAGE_LOG(INFO, "start tx data table self freeze task", K(get_ls_id()));
|
||||
|
||||
if (OB_FAIL(memtable_mgr_->flush(SCN::max_scn(), true))) {
|
||||
if (OB_FAIL(memtable_mgr_->flush(SCN::max_scn(), checkpoint::INVALID_TRACE_ID, true))) {
|
||||
share::ObLSID ls_id = get_ls_id();
|
||||
STORAGE_LOG(WARN, "self freeze of tx data memtable failed.", KR(ret), K(ls_id), KPC(memtable_mgr_));
|
||||
}
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -266,6 +266,7 @@ _balance_wait_killing_transaction_end_threshold
|
||||
_bloom_filter_enabled
|
||||
_bloom_filter_ratio
|
||||
_cache_wash_interval
|
||||
_checkpoint_diagnose_preservation_count
|
||||
_chunk_row_store_mem_limit
|
||||
_ctx_memory_limit
|
||||
_datafile_usage_lower_bound_percentage
|
||||
|
@ -4648,6 +4648,62 @@ error_msg varchar(512) YES NULL
|
||||
select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_clone_job_history;
|
||||
IF(count(*) >= 0, 1, 0)
|
||||
1
|
||||
desc oceanbase.__all_virtual_checkpoint_diagnose_memtable_info;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO PRI NULL
|
||||
trace_id bigint(20) NO PRI NULL
|
||||
svr_ip varchar(46) NO NULL
|
||||
svr_port bigint(20) NO NULL
|
||||
ls_id bigint(20) NO NULL
|
||||
checkpoint_thread_name varchar(16) NO NULL
|
||||
checkpoint_start_time timestamp(6) NO NULL
|
||||
tablet_id bigint(20) NO NULL
|
||||
ptr varchar(128) NO NULL
|
||||
start_scn bigint(20) unsigned NO NULL
|
||||
end_scn bigint(20) unsigned NO NULL
|
||||
rec_scn bigint(20) unsigned NO NULL
|
||||
create_flush_dag_time timestamp(6) NO NULL
|
||||
merge_finish_time timestamp(6) NO NULL
|
||||
release_time timestamp(6) NO NULL
|
||||
frozen_finish_time timestamp(6) NO NULL
|
||||
merge_start_time timestamp(6) NO NULL
|
||||
start_gc_time timestamp(6) NO NULL
|
||||
memtable_occupy_size bigint(20) NO NULL
|
||||
occupy_size bigint(20) NO NULL
|
||||
concurrent_cnt bigint(20) NO NULL
|
||||
desc oceanbase.__all_virtual_checkpoint_diagnose_checkpoint_unit_info;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO PRI NULL
|
||||
trace_id bigint(20) NO PRI NULL
|
||||
svr_ip varchar(46) NO NULL
|
||||
svr_port bigint(20) NO NULL
|
||||
ls_id bigint(20) NO NULL
|
||||
checkpoint_thread_name varchar(16) NO NULL
|
||||
checkpoint_start_time timestamp(6) NO NULL
|
||||
tablet_id bigint(20) NO NULL
|
||||
ptr varchar(128) NO NULL
|
||||
start_scn bigint(20) unsigned NO NULL
|
||||
end_scn bigint(20) unsigned NO NULL
|
||||
rec_scn bigint(20) unsigned NO NULL
|
||||
create_flush_dag_time timestamp(6) NO NULL
|
||||
merge_finish_time timestamp(6) NO NULL
|
||||
start_gc_time timestamp(6) NO NULL
|
||||
desc oceanbase.__all_virtual_checkpoint_diagnose_info;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO NULL
|
||||
svr_ip varchar(46) NO NULL
|
||||
svr_port bigint(20) NO NULL
|
||||
ls_id bigint(20) NO NULL
|
||||
trace_id bigint(20) NO NULL
|
||||
freeze_clock int(10) unsigned NO NULL
|
||||
checkpoint_thread_name varchar(16) NO NULL
|
||||
checkpoint_start_time timestamp(6) NO NULL
|
||||
select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_checkpoint_diagnose_info;
|
||||
IF(count(*) >= 0, 1, 0)
|
||||
1
|
||||
"oceanbase.__all_virtual_checkpoint_diagnose_info runs in single server"
|
||||
IF(count(*) >= 0, 1, 0)
|
||||
1
|
||||
desc oceanbase.__all_virtual_tenant_snapshot_job;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO PRI NULL
|
||||
|
@ -8930,6 +8930,23 @@ comment longtext YES
|
||||
select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_import_table_task_history;
|
||||
IF(count(*) >= 0, 1, 0)
|
||||
1
|
||||
desc oceanbase.__all_virtual_data_activity_metrics;
|
||||
Field Type Null Key Default Extra
|
||||
svr_ip varchar(46) NO PRI NULL
|
||||
svr_port bigint(20) NO PRI NULL
|
||||
tenant_id bigint(20) NO PRI NULL
|
||||
activity_timestamp timestamp(6) NO PRI NULL
|
||||
modification_size bigint(20) NO NULL
|
||||
freeze_times bigint(20) NO NULL
|
||||
mini_merge_cost bigint(20) NO NULL
|
||||
mini_merge_times bigint(20) NO NULL
|
||||
minor_merge_cost bigint(20) NO NULL
|
||||
minor_merge_times bigint(20) NO NULL
|
||||
major_merge_cost bigint(20) NO NULL
|
||||
major_merge_times bigint(20) NO NULL
|
||||
select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_data_activity_metrics;
|
||||
IF(count(*) >= 0, 1, 0)
|
||||
1
|
||||
desc oceanbase.__all_virtual_column_group_mapping;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO PRI NULL
|
||||
@ -9023,6 +9040,62 @@ error_msg varchar(512) YES NULL
|
||||
select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_clone_job_history;
|
||||
IF(count(*) >= 0, 1, 0)
|
||||
1
|
||||
desc oceanbase.__all_virtual_checkpoint_diagnose_memtable_info;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO PRI NULL
|
||||
trace_id bigint(20) NO PRI NULL
|
||||
svr_ip varchar(46) NO NULL
|
||||
svr_port bigint(20) NO NULL
|
||||
ls_id bigint(20) NO NULL
|
||||
checkpoint_thread_name varchar(16) NO NULL
|
||||
checkpoint_start_time timestamp(6) NO NULL
|
||||
tablet_id bigint(20) NO NULL
|
||||
ptr varchar(128) NO NULL
|
||||
start_scn bigint(20) unsigned NO NULL
|
||||
end_scn bigint(20) unsigned NO NULL
|
||||
rec_scn bigint(20) unsigned NO NULL
|
||||
create_flush_dag_time timestamp(6) NO NULL
|
||||
merge_finish_time timestamp(6) NO NULL
|
||||
release_time timestamp(6) NO NULL
|
||||
frozen_finish_time timestamp(6) NO NULL
|
||||
merge_start_time timestamp(6) NO NULL
|
||||
start_gc_time timestamp(6) NO NULL
|
||||
memtable_occupy_size bigint(20) NO NULL
|
||||
occupy_size bigint(20) NO NULL
|
||||
concurrent_cnt bigint(20) NO NULL
|
||||
desc oceanbase.__all_virtual_checkpoint_diagnose_checkpoint_unit_info;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO PRI NULL
|
||||
trace_id bigint(20) NO PRI NULL
|
||||
svr_ip varchar(46) NO NULL
|
||||
svr_port bigint(20) NO NULL
|
||||
ls_id bigint(20) NO NULL
|
||||
checkpoint_thread_name varchar(16) NO NULL
|
||||
checkpoint_start_time timestamp(6) NO NULL
|
||||
tablet_id bigint(20) NO NULL
|
||||
ptr varchar(128) NO NULL
|
||||
start_scn bigint(20) unsigned NO NULL
|
||||
end_scn bigint(20) unsigned NO NULL
|
||||
rec_scn bigint(20) unsigned NO NULL
|
||||
create_flush_dag_time timestamp(6) NO NULL
|
||||
merge_finish_time timestamp(6) NO NULL
|
||||
start_gc_time timestamp(6) NO NULL
|
||||
desc oceanbase.__all_virtual_checkpoint_diagnose_info;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO NULL
|
||||
svr_ip varchar(46) NO NULL
|
||||
svr_port bigint(20) NO NULL
|
||||
ls_id bigint(20) NO NULL
|
||||
trace_id bigint(20) NO NULL
|
||||
freeze_clock int(10) unsigned NO NULL
|
||||
checkpoint_thread_name varchar(16) NO NULL
|
||||
checkpoint_start_time timestamp(6) NO NULL
|
||||
select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_checkpoint_diagnose_info;
|
||||
IF(count(*) >= 0, 1, 0)
|
||||
1
|
||||
"oceanbase.__all_virtual_checkpoint_diagnose_info runs in single server"
|
||||
IF(count(*) >= 0, 1, 0)
|
||||
1
|
||||
desc oceanbase.__all_virtual_aux_stat;
|
||||
Field Type Null Key Default Extra
|
||||
tenant_id bigint(20) NO PRI NULL
|
||||
|
@ -697,11 +697,15 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr
|
||||
12425 __all_virtual_import_table_job_history 2 201001 1
|
||||
12426 __all_virtual_import_table_task 2 201001 1
|
||||
12427 __all_virtual_import_table_task_history 2 201001 1
|
||||
12429 __all_virtual_data_activity_metrics 2 201001 1
|
||||
12430 __all_virtual_column_group_mapping 2 201001 1
|
||||
12431 __all_virtual_column_group_history 2 201001 1
|
||||
12432 __all_virtual_column_group_mapping_history 2 201001 1
|
||||
12435 __all_virtual_clone_job 2 201001 1
|
||||
12436 __all_virtual_clone_job_history 2 201001 1
|
||||
12437 __all_virtual_checkpoint_diagnose_memtable_info 2 201001 1
|
||||
12438 __all_virtual_checkpoint_diagnose_checkpoint_unit_info 2 201001 1
|
||||
12439 __all_virtual_checkpoint_diagnose_info 2 201001 1
|
||||
12447 __all_virtual_aux_stat 2 201001 1
|
||||
12453 __all_virtual_tenant_snapshot_job 2 201001 1
|
||||
12458 __all_virtual_ls_snapshot 2 201001 1
|
||||
|
@ -38,7 +38,9 @@ when "$table_name" in (
|
||||
"__all_virtual_proxy_schema" ,
|
||||
"__all_virtual_proxy_partition_info" ,
|
||||
"__all_virtual_proxy_partition" ,
|
||||
"__all_virtual_proxy_sub_partition") then 3
|
||||
"__all_virtual_proxy_sub_partition",
|
||||
"__all_virtual_checkpoint_diagnose_memtable_info",
|
||||
"__all_virtual_checkpoint_diagnose_checkpoint_unit_info") then 3
|
||||
when "$table_name" in (
|
||||
"__all_virtual_sysstat",
|
||||
"__all_virtual_sesstat",
|
||||
|
@ -43,7 +43,9 @@ when "$table_name" in (
|
||||
"__all_virtual_sesstat",
|
||||
"__all_virtual_session_event") then 4
|
||||
when "$table_name" in (
|
||||
"__all_virtual_tx_data") then 5
|
||||
"__all_virtual_tx_data",
|
||||
"__all_virtual_checkpoint_diagnose_memtable_info",
|
||||
"__all_virtual_checkpoint_diagnose_checkpoint_unit_info") then 5
|
||||
else 1 end) as query_type from dual`;
|
||||
|
||||
|
||||
|
@ -63,6 +63,7 @@ storage_unittest(test_is_old_mds multi_data_source/test_is_old_mds.cpp)
|
||||
storage_unittest(test_fixed_size_block_allocator)
|
||||
storage_unittest(test_dag_warning_history)
|
||||
storage_unittest(test_storage_schema)
|
||||
storage_unittest(test_checkpoint_diagnose checkpoint/test_checkpoint_diagnose.cpp)
|
||||
#storage_unittest(test_storage_schema_mgr)
|
||||
#storage_unittest(test_create_tablet_memtable test_create_tablet_memtable.cpp)
|
||||
storage_unittest(test_tenant_meta_obj_pool test_tenant_meta_obj_pool.cpp)
|
||||
|
207
unittest/storage/checkpoint/test_checkpoint_diagnose.cpp
Normal file
207
unittest/storage/checkpoint/test_checkpoint_diagnose.cpp
Normal file
@ -0,0 +1,207 @@
|
||||
/**
|
||||
* 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 "share/ob_errno.h"
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#define USING_LOG_PREFIX STORAGE
|
||||
|
||||
#define protected public
|
||||
#define private public
|
||||
|
||||
#define UNITTEST
|
||||
#include "storage/checkpoint/ob_checkpoint_diagnose.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace storage
|
||||
{
|
||||
namespace checkpoint
|
||||
{
|
||||
class TestChekpointDiagnose : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
TestChekpointDiagnose() {}
|
||||
virtual ~TestChekpointDiagnose() = default;
|
||||
virtual void SetUp() override {}
|
||||
virtual void TearDown() override {}
|
||||
};
|
||||
|
||||
TEST_F(TestChekpointDiagnose, trace_info) {
|
||||
ObCheckpointDiagnoseMgr mgr(2);
|
||||
mgr.init();
|
||||
int64_t trace_id1 = -1;
|
||||
int64_t trace_id2 = -1;
|
||||
int64_t trace_id3 = -1;
|
||||
int64_t trace_id4 = -1;
|
||||
share::ObLSID ls_id1(1);
|
||||
share::ObLSID ls_id2(1);
|
||||
share::ObLSID ls_id3(1);
|
||||
share::ObLSID ls_id4(1);
|
||||
share::SCN scn1;
|
||||
mgr.acquire_trace_id(ls_id1, trace_id1);
|
||||
ASSERT_EQ(0, trace_id1);
|
||||
ASSERT_EQ(0, mgr.first_pos_);
|
||||
ASSERT_EQ(0, mgr.last_pos_);
|
||||
ASSERT_EQ(1, mgr.get_trace_info_count());
|
||||
ASSERT_EQ(trace_id1, mgr.trace_info_arr_[trace_id1].trace_id_);
|
||||
|
||||
mgr.acquire_trace_id(ls_id2, trace_id2);
|
||||
ASSERT_EQ(1, trace_id2);
|
||||
ASSERT_EQ(0, mgr.first_pos_);
|
||||
ASSERT_EQ(1, mgr.last_pos_);
|
||||
ASSERT_EQ(2, mgr.get_trace_info_count());
|
||||
ASSERT_EQ(trace_id2, mgr.trace_info_arr_[trace_id2].trace_id_);
|
||||
|
||||
mgr.acquire_trace_id(ls_id3, trace_id3);
|
||||
ASSERT_EQ(2, trace_id3);
|
||||
ASSERT_EQ(1, mgr.first_pos_);
|
||||
ASSERT_EQ(2, mgr.last_pos_);
|
||||
ASSERT_EQ(2, mgr.get_trace_info_count());
|
||||
ASSERT_EQ(trace_id3, mgr.trace_info_arr_[trace_id3].trace_id_);
|
||||
ASSERT_EQ(-1, mgr.trace_info_arr_[0].trace_id_);
|
||||
|
||||
mgr.read_trace_info([&](const ObTraceInfo &trace_info) -> int {
|
||||
if (trace_info.trace_id_ == trace_id1) {
|
||||
OB_ASSERT(true);
|
||||
} else if (trace_info.trace_id_ == trace_id2) {
|
||||
OB_ASSERT(trace_info.ls_id_ == ls_id2);
|
||||
} else if (trace_info.trace_id_ == trace_id3) {
|
||||
OB_ASSERT(trace_info.ls_id_ == ls_id3);
|
||||
} else {
|
||||
OB_ASSERT(false);
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
});
|
||||
|
||||
mgr.update_max_trace_info_size(1);
|
||||
ASSERT_EQ(1, mgr.get_trace_info_count());
|
||||
ASSERT_EQ(2, mgr.first_pos_);
|
||||
ASSERT_EQ(2, mgr.last_pos_);
|
||||
ASSERT_EQ(-1, mgr.trace_info_arr_[0].trace_id_);
|
||||
ASSERT_EQ(-1, mgr.trace_info_arr_[1].trace_id_);
|
||||
ASSERT_EQ(trace_id3, mgr.trace_info_arr_[2].trace_id_);
|
||||
|
||||
mgr.acquire_trace_id(ls_id4, trace_id4);
|
||||
ASSERT_EQ(3, trace_id4);
|
||||
ASSERT_EQ(3, mgr.first_pos_);
|
||||
ASSERT_EQ(3, mgr.last_pos_);
|
||||
ASSERT_EQ(1, mgr.get_trace_info_count());
|
||||
ASSERT_EQ(trace_id4, mgr.trace_info_arr_[trace_id4].trace_id_);
|
||||
ASSERT_EQ(-1, mgr.trace_info_arr_[0].trace_id_);
|
||||
ASSERT_EQ(-1, mgr.trace_info_arr_[1].trace_id_);
|
||||
ASSERT_EQ(-1, mgr.trace_info_arr_[2].trace_id_);
|
||||
|
||||
}
|
||||
|
||||
TEST_F(TestChekpointDiagnose, diagnose_info) {
|
||||
int ret = OB_SUCCESS;
|
||||
share::SCN scn1;
|
||||
scn1.set_base();
|
||||
share::SCN scn2 = share::SCN::scn_inc(scn1);
|
||||
share::SCN scn3 = share::SCN::scn_inc(scn2);
|
||||
ObCheckpointDiagnoseMgr mgr;
|
||||
mgr.init();
|
||||
int64_t trace_id1 = -1;
|
||||
int64_t trace_id2 = -1;
|
||||
ObTabletID tablet_id1 = ObTabletID(1);
|
||||
ObTabletID tablet_id2 = ObTabletID(2);
|
||||
ObTabletID tablet_id3 = ObTabletID(3);
|
||||
share::ObLSID ls_id(1);
|
||||
void *ptr = (void*)1;
|
||||
uint32_t freeze_clock = 1;
|
||||
|
||||
// batch tablet freeze
|
||||
ret = mgr.acquire_trace_id(ls_id, trace_id1);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ObCheckpointDiagnoseParam param1(trace_id1, tablet_id1, ptr);
|
||||
ObCheckpointDiagnoseParam param2(trace_id1, tablet_id2, ptr);
|
||||
|
||||
ret = mgr.add_diagnose_info<ObMemtableDiagnoseInfo>(param1);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ret = mgr.update_freeze_info(param1, scn1, scn1, scn1, 0);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ret = mgr.update_schedule_dag_time(param1);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
|
||||
ret = mgr.add_diagnose_info<ObMemtableDiagnoseInfo>(param2);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ret = mgr.update_freeze_info(param2, scn2, scn2, scn2, 0);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ret = mgr.update_schedule_dag_time(param2);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
|
||||
ObMemtableDiagnoseInfo info1;
|
||||
ret = mgr.read_diagnose_info<ObMemtableDiagnoseInfo>(trace_id1, [&](const ObTraceInfo &trace_info, const ObCheckpointDiagnoseKey& key, const ObMemtableDiagnoseInfo &info) -> int {
|
||||
if (key.tablet_id_ == tablet_id1) {
|
||||
OB_ASSERT(trace_info.trace_id_ == trace_id1);
|
||||
OB_ASSERT(trace_info.memtable_diagnose_info_map_.size() == 2);
|
||||
OB_ASSERT(info.rec_scn_ == scn1);
|
||||
OB_ASSERT(info.start_scn_ == scn1);
|
||||
OB_ASSERT(info.end_scn_ == scn1);
|
||||
OB_ASSERT(info.create_flush_dag_time_ != 0);
|
||||
} else if (key.tablet_id_ == tablet_id2) {
|
||||
OB_ASSERT(info.rec_scn_ == scn2);
|
||||
OB_ASSERT(info.start_scn_ == scn2);
|
||||
OB_ASSERT(info.end_scn_ == scn2);
|
||||
OB_ASSERT(info.create_flush_dag_time_ != 0);
|
||||
} else {
|
||||
OB_ASSERT(false);
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
});
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
|
||||
// logstream freeze
|
||||
ret = mgr.acquire_trace_id(ls_id, trace_id2);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ret = mgr.update_freeze_clock(ls_id, trace_id2, freeze_clock);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ObCheckpointDiagnoseParam param3(ls_id.id(), freeze_clock - 1, tablet_id3, ptr);
|
||||
ret = mgr.add_diagnose_info<ObMemtableDiagnoseInfo>(param3);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ret = mgr.update_freeze_info(param3, scn3, scn3, scn3, 0);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
ret = mgr.update_schedule_dag_time(param3);
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
|
||||
ret = mgr.read_diagnose_info<ObMemtableDiagnoseInfo>(trace_id2, [&](const ObTraceInfo &trace_info, const ObCheckpointDiagnoseKey& key, const ObMemtableDiagnoseInfo &info) -> int {
|
||||
if (key.tablet_id_ == tablet_id3) {
|
||||
OB_ASSERT(trace_info.trace_id_ == trace_id2);
|
||||
OB_ASSERT(trace_info.memtable_diagnose_info_map_.size() == 1);
|
||||
OB_ASSERT(info.rec_scn_ == scn3);
|
||||
OB_ASSERT(info.start_scn_ == scn3);
|
||||
OB_ASSERT(info.end_scn_ == scn3);
|
||||
OB_ASSERT(info.create_flush_dag_time_ != 0);
|
||||
} else {
|
||||
OB_ASSERT(false);
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
});
|
||||
ASSERT_EQ(ret, OB_SUCCESS);
|
||||
}
|
||||
|
||||
}
|
||||
} // end namespace storage
|
||||
} // end namespace oceanbase
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
int ret = 0;
|
||||
system("rm -f test_checkpoint_diagnose.log*");
|
||||
OB_LOGGER.set_file_name("test_checkpoint_diagnose.log", true);
|
||||
OB_LOGGER.set_log_level("INFO");
|
||||
signal(49, SIG_IGN);
|
||||
testing::InitGoogleTest(&argc, argv);
|
||||
ret = RUN_ALL_TESTS();
|
||||
return ret;
|
||||
}
|
@ -160,7 +160,7 @@ TEST_F(TestMdsTableFlush, flusher_for_all_order_with_enough_memory) {
|
||||
p_mds_table->rec_scn_ = v_key[i].rec_scn_;
|
||||
v.push_back(mds_table);
|
||||
}
|
||||
ASSERT_EQ(OB_SUCCESS, mgr.flush(share::SCN::max_scn()));
|
||||
ASSERT_EQ(OB_SUCCESS, mgr.flush(share::SCN::max_scn(), -1));
|
||||
ASSERT_EQ(TEST_ALL_SIZE, V_ActualDoFlushKey.size());
|
||||
for (int i = 0; i < V_ActualDoFlushKey.size(); ++i) {
|
||||
if (V_ActualDoFlushKey[i].rec_scn_ != mock_scn(100 + i)) {
|
||||
@ -194,7 +194,7 @@ TEST_F(TestMdsTableFlush, flusher_for_all_order_with_limitted_memory_reserve_fai
|
||||
p_mds_table->rec_scn_ = v_key[i].rec_scn_;
|
||||
v.push_back(mds_table);
|
||||
}
|
||||
ASSERT_EQ(OB_SUCCESS, mgr.flush(share::SCN::max_scn()));
|
||||
ASSERT_EQ(OB_SUCCESS, mgr.flush(share::SCN::max_scn(), -1));
|
||||
ASSERT_EQ(TEST_ALL_SIZE + FLUSH_FOR_ALL_SIZE, V_ActualDoFlushKey.size());// 只保证最前面的TEST_ALL_SIZE的tablet是有序的,并且rec scn最小
|
||||
for (int i = 0; i < FLUSH_FOR_ALL_SIZE; ++i) {
|
||||
if (V_ActualDoFlushKey[i].rec_scn_ != mock_scn(100 + i)) {
|
||||
|
Loading…
x
Reference in New Issue
Block a user