[FEAT MERGE] [PATCH FROM 4.2.3] disk_space usage query optimization

This commit is contained in:
gaishun
2024-05-27 11:12:35 +00:00
committed by ob-robot
parent cc18ead6d9
commit a718b67350
11 changed files with 135 additions and 54 deletions

View File

@ -146,6 +146,8 @@ int ObAllVirtualTabletPtr::process_curr_tenant(ObNewRow *&row)
ls_id = key.ls_id_;
tablet_id = key.tablet_id_;
tablet_pointer = static_cast<const ObTabletPointer*>(ptr_hdl.get_resource_ptr());
ObTabletResidentInfo tablet_info(key, *tablet_pointer);
const int64_t col_cnt = output_column_ids_.count();
for (int64_t i = 0; OB_SUCC(ret) && i < col_cnt; i++) {
const uint64_t col_id = output_column_ids_.at(i);
@ -208,10 +210,10 @@ int ObAllVirtualTabletPtr::process_curr_tenant(ObNewRow *&row)
}
break;
case DATA_OCCUPIED:
cur_row_.cells_[i].set_int(0);
cur_row_.cells_[i].set_int(tablet_info.get_occupy_size());
break;
case DATA_REQUIRED:
cur_row_.cells_[i].set_int(0);
cur_row_.cells_[i].set_int(tablet_info.get_required_size());
break;
default:
ret = OB_ERR_UNEXPECTED;

View File

@ -51,7 +51,7 @@ ObTabletPointer::ObTabletPointer()
attr_()
{
#if defined(__x86_64__) && !defined(ENABLE_OBJ_LEAK_CHECK)
static_assert(sizeof(ObTabletPointer) == 296, "The size of ObTabletPointer will affect the meta memory manager, and the necessity of adding new fields needs to be considered.");
static_assert(sizeof(ObTabletPointer) == 320, "The size of ObTabletPointer will affect the meta memory manager, and the necessity of adding new fields needs to be considered.");
#endif
}
@ -675,7 +675,7 @@ int ObTabletPointer::set_tablet_attr(const ObTabletAttr &attr)
return ret;
}
ObTabletResidentInfo::ObTabletResidentInfo(const ObTabletMapKey &key, ObTabletPointer &tablet_ptr)
ObTabletResidentInfo::ObTabletResidentInfo(const ObTabletMapKey &key, const ObTabletPointer &tablet_ptr)
: attr_(tablet_ptr.attr_), tablet_addr_(tablet_ptr.phy_addr_)
{
tablet_id_ = key.tablet_id_;

View File

@ -34,13 +34,17 @@ struct ObTabletAttr final
public:
ObTabletAttr()
:v_(0),
ha_status_(0)
ha_status_(0),
occupy_bytes_(0),
required_bytes_(0),
tablet_meta_bytes_(0)
{}
~ObTabletAttr() { reset(); }
void reset() { v_ = 0; ha_status_ = 0; }
void reset() { v_ = 0; ha_status_ = 0; occupy_bytes_ = 0; required_bytes_ = 0; tablet_meta_bytes_ = 0; }
bool is_valid() const { return valid_; }
TO_STRING_KV(K_(valid), K_(is_empty_shell), K_(has_transfer_table),
K_(has_next_tablet), K_(has_nested_table), K_(ha_status));
K_(has_next_tablet), K_(has_nested_table), K_(ha_status),
K_(occupy_bytes), K_(required_bytes), K_(tablet_meta_bytes));
public:
union {
int64_t v_;
@ -54,6 +58,9 @@ public:
};
int64_t ha_status_;
int64_t occupy_bytes_;
int64_t required_bytes_;
int64_t tablet_meta_bytes_;
};
class ObTabletPointer final
@ -141,8 +148,8 @@ private:
ObByteLock ddl_kv_mgr_lock_; // 1B
mds::ObMdsTableHandler mds_table_handler_;// 48B
ObTablet *old_version_chain_; // 8B
ObTabletAttr attr_; // 16B // protected by rw lock of tablet_map_
DISALLOW_COPY_AND_ASSIGN(ObTabletPointer); // 288B
ObTabletAttr attr_; // 32B // protected by rw lock of tablet_map_
DISALLOW_COPY_AND_ASSIGN(ObTabletPointer); // 312B
};
struct ObTabletResidentInfo final
@ -152,16 +159,19 @@ public:
: attr_(attr), tablet_addr_(), tablet_id_(tablet_id), ls_id_(ls_id)
{}
ObTabletResidentInfo(const ObTabletMapKey &key, ObTabletPointer &tablet_ptr);
ObTabletResidentInfo(const ObTabletMapKey &key, const ObTabletPointer &tablet_ptr);
~ObTabletResidentInfo() = default;
bool is_valid() const { return attr_.valid_ && tablet_id_.is_valid() && tablet_addr_.is_valid(); }
bool has_transfer_table() const { return attr_.has_transfer_table_; }
bool is_empty_shell() const { return attr_.is_empty_shell_; }
bool has_next_tablet() const { return attr_.has_next_tablet_; }
bool has_nested_table() const { return attr_.has_nested_table_; }
int64_t get_required_size() const { return attr_.required_bytes_; }
int64_t get_occupy_size() const { return attr_.occupy_bytes_; }
int64_t get_meta_size() const { return attr_.tablet_meta_bytes_; }
TO_STRING_KV(K_(ls_id), K_(tablet_id), K_(tablet_addr), K_(attr));
public:
ObTabletAttr &attr_;
const ObTabletAttr &attr_;
ObMetaDiskAddr tablet_addr_; // used to identify one tablet
ObTabletID tablet_id_;
share::ObLSID ls_id_;

