[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