提交 24c01952 编写于 作者: R renju96 提交者: ob-robot

[CP]add allocator label

上级 6f9f8255
......@@ -69,6 +69,9 @@ ObTmpFileIOHandle::ObTmpFileIOHandle()
last_fd_(OB_INVALID_FD),
last_extent_id_(0)
{
io_handles_.set_attr(ObMemAttr(MTL_ID(), "TMP_IO_HDL"));
page_cache_handles_.set_attr(ObMemAttr(MTL_ID(), "TMP_PCACHE_HDL"));
block_cache_handles_.set_attr(ObMemAttr(MTL_ID(), "TMP_BCACHE_HDL"));
}
ObTmpFileIOHandle::~ObTmpFileIOHandle()
......
......@@ -222,7 +222,10 @@ private:
class ObTmpFileMeta final
{
public:
explicit ObTmpFileMeta() : fd_(-1), dir_id_(-1), allocator_(NULL), extents_() {}
explicit ObTmpFileMeta() : fd_(-1), dir_id_(-1), allocator_(NULL), extents_()
{
extents_.set_attr(ObMemAttr(MTL_ID(), "TMP_META"));
}
~ObTmpFileMeta();
int clear();
int init(const int64_t fd, const int64_t dir_id, common::ObIAllocator *allocator);
......
......@@ -976,7 +976,7 @@ int ObComplementWriteTask::do_local_scan()
false,
false);
ObStoreRange range;
ObArenaAllocator allocator;
ObArenaAllocator allocator("cmplt_write", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObDatumRange datum_range;
const bool allow_not_ready = false;
ObLSHandle ls_handle;
......@@ -1155,7 +1155,7 @@ int ObComplementWriteTask::append_row(ObScan *scan)
ObArenaAllocator lob_allocator(ObModIds::OB_LOB_ACCESS_BUFFER, OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObStoreRow reshaped_row;
reshaped_row.flag_.set_flag(ObDmlFlag::DF_INSERT);
ObArenaAllocator allocator(lib::ObLabel("CompDataTaskTmp"));
ObArenaAllocator allocator(lib::ObLabel("CompDataTaskTmp"), OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObRowReshape *reshape_ptr = nullptr;
ObSQLMode sql_mode_for_ddl_reshape = SMO_TRADITIONAL;
ObDatumRow datum_row;
......@@ -1497,8 +1497,9 @@ int ObComplementMergeTask::add_build_hidden_table_sstable()
ObLocalScan::ObLocalScan() : is_inited_(false), tenant_id_(OB_INVALID_TENANT_ID), table_id_(OB_INVALID_ID),
dest_table_id_(OB_INVALID_ID), schema_version_(0), extended_gc_(), snapshot_version_(common::OB_INVALID_VERSION),
txs_(nullptr), default_row_(), tmp_row_(), row_iter_(nullptr), scan_merge_(nullptr), ctx_(), access_param_(),
access_ctx_(), get_table_param_(), allocator_("ObLocalScan"), calc_buf_(ObModIds::OB_SQL_EXPR_CALC),
col_params_(), read_info_(), exist_column_mapping_(allocator_), checksum_calculator_()
access_ctx_(), get_table_param_(), allocator_("ObLocalScan", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()),
calc_buf_(ObModIds::OB_SQL_EXPR_CALC, OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()), col_params_(), read_info_(),
exist_column_mapping_(allocator_), checksum_calculator_()
{}
ObLocalScan::~ObLocalScan()
......@@ -1956,7 +1957,7 @@ ObRemoteScan::ObRemoteScan()
row_with_reshape_(),
res_(),
result_(nullptr),
allocator_("DDLRemoteScan"),
allocator_("DDLRemoteScan", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()),
org_col_ids_(),
column_names_(),
checksum_calculator_()
......
......@@ -43,7 +43,7 @@ public:
is_inited_(false), orig_tenant_id_(common::OB_INVALID_TENANT_ID), dest_tenant_id_(common::OB_INVALID_TENANT_ID),
orig_ls_id_(share::ObLSID::INVALID_LS_ID), dest_ls_id_(share::ObLSID::INVALID_LS_ID), orig_table_id_(common::OB_INVALID_ID),
dest_table_id_(common::OB_INVALID_ID), orig_tablet_id_(ObTabletID::INVALID_TABLET_ID), dest_tablet_id_(ObTabletID::INVALID_TABLET_ID),
allocator_("CompleteDataPar"), row_store_type_(common::ENCODING_ROW_STORE), orig_schema_version_(0), dest_schema_version_(0),
allocator_("CompleteDataPar", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()), row_store_type_(common::ENCODING_ROW_STORE), orig_schema_version_(0), dest_schema_version_(0),
snapshot_version_(0), concurrent_cnt_(0), task_id_(0), execution_id_(-1), tablet_task_id_(0), compat_mode_(lib::Worker::CompatMode::INVALID), data_format_version_(0)
{}
~ObComplementDataParam() { destroy(); }
......@@ -120,7 +120,7 @@ struct ObComplementDataContext final
public:
ObComplementDataContext():
is_inited_(false), is_major_sstable_exist_(false), complement_data_ret_(common::OB_SUCCESS),
allocator_("CompleteDataCtx"), lock_(ObLatchIds::COMPLEMENT_DATA_CONTEXT_LOCK), concurrent_cnt_(0),
allocator_("CompleteDataCtx", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()), lock_(ObLatchIds::COMPLEMENT_DATA_CONTEXT_LOCK), concurrent_cnt_(0),
data_sstable_redo_writer_(), index_builder_(nullptr), ddl_kv_mgr_handle_(), row_scanned_(0), row_inserted_(0)
{}
~ObComplementDataContext() { destroy(); }
......
......@@ -138,7 +138,7 @@ void ObDDLStartClogCb::try_release()
ObDDLMacroBlockClogCb::ObDDLMacroBlockClogCb()
: is_inited_(false), status_(), ls_id_(), redo_info_(), macro_block_id_(),
arena_("ddl_clog_cb", OB_MALLOC_BIG_BLOCK_SIZE), data_buffer_lock_(), is_data_buffer_freed_(false), ddl_kv_mgr_handle_()
data_buffer_lock_(), is_data_buffer_freed_(false), ddl_kv_mgr_handle_()
{
}
......
......@@ -122,7 +122,6 @@ private:
share::ObLSID ls_id_;
blocksstable::ObDDLMacroBlockRedoInfo redo_info_;
blocksstable::MacroBlockId macro_block_id_;
ObArenaAllocator arena_;
ObSpinLock data_buffer_lock_;
bool is_data_buffer_freed_;
ObTabletHandle tablet_handle_;
......
......@@ -311,7 +311,7 @@ int ObDDLTableMergeTask::process()
ObTableStoreIterator ddl_table_iter;
ObTabletMemberWrapper<ObTabletTableStore> table_store_wrapper;
const uint64_t tenant_id = MTL_ID();
common::ObArenaAllocator allocator("DDLMergeTask");
common::ObArenaAllocator allocator("DDLMergeTask", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObSSTable compact_sstable;
ObSSTable *sstable = nullptr;
bool skip_major_process = false;
......@@ -506,7 +506,7 @@ int ObTabletDDLUtil::prepare_index_data_desc(ObTablet &tablet,
int ret = OB_SUCCESS;
data_desc.reset();
ObLSService *ls_service = MTL(ObLSService *);
ObArenaAllocator tmp_arena("DDLIdxDescTmp");
ObArenaAllocator tmp_arena("DDLIdxDescTmp", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObStorageSchema *storage_schema = nullptr;
const ObTabletID &tablet_id = tablet.get_tablet_meta().tablet_id_;
const ObLSID &ls_id = tablet.get_tablet_meta().ls_id_;
......@@ -642,7 +642,7 @@ int ObTabletDDLUtil::create_ddl_sstable(ObTablet &tablet,
blocksstable::ObSSTable &sstable)
{
int ret = OB_SUCCESS;
ObArenaAllocator tmp_arena("CreateDDLSstTmp");
ObArenaAllocator tmp_arena("CreateDDLSstTmp", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObStorageSchema *storage_schema = nullptr;
SMART_VAR(ObSSTableMergeRes, res) {
if (OB_UNLIKELY(nullptr == sstable_index_builder || !ddl_param.is_valid())) {
......@@ -749,7 +749,7 @@ int ObTabletDDLUtil::update_ddl_table_store(ObTablet &tablet,
ObLSService *ls_service = MTL(ObLSService *);
ObLSHandle ls_handle;
ObTabletHandle tablet_handle;
ObArenaAllocator allocator;
ObArenaAllocator allocator("DDLUtil_update", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObStorageSchema *tablet_storage_schema = nullptr;
if (OB_FAIL(ls_service->get_ls(ddl_param.ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) {
LOG_WARN("get ls failed", K(ret), K(ddl_param));
......@@ -798,7 +798,7 @@ int ObTabletDDLUtil::compact_ddl_sstable(ObTablet &tablet,
blocksstable::ObSSTable &sstable)
{
int ret = OB_SUCCESS;
ObArenaAllocator arena;
ObArenaAllocator arena("compact_sst", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObBlockMetaTree meta_tree;
ObArray<const ObDataMacroBlockMeta *> sorted_metas;
bool is_data_complete = false;
......
......@@ -509,7 +509,7 @@ int ObSSTableInsertTabletContext::build_sstable_slice(
const ObTabletID &tablet_id = build_param.tablet_id_;
ObSchemaGetterGuard schema_guard;
const ObTableSchema *table_schema = nullptr;
ObArenaAllocator allocator(lib::ObLabel("PartInsSstTmp"));
ObArenaAllocator allocator(lib::ObLabel("PartInsSstTmp"), OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id);
ObSSTableInsertSliceWriter *sstable_slice_writer = nullptr;
bool ddl_committed = false;
if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
......
......@@ -38,9 +38,9 @@ using namespace oceanbase::share::schema;
ObBlockMetaTree::ObBlockMetaTree()
: is_inited_(false), arena_(), tree_allocator_(arena_), block_tree_(tree_allocator_)
: is_inited_(false), macro_blocks_(), arena_("DDL_Btree", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()), tree_allocator_(arena_), block_tree_(tree_allocator_)
{
macro_blocks_.set_attr(ObMemAttr(MTL_ID(), "DDL_Btree"));
}
ObBlockMetaTree::~ObBlockMetaTree()
......@@ -89,7 +89,7 @@ int ObDDLKV::init_sstable_param(ObTablet &tablet,
{
int ret = OB_SUCCESS;
ObStorageSchema *storage_schema_ptr = nullptr;
ObArenaAllocator allocator;
ObArenaAllocator allocator("DDLKV", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
if (OB_UNLIKELY(!table_key.is_valid() || !ddl_start_scn.is_valid_and_not_min())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(table_key), K(ddl_start_scn));
......@@ -439,7 +439,7 @@ int ObBlockMetaTree::get_last_rowkey(const ObDatumRowkey *&last_rowkey)
ObDDLKV::ObDDLKV()
: is_inited_(false), ls_id_(), tablet_id_(), ddl_start_scn_(SCN::min_scn()), snapshot_version_(0),
lock_(), arena_allocator_("DDL_KV"), is_freezed_(false), is_closed_(false), last_freezed_scn_(SCN::min_scn()),
lock_(), arena_allocator_("DDL_KV", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()), is_freezed_(false), is_closed_(false), last_freezed_scn_(SCN::min_scn()),
min_scn_(SCN::max_scn()), max_scn_(SCN::min_scn()), freeze_scn_(SCN::max_scn()), pending_cnt_(0), data_format_version_(0)
{
}
......@@ -697,7 +697,7 @@ int ObDDLKV::close(ObTablet &tablet)
} else if (OB_FAIL(block_meta_tree_.get_sorted_meta_array(meta_array))) {
LOG_WARN("get sorted meta array failed", K(ret));
} else {
ObArenaAllocator allocator("DDLUpTabStore");
ObArenaAllocator allocator("DDLUpTabStore", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
ObSSTable sstable;
ObTabletDDLParam ddl_param;
ddl_param.tenant_id_ = MTL_ID();
......
......@@ -26,8 +26,9 @@ using namespace table;
*/
ObDirectLoadInsertTableParam::ObDirectLoadInsertTableParam()
: table_id_(OB_INVALID_ID), schema_version_(0), snapshot_version_(0), execution_id_(0), ddl_task_id_(0)
: table_id_(OB_INVALID_ID), schema_version_(0), snapshot_version_(0), execution_id_(0), ddl_task_id_(0), ls_partition_ids_()
{
ls_partition_ids_.set_attr(ObMemAttr(MTL_ID(), "DLITP_ids"));
}
ObDirectLoadInsertTableParam::~ObDirectLoadInsertTableParam()
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册