View File

@ -153,7 +153,7 @@ int ObDiskUsageReportTask::report_tenant_disk_usage(const char *svr_ip,
// to reduce the locking time of the result_map_,
// copy the value to array and then update the usage table,
ObArray<hash::HashMapPair<ObDiskUsageReportKey, int64_t>> result_arr;
ObArray<ObDiskUsageReportMap> result_arr;
ObReportResultGetter copy_result(result_arr);
if (OB_FAIL(ret)) {
@ -162,10 +162,10 @@ int ObDiskUsageReportTask::report_tenant_disk_usage(const char *svr_ip,
STORAGE_LOG(WARN, "fail to copy result", K(ret));
}
for (int64_t i = 0; i < result_arr.count() && OB_SUCC(ret); ++i) {
const hash::HashMapPair<ObDiskUsageReportKey, int64_t> &pair = result_arr.at(i);
const ObDiskUsageReportMap &pair = result_arr.at(i);
if (OB_FAIL(disk_usage_table_operator_.update_tenant_space_usage(
pair.first.tenant_id_, svr_ip, svr_port, seq_num,
pair.first.file_type_, pair.second, pair.second))) {
pair.first.file_type_, pair.second.first, pair.second.second))) {
STORAGE_LOG(WARN, "failed to update disk usage of log and meta", K(ret), K(pair.first));
}
}
@ -182,39 +182,36 @@ int ObDiskUsageReportTask::count_tenant_data(const uint64_t tenant_id)
ObDiskUsageReportKey data_key;
int64_t meta_size = 0;
int64_t data_size = 0;
int64_t tmp_meta_size = 0;
int64_t tmp_data_size = 0;
int64_t occupy_size = 0;
data_size = 0;
if (OB_FAIL(MTL(ObTenantCheckpointSlogHandler*)->get_meta_block_list(block_list))) {
STORAGE_LOG(WARN, "failed to get tenant's meta block list", K(ret));
} else {
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObArenaAllocator iter_allocator("DiskReport", OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id);
ObTenantTabletIterator tablet_iter(*t3m, iter_allocator, nullptr/*no op*/);
ObTabletHandle tablet_handle;
while (OB_SUCC(ret) && OB_SUCC(tablet_iter.get_next_tablet(tablet_handle))) {
if (OB_UNLIKELY(!tablet_handle.is_valid())) {
ObTenantTabletPtrWithInMemObjIterator tablet_ptr_iter(*t3m);
ObTabletPointerHandle pointer_handle;
ObTabletHandle unused_tablet_handle;
ObTabletMapKey tablet_map_key;
const ObTabletPointer *tablet_pointer = nullptr;
while (OB_SUCC(ret) && OB_SUCC(tablet_ptr_iter.get_next_tablet_pointer(tablet_map_key, pointer_handle, unused_tablet_handle))) {
if (OB_UNLIKELY(!pointer_handle.is_valid())) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "unexpected invalid tablet", K(ret), K(tablet_handle));
} else if (tablet_handle.get_obj()->is_empty_shell()) {
// skip empty shell
} else if (OB_FAIL(tablet_handle.get_obj()->get_tablet_size(
true /*ignore shared block*/, tmp_meta_size, tmp_data_size))) {
STORAGE_LOG(WARN, "failed to get tablet's meta and data size", K(ret));
STORAGE_LOG(WARN, "unexpected invalid tablet", K(ret), K(pointer_handle));
} else if (OB_ISNULL(tablet_pointer = static_cast<const ObTabletPointer*>(pointer_handle.get_resource_ptr()))) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "failed to cast ptr to ObTabletPointer*", K(ret), K(pointer_handle));
} else {
meta_size += tmp_meta_size;
data_size += tmp_data_size;
ObTabletResidentInfo tablet_info(tablet_map_key, *tablet_pointer);
occupy_size += tablet_info.get_occupy_size();
data_size += tablet_info.get_required_size();
meta_size += tablet_info.get_meta_size();
}
tablet_handle.reset();
iter_allocator.reuse();
tmp_meta_size = 0;
tmp_data_size = 0;
pointer_handle.reset();
}
if (OB_ITER_END == ret || OB_SUCCESS == ret) {
ret = OB_SUCCESS;
data_size += MTL(ObSharedMacroBlockMgr*)->get_shared_block_cnt() * OB_DEFAULT_MACRO_BLOCK_SIZE;
meta_size += block_list.count() * OB_DEFAULT_MACRO_BLOCK_SIZE;
}
}
@ -224,10 +221,10 @@ int ObDiskUsageReportTask::count_tenant_data(const uint64_t tenant_id)
meta_key.file_type_ = ObDiskReportFileType::TENANT_META_DATA;
data_key.tenant_id_ = tenant_id;
data_key.file_type_ = ObDiskReportFileType::TENANT_DATA;
if (OB_FAIL(result_map_.set_refactored(meta_key, meta_size, 1 /* whether allowed to override */))) {
if (OB_FAIL(result_map_.set_refactored(meta_key, std::make_pair(meta_size, meta_size), 1 /* whether allowed to override */))) {
STORAGE_LOG(WARN, "failed to insert meta info result_map_", K(ret), K(meta_key), K(meta_size));
} else if (OB_FAIL(result_map_.set_refactored(data_key, data_size, 1 /* whether allowed to override */))) {
STORAGE_LOG(WARN, "failed to insert data info result_map_", K(ret), K(data_key), K(data_size));
} else if (OB_FAIL(result_map_.set_refactored(data_key,std::make_pair(occupy_size, data_size), 1 /* whether allowed to override */))) {
STORAGE_LOG(WARN, "failed to insert data info result_map_", K(ret), K(data_key), K(occupy_size), K(data_size));
}
}
return ret;
@ -294,7 +291,7 @@ int ObDiskUsageReportTask::count_tenant_slog(const uint64_t tenant_id)
} else {
report_key.file_type_ = ObDiskReportFileType::TENANT_SLOG_DATA;
report_key.tenant_id_ = tenant_id;
if (OB_FAIL(result_map_.set_refactored(report_key, slog_space, 1))) {
if (OB_FAIL(result_map_.set_refactored(report_key, std::make_pair(slog_space, slog_space), 1))) {
STORAGE_LOG(WARN, "failed to set result_map_", K(ret), K(report_key), K(slog_space));
}
}
@ -317,7 +314,7 @@ int ObDiskUsageReportTask::count_tenant_clog(const uint64_t tenant_id)
} else {
report_key.file_type_ = ObDiskReportFileType::TENANT_CLOG_DATA;
report_key.tenant_id_ = tenant_id;
if (OB_FAIL(result_map_.set_refactored(report_key, clog_space, 1))) {
if (OB_FAIL(result_map_.set_refactored(report_key, std::make_pair(clog_space, clog_space), 1))) {
STORAGE_LOG(WARN, "failed to set result_map_", K(ret), K(report_key), K(clog_space));
}
}
@ -340,7 +337,7 @@ int ObDiskUsageReportTask::count_server_slog()
} else {
report_key.file_type_ = ObDiskReportFileType::TENANT_SLOG_DATA;
report_key.tenant_id_ = OB_SERVER_TENANT_ID;
if (OB_FAIL(result_map_.set_refactored(report_key, slog_space, 1))) {
if (OB_FAIL(result_map_.set_refactored(report_key, std::make_pair(slog_space, slog_space), 1))) {
STORAGE_LOG(WARN, "failed to set result_map_", K(ret), K(report_key), K(slog_space));
}
}
@ -365,7 +362,7 @@ int ObDiskUsageReportTask::count_server_clog()
report_key.file_type_ = ObDiskReportFileType::TENANT_CLOG_DATA;
report_key.tenant_id_ = OB_SERVER_TENANT_ID;
int64_t clog_space = clog_in_use_size_byte;
if (OB_FAIL(result_map_.set_refactored(report_key, clog_space, 1))) {
if (OB_FAIL(result_map_.set_refactored(report_key, std::make_pair(clog_space, clog_space), 1))) {
STORAGE_LOG(WARN, "failed to set result_map_", K(ret), K(report_key), K(clog_space));
}
}
@ -382,7 +379,8 @@ int ObDiskUsageReportTask::count_server_meta()
} else {
report_key.tenant_id_ = OB_SERVER_TENANT_ID;
report_key.file_type_ = ObDiskReportFileType::TENANT_META_DATA;
if (OB_FAIL(result_map_.set_refactored(report_key, block_list.count() * common::OB_DEFAULT_MACRO_BLOCK_SIZE, 1))) {
int64_t server_meta_size = block_list.count() * common::OB_DEFAULT_MACRO_BLOCK_SIZE;
if (OB_FAIL(result_map_.set_refactored(report_key, std::make_pair(server_meta_size, server_meta_size), 1))) {
STORAGE_LOG(WARN, "failed to set result_map_", K(ret), K(report_key), K(block_list.count()));
}
}
@ -402,7 +400,8 @@ int ObDiskUsageReportTask::count_tenant_tmp()
for (int64_t i = 0; OB_SUCC(ret) && i < tenant_block_cnt_pairs.count(); ++i) {
report_key.tenant_id_ = tenant_block_cnt_pairs.at(i).first;
macro_block_cnt = tenant_block_cnt_pairs.at(i).second;
if (OB_FAIL(result_map_.set_refactored(report_key, macro_block_cnt * common::OB_DEFAULT_MACRO_BLOCK_SIZE, 1))) {
int64_t tenant_tmp_size = macro_block_cnt * common::OB_DEFAULT_MACRO_BLOCK_SIZE;
if (OB_FAIL(result_map_.set_refactored(report_key, std::make_pair(tenant_tmp_size, tenant_tmp_size), 1))) {
STORAGE_LOG(WARN, "failed to set tenant tmp usage into result map", K(ret), K(report_key), K(macro_block_cnt));
}
}
@ -473,7 +472,7 @@ int ObDiskUsageReportTask::get_data_disk_used_size(const uint64_t tenant_id, int
};
ObDiskUsageReportKey key;
key.tenant_id_ = tenant_id;
int64_t size = 0;
std::pair<int64_t, int64_t> size = std::make_pair(0, 0);
for (int64_t i = 0; i < need_cnt && OB_SUCC(ret); i++) {
key.file_type_ = file_types_need[i];
@ -483,7 +482,7 @@ int ObDiskUsageReportTask::get_data_disk_used_size(const uint64_t tenant_id, int
} else if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
used_size += size;
used_size += size.second;
}
}
}
@ -503,13 +502,13 @@ int ObDiskUsageReportTask::get_clog_disk_used_size(const uint64_t tenant_id, int
key.tenant_id_ = tenant_id;
key.file_type_ = ObDiskReportFileType::TENANT_CLOG_DATA;
int64_t size = 0;
std::pair<int64_t, int64_t> size = std::make_pair(0, 0);
if (OB_FAIL(result_map_.get_refactored(key, size)) && OB_HASH_NOT_EXIST != ret) {
STORAGE_LOG(WARN, "fail to get file type size", K(ret), K(key));
} else if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
used_size = size;
used_size = size.second;
}
}
return ret;

