提交 50b76d02 编写于 作者: O obdev 提交者: wangzelin.wzl

[CP] [CP] Add diagnostic log for detection of schema mgr ref leak

上级 4e81072c
......@@ -46,6 +46,7 @@ ObInnerSQLResult::ObInnerSQLResult(ObSQLSessionInfo& session)
mem_context_(nullptr),
mem_context_destroy_guard_(mem_context_),
sql_ctx_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_INNER_SQL_RESULT),
opened_(false),
session_(session),
result_set_(nullptr),
......
......@@ -36,7 +36,7 @@ ObTableApiRowIterator::ObTableApiRowIterator()
: part_service_(NULL),
schema_service_(NULL),
ctx_(NULL),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_TABLE_API_ROW_ITER),
table_schema_(NULL),
table_id_(0),
tenant_id_(0),
......
......@@ -24,7 +24,10 @@ namespace oceanbase {
namespace observer {
ObAllVirtualProxyBaseIterator::ObAllVirtualProxyBaseIterator()
: ObVirtualTableIterator(), schema_service_(NULL), full_schema_guard_(), calc_buf_(NULL)
: ObVirtualTableIterator(),
schema_service_(NULL),
full_schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
calc_buf_(NULL)
{}
ObAllVirtualProxyBaseIterator::~ObAllVirtualProxyBaseIterator()
......
......@@ -53,7 +53,7 @@ ObAllVirtualProxySchema::ObAllVirtualProxySchema()
routine_databases_(),
routine_names_(),
schema_service_(NULL),
full_schema_guard_(),
full_schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
complex_table_type_(CT_DEFAULT),
force_sql_refresh_(false)
{
......
......@@ -65,7 +65,7 @@ ObPhysicalRestoreStat::ObPhysicalRestoreStat(share::schema::ObMultiVersionSchema
statistic_(),
only_sys_(only_sys),
sql_proxy_(sql_proxy),
guard_(),
guard_(share::schema::ObSchemaMgrItem::MOD_PHY_RES_STAT),
schema_service_(schema_service),
pt_operator_(pt_operator),
stop_(stop)
......
......@@ -36,7 +36,7 @@ ObAllRebalanceMapItemStat::ObAllRebalanceMapItemStat()
cur_map_size_(-1),
cur_zone_idx_(-1),
cur_item_idx_(-1),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
table_schema_(NULL),
columns_()
{}
......
......@@ -31,7 +31,7 @@ ObAllRebalanceMapStat::ObAllRebalanceMapStat()
tenant_maps_(),
cur_tenant_idx_(-1),
cur_map_idx_(-1),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
table_schema_(NULL),
columns_()
{}
......
......@@ -31,7 +31,7 @@ ObAllRebalanceTenantStat::ObAllRebalanceTenantStat()
all_tenants_(),
cur_tenant_idx_(-1),
cur_zone_idx_(-1),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
table_schema_(NULL),
columns_()
{}
......
......@@ -21,7 +21,7 @@ using namespace oceanbase::share::schema;
ObAllRebalanceUnitDistributionStat::ObAllRebalanceUnitDistributionStat(ObUnitManager& unit_mgr,
ObILeaderCoordinator& leader_coordinator, ObServerManager& server_mgr, ObZoneManager& zone_mgr)
: server_balance_plan_(unit_mgr, leader_coordinator, server_mgr, zone_mgr),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
svr_ip_buf_(),
unit_info_(),
inited_(false)
......
......@@ -21,7 +21,7 @@ using namespace oceanbase::share::schema;
ObAllRebalanceUnitMigrateStat::ObAllRebalanceUnitMigrateStat(ObUnitManager& unit_mgr,
ObILeaderCoordinator& leader_coordinator, ObServerManager& server_mgr, ObZoneManager& zone_mgr)
: server_balance_plan_(unit_mgr, leader_coordinator, server_mgr, zone_mgr),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
src_ip_buf_(),
dst_ip_buf_(),
task_(),
......
......@@ -25,7 +25,7 @@ namespace rootserver {
class ObAllVirtualLeaderStat : public common::ObVirtualTableProjector {
public:
ObAllVirtualLeaderStat()
: schema_guard_(),
: schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
all_tenants_(),
cur_tenant_idx_(-1),
columns_(),
......
......@@ -301,7 +301,8 @@ public:
share::schema::ObSchemaGetterGuard& get_schema_guard()
{
if (OB_UNLIKELY(NULL == schema_guard_)) {
schema_guard_ = new (schema_guard_buf_) share::schema::ObSchemaGetterGuard;
schema_guard_ = new (schema_guard_buf_)
share::schema::ObSchemaGetterGuard(share::schema::ObSchemaMgrItem::MOD_VTABLE_SCAN_PARAM);
}
return *schema_guard_;
}
......
......@@ -75,7 +75,12 @@ int ObVirtualTableProjector::check_column_exist(
}
////////////////////////////////////////////////////////////////
ObSimpleVirtualTableIterator::ObSimpleVirtualTableIterator(uint64_t tenant_id, uint64_t table_id)
: tenant_id_(tenant_id), table_id_(table_id), schema_service_(NULL), table_schema_(NULL)
: tenant_id_(tenant_id),
table_id_(table_id),
columns_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_VIRTUAL_TABLE),
schema_service_(NULL),
table_schema_(NULL)
{}
int ObSimpleVirtualTableIterator::init(share::schema::ObMultiVersionSchemaService* schema_service)
......
......@@ -363,6 +363,7 @@ ObTenantPartitionIterator::ObTenantPartitionIterator()
: inited_(false),
pt_operator_(NULL),
schema_service_(NULL),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_TENANT_PT_ITER),
partition_entity_iterator_(),
allocator_(ObModIds::OB_RS_PARTITION_TABLE_TEMP),
filters_(),
......
......@@ -1735,7 +1735,7 @@ int ObMultiVersionSchemaService::add_schema_mgr_info(ObSchemaGetterGuard& schema
ret = OB_SCHEMA_EAGAIN;
LOG_WARN("tenant schema is not readable now", K(ret), K(tenant_id), K(snapshot_version), K(schema_status));
} else {
ObSchemaMgrHandle handle;
ObSchemaMgrHandle handle(schema_guard.mod_);
ObSchemaMgrInfo schema_mgr_info(tenant_id, snapshot_version, schema_mgr, handle, schema_status);
int64_t count = schema_guard.schema_mgr_infos_.count();
// Guaranteed to be monotonically increasing when inserted
......
......@@ -91,6 +91,26 @@ ObSchemaGetterGuard::ObSchemaGetterGuard()
local_allocator_(ObModIds::OB_SCHEMA_MGR_INFO_ARRAY),
schema_mgr_infos_(DEFAULT_TENANT_NUM, ModulePageAllocator(local_allocator_)),
schema_objs_(DEFAULT_RESERVE_SIZE, ModulePageAllocator(local_allocator_)),
mod_(ObSchemaMgrItem::MOD_STACK),
schema_guard_type_(INVALID_SCHEMA_GUARD_TYPE),
is_standby_cluster_(false),
is_inited_(false)
{}
ObSchemaGetterGuard::ObSchemaGetterGuard(const ObSchemaMgrItem::Mod mod)
: schema_service_(NULL),
snapshot_version_(OB_INVALID_VERSION),
session_id_(0),
mgr_(NULL),
mock_allocator_(ObModIds::OB_MOCK_SCHEMA),
mock_gts_schema_(nullptr),
mock_simple_gts_schema_(nullptr),
schema_helper_(mock_allocator_),
tenant_id_(OB_INVALID_TENANT_ID),
local_allocator_(ObModIds::OB_SCHEMA_MGR_INFO_ARRAY),
schema_mgr_infos_(DEFAULT_TENANT_NUM, ModulePageAllocator(local_allocator_)),
schema_objs_(DEFAULT_RESERVE_SIZE, ModulePageAllocator(local_allocator_)),
mod_(mod),
schema_guard_type_(INVALID_SCHEMA_GUARD_TYPE),
is_standby_cluster_(false),
is_inited_(false)
......@@ -140,6 +160,8 @@ int ObSchemaGetterGuard::reset()
schema_mgr_infos_.reset();
local_allocator_.reuse();
// mod_ should not be reset
is_inited_ = false;
return ret;
}
......
......@@ -171,6 +171,7 @@ public:
};
ObSchemaGetterGuard();
explicit ObSchemaGetterGuard(const ObSchemaMgrItem::Mod mod);
virtual ~ObSchemaGetterGuard();
int reset();
void dump();
......@@ -659,6 +660,7 @@ private:
// for new lazy logic
SchemaObjs schema_objs_;
ObSchemaMgrItem::Mod mod_;
SchemaGuardType schema_guard_type_;
bool is_standby_cluster_;
bool is_inited_;
......
......@@ -25,7 +25,11 @@ using namespace common;
namespace share {
namespace schema {
ObSchemaMgrHandle::ObSchemaMgrHandle() : schema_mgr_item_(NULL), ref_timestamp_(0)
ObSchemaMgrHandle::ObSchemaMgrHandle() : schema_mgr_item_(NULL), ref_timestamp_(0), mod_(ObSchemaMgrItem::MOD_STACK)
{}
ObSchemaMgrHandle::ObSchemaMgrHandle(const ObSchemaMgrItem::Mod mod)
: schema_mgr_item_(NULL), ref_timestamp_(0), mod_(mod)
{}
ObSchemaMgrHandle::~ObSchemaMgrHandle()
......@@ -43,8 +47,10 @@ ObSchemaMgrHandle& ObSchemaMgrHandle::operator=(const ObSchemaMgrHandle& other)
if (this != &other) {
reset();
schema_mgr_item_ = other.schema_mgr_item_;
mod_ = other.mod_;
if (NULL != schema_mgr_item_) {
(void)ATOMIC_FAA(&schema_mgr_item_->ref_cnt_, 1);
(void)ATOMIC_FAA(&schema_mgr_item_->mod_ref_cnt_[mod_], 1);
}
}
return *this;
......@@ -55,6 +61,7 @@ void ObSchemaMgrHandle::reset()
revert();
schema_mgr_item_ = NULL;
ref_timestamp_ = 0;
// mod_ should not be reset
}
bool ObSchemaMgrHandle::is_valid()
......@@ -63,12 +70,12 @@ bool ObSchemaMgrHandle::is_valid()
if (NULL != schema_mgr_item_) {
ref_cnt = ATOMIC_LOAD(&schema_mgr_item_->ref_cnt_);
}
return NULL != schema_mgr_item_ && ref_cnt > 0;
return NULL != schema_mgr_item_ && ref_cnt > 0 && mod_ >= 0 && mod_ < ObSchemaMgrItem::MOD_MAX;
}
void ObSchemaMgrHandle::dump() const
{
LOG_INFO("schema mgr item ptr", K(schema_mgr_item_), K(ref_timestamp_));
LOG_INFO("schema mgr item ptr", K(schema_mgr_item_), K(ref_timestamp_), K_(mod));
}
inline void ObSchemaMgrHandle::revert()
......@@ -89,6 +96,7 @@ inline void ObSchemaMgrHandle::revert()
K(lbt()));
}
(void)ATOMIC_FAA(&schema_mgr_item_->ref_cnt_, -1);
(void)ATOMIC_FAA(&schema_mgr_item_->mod_ref_cnt_[mod_], -1);
}
}
......@@ -126,6 +134,7 @@ int ObSchemaMgrCache::init(int64_t init_cached_num, Mode mode)
ObSchemaMgrItem& schema_mgr_item = schema_mgr_items_[i];
schema_mgr_item.schema_mgr_ = NULL;
schema_mgr_item.ref_cnt_ = 0;
MEMSET(schema_mgr_item.mod_ref_cnt_, 0, ObSchemaMgrItem::MOD_MAX);
}
}
}
......@@ -214,6 +223,7 @@ int ObSchemaMgrCache::get(const int64_t schema_version, const ObSchemaMgr*& sche
ret = OB_ENTRY_NOT_EXIST;
} else {
(void)ATOMIC_FAA(&dst_item->ref_cnt_, 1);
(void)ATOMIC_FAA(&dst_item->mod_ref_cnt_[handle.mod_], 1);
schema_mgr = dst_item->schema_mgr_;
handle.schema_mgr_item_ = dst_item;
handle.ref_timestamp_ = ObClockGenerator::getClock();
......@@ -259,6 +269,7 @@ int ObSchemaMgrCache::get_nearest(
if (OB_SUCC(ret)) {
dst_item = &schema_mgr_items_[nearest_pos];
(void)ATOMIC_FAA(&dst_item->ref_cnt_, 1);
(void)ATOMIC_FAA(&dst_item->mod_ref_cnt_[handle.mod_], 1);
schema_mgr = dst_item->schema_mgr_;
handle.schema_mgr_item_ = dst_item;
handle.ref_timestamp_ = ObTimeUtility::current_time();
......@@ -374,7 +385,9 @@ int ObSchemaMgrCache::put(ObSchemaMgr* schema_mgr, ObSchemaMgr*& eli_schema_mgr,
K(schema_version),
K(schema_mgr),
"ref_cnt",
schema_mgr_item.ref_cnt_);
schema_mgr_item.ref_cnt_,
"mod_ref_cnt",
ObArrayWrap<int64_t>(schema_mgr_item.mod_ref_cnt_, ObSchemaMgrItem::MOD_MAX));
}
}
} else {
......@@ -386,8 +399,12 @@ int ObSchemaMgrCache::put(ObSchemaMgr* schema_mgr, ObSchemaMgr*& eli_schema_mgr,
K(target_pos),
K(common::lbt()));
(void)ATOMIC_STORE(&dst_item->ref_cnt_, 0);
for (int64_t i = 0; i < ObSchemaMgrItem::MOD_MAX; i++) {
(void)ATOMIC_STORE(&dst_item->mod_ref_cnt_[i], 0);
}
if (NULL != handle) {
(void)ATOMIC_FAA(&dst_item->ref_cnt_, 1);
(void)ATOMIC_FAA(&dst_item->mod_ref_cnt_[handle->mod_], 1);
handle->schema_mgr_item_ = dst_item;
}
if (OB_NOT_NULL(eli_schema_mgr)) {
......@@ -429,6 +446,9 @@ int ObSchemaMgrCache::try_gc_tenant_schema_mgr(ObSchemaMgr*& eli_schema_mgr)
eli_schema_mgr = tmp_schema_mgr;
schema_mgr_item.schema_mgr_ = NULL;
(void)ATOMIC_STORE(&schema_mgr_item.ref_cnt_, 0);
for (int64_t i = 0; i < ObSchemaMgrItem::MOD_MAX; i++) {
(void)ATOMIC_STORE(&schema_mgr_item.mod_ref_cnt_[i], 0);
}
is_stop = true;
}
}
......@@ -469,6 +489,9 @@ int ObSchemaMgrCache::try_elimiante_schema_mgr(ObSchemaMgr*& eli_schema_mgr)
eli_schema_mgr = tmp_schema_mgr;
schema_mgr_item.schema_mgr_ = NULL;
(void)ATOMIC_STORE(&schema_mgr_item.ref_cnt_, 0);
for (int64_t i = 0; i < ObSchemaMgrItem::MOD_MAX; i++) {
(void)ATOMIC_STORE(&schema_mgr_item.mod_ref_cnt_[i], 0);
}
found = true;
}
}
......@@ -518,7 +541,9 @@ void ObSchemaMgrCache::dump() const
K(schema_count),
K(schema_size),
"ref_cnt",
schema_mgr_item.ref_cnt_);
schema_mgr_item.ref_cnt_,
"mod_ref_cnt",
ObArrayWrap<int64_t>(schema_mgr_item.mod_ref_cnt_, ObSchemaMgrItem::MOD_MAX));
}
}
LOG_INFO("[SCHEMA_STATISTICS] dump schema_mgr_cache", K(ret), K(total_count), K(total_size));
......
......@@ -25,16 +25,50 @@ class ObSchemaMgr;
class ObSchemaMgrCache;
struct ObSchemaMgrItem {
enum Mod {
MOD_STACK = 0,
MOD_VTABLE_SCAN_PARAM = 1,
MOD_INNER_SQL_RESULT = 2,
MOD_TABLE_API_ROW_ITER = 3,
MOD_STAT_CONV_INFO = 4,
MOD_SHUFFLE_TASK_HANDLE = 5,
MOD_LOAD_DATA_IMPL = 6,
MOD_PX_TASK_PROCESSS = 7,
MOD_TABLE_SCAN = 8,
MOD_DIST_EXECUTER = 9,
MOD_MINI_TASK_BASE = 10,
MOD_REMOTE_EXE = 11,
MOD_CACHED_GUARD = 12,
MOD_UNIQ_CHECK = 13,
MOD_LOGIC_ROW = 14,
MOD_TAILORED_ROW_ITER = 15,
MOD_SSTABLE_MERGE_CTX = 16,
MOD_SSTABLE_SPLIT_CTX = 17,
MOD_RELATIVE_TABLE = 18,
MOD_RECOVER_POINT = 19,
MOD_PART_SCHEMA_RECORDER = 20,
MOD_VIRTUAL_TABLE = 21,
MOD_PHY_RES_STAT = 22,
MOD_TENANT_PT_ITER = 23,
MOD_INDEX_PARAM = 24,
MOD_BACKUP_CHECKER = 25,
MOD_DIS_TASK_SPLITER = 26,
MOD_MAX
};
ObSchemaMgrItem() : schema_mgr_(NULL), ref_cnt_(0)
{}
{
MEMSET(mod_ref_cnt_, 0, MOD_MAX);
}
ObSchemaMgr* schema_mgr_;
int64_t ref_cnt_ CACHE_ALIGNED;
int64_t mod_ref_cnt_[MOD_MAX] CACHE_ALIGNED;
};
class ObSchemaMgrHandle {
public:
ObSchemaMgrHandle();
explicit ObSchemaMgrHandle(const ObSchemaMgrItem::Mod mod);
ObSchemaMgrHandle(const ObSchemaMgrHandle& other);
virtual ~ObSchemaMgrHandle();
ObSchemaMgrHandle& operator=(const ObSchemaMgrHandle& other);
......@@ -50,6 +84,7 @@ private:
static const int64_t REF_TIME_THRESHOLD = 60 * 1000 * 1000L;
ObSchemaMgrItem* schema_mgr_item_;
int64_t ref_timestamp_;
ObSchemaMgrItem::Mod mod_;
};
class ObSchemaMgrCache {
......
......@@ -35,7 +35,7 @@ struct ObStatConverterInfo {
real_table_id_(common::OB_INVALID_ID),
tenant_id_col_id_(UINT64_MAX),
vt_result_converter_(),
schema_guard_()
schema_guard_(share::schema::ObSchemaMgrItem::MOD_STAT_CONV_INFO)
{}
ObArray<bool> output_column_with_tenant_ids_;
......
......@@ -2666,6 +2666,7 @@ ObShuffleTaskHandle::ObShuffleTaskHandle(
ObInsertValueGenerator& main_generator, ObPartIdCalculator& main_calculator, ObDataFragMgr& main_datafrag_mgr)
: exec_ctx(GCTX.session_mgr_),
data_buffer(NULL),
schema_guard(share::schema::ObSchemaMgrItem::MOD_SHUFFLE_TASK_HANDLE),
generator(main_generator),
calculator(main_calculator),
datafrag_mgr(main_datafrag_mgr)
......
......@@ -924,6 +924,7 @@ public:
ObLoadDataImpl()
: // arguments
schema_guard_(share::schema::ObSchemaMgrItem::MOD_LOAD_DATA_IMPL),
part_level_(share::schema::PARTITION_LEVEL_ZERO),
part_num_(0),
initial_step_(FIND_LINE_START),
......
......@@ -67,7 +67,6 @@ private:
sql::ObPhysicalPlan phy_plan_;
common::ObArenaAllocator des_allocator_;
// observer::ObVirtualTableIteratorFactory vt_iter_factory_;
// share::schema::ObSchemaGetterGuard schema_guard_;
};
class ObInitFastSqcP : public obrpc::ObRpcProcessor<obrpc::ObPxRpcProxy::ObRpc<obrpc::OB_PX_FAST_INIT_SQC>> {
......
......@@ -46,7 +46,7 @@ ObPxTaskProcess::ObPxTaskProcess(const observer::ObGlobalContext& gctx, ObPxRpcI
: gctx_(gctx),
arg_(arg),
sql_ctx_(),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_PX_TASK_PROCESSS),
vt_iter_factory_(*gctx.vt_iter_creator_),
enqueue_timestamp_(0),
process_timestamp_(0),
......
......@@ -112,6 +112,7 @@ public:
output_row_count_(-1),
iter_end_(false),
partition_id_(OB_INVALID_INDEX),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_TABLE_SCAN),
iterated_rows_(0),
is_partition_list_empty_(false),
got_feedback_(false),
......
......@@ -443,6 +443,7 @@ ObTableScanOp::ObTableScanOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOp
output_row_count_(-1),
iter_end_(false),
partition_id_(OB_INVALID_INDEX),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_TABLE_SCAN),
iterated_rows_(0),
is_partition_list_empty_(false),
got_feedback_(false),
......
......@@ -91,6 +91,7 @@ public:
vt_iter_factory_(*gctx_.vt_iter_creator_),
phy_plan_(),
sql_ctx_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_DIST_EXECUTER),
trans_state_(),
exec_record_(),
process_timestamp_(0),
......@@ -249,6 +250,7 @@ public:
vt_iter_factory_(*gctx_.vt_iter_creator_),
phy_plan_(),
sql_ctx_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_MINI_TASK_BASE),
trans_state_(),
is_rollback_(false),
process_timestamp_(0),
......
......@@ -251,7 +251,7 @@ bool ObDistributedTaskSpliter::ObSliceComparer::operator()(const ObSliceEvent* s
}
ObDistributedTaskSpliter::ObDistributedTaskSpliter()
: schema_guard_(),
: schema_guard_(share::schema::ObSchemaMgrItem::MOD_DIS_TASK_SPLITER),
table_locations_(ObModIds::OB_SQL_EXECUTOR_TASK_SPLITER, OB_MALLOC_NORMAL_BLOCK_SIZE),
part_shuffle_keys_(ObModIds::OB_SQL_EXECUTOR_TASK_SPLITER, OB_MALLOC_NORMAL_BLOCK_SIZE),
part_idxs_(ObModIds::OB_SQL_EXECUTOR_TASK_SPLITER, OB_MALLOC_NORMAL_BLOCK_SIZE),
......
......@@ -27,6 +27,7 @@ public:
exec_ctx_(CURRENT_CONTEXT->get_arena_allocator(), gctx.session_mgr_),
vt_iter_factory_(*gctx_.vt_iter_creator_),
sql_ctx_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_REMOTE_EXE),
trans_state_(),
exec_errcode_(common::OB_SUCCESS),
process_timestamp_(0),
......
......@@ -125,6 +125,7 @@ struct ObSessionStat final {
class ObTenantCachedSchemaGuardInfo {
public:
ObTenantCachedSchemaGuardInfo()
: schema_guard_(share::schema::ObSchemaMgrItem::MOD_CACHED_GUARD)
{
reset();
}
......
......@@ -1834,7 +1834,7 @@ ObUniqueCheckingDag::ObUniqueCheckingDag()
part_service_(NULL),
pkey_(),
schema_service_(NULL),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_UNIQ_CHECK),
index_schema_(NULL),
data_table_schema_(NULL),
callback_(NULL),
......
......@@ -1299,6 +1299,7 @@ ObLogicRowProducer::ObLogicRowProducer()
guard_(),
allocator_(ObNewModIds::OB_PARTITION_MIGRATE),
storage_(NULL),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_LOGIC_ROW),
table_schema_(NULL),
memtable_(NULL),
arg_(),
......@@ -2474,7 +2475,7 @@ ObTailoredRowIterator::ObTailoredRowIterator()
is_inited_(false),
row_iter_(),
allocator_(),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_TAILORED_ROW_ITER),
table_schema_(NULL),
snapshot_version_(0)
{}
......
......@@ -560,7 +560,7 @@ ObSSTableMergeCtx::ObSSTableMergeCtx()
index_stats_(),
schema_version_(0),
table_schema_(NULL),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_SSTABLE_MERGE_CTX),
bf_rowkey_prefix_(0),
is_full_merge_(false),
stat_sampling_ratio_(0),
......
......@@ -53,9 +53,9 @@ ObSSTableSplitCtx::ObSSTableSplitCtx()
schema_version_(-1),
split_schema_version_(-1),
table_schema_(NULL),
schema_guard_(),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_SSTABLE_SPLIT_CTX),
split_table_schema_(NULL),
split_schema_guard_(),
split_schema_guard_(share::schema::ObSchemaMgrItem::MOD_SSTABLE_SPLIT_CTX),
split_handle_(),
remain_handle_(),
split_cnt_(0),
......
......@@ -111,6 +111,7 @@ public:
index_tables_(),
index_tables_buf_count_(0),
allocator_(allocator),
schema_guard_(share::schema::ObSchemaMgrItem::MOD_RELATIVE_TABLE),
table_param_(NULL),
use_table_param_(false)
{}
......
......@@ -896,8 +896,8 @@ ObRecoveryPointSchemaFilter::ObRecoveryPointSchemaFilter()
tenant_id_(OB_INVALID_ID),
tenant_recovery_point_schema_version_(OB_INVALID_VERSION),
tenant_current_schema_version_(OB_INVALID_VERSION),
recovery_point_schema_guard_(),
current_schema_guard_(),
recovery_point_schema_guard_(share::schema::ObSchemaMgrItem::MOD_RECOVER_POINT),
current_schema_guard_(share::schema::ObSchemaMgrItem::MOD_RECOVER_POINT),
is_schema_version_same_(false)
{}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册