diff --git a/src/storage/blocksstable/ob_tmp_file.cpp b/src/storage/blocksstable/ob_tmp_file.cpp index 12a71944db..6ab4b26e82 100644 --- a/src/storage/blocksstable/ob_tmp_file.cpp +++ b/src/storage/blocksstable/ob_tmp_file.cpp @@ -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() diff --git a/src/storage/blocksstable/ob_tmp_file.h b/src/storage/blocksstable/ob_tmp_file.h index 5c32842035..bd8b6fa8fd 100644 --- a/src/storage/blocksstable/ob_tmp_file.h +++ b/src/storage/blocksstable/ob_tmp_file.h @@ -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); diff --git a/src/storage/ddl/ob_complement_data_task.cpp b/src/storage/ddl/ob_complement_data_task.cpp index 3c8e9d50d2..cee45b0c45 100644 --- a/src/storage/ddl/ob_complement_data_task.cpp +++ b/src/storage/ddl/ob_complement_data_task.cpp @@ -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_() diff --git a/src/storage/ddl/ob_complement_data_task.h b/src/storage/ddl/ob_complement_data_task.h index e4e9a825df..e479f7ed9e 100644 --- a/src/storage/ddl/ob_complement_data_task.h +++ b/src/storage/ddl/ob_complement_data_task.h @@ -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(); } diff --git a/src/storage/ddl/ob_ddl_clog.cpp b/src/storage/ddl/ob_ddl_clog.cpp index 3318de143c..e272036483 100644 --- a/src/storage/ddl/ob_ddl_clog.cpp +++ b/src/storage/ddl/ob_ddl_clog.cpp @@ -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_() { } diff --git a/src/storage/ddl/ob_ddl_clog.h b/src/storage/ddl/ob_ddl_clog.h index 11a723ff4c..fb4c6d9877 100644 --- a/src/storage/ddl/ob_ddl_clog.h +++ b/src/storage/ddl/ob_ddl_clog.h @@ -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_; diff --git a/src/storage/ddl/ob_ddl_merge_task.cpp b/src/storage/ddl/ob_ddl_merge_task.cpp index f64af24a8e..213286a9e8 100644 --- a/src/storage/ddl/ob_ddl_merge_task.cpp +++ b/src/storage/ddl/ob_ddl_merge_task.cpp @@ -311,7 +311,7 @@ int ObDDLTableMergeTask::process() ObTableStoreIterator ddl_table_iter; ObTabletMemberWrapper 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 sorted_metas; bool is_data_complete = false; diff --git a/src/storage/ddl/ob_direct_insert_sstable_ctx.cpp b/src/storage/ddl/ob_direct_insert_sstable_ctx.cpp index acd5d79ac6..51f273f201 100644 --- a/src/storage/ddl/ob_direct_insert_sstable_ctx.cpp +++ b/src/storage/ddl/ob_direct_insert_sstable_ctx.cpp @@ -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( diff --git a/src/storage/ddl/ob_tablet_ddl_kv.cpp b/src/storage/ddl/ob_tablet_ddl_kv.cpp index 1be0f1ec2a..64b8f7aa22 100644 --- a/src/storage/ddl/ob_tablet_ddl_kv.cpp +++ b/src/storage/ddl/ob_tablet_ddl_kv.cpp @@ -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(); diff --git a/src/storage/direct_load/ob_direct_load_insert_table_ctx.cpp b/src/storage/direct_load/ob_direct_load_insert_table_ctx.cpp index 3abbdd57f1..ee432c5a80 100644 --- a/src/storage/direct_load/ob_direct_load_insert_table_ctx.cpp +++ b/src/storage/direct_load/ob_direct_load_insert_table_ctx.cpp @@ -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()