View File

@ -31,6 +31,7 @@ namespace storage
{
class ObTenantTabletIterator;
struct ObDiskUsageReportKey
{
ObDiskReportFileType file_type_;
@ -56,6 +57,8 @@ struct ObDiskUsageReportKey
TO_STRING_KV(K_(file_type), K_(tenant_id));
};
typedef hash::HashMapPair<ObDiskUsageReportKey, std::pair<int64_t, int64_t>> ObDiskUsageReportMap;// pair(occupy_size, required_size)
class ObDiskUsageReportTask : public common::ObTimerTask, public observer::ObIDiskReport
{
public:
@ -73,11 +76,11 @@ private:
class ObReportResultGetter final
{
public:
explicit ObReportResultGetter(ObArray<hash::HashMapPair<ObDiskUsageReportKey, int64_t>> &result_arr)
explicit ObReportResultGetter(ObArray<ObDiskUsageReportMap> &result_arr)
: result_arr_(result_arr)
{}
~ObReportResultGetter() = default;
int operator()(const hash::HashMapPair<ObDiskUsageReportKey, int64_t> &pair)
int operator()(const ObDiskUsageReportMap &pair)
{
int ret = OB_SUCCESS;
if (OB_FAIL(result_arr_.push_back(pair))) {
@ -86,7 +89,7 @@ private:
return ret;
}
private:
ObArray<hash::HashMapPair<ObDiskUsageReportKey, int64_t>> &result_arr_;
ObArray<ObDiskUsageReportMap> &result_arr_;
DISALLOW_COPY_AND_ASSIGN(ObReportResultGetter);
};
@ -112,7 +115,7 @@ private:
const int64_t seq_num);
virtual void runTimerTask();
typedef common::hash::ObHashMap<ObDiskUsageReportKey, int64_t> ReportResultMap;
typedef common::hash::ObHashMap<ObDiskUsageReportKey, std::pair<int64_t, int64_t>> ReportResultMap; // pair(occupy_size, required_size)
private:
bool is_inited_;
ReportResultMap result_map_;

View File

@ -5612,7 +5612,9 @@ int ObTablet::calc_tablet_attr(ObTabletAttr &attr) const
attr.is_empty_shell_ = table_store_addr_.addr_.is_none();
attr.has_next_tablet_ = tablet_meta_.has_next_tablet_;
attr.ha_status_ = tablet_meta_.ha_status_.get_ha_status();
attr.required_bytes_ = tablet_meta_.space_usage_.data_size_ + tablet_meta_.space_usage_.shared_data_size_;
attr.occupy_bytes_ = tablet_meta_.space_usage_.occupy_bytes_;
attr.tablet_meta_bytes_ = tablet_meta_.space_usage_.shared_meta_size_ + tablet_meta_.space_usage_.meta_size_;
attr.has_nested_table_ = false;
ObTabletMemberWrapper<ObTabletTableStore> wrapper;
const ObTabletTableStore *table_store = nullptr;
@ -6621,6 +6623,47 @@ int ObTablet::init_aggregated_info(common::ObArenaAllocator &allocator, ObLinked
return ret;
}
int ObTablet::calc_sstable_occupy_size(int64_t &occupy_size)
{
int ret = OB_SUCCESS;
occupy_size = 0;
ObTableStoreIterator iter;
if (OB_FAIL(inner_get_all_sstables(iter))) {
LOG_WARN("fail to get all sstables", K(ret));
}
int64_t occupy_bytes = 0;
while (OB_SUCC(ret)) {
ObITable *table = nullptr;
ObSSTable *sstable = nullptr;
ObSSTableMetaHandle meta_handle;
if (OB_FAIL(iter.get_next(table))) {
if (OB_UNLIKELY(OB_ITER_END == ret)) {
ret = OB_SUCCESS;
break;
} else {
LOG_WARN("fail to get next table from iter", K(ret), K(iter));
}
} else if (FALSE_IT(sstable = static_cast<ObSSTable *>(table))) {
} else if (OB_ISNULL(sstable) || !sstable->is_valid()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the sstable is null or invalid", K(ret), KPC(sstable));
} else if (OB_FAIL(sstable->get_meta(meta_handle))) {
LOG_WARN("fail to get sstable meta handle", K(ret), KPC(sstable));
} else if (!meta_handle.is_valid()) {
LOG_WARN("meta_handle is not valid", K(ret), K(meta_handle), KPC(sstable));
} else if (sstable->is_small_sstable()) {
occupy_bytes += sstable->get_macro_read_size(); /*nested_size_*/
} else {
const ObSSTableMeta &sstable_meta = meta_handle.get_sstable_meta();
occupy_bytes += sstable_meta.get_occupy_size();
}
}
if (OB_SUCC(ret)) {
occupy_size = occupy_bytes;
}
return ret;
}
int ObTablet::load_medium_info_list(
common::ObArenaAllocator &allocator,
const ObTabletComplexAddr<oceanbase::storage::ObTabletDumpedMediumInfo> &complex_addr,

View File

@ -33,6 +33,7 @@
#include "storage/tablet/ob_tablet_mds_data_cache.h"
#include "storage/tablet/ob_tablet_block_aggregated_info.h"
#include "storage/tablet/ob_tablet_block_header.h"
#include "storage/tablet/ob_tablet_space_usage.h"
#include "storage/tx/ob_trans_define.h"
#include "share/scn.h"
#include "ob_i_tablet_mds_interface.h"
@ -602,6 +603,8 @@ private:
static void dec_linked_block_ref_cnt(const ObMetaDiskAddr &head_addr);
int64_t get_try_cache_size() const;
int inner_release_memtables(const share::SCN scn);
int calc_sstable_occupy_size(int64_t &occupy_size);
inline void set_space_usage_(const ObTabletSpaceUsage &space_usage) { tablet_meta_.set_space_usage_(space_usage); }
private:
static bool ignore_ret(const int ret);
int inner_check_valid(const bool ignore_ha_status = false) const;

View File

@ -181,6 +181,7 @@ int ObTabletMeta::init(
ddl_data_format_version_ = MAX(old_tablet_meta.ddl_data_format_version_, ddl_info.data_format_version_);
mds_checkpoint_scn_ = old_tablet_meta.mds_checkpoint_scn_;
transfer_info_ = old_tablet_meta.transfer_info_;
space_usage_ = old_tablet_meta.space_usage_;
is_inited_ = true;
}
@ -229,6 +230,7 @@ int ObTabletMeta::init(
max_serialized_medium_scn_ = old_tablet_meta.max_serialized_medium_scn_;
mds_checkpoint_scn_ = flush_scn;
transfer_info_ = old_tablet_meta.transfer_info_;
space_usage_ = old_tablet_meta.space_usage_;
is_inited_ = true;
}
@ -323,6 +325,7 @@ int ObTabletMeta::init(
ddl_data_format_version_ = old_tablet_meta.ddl_data_format_version_;
mds_checkpoint_scn_ = old_tablet_meta.mds_checkpoint_scn_;
transfer_info_ = old_tablet_meta.transfer_info_;
space_usage_ = old_tablet_meta.space_usage_;
is_inited_ = true;
}
@ -410,6 +413,7 @@ int ObTabletMeta::init(
ddl_data_format_version_ = old_tablet_meta.ddl_data_format_version_;
transfer_info_ = transfer_info;
mds_checkpoint_scn_ = old_tablet_meta.mds_checkpoint_scn_;
space_usage_ = old_tablet_meta.space_usage_;
is_inited_ = true;
}
@ -474,7 +478,8 @@ bool ObTabletMeta::is_valid() const
&& clog_checkpoint_scn_ >= INIT_CLOG_CHECKPOINT_SCN
&& start_scn_ >= INIT_CLOG_CHECKPOINT_SCN
&& start_scn_ <= clog_checkpoint_scn_))
&& create_schema_version_ >= 0;
&& create_schema_version_ >= 0
&& space_usage_.is_valid();
}
int ObTabletMeta::assign(const ObTabletMeta &other)

