disable trans data structure's copy operator

This commit is contained in:
chinaxing 2024-09-18 07:03:36 +00:00 committed by ob-robot
parent ffe2dddd6b
commit f91c51797c
22 changed files with 165 additions and 54 deletions

View File

@ -133,7 +133,7 @@ void TestTableScanPureDataTable::insert_data_to_tablet(MockObAccessService *acce
dml_param.schema_version_ = share::OB_CORE_SCHEMA_VERSION + 1;
dml_param.tenant_schema_version_ = share::OB_CORE_SCHEMA_VERSION + 1;
dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_;
dml_param.snapshot_ = read_snapshot;
ASSERT_EQ(OB_SUCCESS, dml_param.snapshot_.assign(read_snapshot));
dml_param.store_ctx_guard_ = &store_ctx_guard;
ObArenaAllocator allocator;

View File

@ -118,7 +118,7 @@ public:
ASSERT_TRUE(MockTenantModuleEnv::get_instance().is_inited());
}
void create_ls(uint64_t tenant_id, ObLSID &ls_id, ObLS *&ls);
void insert_rows(ObLSID &ls_id, ObTabletID &tablet_id, ObTxDesc &tx_desc, ObTxReadSnapshot snapshot, const char* in_str);
void insert_rows(ObLSID &ls_id, ObTabletID &tablet_id, ObTxDesc &tx_desc, ObTxReadSnapshot &snapshot, const char* in_str);
void prepare_tx_desc(ObTxDesc *&tx_desc, ObTxReadSnapshot &snapshot);
private:
static share::schema::ObTableSchema table_schema_;
@ -161,7 +161,7 @@ void TestTrans::create_ls(uint64_t tenant_id, ObLSID &ls_id, ObLS *&ls)
}
}
void TestTrans::insert_rows(ObLSID &ls_id, ObTabletID &tablet_id, ObTxDesc &tx_desc, ObTxReadSnapshot snapshot, const char* ins_str)
void TestTrans::insert_rows(ObLSID &ls_id, ObTabletID &tablet_id, ObTxDesc &tx_desc, ObTxReadSnapshot &snapshot, const char* ins_str)
{
int64_t affected_rows = 0;
ObMockDatumRowIterator ins_iter;
@ -179,7 +179,7 @@ void TestTrans::insert_rows(ObLSID &ls_id, ObTabletID &tablet_id, ObTxDesc &tx_d
dml_param.timeout_ = ObTimeUtility::current_time() + 100000000;
dml_param.schema_version_ = 1000;
dml_param.table_param_ = &table_dml_param;
dml_param.snapshot_ = snapshot;
ASSERT_EQ(OB_SUCCESS, dml_param.snapshot_.assign(snapshot));
dml_param.store_ctx_guard_ = &store_ctx_guard;
auto as = MTL(ObAccessService*);

View File

@ -2090,7 +2090,7 @@ int ObPLCursorInfo::deep_copy(ObPLCursorInfo &src, common::ObIAllocator *allocat
forall_rollback_ = src.forall_rollback_;
trans_id_ = src.trans_id_;
is_scrollable_ = src.is_scrollable_;
snapshot_ = src.snapshot_;
OZ (snapshot_.assign(src.snapshot_));
is_need_check_snapshot_ = src.is_need_check_snapshot_;
last_execute_time_ = src.last_execute_time_;
sql_trace_id_ = src.sql_trace_id_;

View File

@ -11093,7 +11093,9 @@ bool ObRegisterTxDataResult::is_valid() const
int ObRegisterTxDataResult::init(const ObTxExecResult &tx_result)
{
int ret = OB_SUCCESS;
tx_result_ = tx_result;
if (OB_FAIL(tx_result_.assign(tx_result))) {
LOG_WARN("assign tx result fail", K(ret));
}
return ret;
}

View File

@ -320,9 +320,14 @@ int ObHNSWSerializeCallback::operator()(const char *data, const int64_t data_siz
lob_param.sql_mode_ = SMO_DEFAULT;
lob_param.timeout_ = param.timeout_;
lob_param.lob_common_ = nullptr;
lob_param.snapshot_ = *reinterpret_cast<transaction::ObTxReadSnapshot*>(param.snapshot_);
lob_param.tx_desc_ = reinterpret_cast<transaction::ObTxDesc*>(param.tx_desc_);
if (OB_ISNULL(lob_mngr)) {
ret = lob_param.snapshot_.assign(*reinterpret_cast<transaction::ObTxReadSnapshot*>(param.snapshot_));
if (OB_FAIL(ret)) {
LOG_WARN("assign snapshot fail", K(ret));
} else {
lob_param.tx_desc_ = reinterpret_cast<transaction::ObTxDesc*>(param.tx_desc_);
}
if (OB_FAIL(ret)) {
} else if (OB_ISNULL(lob_mngr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get lob manager nullptr", K(ret));
} else if (OB_FAIL(lob_mngr->append(lob_param, src_lob))) {
@ -339,4 +344,4 @@ int ObHNSWSerializeCallback::operator()(const char *data, const int64_t data_siz
}
};
};
};

View File

@ -138,7 +138,9 @@ int ObDASLocalLookupIter::init_scan_param(ObTableScanParam &param, const ObDASSc
param.trans_desc_ = trans_desc_;
}
if (OB_NOT_NULL(snapshot_)) {
param.snapshot_ = *snapshot_;
if (OB_FAIL(param.snapshot_.assign(*snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null snapshot", K(ret), KPC(this));
@ -152,7 +154,7 @@ int ObDASLocalLookupIter::init_scan_param(ObTableScanParam &param, const ObDASSc
param.op_filters_ = &ctdef->pd_expr_spec_.pushdown_filters_;
}
param.pd_storage_filters_ = rtdef->p_pd_expr_op_->pd_storage_filters_;
if (OB_FAIL(param.column_ids_.assign(ctdef->access_column_ids_))) {
if (FAILEDx(param.column_ids_.assign(ctdef->access_column_ids_))) {
LOG_WARN("failed to assign column ids", K(ret));
}
if (rtdef->sample_info_ != nullptr) {

View File

@ -385,7 +385,9 @@ int ObDASTextRetrievalIter::init_base_idx_scan_param(
}
if (OB_NOT_NULL(snapshot)) {
scan_param.snapshot_ = *snapshot;
if (OB_FAIL(scan_param.snapshot_.assign(*snapshot))) {
LOG_WARN("assign snapshot fail", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("null snapshot", K(ret), KP(snapshot));

View File

@ -639,7 +639,9 @@ int ObDASTextRetrievalMergeIter::init_total_doc_cnt_param(
}
if (OB_NOT_NULL(snapshot)) {
scan_param.snapshot_ = *snapshot;
if (OB_FAIL(scan_param.snapshot_.assign(*snapshot))) {
LOG_WARN("assign snapshot fail", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("null snapshot", K(ret), KP(snapshot));

View File

@ -282,7 +282,9 @@ int ObDASVIdMergeIter::init_rowkey_vid_scan_param(
rowkey_vid_scan_param_.trans_desc_ = trans_desc;
}
if (OB_NOT_NULL(snapshot)) {
rowkey_vid_scan_param_.snapshot_ = *snapshot;
if (OB_FAIL(rowkey_vid_scan_param_.snapshot_.assign(*snapshot))) {
LOG_WARN("assign snapshot fail", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null snapshot", K(ret), KPC(ctdef), KPC(rtdef));
@ -296,7 +298,7 @@ int ObDASVIdMergeIter::init_rowkey_vid_scan_param(
rowkey_vid_scan_param_.op_filters_ = &ctdef->pd_expr_spec_.pushdown_filters_;
}
rowkey_vid_scan_param_.pd_storage_filters_ = rtdef->p_pd_expr_op_->pd_storage_filters_;
if (OB_FAIL(rowkey_vid_scan_param_.column_ids_.assign(ctdef->access_column_ids_))) {
if (FAILEDx(rowkey_vid_scan_param_.column_ids_.assign(ctdef->access_column_ids_))) {
LOG_WARN("failed to assign column ids", K(ret));
}
if (rtdef->sample_info_ != nullptr) {
@ -782,4 +784,4 @@ int ObDASVIdMergeIter::fill_vid_ids_in_data_table(const common::ObIArray<int64_t
}
} // end namespace sql
} // end namespace oceanbase
} // end namespace oceanbase

View File

@ -280,7 +280,9 @@ int ObDASScanOp::init_scan_param()
scan_param_.sample_info_ = *scan_rtdef_->sample_info_;
}
if (OB_NOT_NULL(snapshot_)) {
scan_param_.snapshot_ = *snapshot_;
if (OB_FAIL(scan_param_.snapshot_.assign(*snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("snapshot is null", K(ret), KPC(this));
@ -296,7 +298,7 @@ int ObDASScanOp::init_scan_param()
scan_param_.op_filters_ = &scan_ctdef_->pd_expr_spec_.pushdown_filters_;
}
scan_param_.pd_storage_filters_ = scan_rtdef_->p_pd_expr_op_->pd_storage_filters_;
if (OB_FAIL(scan_param_.column_ids_.assign(scan_ctdef_->access_column_ids_))) {
if (FAILEDx(scan_param_.column_ids_.assign(scan_ctdef_->access_column_ids_))) {
LOG_WARN("init column ids failed", K(ret));
}
//external table scan params
@ -1831,7 +1833,9 @@ OB_INLINE int ObLocalIndexLookupOp::init_scan_param()
scan_param_.trans_desc_ = tx_desc_;
}
if (OB_NOT_NULL(snapshot_)) {
scan_param_.snapshot_ = *snapshot_;
if (OB_FAIL(scan_param_.snapshot_.assign(*snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("snapshot is null", K(ret), KPC(this));
@ -1847,7 +1851,7 @@ OB_INLINE int ObLocalIndexLookupOp::init_scan_param()
scan_param_.op_filters_ = &lookup_ctdef_->pd_expr_spec_.pushdown_filters_;
}
scan_param_.pd_storage_filters_ = lookup_rtdef_->p_pd_expr_op_->pd_storage_filters_;
if (OB_FAIL(scan_param_.column_ids_.assign(lookup_ctdef_->access_column_ids_))) {
if (FAILEDx(scan_param_.column_ids_.assign(lookup_ctdef_->access_column_ids_))) {
LOG_WARN("init column ids failed", K(ret));
}
LOG_DEBUG("init local index lookup scan_param", K(scan_param_));

View File

@ -422,7 +422,9 @@ int ObDataAccessService::do_async_remote_das_task(
remote_info.exec_ctx_ = &das_ref.get_exec_ctx();
remote_info.frame_info_ = das_ref.get_expr_frame_info();
remote_info.trans_desc_ = session->get_tx_desc();
remote_info.snapshot_ = *task_arg.get_task_op()->get_snapshot();
if (OB_FAIL(remote_info.snapshot_.assign(*task_arg.get_task_op()->get_snapshot()))) {
LOG_WARN("assign snapshot fail", K(ret));
}
remote_info.need_tx_ = (remote_info.trans_desc_ != nullptr);
session->get_cur_sql_id(remote_info.sql_id_, sizeof(remote_info.sql_id_));
remote_info.user_id_ = session->get_user_id();
@ -433,7 +435,7 @@ int ObDataAccessService::do_async_remote_das_task(
ObDASRemoteInfo::get_remote_info() = &remote_info;
ObIDASTaskResult *op_result = nullptr;
ObRpcDasAsyncAccessCallBack *das_async_cb = nullptr;
if (OB_FAIL(das_ref.allocate_async_das_cb(das_async_cb, task_ops, timeout_ts))) {
if (FAILEDx(das_ref.allocate_async_das_cb(das_async_cb, task_ops, timeout_ts))) {
LOG_WARN("failed to allocate das async cb", K(ret));
}
// prepare op result in advance avoiding racing condition.
@ -516,7 +518,9 @@ int ObDataAccessService::do_sync_remote_das_task(
remote_info.exec_ctx_ = &das_ref.get_exec_ctx();
remote_info.frame_info_ = das_ref.get_expr_frame_info();
remote_info.trans_desc_ = session->get_tx_desc();
remote_info.snapshot_ = *task_arg.get_task_op()->get_snapshot();
if (OB_FAIL(remote_info.snapshot_.assign(*task_arg.get_task_op()->get_snapshot()))) {
LOG_WARN("assign snapshot fail", K(ret));
}
remote_info.need_tx_ = (remote_info.trans_desc_ != nullptr);
session->get_cur_sql_id(remote_info.sql_id_, sizeof(remote_info.sql_id_));
remote_info.user_id_ = session->get_user_id();

View File

@ -317,7 +317,9 @@ int ObDomainIndexLookupOp::set_doc_id_idx_lookup_param(
aux_scan_param.sample_info_ = *aux_lookup_rtdef->sample_info_;
}
if (OB_NOT_NULL(snapshot_)) {
aux_scan_param.snapshot_ = *snapshot_;
if (OB_FAIL(aux_scan_param.snapshot_.assign(*snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("snapshot is null", K(ret), KPC(this));

View File

@ -253,7 +253,9 @@ int ObVectorIndexLookupOp::init_base_idx_scan_param(const share::ObLSID &ls_id,
}
if (OB_NOT_NULL(snapshot)) {
scan_param.snapshot_ = *snapshot;
if (OB_FAIL(scan_param.snapshot_.assign(*snapshot))) {
LOG_WARN("assign snapshot fail", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("null snapshot", K(ret), KP(snapshot));

View File

@ -1208,7 +1208,9 @@ int ObDMLService::init_dml_param(const ObDASDMLBaseCtDef &base_ctdef,
dml_param.prelock_ = base_rtdef.prelock_;
dml_param.is_batch_stmt_ = base_ctdef.is_batch_stmt_;
dml_param.dml_allocator_ = &das_alloc;
dml_param.snapshot_ = snapshot;
if (OB_FAIL(dml_param.snapshot_.assign(snapshot))) {
LOG_WARN("assign snapshot fail", K(ret));
}
dml_param.branch_id_ = write_branch_id;
dml_param.store_ctx_guard_ = &store_ctx_gurad;
if (base_ctdef.is_batch_stmt_) {

View File

@ -3778,8 +3778,9 @@ int ObLobQueryIter::open(ObLobAccessParam &param, ObLobCtx& lob_ctx, common::ObA
if (OB_FAIL(lob_manager->lob_query_with_retry(param, dst_addr, is_remote, meta_iter_,
ObLobQueryArg::QueryType::READ, remote_query_ctx_))) {
LOG_WARN("fail to do lob query with retry", K(ret), K(is_remote), K(dst_addr));
} else if (OB_FAIL(param_.assign(param))) {
LOG_WARN("assign lob access param fail", K(ret), K(param));
} else if (is_remote) { // init remote scan
param_ = param;
is_reverse_ = param.scan_backward_;
cs_type_ = param.coll_type_;
is_inited_ = true;
@ -3791,7 +3792,6 @@ int ObLobQueryIter::open(ObLobAccessParam &param, ObLobCtx& lob_ctx, common::ObA
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc buffer failed.", K(ret), K(last_data_buf_len_));
} else {
param_ = param;
lob_ctx_ = lob_ctx;
is_inited_ = true;
is_in_row_ = false;

View File

@ -358,7 +358,6 @@ int ObPersistentLobApator::build_lob_meta_table_dml(
dml_base_param.tz_info_ = NULL;
dml_base_param.sql_mode_ = SMO_DEFAULT;
dml_base_param.encrypt_meta_ = &dml_base_param.encrypt_meta_legacy_;
dml_base_param.snapshot_ = param.snapshot_;
dml_base_param.check_schema_version_ = false; // lob tablet should not check schema version
dml_base_param.schema_version_ = 0;
dml_base_param.store_ctx_guard_ = store_ctx_guard;
@ -374,6 +373,8 @@ int ObPersistentLobApator::build_lob_meta_table_dml(
if (OB_FAIL(ret)) {
} else if (OB_FAIL(get_meta_table_dml_param(dml_base_param.table_param_))) {
LOG_WARN("get_meta_table_dml_param fail", KR(ret));
} else if (OB_FAIL(dml_base_param.snapshot_.assign(param.snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
}
}
return ret;
@ -695,7 +696,6 @@ int ObPersistentLobApator::build_common_scan_param(
scan_param.limit_param_.limit_ = -1;
scan_param.limit_param_.offset_ = 0;
// sessions
scan_param.snapshot_ = param.snapshot_;
if(param.read_latest_) {
scan_param.tx_id_ = param.snapshot_.core_.tx_id_;
}
@ -715,6 +715,9 @@ int ObPersistentLobApator::build_common_scan_param(
scan_param.need_scn_ = false;
scan_param.pd_storage_flag_ = false;
scan_param.fb_snapshot_ = param.fb_snapshot_;
if (OB_FAIL(scan_param.snapshot_.assign(param.snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
}
}
return ret;
}

View File

@ -37,6 +37,68 @@ ObLobAccessParam::~ObLobAccessParam()
}
}
int ObLobAccessParam::assign(const ObLobAccessParam& other)
{
int ret = OB_SUCCESS;
this->tmp_allocator_ = other.tmp_allocator_;
this->allocator_ = other.allocator_;
this->tx_desc_ = other.tx_desc_;
// use assign
// this->snapshot_ = other.snapshot_;
this->tx_id_ = other.tx_id_;
this->sql_mode_ = other.sql_mode_;
this->dml_base_param_ = other.dml_base_param_;
this->tenant_id_ = other.tenant_id_;
this->src_tenant_id_ = other.src_tenant_id_;
this->ls_id_ = other.ls_id_;
this->tablet_id_ = other.tablet_id_;
this->lob_meta_tablet_id_ = other.lob_meta_tablet_id_;
this->lob_piece_tablet_id_ = other.lob_piece_tablet_id_;
this->coll_type_ = other.coll_type_;
this->lob_locator_ = other.lob_locator_;
this->lob_common_ = other.lob_common_;
this->lob_data_ = other.lob_data_;
this->byte_size_ = other.byte_size_;
this->handle_size_ = other.handle_size_;
this->timeout_ = other.timeout_;
this->fb_snapshot_ = other.fb_snapshot_;
this->offset_ = other.offset_;
this->len_ = other.len_;
this->parent_seq_no_ = other.parent_seq_no_;
this->seq_no_st_ = other.seq_no_st_;
this->used_seq_cnt_ = other.used_seq_cnt_;
this->total_seq_cnt_ = other.total_seq_cnt_;
this->checksum_ = other.checksum_;
this->update_len_ = other.update_len_;
this->op_type_ = other.op_type_;
this->is_total_quantity_log_ = other.is_total_quantity_log_;
this->read_latest_ = other.read_latest_;
this->scan_backward_ = other.scan_backward_;
this->is_fill_zero_ = other.is_fill_zero_;
this->from_rpc_ = other.from_rpc_;
this->inrow_read_nocopy_ = other.inrow_read_nocopy_;
this->is_store_char_len_ = other.is_store_char_len_;
this->inrow_threshold_ = other.inrow_threshold_;
this->schema_chunk_size_ = other.schema_chunk_size_;
this->ext_info_log_ = other.ext_info_log_;
this->access_ctx_ = other.access_ctx_;
this->lob_id_geneator_ = other.lob_id_geneator_;
this->remote_query_ctx_ = other.remote_query_ctx_;
if (OB_FAIL(this->snapshot_.assign(other.snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret), K(other));
}
return ret;
}
ObCollationType ObLobCharsetUtil::get_collation_type(ObObjType type, ObCollationType ori_coll_type)
{
ObCollationType coll_type = ori_coll_type;
@ -244,19 +306,23 @@ int ObInsertLobColumnHelper::insert_lob_column(ObIAllocator &allocator,
ObLobAccessParam lob_param;
lob_param.src_tenant_id_ = src_tenant_id;
lob_param.tx_desc_ = tx_desc;
lob_param.snapshot_ = snapshot;
lob_param.sql_mode_ = SMO_DEFAULT;
lob_param.ls_id_ = ls_id;
lob_param.tablet_id_ = tablet_id;
lob_param.coll_type_ = ObLobCharsetUtil::get_collation_type(obj_type, cs_type);
lob_param.allocator_ = &allocator;
lob_param.lob_common_ = nullptr;
lob_param.timeout_ = timeout_ts;
lob_param.scan_backward_ = false;
lob_param.offset_ = 0;
lob_param.inrow_threshold_ = lob_storage_param.inrow_threshold_;
LOG_DEBUG("lob storage param", K(lob_storage_param), K(cs_type));
if (!src.is_valid()) {
if (OB_FAIL(lob_param.snapshot_.assign(snapshot))) {
LOG_WARN("assign snapshot fail", K(ret));
} else {
lob_param.sql_mode_ = SMO_DEFAULT;
lob_param.ls_id_ = ls_id;
lob_param.tablet_id_ = tablet_id;
lob_param.coll_type_ = ObLobCharsetUtil::get_collation_type(obj_type, cs_type);
lob_param.allocator_ = &allocator;
lob_param.lob_common_ = nullptr;
lob_param.timeout_ = timeout_ts;
lob_param.scan_backward_ = false;
lob_param.offset_ = 0;
lob_param.inrow_threshold_ = lob_storage_param.inrow_threshold_;
LOG_DEBUG("lob storage param", K(lob_storage_param), K(cs_type));
}
if (OB_FAIL(ret)) {
} else if (!src.is_valid()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid src lob locator.", K(ret));
} else if (OB_FAIL(lob_mngr->append(lob_param, src))) {

View File

@ -81,6 +81,8 @@ public:
~ObLobAccessParam();
public:
int assign(const ObLobAccessParam &other);
bool is_full_read() const { return op_type_ == ObLobDataOutRowCtx::OpType::SQL && 0 == offset_ && (len_ == byte_size_ || INT64_MAX == len_ || UINT64_MAX == len_); }
bool is_full_delete() const { return op_type_ == ObLobDataOutRowCtx::OpType::SQL && 0 == offset_ && len_ >= byte_size_; }
bool is_full_insert() const { return op_type_ == ObLobDataOutRowCtx::OpType::SQL && 0 == offset_ && 0 == byte_size_; }
@ -174,6 +176,9 @@ public:
share::ObTabletCacheInterval *lob_id_geneator_;
// remote query ctx
void *remote_query_ctx_;
private:
DISALLOW_COPY_AND_ASSIGN(ObLobAccessParam);
};
struct ObLobMetaInfo {

View File

@ -4410,9 +4410,10 @@ int ObLSTabletService::insert_lob_col(
ObLobAccessParam lob_param;
lob_param.tx_desc_ = run_ctx.store_ctx_.mvcc_acc_ctx_.tx_desc_;
lob_param.parent_seq_no_ = run_ctx.store_ctx_.mvcc_acc_ctx_.tx_scn_;
lob_param.snapshot_ = run_ctx.dml_param_.snapshot_;
lob_param.is_total_quantity_log_ = run_ctx.dml_param_.is_total_quantity_log_;
if (lob_param.snapshot_.is_none_read()) {
if (OB_FAIL(lob_param.snapshot_.assign(run_ctx.dml_param_.snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
} else if (lob_param.snapshot_.is_none_read()) {
// NOTE:
// lob_insert need table_scan, the snapshot already generated in
// run_ctx.store_ctx, use it as an LS ReadSnapshot
@ -4442,7 +4443,7 @@ int ObLSTabletService::insert_lob_col(
// for not strict sql mode, will insert empty string without lob header
bool has_lob_header = datum.has_lob_header() && raw_data.length() > 0;
ObLobLocatorV2 loc(raw_data, has_lob_header);
if (OB_FAIL(set_lob_storage_params(run_ctx, column, lob_param))) {
if (FAILEDx(set_lob_storage_params(run_ctx, column, lob_param))) {
LOG_WARN("set_lob_storage_params fail", K(ret), K(column));
} else if (OB_FAIL(lob_mngr->append(lob_param, loc))) {
LOG_WARN("[STORAGE_LOB]lob append failed.", K(ret));
@ -4822,8 +4823,9 @@ int ObLSTabletService::process_delta_lob(
// init lob param
lob_param.tx_desc_ = run_ctx.store_ctx_.mvcc_acc_ctx_.tx_desc_;
lob_param.parent_seq_no_ = run_ctx.store_ctx_.mvcc_acc_ctx_.tx_scn_;
lob_param.snapshot_ = run_ctx.dml_param_.snapshot_;
if (lob_param.snapshot_.is_none_read()) {
if (OB_FAIL(lob_param.snapshot_.assign(run_ctx.dml_param_.snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
} else if (lob_param.snapshot_.is_none_read()) {
// NOTE:
// lob_insert need table_scan, the snapshot already generated in
// run_ctx.store_ctx, use it as an LS ReadSnapshot
@ -4839,7 +4841,7 @@ int ObLSTabletService::process_delta_lob(
// should use old obj lob
ObLobLocatorV2 old_lob;
ObString old_disk_lob;
if (OB_FAIL(set_lob_storage_params(run_ctx, column, lob_param))) {
if (FAILEDx(set_lob_storage_params(run_ctx, column, lob_param))) {
LOG_WARN("set_lob_storage_params fail", K(ret), K(column));
} else if (FALSE_IT(old_datum.get_mem_lob(old_lob))) {
} else if (!old_lob.is_valid()) {
@ -5818,7 +5820,6 @@ int ObLSTabletService::delete_lob_col(
lob_common = reinterpret_cast<ObLobCommon*>(buf);
lob_param.tx_desc_ = run_ctx.store_ctx_.mvcc_acc_ctx_.tx_desc_;
lob_param.parent_seq_no_ = run_ctx.store_ctx_.mvcc_acc_ctx_.tx_scn_;
lob_param.snapshot_ = run_ctx.dml_param_.snapshot_;
lob_param.tx_id_ = lob_param.tx_desc_->get_tx_id();
lob_param.sql_mode_ = run_ctx.dml_param_.sql_mode_;
lob_param.is_total_quantity_log_ = run_ctx.dml_param_.is_total_quantity_log_;
@ -5834,7 +5835,10 @@ int ObLSTabletService::delete_lob_col(
lob_param.offset_ = 0;
// use byte size to delete all
lob_param.len_ = lob_param.byte_size_; //ObCharset::strlen_char(lob_param.coll_type_, sql_data.ptr(), sql_data.length());
if (lob_param.byte_size_ < 0) {
if (OB_FAIL(lob_param.snapshot_.assign(run_ctx.dml_param_.snapshot_))) {
LOG_WARN("assign snapshot fail", K(ret));
} else if (lob_param.byte_size_ < 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("calc byte size is negative.", K(ret), K(data), K(lob_param));
} else if (OB_FAIL(lob_mngr->erase(lob_param))) {

View File

@ -1454,6 +1454,7 @@ int ObTxExecResult::assign(const ObTxExecResult &r)
TRANS_LOG(WARN, "assign touched_ls_list fail, set incomplete", K(ret), KPC(this));
}
conflict_txs_.assign(r.conflict_txs_);
conflict_info_array_.assign(r.conflict_info_array_);
return ret;
}

View File

@ -311,6 +311,7 @@ struct ObTxReadSnapshot
K_(parts),
K_(committed));
OB_UNIS_VERSION(1);
DISABLE_COPY_ASSIGN(ObTxReadSnapshot);
};
class ObTxSavePoint
@ -386,6 +387,7 @@ public:
int merge_result(const ObTxExecResult &r);
int assign(const ObTxExecResult &r);
const ObSArray<ObTransIDAndAddr> &get_conflict_txs() const { return conflict_txs_; }
DISABLE_COPY_ASSIGN(ObTxExecResult);
};
class RollbackMaskSet
@ -836,6 +838,7 @@ LST_DO(DEF_FREE_ROUTE_DECODE, (;), static, dynamic, parts, extra);
int64_t get_seq_base() const { return seq_base_; }
int add_modified_tables(const ObIArray<uint64_t> &tables);
bool has_modify_table(const uint64_t table_id) const;
DISABLE_COPY_ASSIGN(ObTxDesc);
};
// Is used to store and travserse all TxScheduler's Stat information;

View File

@ -376,8 +376,8 @@ int TestDmlCommon::build_table_scan_param(
ObTableScanParam &scan_param)
{
int ret = build_table_scan_param_base_(tenant_id, table_param, false, scan_param);
if (OB_SUCC(ret)) {
scan_param.snapshot_ = read_snapshot;
if (FAILEDx(scan_param.snapshot_.assign(read_snapshot))) {
STORAGE_LOG(WARN, "assign snapshot fail", K(ret));
}
return ret;
}