[CP]add allocator label
This commit is contained in:
		| @ -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() | ||||
|  | ||||
		Reference in New Issue
	
	Block a user
	 renju96
					renju96