View File

@ -188,6 +188,7 @@ private:
int inner_check_(
const ObTabletMeta &old_tablet_meta,
const ObMigrationTabletParam *tablet_meta);
inline void set_space_usage_ (const ObTabletSpaceUsage &space_usage) { space_usage_ = space_usage; }
private:
static const int32_t TABLET_META_VERSION = 1;
private:

View File

@ -657,6 +657,12 @@ int ObTabletPersister::persist_and_fill_tablet(
}
}
space_usage.shared_data_size_ = shared_data_size;
if (OB_FAIL(ret)) {
} else if (OB_FAIL(new_handle.get_obj()->calc_sstable_occupy_size(space_usage.occupy_bytes_))) {
LOG_WARN("failed to calc tablet occupy_size", K(ret), KPC(new_handle.get_obj()));
} else {
new_handle.get_obj()->set_space_usage_(space_usage);
}
}
return ret;

View File

@ -38,6 +38,15 @@ public:
int serialize(char *buf, const int64_t buf_len, int64_t &pos) const;
int deserialize(const char *buf, const int64_t data_len, int64_t &pos);
int32_t get_serialize_size() const;
bool is_valid() const
{
return (OB_INVALID_SIZE != shared_data_size_)
&& (OB_INVALID_SIZE != data_size_)
&& (OB_INVALID_SIZE != shared_meta_size_)
&& (OB_INVALID_SIZE != meta_size_)
&& (OB_INVALID_SIZE != occupy_bytes_);
}
public:
static const int32_t TABLET_SPACE_USAGE_INFO_VERSION = 1;
public: